浏览代码

HADOOP-2389 Provide multiple language bindings for HBase (Thrift)
Actual code (first commit added thrift lib so this patch would build).


git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@611535 13f79535-47bb-0310-9956-ffa450edef68

Michael Stack 17 年之前
父节点
当前提交
716850dd1f
共有 22 个文件被更改,包括 10004 次插入3 次删除
  1. 1 0
      src/contrib/hbase/CHANGES.txt
  2. 9 3
      src/contrib/hbase/bin/hbase
  3. 238 0
      src/contrib/hbase/src/examples/thrift/DemoClient.cpp
  4. 276 0
      src/contrib/hbase/src/examples/thrift/DemoClient.java
  5. 178 0
      src/contrib/hbase/src/examples/thrift/DemoClient.rb
  6. 18 0
      src/contrib/hbase/src/examples/thrift/Makefile
  7. 15 0
      src/contrib/hbase/src/examples/thrift/README.txt
  8. 12 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
  9. 437 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift
  10. 624 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
  11. 89 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
  12. 111 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
  13. 277 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
  14. 34 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/Constants.java
  15. 6873 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
  16. 112 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
  17. 111 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
  18. 156 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
  19. 111 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/NotFound.java
  20. 111 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/RegionDescriptor.java
  21. 153 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/ScanEntry.java
  22. 58 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/package.html

+ 1 - 0
src/contrib/hbase/CHANGES.txt

@@ -26,6 +26,7 @@ Trunk (unreleased changes)
                 (Bryan Duxbury via Stack)
                 (Bryan Duxbury via Stack)
     HADOOP-2240 Truncate for hbase (Edward Yoon via Stack)
     HADOOP-2240 Truncate for hbase (Edward Yoon via Stack)
     HADOOP-2389 Provide multiple language bindings for HBase (Thrift)
     HADOOP-2389 Provide multiple language bindings for HBase (Thrift)
+                (David Simpson via Stack)
 
 
   OPTIMIZATIONS
   OPTIMIZATIONS
    HADOOP-2479 Save on number of Text object creations
    HADOOP-2479 Save on number of Text object creations

+ 9 - 3
src/contrib/hbase/bin/hbase

@@ -58,9 +58,11 @@ esac
 if [ $# = 0 ]; then
 if [ $# = 0 ]; then
   echo "Usage: hbase [--hadoop=hadoopdir] <command>"
   echo "Usage: hbase [--hadoop=hadoopdir] <command>"
   echo "where <command> is one of:"
   echo "where <command> is one of:"
-  echo "  shell            run the hbase shell"
-  echo "  master           run a hbase HMaster node" 
-  echo "  regionserver     run a hbase HRegionServer node" 
+  echo "  shell            run the Hbase shell"
+  echo "  master           run an Hbase HMaster node" 
+  echo "  regionserver     run an Hbase HRegionServer node" 
+  echo "  rest             run an Hbase REST server" 
+  echo "  thrift           run an Hbase Thrift server" 
   echo " or"
   echo " or"
   echo "  CLASSNAME        run the class named CLASSNAME"
   echo "  CLASSNAME        run the class named CLASSNAME"
   echo "Most commands print help when invoked w/o parameters."
   echo "Most commands print help when invoked w/o parameters."
@@ -213,6 +215,10 @@ elif [ "$COMMAND" = "master" ] ; then
   CLASS='org.apache.hadoop.hbase.HMaster'
   CLASS='org.apache.hadoop.hbase.HMaster'
 elif [ "$COMMAND" = "regionserver" ] ; then
 elif [ "$COMMAND" = "regionserver" ] ; then
   CLASS='org.apache.hadoop.hbase.HRegionServer'
   CLASS='org.apache.hadoop.hbase.HRegionServer'
+elif [ "$COMMAND" = "rest" ] ; then
+  CLASS='org.apache.hadoop.hbase.rest.Dispatcher'
+elif [ "$COMMAND" = "thrift" ] ; then
+  CLASS='org.apache.hadoop.hbase.thrift.ThriftServer'
 else
 else
   CLASS=$COMMAND
   CLASS=$COMMAND
 fi
 fi

+ 238 - 0
src/contrib/hbase/src/examples/thrift/DemoClient.cpp

@@ -0,0 +1,238 @@
+#include <stdio.h>
+#include <unistd.h>
+#include <sys/time.h>
+#include <poll.h>
+
+#include <iostream>
+
+#include <protocol/TBinaryProtocol.h>
+#include <transport/TSocket.h>
+#include <transport/TTransportUtils.h>
+
+#include "Hbase.h"
+
+using namespace facebook::thrift;
+using namespace facebook::thrift::protocol;
+using namespace facebook::thrift::transport;
+
+using namespace apache::hadoop::hbase::thrift;
+
+typedef std::vector<std::string> StrVec;
+typedef std::map<std::string,std::string> StrMap;
+typedef std::vector<ColumnDescriptor> ColVec;
+typedef std::map<std::string,ColumnDescriptor> ColMap;
+
+
+static void
+printRow(const std::string &row, const StrMap &columns)
+{
+  std::cout << "row: " << row << ", cols: ";
+  for (StrMap::const_iterator it = columns.begin(); it != columns.end(); ++it) {
+    std::cout << it->first << " => " << it->second << "; ";
+  }
+  std::cout << std::endl;
+}
+
+static void 
+printEntry(const ScanEntry &entry)
+{
+  printRow(entry.row, entry.columns);
+}
+
+static void
+printVersions(const std::string &row, const StrVec &versions)
+{
+  std::cout << "row: " << row << ", values: ";
+  for (StrVec::const_iterator it = versions.begin(); it != versions.end(); ++it) {
+    std::cout << *it << "; ";
+  }
+  std::cout << std::endl;
+}
+
+int 
+main(int argc, char** argv) 
+{
+  boost::shared_ptr<TTransport> socket(new TSocket("localhost", 9090));
+  boost::shared_ptr<TTransport> transport(new TBufferedTransport(socket));
+  boost::shared_ptr<TProtocol> protocol(new TBinaryProtocol(transport));
+  HbaseClient client(protocol);
+
+  try {
+    transport->open();
+
+    std::string t("demo_table");
+
+    //
+    // Scan all tables, look for the demo table and delete it.
+    //
+    std::cout << "scanning tables..." << std::endl;
+    StrVec tables;
+    client.getTableNames(tables);
+    for (StrVec::const_iterator it = tables.begin(); it != tables.end(); ++it) {
+      std::cout << "  found: " << *it << std::endl;
+      if (t == *it) {
+        std::cout << "    deleting table: " << *it << std::endl;
+        client.deleteTable(*it);
+      }
+    }
+
+    //
+    // Create the demo table with two column families, entry: and unused:
+    //
+    ColVec columns;
+    columns.push_back(ColumnDescriptor());
+    columns.back().name = "entry:";
+    columns.back().maxVersions = 10;
+    columns.push_back(ColumnDescriptor());
+    columns.back().name = "unused:";
+
+    std::cout << "creating table: " << t << std::endl;
+    try {
+      client.createTable(t, columns);
+    } catch (AlreadyExists &ae) {
+      std::cout << "WARN: " << ae.message << std::endl;
+    }
+
+    ColMap columnMap;
+    client.getColumnDescriptors(columnMap, t);
+    std::cout << "column families in " << t << ": " << std::endl;
+    for (ColMap::const_iterator it = columnMap.begin(); it != columnMap.end(); ++it) {
+      std::cout << "  column: " << it->second.name << ", maxVer: " << it->second.maxVersions << std::endl;
+    }
+
+    //
+    // Test UTF-8 handling
+    //
+    std::string invalid("foo-\xfc\xa1\xa1\xa1\xa1\xa1");
+    std::string valid("foo-\xE7\x94\x9F\xE3\x83\x93\xE3\x83\xBC\xE3\x83\xAB");
+
+    // non-utf8 is fine for data
+    client.put(t, "foo", "entry:foo", invalid);
+
+    // try empty strings
+    client.put(t, "", "entry:", "");
+
+    // this row name is valid utf8
+    client.put(t, valid, "entry:foo", valid);
+
+    // non-utf8 is not allowed in row names
+    try {
+      client.put(t, invalid, "entry:foo", invalid);
+      std::cout << "FATAL: shouldn't get here!" << std::endl;
+      exit(-1);
+    } catch (IOError e) {
+      std::cout << "expected error: " << e.message << std::endl;
+    }
+
+    // Run a scanner on the rows we just created
+    StrVec columnNames;
+    columnNames.push_back("entry:");
+
+    std::cout << "Starting scanner..." << std::endl;
+    int scanner = client.scannerOpen(t, "", columnNames);
+    try {
+      while (true) {
+        ScanEntry value;
+        client.scannerGet(value, scanner);
+        printEntry(value);
+      }
+    } catch (NotFound &nf) {
+      client.scannerClose(scanner);
+      std::cout << "Scanner finished" << std::endl;
+    }
+
+    //
+    // Run some operations on a bunch of rows.
+    //
+    for (int i = 100; i >= 0; --i) {
+      // format row keys as "00000" to "00100"
+      char buf[32];
+      sprintf(buf, "%0.5d", i);
+      std::string row(buf);
+      
+      StrMap values;
+
+      client.put(t, row, "unused:", "DELETE_ME");
+      client.getRow(values, t, row);
+      printRow(row, values);
+      client.deleteAllRow(t, row);
+
+      client.put(t, row, "entry:num", "0");
+      client.put(t, row, "entry:foo", "FOO");
+      client.getRow(values, t, row);
+      printRow(row, values);
+
+      // sleep to force later timestamp 
+      poll(0, 0, 50);
+
+      std::vector<Mutation> mutations;
+      mutations.push_back(Mutation());
+      mutations.back().column = "entry:foo";
+      mutations.back().isDelete = true;
+      mutations.push_back(Mutation());
+      mutations.back().column = "entry:num";
+      mutations.back().value = "-1";
+      client.mutateRow(t, row, mutations);
+      client.getRow(values, t, row);
+      printRow(row, values);
+      
+      client.put(t, row, "entry:num", boost::lexical_cast<std::string>(i));
+      client.put(t, row, "entry:sqr", boost::lexical_cast<std::string>(i*i));
+      client.getRow(values, t, row);
+      printRow(row, values);
+
+      mutations.clear();
+      mutations.push_back(Mutation());
+      mutations.back().column = "entry:num";
+      mutations.back().value = "-999";
+      mutations.push_back(Mutation());
+      mutations.back().column = "entry:sqr";
+      mutations.back().isDelete = true;
+      client.mutateRowTs(t, row, mutations, 1); // shouldn't override latest
+      client.getRow(values, t, row);
+      printRow(row, values);
+
+      StrVec versions;
+      client.getVer(versions, t, row, "entry:num", 10);
+      printVersions(row, versions);
+      assert(versions.size() == 4);
+      std::cout << std::endl;
+
+      try {
+        std::string value;
+        client.get(value, t, row, "entry:foo");
+        std::cout << "FATAL: shouldn't get here!" << std::endl;
+        exit(-1);
+      } catch (NotFound &nf) {
+        // blank
+      }
+    }
+
+    // scan all rows/columns
+
+    columnNames.clear();
+    client.getColumnDescriptors(columnMap, t);
+    for (ColMap::const_iterator it = columnMap.begin(); it != columnMap.end(); ++it) {
+      columnNames.push_back(it->first);
+    }
+
+    std::cout << "Starting scanner..." << std::endl;
+    scanner = client.scannerOpenWithStop(t, "00020", "00040", columnNames);
+    try {
+      while (true) {
+        ScanEntry value;
+        client.scannerGet(value, scanner);
+        printEntry(value);
+      }
+    } catch (NotFound &nf) {
+      client.scannerClose(scanner);
+      std::cout << "Scanner finished" << std::endl;
+    }
+
+    transport->close();
+  } 
+  catch (TException &tx) {
+    printf("ERROR: %s\n", tx.what());
+  }
+
+}

+ 276 - 0
src/contrib/hbase/src/examples/thrift/DemoClient.java

@@ -0,0 +1,276 @@
+package org.apache.hadoop.hbase.thrift;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.text.NumberFormat;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.TreeMap;
+import java.util.SortedMap;
+
+import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
+import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
+import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.thrift.generated.IOError;
+import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
+import org.apache.hadoop.hbase.thrift.generated.Mutation;
+import org.apache.hadoop.hbase.thrift.generated.NotFound;
+import org.apache.hadoop.hbase.thrift.generated.ScanEntry;
+import com.facebook.thrift.TException;
+import com.facebook.thrift.protocol.TBinaryProtocol;
+import com.facebook.thrift.protocol.TProtocol;
+import com.facebook.thrift.transport.TSocket;
+import com.facebook.thrift.transport.TTransport;
+
+public class DemoClient {
+  
+  protected int port = 9090;
+  CharsetDecoder decoder = null;
+
+  public static void main(String[] args) 
+    throws IOError, TException, NotFound, UnsupportedEncodingException, IllegalArgument, AlreadyExists 
+  {
+    DemoClient client = new DemoClient();
+    client.run();
+  }
+
+  DemoClient() {
+    decoder = Charset.forName("UTF-8").newDecoder();
+  }
+  
+  // Helper to translate byte[]'s to UTF8 strings
+  private String utf8(byte[] buf) {
+    try {
+      return decoder.decode(ByteBuffer.wrap(buf)).toString();
+    } catch (CharacterCodingException e) {
+      return "[INVALID UTF-8]";
+    }
+  }
+  
+  // Helper to translate strings to UTF8 bytes
+  private byte[] bytes(String s) {
+    try {
+      return s.getBytes("UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      e.printStackTrace();
+      return null;
+    }
+  }
+  
+  private void run() throws IOError, TException, NotFound, IllegalArgument,
+      AlreadyExists {
+    TTransport transport = new TSocket("localhost", port);
+    TProtocol protocol = new TBinaryProtocol(transport, true, true);
+    Hbase.Client client = new Hbase.Client(protocol);
+
+    transport.open();
+
+    byte[] t = bytes("demo_table");
+    
+    //
+    // Scan all tables, look for the demo table and delete it.
+    //
+    System.out.println("scanning tables...");
+    for (byte[] name : client.getTableNames()) {
+      System.out.println("  found: " + utf8(name));
+      if (utf8(name).equals(utf8(t))) {
+        System.out.println("    deleting table: " + utf8(name));  
+        client.deleteTable(name);
+      }
+    }
+    
+    //
+    // Create the demo table with two column families, entry: and unused:
+    //
+    ArrayList<ColumnDescriptor> columns = new ArrayList<ColumnDescriptor>();
+    ColumnDescriptor col = null;
+    col = new ColumnDescriptor();
+    col.name = bytes("entry:");
+    col.maxVersions = 10;
+    columns.add(col);
+    col = new ColumnDescriptor();
+    col.name = bytes("unused:");
+    columns.add(col);
+
+    System.out.println("creating table: " + utf8(t));
+    try {
+      client.createTable(t, columns);
+    } catch (AlreadyExists ae) {
+      System.out.println("WARN: " + ae.message);
+    }
+    
+    System.out.println("column families in " + utf8(t) + ": ");
+    AbstractMap<byte[], ColumnDescriptor> columnMap = client.getColumnDescriptors(t);
+    for (ColumnDescriptor col2 : columnMap.values()) {
+      System.out.println("  column: " + utf8(col2.name) + ", maxVer: " + Integer.toString(col2.maxVersions));
+    }
+    
+    //
+    // Test UTF-8 handling
+    //
+    byte[] invalid = { (byte) 'f', (byte) 'o', (byte) 'o', (byte) '-', (byte) 0xfc, (byte) 0xa1, (byte) 0xa1, (byte) 0xa1, (byte) 0xa1 };
+    byte[] valid = { (byte) 'f', (byte) 'o', (byte) 'o', (byte) '-', (byte) 0xE7, (byte) 0x94, (byte) 0x9F, (byte) 0xE3, (byte) 0x83, (byte) 0x93, (byte) 0xE3, (byte) 0x83, (byte) 0xBC, (byte) 0xE3, (byte) 0x83, (byte) 0xAB};
+
+    // non-utf8 is fine for data
+    client.put(t, bytes("foo"), bytes("entry:foo"), invalid);
+
+    // try empty strings
+    client.put(t, bytes(""), bytes("entry:"), bytes(""));
+    
+    // this row name is valid utf8
+    client.put(t, valid, bytes("entry:foo"), valid);
+    
+    // non-utf8 is not allowed in row names
+    try {
+      client.put(t, invalid, bytes("entry:foo"), invalid);
+      System.out.println("FATAL: shouldn't get here");
+      System.exit(-1);
+    } catch (IOError e) {
+      System.out.println("expected error: " + e.message);
+    }
+    
+    // Run a scanner on the rows we just created
+    ArrayList<byte[]> columnNames = new ArrayList<byte[]>();
+    columnNames.add(bytes("entry:"));
+    
+    System.out.println("Starting scanner...");
+    int scanner = client
+        .scannerOpen(t, bytes(""), columnNames);
+    try {
+      while (true) {
+        ScanEntry value = client.scannerGet(scanner);
+        printEntry(value);
+      }
+    } catch (NotFound nf) {
+      client.scannerClose(scanner);
+      System.out.println("Scanner finished");
+    }
+    
+    //
+    // Run some operations on a bunch of rows
+    //
+    for (int i = 100; i >= 0; --i) {
+      // format row keys as "00000" to "00100"
+      NumberFormat nf = NumberFormat.getInstance();
+      nf.setMinimumIntegerDigits(5);
+      nf.setGroupingUsed(false);
+      byte[] row = bytes(nf.format(i));
+      
+      client.put(t, row, bytes("unused:"), bytes("DELETE_ME"));
+      printRow(row, client.getRow(t, row));
+      client.deleteAllRow(t, row);
+
+      client.put(t, row, bytes("entry:num"), bytes("0"));
+      client.put(t, row, bytes("entry:foo"), bytes("FOO"));
+      printRow(row, client.getRow(t, row));
+
+      Mutation m = null;      
+      ArrayList<Mutation> mutations = new ArrayList<Mutation>();
+      m = new Mutation();
+      m.column = bytes("entry:foo");
+      m.isDelete = true;
+      mutations.add(m);
+      m = new Mutation();
+      m.column = bytes("entry:num");
+      m.value = bytes("-1");
+      mutations.add(m);
+      client.mutateRow(t, row, mutations);
+      printRow(row, client.getRow(t, row));
+      
+      client.put(t, row, bytes("entry:num"), bytes(Integer.toString(i)));
+      client.put(t, row, bytes("entry:sqr"), bytes(Integer.toString(i * i)));
+      printRow(row, client.getRow(t, row));
+
+      // sleep to force later timestamp 
+      try {
+        Thread.sleep(50);
+      } catch (InterruptedException e) {
+        // no-op
+      }
+      
+      mutations.clear();
+      m = new Mutation();
+      m.column = bytes("entry:num");
+      m.value = bytes("-999");
+      mutations.add(m);
+      m = new Mutation();
+      m.column = bytes("entry:sqr");
+      m.isDelete = true;
+      client.mutateRowTs(t, row, mutations, 1); // shouldn't override latest
+      printRow(row, client.getRow(t, row));
+
+      ArrayList<byte[]> versions = client.getVer(t, row, bytes("entry:num"), 10);
+      printVersions(row, versions);
+      if (versions.size() != 4) {
+        System.out.println("FATAL: wrong # of versions");
+        System.exit(-1);
+      }
+      
+      try {
+        client.get(t, row, bytes("entry:foo"));
+        System.out.println("FATAL: shouldn't get here");
+        System.exit(-1);
+      } catch (NotFound nf2) {
+        // blank
+      }
+
+      System.out.println("");
+    }
+    
+    // scan all rows/columnNames
+    
+    columnNames.clear();
+    for (ColumnDescriptor col2 : client.getColumnDescriptors(t).values()) {
+      columnNames.add(col2.name);
+    }
+    
+    System.out.println("Starting scanner...");
+    scanner = client.scannerOpenWithStop(t, bytes("00020"), bytes("00040"),
+        columnNames);
+    try {
+      while (true) {
+        ScanEntry value = client.scannerGet(scanner);
+        printEntry(value);
+      }
+    } catch (NotFound nf) {
+      client.scannerClose(scanner);
+      System.out.println("Scanner finished");
+    }
+    
+    transport.close();
+  }
+  
+  private final void printVersions(byte[] row, ArrayList<byte[]> values) {
+    StringBuilder rowStr = new StringBuilder();
+    for (byte[] value : values) {
+      rowStr.append(utf8(value));
+      rowStr.append("; ");
+    }
+    System.out.println("row: " + utf8(row) + ", values: " + rowStr);
+  }
+  
+  private final void printEntry(ScanEntry entry) {
+    printRow(entry.row, entry.columns);
+  }
+  
+  private final void printRow(byte[] row, AbstractMap<byte[], byte[]> values) {
+    // copy values into a TreeMap to get them in sorted order
+    
+    TreeMap<String,byte[]> sorted = new TreeMap<String,byte[]>();
+    for (AbstractMap.Entry<byte[], byte[]> entry : values.entrySet()) {
+      sorted.put(utf8(entry.getKey()), entry.getValue());
+    }
+    
+    StringBuilder rowStr = new StringBuilder();
+    for (SortedMap.Entry<String, byte[]> entry : sorted.entrySet()) {
+      rowStr.append(entry.getKey());
+      rowStr.append(" => ");
+      rowStr.append(utf8(entry.getValue()));
+      rowStr.append("; ");
+    }
+    System.out.println("row: " + utf8(row) + ", cols: " + rowStr);
+  }
+}

+ 178 - 0
src/contrib/hbase/src/examples/thrift/DemoClient.rb

@@ -0,0 +1,178 @@
+#!/usr/bin/ruby
+
+$:.push('~/thrift/trunk/lib/rb/lib')
+$:.push('./gen-rb')
+
+require 'thrift/transport/tsocket'
+require 'thrift/protocol/tbinaryprotocol'
+
+require 'Hbase'
+
+def printRow(row, values)
+  print "row: #{row}, cols: "
+  values.sort.each do |k,v|
+    print "#{k} => #{v}; "
+  end
+  puts ""
+end
+
+def printEntry(entry)
+  printRow(entry.row, entry.columns)
+end
+
+transport = TBufferedTransport.new(TSocket.new("localhost", 9090))
+protocol = TBinaryProtocol.new(transport)
+client = Apache::Hadoop::Hbase::Thrift::Hbase::Client.new(protocol)
+
+transport.open()
+
+t = "demo_table"
+
+#
+# Scan all tables, look for the demo table and delete it.
+#
+puts "scanning tables..."
+client.getTableNames().sort.each do |name|
+  puts "  found: #{name}"
+  if (name == t) 
+    puts "    deleting table: #{name}" 
+    client.deleteTable(name)
+  end
+end
+
+#
+# Create the demo table with two column families, entry: and unused:
+#
+columns = []
+col = Apache::Hadoop::Hbase::Thrift::ColumnDescriptor.new
+col.name = "entry:"
+col.maxVersions = 10
+columns << col;
+col = Apache::Hadoop::Hbase::Thrift::ColumnDescriptor.new
+col.name = "unused:"
+columns << col;
+
+puts "creating table: #{t}"
+begin
+  client.createTable(t, columns)
+rescue Apache::Hadoop::Hbase::Thrift::AlreadyExists => ae
+  puts "WARN: #{ae.message}"
+end
+
+puts "column families in #{t}: "
+client.getColumnDescriptors(t).sort.each do |key, col|
+  puts "  column: #{col.name}, maxVer: #{col.maxVersions}"
+end
+
+#
+# Test UTF-8 handling
+#
+invalid = "foo-\xfc\xa1\xa1\xa1\xa1\xa1"
+valid = "foo-\xE7\x94\x9F\xE3\x83\x93\xE3\x83\xBC\xE3\x83\xAB";
+
+# non-utf8 is fine for data
+client.put(t, "foo", "entry:foo", invalid)
+
+# try empty strings
+client.put(t, "", "entry:", "");
+
+# this row name is valid utf8
+client.put(t, valid, "entry:foo", valid)
+
+# non-utf8 is not allowed in row names
+begin
+  client.put(t, invalid, "entry:foo", invalid)
+  raise "shouldn't get here!"
+rescue Apache::Hadoop::Hbase::Thrift::IOError => e
+  puts "expected error: #{e.message}"
+end
+
+# Run a scanner on the rows we just created
+puts "Starting scanner..."
+scanner = client.scannerOpen(t, "", ["entry:"])
+begin
+  while (true) 
+    printEntry(client.scannerGet(scanner))
+  end
+rescue Apache::Hadoop::Hbase::Thrift::NotFound => nf
+  client.scannerClose(scanner)
+  puts "Scanner finished"
+end
+
+#
+# Run some operations on a bunch of rows.
+#
+(0..100).to_a.reverse.each do |e|
+  # format row keys as "00000" to "00100"
+  row = format("%0.5d", e)
+
+  client.put(t, row, "unused:", "DELETE_ME");
+  printRow(row, client.getRow(t, row));
+  client.deleteAllRow(t, row)
+
+  client.put(t, row, "entry:num", "0")
+  client.put(t, row, "entry:foo", "FOO")
+  printRow(row, client.getRow(t, row));
+
+  mutations = []
+  m = Apache::Hadoop::Hbase::Thrift::Mutation.new
+  m.column = "entry:foo"
+  m.isDelete = 1
+  mutations << m
+  m = Apache::Hadoop::Hbase::Thrift::Mutation.new
+  m.column = "entry:num"
+  m.value = "-1"
+  mutations << m
+  client.mutateRow(t, row, mutations)
+  printRow(row, client.getRow(t, row));
+
+  client.put(t, row, "entry:num", e.to_s)
+  client.put(t, row, "entry:sqr", (e*e).to_s)
+  printRow(row, client.getRow(t, row));
+  
+  mutations = []
+  m = Apache::Hadoop::Hbase::Thrift::Mutation.new
+  m.column = "entry:num"
+  m.value = "-999"
+  mutations << m
+  m = Apache::Hadoop::Hbase::Thrift::Mutation.new
+  m.column = "entry:sqr"
+  m.isDelete = 1
+  mutations << m
+  client.mutateRowTs(t, row, mutations, 1) # shouldn't override latest
+  printRow(row, client.getRow(t, row));
+
+  versions = client.getVer(t, row, "entry:num", 10)
+  print "row: #{row}, values: "
+  versions.each do |v|
+    print "#{v}; "
+  end
+  puts ""    
+  
+  begin
+    client.get(t, row, "entry:foo")
+    raise "shouldn't get here!"
+  rescue Apache::Hadoop::Hbase::Thrift::NotFound => nf
+    # blank
+  end
+
+  puts ""
+end 
+
+columns = []
+client.getColumnDescriptors(t).each do |col, desc|
+  columns << col
+end
+
+puts "Starting scanner..."
+scanner = client.scannerOpenWithStop(t, "00020", "00040", columns)
+begin
+  while (true) 
+    printEntry(client.scannerGet(scanner))
+  end
+rescue Apache::Hadoop::Hbase::Thrift::NotFound => nf
+  client.scannerClose(scanner)
+  puts "Scanner finished"
+end
+  
+transport.close()

+ 18 - 0
src/contrib/hbase/src/examples/thrift/Makefile

@@ -0,0 +1,18 @@
+# Makefile for C++ Hbase Thrift DemoClient
+#
+# NOTE: run 'thrift -cpp Hbase.thrift' first
+
+THRIFT_DIR = /usr/local/include/thrift
+LIB_DIR = /usr/local/lib
+
+GEN_SRC = ./gen-cpp/Hbase.cpp \
+	  ./gen-cpp/Hbase_types.cpp \
+	  ./gen-cpp/Hbase_constants.cpp
+
+default: DemoClient
+
+DemoClient: DemoClient.cpp
+	g++ -o DemoClient -I${THRIFT_DIR}  -I./gen-cpp -L${LIB_DIR} -lthrift DemoClient.cpp ${GEN_SRC}
+
+clean:
+	rm -rf DemoClient

+ 15 - 0
src/contrib/hbase/src/examples/thrift/README.txt

@@ -0,0 +1,15 @@
+Hbase Thrift Client Examples
+============================
+
+Included in this directory are sample clients of the HBase ThriftServer.  They
+all perform the same actions but are implemented in C++, Java, and Ruby
+respectively.
+
+To run/compile this clients, you will first need to install the thrift package
+(from http://developers.facebook.com/thrift/) and then run thrift to generate
+the language files:
+
+thrift -cpp -java -rb \
+    ../../../src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift
+
+

+ 12 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java

@@ -152,6 +152,18 @@ public class BloomFilterDescriptor implements WritableComparable {
     return value.toString();
     return value.toString();
   }
   }
 
 
+  public BloomFilterType getType() {
+    return filterType;
+  }
+  
+  public int getVectorSize() {
+    return vectorSize;
+  }
+  
+  public int getNbHash() {
+    return nbHash;
+  }
+  
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object obj) {
   public boolean equals(Object obj) {

+ 437 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift

@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// ----------------------------------------------------------------
+// HBase.thrift -
+//
+// This is a Thrift interface definition file for the Hbase service.
+// Target language libraries for C++, Java, Ruby, PHP, (and more) are
+// generated by running this file through the Thrift compiler with the
+// appropriate flags.  The Thrift compiler binary and runtime
+// libraries for various languages is currently available from
+// Facebook (http://developers.facebook.com/thrift/).  The intent is
+// for the Thrift project to migrate to Apache Incubator.
+//
+// See the package.html file for information on the version of Thrift
+// used to generate the *.java files checked into the Hbase project.
+// ----------------------------------------------------------------
+
+java_package org.apache.hadoop.hbase.thrift.generated
+cpp_namespace apache.hadoop.hbase.thrift
+ruby_namespace Apache.Hadoop.Hbase.Thrift
+// note: other language namespaces tbd...
+
+//
+// Types
+//
+
+// NOTE: all variables with the Text type are assumed to be correctly
+// formatted UTF-8 strings.  This is a programming language and locale
+// dependent property that the client application is repsonsible for
+// maintaining.  If strings with an invalid encoding are sent, an
+// IOError will be thrown.
+
+typedef binary Text
+typedef binary Bytes
+typedef i32    ScannerID
+
+
+/**
+ * An HColumnDescriptor contains information about a column family
+ * such as the number of versions, compression settings, etc. It is
+ * used as input when creating a table or adding a column.
+ */
+struct ColumnDescriptor {
+  1:Text name,
+  2:i32 maxVersions = 3,
+  3:string compression = "NONE",
+  4:bool inMemory = 0,
+  5:i32 maxValueLength = 2147483647,
+  6:string bloomFilterType = "NONE",
+  7:i32 bloomFilterVectorSize = 0,
+  8:i32 bloomFilterNbHashes = 0
+}
+
+/**
+ * A RegionDescriptor contains informationa about an HTable region.
+ * Currently, this is just the startKey of the region.
+ */
+struct RegionDescriptor {
+  1:Text startKey,
+}
+
+/**
+ * A Mutation object is used to either update or delete a column-value.
+ */
+struct Mutation {
+  1:bool isDelete = 0,
+  2:Text column,
+  3:Text value
+}
+
+/**
+ * A ScanEntry contains the row, column, and value information for a scanner's
+ * current location.
+ */
+struct ScanEntry {
+  1:Text row,
+  2:map<Text, Bytes> columns
+}
+
+//
+// Exceptions
+//
+/**
+ * An IOError exception signals that an error occurred communicating
+ * to the Hbase master or an Hbase region server.  Also used to return
+ * more general Hbase error conditions.
+ */
+exception IOError {
+  1:string message
+}
+
+/**
+ * An IllegalArgument exception indicates an illegal or invalid
+ * argument was passed into a procedure.
+ */
+exception IllegalArgument {
+  1:string message
+}
+
+/**
+ * A NotFound exception is used to indicate that no value was found
+ * for a query, or that a scanner has reached it's end.
+ */
+exception NotFound {
+  1:string message
+}
+
+/**
+ * An AlreadyExists exceptions signals that a table with the specified
+ * name already exists
+ */
+exception AlreadyExists {
+  1:string message
+}
+
+//
+// Service 
+//
+
+service Hbase {
+
+  /**
+   * List all the userspace tables.
+   * @return - returns a list of names
+   */
+  list<Text> getTableNames()
+    throws (1:IOError io)
+
+  /**
+   * List all the column families assoicated with a table.
+   * @param tableName table name
+   * @return list of column family descriptors
+   */
+  map<Text,ColumnDescriptor> getColumnDescriptors (1:Text tableName)
+    throws (1:IOError io)
+
+  /**
+   * List the regions associated with a table.
+   * @param tableName table name
+   * @return list of region descriptors
+   */
+  list<RegionDescriptor> getTableRegions(1:Text tableName) 
+    throws (1:IOError io)
+
+  /**
+   * Create a table with the specified column families.  The name
+   * field for each ColumnDescriptor must be set and must end in a
+   * colon (:).  All other fields are optional and will get default
+   * values if not explicitly specified.
+   *
+   * @param tableName name of table to create
+   * @param columnFamilies list of column family descriptors
+   *
+   * @throws IllegalArgument if an input parameter is invalid
+   * @throws AlreadyExists if the table name already exists
+   */ 
+  void createTable(1:Text tableName, 2:list<ColumnDescriptor> columnFamilies)
+    throws (1:IOError io, 2:IllegalArgument ia, 3:AlreadyExists exist)
+
+  /**
+   * Deletes a table
+   * @param tableName name of table to delete
+   * @throws NotFound if table doesn't exist on server
+   */
+  void deleteTable(1:Text tableName)
+    throws (1:IOError io, 2:NotFound nf)
+
+  /** 
+   * Get a single value for the specified table, row, and column at the
+   * latest timestamp.
+   *
+   * @param tableName name of table
+   * @param row row key
+   * @param column column name
+   * @return value for specified row/column
+   */
+  Bytes get(1:Text tableName, 2:Text row, 3:Text column) 
+    throws (1:IOError io, 2:NotFound nf)
+
+  /** 
+   * Get the specified number of versions for the specified table,
+   * row, and column.
+   *
+   * @param tableName name of table
+   * @param row row key
+   * @param column column name
+   * @param numVersions number of versions to retrieve
+   * @return list of values for specified row/column
+   */
+  list<Bytes> getVer(1:Text tableName, 2:Text row, 3:Text column, 4:i32 numVersions) 
+    throws (1:IOError io, 2:NotFound nf)
+
+  /** 
+   * Get the specified number of versions for the specified table,
+   * row, and column.  Only versions less than or equal to the specified
+   * timestamp will be returned.
+   *
+   * @param tableName name of table
+   * @param row row key
+   * @param column column name
+   * @param timestamp timestamp
+   * @param numVersions number of versions to retrieve
+   * @return list of values for specified row/column
+   */
+  list<Bytes> getVerTs(1:Text tableName, 2:Text row, 3:Text column, 4:i64 timestamp,  5:i32 numVersions)
+    throws (1:IOError io, 2:NotFound nf)
+
+  /** 
+   * Get all the data for the specified table and row at the latest
+   * timestamp.
+   * 
+   * @param tableName name of table
+   * @param row row key
+   * @return Map of columns to values.  Map is empty if row does not exist.
+   */
+  map<Text, Bytes> getRow(1:Text tableName, 2:Text row)
+    throws (1:IOError io)
+
+  /** 
+   * Get all the data for the specified table and row at the specified
+   * timestamp.
+   * 
+   * @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.
+   */
+  map<Text, Bytes> getRowTs(1:Text tableName, 2:Text row, 3:i64 timestamp)
+    throws (1:IOError io)
+
+  /** 
+   * 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
+   */
+  void put(1:Text tableName, 2:Text row, 3:Text column, 4:Bytes value)
+    throws (1:IOError io)
+
+  /** 
+   * Apply a series of mutations (updates/deletes) to a row in a
+   * single transaction.  If an exception is thrown, then the
+   * transaction is aborted.  Default current timestamp is used, and
+   * all entries will have an identical timestamp.
+   *
+   * @param tableName name of table
+   * @param row row key
+   * @param mutations list of mutation commands
+   */
+  void mutateRow(1:Text tableName, 2:Text row, 3:list<Mutation> mutations)
+    throws (1:IOError io)
+
+  /** 
+   * Apply a series of mutations (updates/deletes) to a row in a
+   * single transaction.  If an exception is thrown, then the
+   * transaction is aborted.  The specified timestamp is used, and
+   * all entries will have an identical timestamp.
+   *
+   * @param tableName name of table
+   * @param row row key
+   * @param mutations list of mutation commands
+   * @param timestamp timestamp
+   */
+  void mutateRowTs(1:Text tableName, 2:Text row, 3:list<Mutation> mutations, 4:i64 timestamp)
+    throws (1:IOError io)
+
+  /** 
+   * Delete all cells that match the passed row and column.
+   *
+   * @param tableName name of table
+   * @param row Row to update
+   * @param column name of column whose value is to be deleted
+   */
+  void deleteAll(1:Text tableName, 2:Text row, 3:Text column)
+    throws (1:IOError io)
+
+  /** 
+   * Delete all cells that match the passed row and column and whose
+   * timestamp is equal-to or older than the passed timestamp.
+   *
+   * @param tableName name of table
+   * @param row Row to update
+   * @param column name of column whose value is to be deleted
+   * @param timestamp timestamp
+   */
+  void deleteAllTs(1:Text tableName, 2:Text row, 3:Text column, 4:i64 timestamp)
+    throws (1:IOError io)
+
+  /**
+   * Completely delete the row's cells.
+   *
+   * @param tableName name of table
+   * @param row key of the row to be completely deleted.
+   */
+  void deleteAllRow(1:Text tableName, 2:Text row)
+    throws (1:IOError io)
+
+  /**
+   * Completely delete the row's cells marked with a timestamp
+   * equal-to or older than the passed timestamp.
+   *
+   * @param tableName name of table
+   * @param row key of the row to be completely deleted.
+   * @param timestamp timestamp
+   */
+  void deleteAllRowTs(1:Text tableName, 2:Text row, 3:i64 timestamp)
+    throws (1:IOError io)
+
+  /** 
+   * Get a scanner on the current table starting at the specified row and
+   * ending at the last row in the table.  Return the specified columns.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier.
+   * @param tableName name of table
+   * @param startRow starting row in table to scan.  send "" (empty string) to
+   *                 start at the first row.
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpen(1:Text tableName, 
+                        2:Text startRow,
+                        3:list<Text> columns)
+    throws (1:IOError io)
+
+  /** 
+   * Get a scanner on the current table starting and stopping at the
+   * specified rows.  ending at the last row in the table.  Return the
+   * specified columns.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier.
+   * @param tableName name of table
+   * @param startRow starting row in table to scan.  send "" (empty string) to
+   *                 start at the first row.
+   * @param stopRow row to stop scanning on.  This row is *not* included in the
+   *                scanner's results
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenWithStop(1:Text tableName, 
+                                2:Text startRow,
+                                3:Text stopRow, 
+                                4:list<Text> columns)
+    throws (1:IOError io)
+
+  /** 
+   * Get a scanner on the current table starting at the specified row and
+   * ending at the last row in the table.  Return the specified columns.
+   * Only values with the specified timestamp are returned.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier.
+   * @param tableName name of table
+   * @param startRow starting row in table to scan.  send "" (empty string) to
+   *                 start at the first row.
+   * @param timestamp timestamp
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenTs(1:Text tableName, 
+                          2:Text startRow,
+                          3:list<Text> columns,
+                          4:i64 timestamp)
+    throws (1:IOError io)
+
+  /** 
+   * Get a scanner on the current table starting and stopping at the
+   * specified rows.  ending at the last row in the table.  Return the
+   * specified columns.  Only values with the specified timestamp are
+   * returned.
+   *
+   * @param columns columns to scan. If column name is a column family, all
+   * columns of the specified column family are returned.  Its also possible
+   * to pass a regex in the column qualifier.
+   * @param tableName name of table
+   * @param startRow starting row in table to scan.  send "" (empty string) to
+   *                 start at the first row.
+   * @param stopRow row to stop scanning on.  This row is *not* included
+   *                in the scanner's results
+   * @param timestamp timestamp
+   *
+   * @return scanner id to be used with other scanner procedures
+   */
+  ScannerID scannerOpenWithStopTs(1:Text tableName, 
+                                  2:Text startRow,
+                                  3:Text stopRow, 
+                                  4:list<Text> columns,
+                                  5:i64 timestamp)
+    throws (1:IOError io)
+
+  /**
+   * Returns the scanner's current row value and advances to the next
+   * row in the table.  When there are no more rows in the table, or a key
+   * greater-than-or-equal-to the scanner's specified stopRow is reached,
+   * a NotFound exception is returned.
+   *
+   * @param id id of a scanner returned by scannerOpen
+   * @return a ScanEntry object representing the current row's values
+   * @throws IllegalArgument if ScannerID is invalid
+   * @throws NotFound when the scanner reaches the end
+   */
+  ScanEntry scannerGet(1:ScannerID id)
+    throws (1:IOError io, 2:IllegalArgument ia, 3:NotFound nf)
+
+  /**
+   * Closes the server-state associated with an open scanner.
+   *
+   * @param id id of a scanner returned by scannerOpen
+   * @throws IllegalArgument if ScannerID is invalid
+   */
+  void scannerClose(1:ScannerID id)
+    throws (1:IOError io, 2:IllegalArgument ia)
+}

+ 624 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java

@@ -0,0 +1,624 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.thrift;
+
+import java.io.IOException;
+import java.nio.charset.MalformedInputException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.Map.Entry;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseAdmin;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HScannerInterface;
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.HTable;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.io.Text;
+
+import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
+import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
+import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.thrift.generated.IOError;
+import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
+import org.apache.hadoop.hbase.thrift.generated.Mutation;
+import org.apache.hadoop.hbase.thrift.generated.NotFound;
+import org.apache.hadoop.hbase.thrift.generated.RegionDescriptor;
+import org.apache.hadoop.hbase.thrift.generated.ScanEntry;
+
+import com.facebook.thrift.TException;
+import com.facebook.thrift.protocol.TBinaryProtocol;
+import com.facebook.thrift.protocol.TProtocolFactory;
+import com.facebook.thrift.server.TServer;
+import com.facebook.thrift.server.TThreadPoolServer;
+import com.facebook.thrift.transport.TServerSocket;
+import com.facebook.thrift.transport.TServerTransport;
+
+/**
+ * ThriftServer - this class starts up a Thrift server which implements the
+ * Hbase API specified in the Hbase.thrift IDL file.
+ */
+public class ThriftServer {
+  
+  /**
+   * The HBaseHandler is a glue object that connects Thrift RPC calls to the
+   * HBase client API primarily defined in the HBaseAdmin and HTable objects.
+   */
+  public static class HBaseHandler implements Hbase.Iface {
+    protected HBaseConfiguration conf = new HBaseConfiguration();
+    protected HBaseAdmin admin = null;
+    protected final Log LOG = LogFactory.getLog(this.getClass().getName());
+
+    // nextScannerId and scannerMap are used to manage scanner state
+    protected int nextScannerId = 0;
+    protected HashMap<Integer, HScannerInterface> scannerMap = null;
+    
+    /**
+     * Creates and returns an HTable instance from a given table name.
+     * 
+     * @param tableName
+     *          name of table
+     * @return HTable object
+     * @throws IOException
+     * @throws IOException
+     */
+    protected HTable getTable(final byte[] tableName) throws IOError,
+        IOException {
+      return new HTable(this.conf, getText(tableName));
+    }
+    
+    /**
+     * Assigns a unique ID to the scanner and adds the mapping to an internal
+     * hash-map.
+     * 
+     * @param scanner
+     * @return integer scanner id
+     */
+    protected synchronized int addScanner(HScannerInterface scanner) {
+      int id = nextScannerId++;
+      scannerMap.put(id, scanner);
+      return id;
+    }
+    
+    /**
+     * Returns the scanner associated with the specified ID.
+     * 
+     * @param id
+     * @return a HScannerInterface, or null if ID was invalid.
+     */
+    protected synchronized HScannerInterface getScanner(int id) {
+      return scannerMap.get(id);
+    }
+    
+    /**
+     * Removes the scanner associated with the specified ID from the internal
+     * id->scanner hash-map.
+     * 
+     * @param id
+     * @return a HScannerInterface, or null if ID was invalid.
+     */
+    protected synchronized HScannerInterface removeScanner(int id) {
+      return scannerMap.remove(id);
+    }
+    
+    /**
+     * Constructs an HBaseHandler object.
+     * 
+     * @throws MasterNotRunningException
+     */
+    HBaseHandler() throws MasterNotRunningException {
+      conf = new HBaseConfiguration();
+      admin = new HBaseAdmin(conf);
+      scannerMap = new HashMap<Integer, HScannerInterface>();
+    }
+    
+    /**
+     * Converts a byte array to a Text object after validating the UTF-8
+     * encoding.
+     * 
+     * @param buf
+     *          UTF-8 encoded bytes
+     * @return Text object
+     * @throws IllegalArgument
+     */
+    Text getText(byte[] buf) throws IOError {
+      try {
+        Text.validateUTF8(buf);
+      } catch (MalformedInputException e) {
+        throw new IOError("invalid UTF-8 encoding in row or column name");
+      }
+      return new Text(buf);
+    }
+    
+    //
+    // The Thrift Hbase.Iface interface is implemented below.
+    // Documentation for the methods and datastructures is the Hbase.thrift file
+    // used to generate the interface.
+    //
+    
+    public ArrayList<byte[]> getTableNames() throws IOError {
+      LOG.debug("getTableNames");
+      try {
+        HTableDescriptor[] tables = this.admin.listTables();
+        ArrayList<byte[]> list = new ArrayList<byte[]>(tables.length);
+        for (int i = 0; i < tables.length; i++) {
+          list.add(tables[i].getName().toString().getBytes());
+        }
+        return list;
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public ArrayList<RegionDescriptor> getTableRegions(byte[] tableName)
+        throws IOError {
+      LOG.debug("getTableRegions: " + new String(tableName));
+      try {
+        HTable table = getTable(tableName);
+        Text[] startKeys = table.getStartKeys();
+        ArrayList<RegionDescriptor> regions = new ArrayList<RegionDescriptor>();
+        for (int i = 0; i < startKeys.length; i++) {
+          RegionDescriptor region = new RegionDescriptor();
+          region.startKey = startKeys[i].toString().getBytes();
+          regions.add(region);
+        }
+        return regions;
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public byte[] get(byte[] tableName, byte[] row, byte[] column)
+        throws NotFound, IOError {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("get: table=" + new String(tableName) + ", row="
+            + new String(row) + ", col=" + new String(column));
+      }
+      try {
+        HTable table = getTable(tableName);
+        byte[] value = table.get(getText(row), getText(column));
+        if (value == null) {
+          throw new NotFound();
+        }
+        return value;
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public ArrayList<byte[]> getVer(byte[] tableName, byte[] row,
+        byte[] column, int numVersions) throws IOError, NotFound {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("getVer: table=" + new String(tableName) + ", row="
+            + new String(row) + ", col=" + new String(column) + ", numVers="
+            + numVersions);
+      }
+      try {
+        HTable table = getTable(tableName);
+        byte[][] values = table.get(getText(row), getText(column), numVersions);
+        if (values == null) {
+          throw new NotFound();
+        }
+        return new ArrayList<byte[]>(Arrays.asList(values));
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public ArrayList<byte[]> getVerTs(byte[] tableName, byte[] row,
+        byte[] column, long timestamp, int numVersions) throws IOError,
+        NotFound {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("getVerTs: table=" + new String(tableName) + ", row="
+            + new String(row) + ", col=" + new String(column) + ", ts="
+            + timestamp + ", numVers=" + numVersions);
+      }
+      try {
+        HTable table = getTable(tableName);
+        byte[][] values = table.get(getText(row), getText(column), timestamp,
+            numVersions);
+        if (values == null) {
+          throw new NotFound();
+        }
+        return new ArrayList<byte[]>(Arrays.asList(values));
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public AbstractMap<byte[], byte[]> getRow(byte[] tableName, byte[] row)
+        throws IOError {
+      return getRowTs(tableName, row, HConstants.LATEST_TIMESTAMP);
+    }
+    
+    public AbstractMap<byte[], byte[]> getRowTs(byte[] tableName, byte[] row,
+        long timestamp) throws IOError {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("getRowTs: table=" + new String(tableName) + ", row="
+            + new String(row) + ", ts=" + timestamp);
+      }
+      try {
+        HTable table = getTable(tableName);
+        SortedMap<Text, byte[]> values = table.getRow(getText(row), timestamp);
+        // copy the map from type <Text, byte[]> to <byte[], byte[]>
+        HashMap<byte[], byte[]> returnValues = new HashMap<byte[], byte[]>();
+        for (Entry<Text, byte[]> e : values.entrySet()) {
+          returnValues.put(e.getKey().getBytes(), e.getValue());
+        }
+        return returnValues;
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public void put(byte[] tableName, byte[] row, byte[] column, byte[] value)
+        throws IOError {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("put: table=" + new String(tableName) + ", row="
+            + new String(row) + ", col=" + new String(column)
+            + ", value.length=" + value.length);
+      }
+      try {
+        HTable table = getTable(tableName);
+        long lockid = table.startUpdate(getText(row));
+        table.put(lockid, getText(column), value);
+        table.commit(lockid);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public void deleteAll(byte[] tableName, byte[] row, byte[] column)
+        throws IOError {
+      deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP);
+    }
+    
+    public void deleteAllTs(byte[] tableName, byte[] row, byte[] column,
+        long timestamp) throws IOError {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("deleteAllTs: table=" + new String(tableName) + ", row="
+            + new String(row) + ", col=" + new String(column) + ", ts="
+            + timestamp);
+      }
+      try {
+        HTable table = getTable(tableName);
+        table.deleteAll(getText(row), getText(column), timestamp);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public void deleteAllRow(byte[] tableName, byte[] row) throws IOError {
+      deleteAllRowTs(tableName, row, HConstants.LATEST_TIMESTAMP);
+    }
+    
+    public void deleteAllRowTs(byte[] tableName, byte[] row, long timestamp)
+        throws IOError {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("deleteAllRowTs: table=" + new String(tableName) + ", row="
+            + new String(row) + ", ts=" + timestamp);
+      }
+      try {
+        HTable table = getTable(tableName);
+        table.deleteAll(getText(row), timestamp);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public void createTable(byte[] tableName,
+        ArrayList<ColumnDescriptor> columnFamilies) throws IOError,
+        IllegalArgument, AlreadyExists {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("createTable: table=" + new String(tableName));
+      }
+      try {
+        Text tableStr = getText(tableName);
+        if (admin.tableExists(tableStr)) {
+          throw new AlreadyExists("table name already in use");
+        }
+        HTableDescriptor desc = new HTableDescriptor(tableStr.toString());
+        for (ColumnDescriptor col : columnFamilies) {
+          HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col);
+          desc.addFamily(colDesc);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("createTable:     col=" + colDesc.getName());
+          }
+        }
+        admin.createTable(desc);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      } catch (IllegalArgumentException e) {
+        throw new IllegalArgument(e.getMessage());
+      }
+    }
+    
+    public void deleteTable(byte[] tableName) throws IOError, NotFound {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("deleteTable: table=" + new String(tableName));
+      }
+      try {
+        Text tableStr = getText(tableName);
+        if (!admin.tableExists(tableStr)) {
+          throw new NotFound();
+        }
+        admin.deleteTable(tableStr);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public void mutateRow(byte[] tableName, byte[] row,
+        ArrayList<Mutation> mutations) throws IOError {
+      mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP);
+    }
+    
+    public void mutateRowTs(byte[] tableName, byte[] row,
+        ArrayList<Mutation> mutations, long timestamp) throws IOError {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("mutateRowTs: table=" + new String(tableName) + ", row="
+            + new String(row) + ", ts=" + timestamp + " mutations="
+            + mutations.size());
+        for (Mutation m : mutations) {
+          if (m.isDelete) {
+            LOG.debug("mutateRowTs:    : delete - " + getText(m.column));
+          } else {
+            LOG.debug("mutateRowTs:    : put - " + getText(m.column) + " => "
+                + m.value);
+          }
+        }
+      }
+      
+      Long lockid = null;
+      HTable table = null;
+      
+      try {
+        table = getTable(tableName);
+        lockid = table.startUpdate(getText(row));
+        for (Mutation m : mutations) {
+          if (m.isDelete) {
+            table.delete(lockid, getText(m.column));
+          } else {
+            table.put(lockid, getText(m.column), m.value);
+          }
+        }
+        table.commit(lockid, timestamp);
+      } catch (IOException e) {
+        if (lockid != null) {
+          table.abort(lockid);
+        }
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public void scannerClose(int id) throws IOError, IllegalArgument {
+      LOG.debug("scannerClose: id=" + id);
+      HScannerInterface scanner = getScanner(id);
+      if (scanner == null) {
+        throw new IllegalArgument("scanner ID is invalid");
+      }
+      try {
+        scanner.close();
+        removeScanner(id);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public ScanEntry scannerGet(int id) throws IllegalArgument, NotFound,
+        IOError {
+      LOG.debug("scannerGet: id=" + id);
+      HScannerInterface scanner = getScanner(id);
+      if (scanner == null) {
+        throw new IllegalArgument("scanner ID is invalid");
+      }
+      
+      HStoreKey key = new HStoreKey();
+      TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
+      
+      try {
+        if (scanner.next(key, results) == false) {
+          throw new NotFound("end of scanner reached");
+        }
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+      
+      ScanEntry retval = new ScanEntry();
+      retval.row = key.getRow().getBytes();
+      retval.columns = new HashMap<byte[], byte[]>(results.size());
+      
+      for (SortedMap.Entry<Text, byte[]> e : results.entrySet()) {
+        retval.columns.put(e.getKey().getBytes(), e.getValue());
+      }
+      return retval;
+    }
+    
+    public int scannerOpen(byte[] tableName, byte[] startRow,
+        ArrayList<byte[]> columns) throws IOError {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("scannerOpen: table=" + getText(tableName) + ", start="
+            + getText(startRow) + ", columns=" + columns.toString());
+      }
+      try {
+        HTable table = getTable(tableName);
+        Text[] columnsText = new Text[columns.size()];
+        for (int i = 0; i < columns.size(); ++i) {
+          columnsText[i] = getText(columns.get(i));
+        }
+        HScannerInterface scanner = table.obtainScanner(columnsText,
+            getText(startRow));
+        return addScanner(scanner);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public int scannerOpenWithStop(byte[] tableName, byte[] startRow,
+        byte[] stopRow, ArrayList<byte[]> columns) throws IOError, TException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("scannerOpen: table=" + getText(tableName) + ", start="
+            + getText(startRow) + ", stop=" + getText(stopRow) + ", columns="
+            + columns.toString());
+      }
+      try {
+        HTable table = getTable(tableName);
+        Text[] columnsText = new Text[columns.size()];
+        for (int i = 0; i < columns.size(); ++i) {
+          columnsText[i] = getText(columns.get(i));
+        }
+        HScannerInterface scanner = table.obtainScanner(columnsText,
+            getText(startRow), getText(stopRow));
+        return addScanner(scanner);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public int scannerOpenTs(byte[] tableName, byte[] startRow,
+        ArrayList<byte[]> columns, long timestamp) throws IOError, TException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("scannerOpen: table=" + getText(tableName) + ", start="
+            + getText(startRow) + ", columns=" + columns.toString()
+            + ", timestamp=" + timestamp);
+      }
+      try {
+        HTable table = getTable(tableName);
+        Text[] columnsText = new Text[columns.size()];
+        for (int i = 0; i < columns.size(); ++i) {
+          columnsText[i] = getText(columns.get(i));
+        }
+        HScannerInterface scanner = table.obtainScanner(columnsText,
+            getText(startRow), timestamp);
+        return addScanner(scanner);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow,
+        byte[] stopRow, ArrayList<byte[]> columns, long timestamp)
+        throws IOError, TException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("scannerOpen: table=" + getText(tableName) + ", start="
+            + getText(startRow) + ", stop=" + getText(stopRow) + ", columns="
+            + columns.toString() + ", timestamp=" + timestamp);
+      }
+      try {
+        HTable table = getTable(tableName);
+        Text[] columnsText = new Text[columns.size()];
+        for (int i = 0; i < columns.size(); ++i) {
+          columnsText[i] = getText(columns.get(i));
+        }
+        HScannerInterface scanner = table.obtainScanner(columnsText,
+            getText(startRow), getText(stopRow), timestamp);
+        return addScanner(scanner);
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+    
+    public AbstractMap<byte[], ColumnDescriptor> getColumnDescriptors(
+        byte[] tableName) throws IOError, TException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("getColumnDescriptors: table=" + new String(tableName));
+      }
+      try {
+        HashMap<byte[], ColumnDescriptor> columns = new HashMap<byte[], ColumnDescriptor>();
+        
+        HTable table = getTable(tableName);
+        HTableDescriptor desc = table.getMetadata();
+        
+        for (Entry<Text, HColumnDescriptor> e : desc.families().entrySet()) {
+          ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e.getValue());
+          columns.put(col.name, col);
+        }
+        return columns;
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+  }
+  
+  public static void main(String[] args) {
+    Log LOG = LogFactory.getLog("ThriftServer");
+    
+    // Parse command-line
+    //
+    Options options = new Options();
+    options.addOption("h", "help", false, "print this message");
+    options.addOption("p", "port", true,
+        "server listening port (default: 9090)");
+    CommandLineParser parser = new GnuParser();
+    CommandLine line;
+    
+    try {
+      line = parser.parse(options, args);
+    } catch (ParseException e) {
+      System.out.println("ERROR: " + e.getMessage());
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp("ThriftServer [options]", options);
+      return;
+    }
+    
+    if (line.hasOption("h")) {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp("ThriftServer [options]", options);
+      return;
+    }
+    
+    int port = Integer.parseInt(line.getOptionValue("p", "9090"));
+    
+    // Launch Thrift Server
+    //
+    try {
+      LOG
+          .info("starting HBase Thrift server on port "
+              + Integer.toString(port));
+      HBaseHandler handler = new HBaseHandler();
+      Hbase.Processor processor = new Hbase.Processor(handler);
+      TServerTransport serverTransport = new TServerSocket(port);
+      TProtocolFactory protFactory = new TBinaryProtocol.Factory(true, true);
+      TServer server = new TThreadPoolServer(processor, serverTransport,
+          protFactory);
+      
+      LOG.info("Starting the server...");
+      server.serve();
+      
+    } catch (Exception x) {
+      x.printStackTrace();
+    }
+    LOG.info("done.");
+  }
+}

+ 89 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java

@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.thrift;
+
+import org.apache.hadoop.hbase.BloomFilterDescriptor;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
+import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
+import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
+import org.apache.hadoop.io.Text;
+
+public class ThriftUtilities {
+  
+  /**
+   * This utility method creates a new Hbase HColumnDescriptor object based on a
+   * Thrift ColumnDescriptor "struct".
+   * 
+   * @param in
+   *          Thrift ColumnDescriptor object
+   * @return HColumnDescriptor
+   * @throws IllegalArgument
+   */
+  static public HColumnDescriptor colDescFromThrift(ColumnDescriptor in)
+      throws IllegalArgument {
+    CompressionType comp = CompressionType.valueOf(in.compression);
+    BloomFilterDescriptor bloom = null;
+    if (in.bloomFilterType.compareTo("NONE") != 0) {
+      if (in.bloomFilterVectorSize > 0 && in.bloomFilterNbHashes > 0) {
+        bloom = new BloomFilterDescriptor(BloomFilterDescriptor.BloomFilterType
+            .valueOf(in.bloomFilterType), in.bloomFilterVectorSize,
+            in.bloomFilterNbHashes);
+      } else if (in.bloomFilterVectorSize > 0) {
+        bloom = new BloomFilterDescriptor(BloomFilterDescriptor.BloomFilterType
+            .valueOf(in.bloomFilterType), in.bloomFilterVectorSize);
+      } else {
+        throw new IllegalArgument(
+            "must specify number of entries for bloom filter");
+      }
+    }
+    
+    if (in.name == null || in.name.length <= 0) {
+      throw new IllegalArgument("column name is empty");
+    }
+    HColumnDescriptor col = new HColumnDescriptor(new Text(in.name),
+        in.maxVersions, comp, in.inMemory, in.maxValueLength, bloom);
+    return col;
+  }
+  
+  /**
+   * This utility method creates a new Thrift ColumnDescriptor "struct" based on
+   * an Hbase HColumnDescriptor object.
+   * 
+   * @param in
+   *          Hbase HColumnDescriptor object
+   * @return Thrift ColumnDescriptor
+   */
+  static public ColumnDescriptor colDescFromHbase(HColumnDescriptor in) {
+    ColumnDescriptor col = new ColumnDescriptor();
+    col.name = in.getName().getBytes();
+    col.maxVersions = in.getMaxVersions();
+    col.compression = in.getCompression().toString();
+    col.inMemory = in.isInMemory();
+    col.maxValueLength = in.getMaxValueLength();
+    BloomFilterDescriptor bloom = in.getBloomFilter();
+    if (bloom != null) {
+      col.bloomFilterType = bloom.getType().toString();
+      col.bloomFilterVectorSize = bloom.getVectorSize();
+      col.bloomFilterNbHashes = bloom.getNbHash();
+    }
+    return col;
+  }
+  
+}

+ 111 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java

@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+/**
+ * An AlreadyExists exceptions signals that a table with the specified
+ * name already exists
+ */
+public class AlreadyExists extends Exception implements TBase, java.io.Serializable {
+  public String message;
+
+  public final Isset __isset = new Isset();
+  public static final class Isset {
+    public boolean message = false;
+  }
+
+  public AlreadyExists() {
+  }
+
+  public AlreadyExists(
+    String message)
+  {
+    this();
+    this.message = message;
+    this.__isset.message = 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.message = iprot.readString();
+            this.__isset.message = 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("AlreadyExists");
+    oprot.writeStructBegin(struct);
+    TField field = new TField();
+    if (this.message != null) {
+      field.name = "message";
+      field.type = TType.STRING;
+      field.id = 1;
+      oprot.writeFieldBegin(field);
+      oprot.writeString(this.message);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("AlreadyExists(");
+    sb.append("message:");
+    sb.append(this.message);
+    sb.append(")");
+    return sb.toString();
+  }
+
+}
+

+ 277 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java

@@ -0,0 +1,277 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+/**
+ * An HColumnDescriptor contains information about a column family
+ * such as the number of versions, compression settings, etc. It is
+ * used as input when creating a table or adding a column.
+ */
+public class ColumnDescriptor implements TBase, java.io.Serializable {
+  public byte[] name;
+  public int maxVersions;
+  public String compression;
+  public boolean inMemory;
+  public int maxValueLength;
+  public String bloomFilterType;
+  public int bloomFilterVectorSize;
+  public int bloomFilterNbHashes;
+
+  public final Isset __isset = new Isset();
+  public static final class Isset {
+    public boolean name = false;
+    public boolean maxVersions = false;
+    public boolean compression = false;
+    public boolean inMemory = false;
+    public boolean maxValueLength = false;
+    public boolean bloomFilterType = false;
+    public boolean bloomFilterVectorSize = false;
+    public boolean bloomFilterNbHashes = false;
+  }
+
+  public ColumnDescriptor() {
+    this.maxVersions = 3;
+
+    this.compression = "NONE";
+
+    this.inMemory = false;
+
+    this.maxValueLength = 2147483647;
+
+    this.bloomFilterType = "NONE";
+
+    this.bloomFilterVectorSize = 0;
+
+    this.bloomFilterNbHashes = 0;
+
+  }
+
+  public ColumnDescriptor(
+    byte[] name,
+    int maxVersions,
+    String compression,
+    boolean inMemory,
+    int maxValueLength,
+    String bloomFilterType,
+    int bloomFilterVectorSize,
+    int bloomFilterNbHashes)
+  {
+    this();
+    this.name = name;
+    this.__isset.name = true;
+    this.maxVersions = maxVersions;
+    this.__isset.maxVersions = true;
+    this.compression = compression;
+    this.__isset.compression = true;
+    this.inMemory = inMemory;
+    this.__isset.inMemory = true;
+    this.maxValueLength = maxValueLength;
+    this.__isset.maxValueLength = true;
+    this.bloomFilterType = bloomFilterType;
+    this.__isset.bloomFilterType = true;
+    this.bloomFilterVectorSize = bloomFilterVectorSize;
+    this.__isset.bloomFilterVectorSize = true;
+    this.bloomFilterNbHashes = bloomFilterNbHashes;
+    this.__isset.bloomFilterNbHashes = 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.name = iprot.readBinary();
+            this.__isset.name = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2:
+          if (field.type == TType.I32) {
+            this.maxVersions = iprot.readI32();
+            this.__isset.maxVersions = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3:
+          if (field.type == TType.STRING) {
+            this.compression = iprot.readString();
+            this.__isset.compression = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4:
+          if (field.type == TType.BOOL) {
+            this.inMemory = iprot.readBool();
+            this.__isset.inMemory = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 5:
+          if (field.type == TType.I32) {
+            this.maxValueLength = iprot.readI32();
+            this.__isset.maxValueLength = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 6:
+          if (field.type == TType.STRING) {
+            this.bloomFilterType = iprot.readString();
+            this.__isset.bloomFilterType = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 7:
+          if (field.type == TType.I32) {
+            this.bloomFilterVectorSize = iprot.readI32();
+            this.__isset.bloomFilterVectorSize = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 8:
+          if (field.type == TType.I32) {
+            this.bloomFilterNbHashes = iprot.readI32();
+            this.__isset.bloomFilterNbHashes = 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("ColumnDescriptor");
+    oprot.writeStructBegin(struct);
+    TField field = new TField();
+    if (this.name != null) {
+      field.name = "name";
+      field.type = TType.STRING;
+      field.id = 1;
+      oprot.writeFieldBegin(field);
+      oprot.writeBinary(this.name);
+      oprot.writeFieldEnd();
+    }
+    field.name = "maxVersions";
+    field.type = TType.I32;
+    field.id = 2;
+    oprot.writeFieldBegin(field);
+    oprot.writeI32(this.maxVersions);
+    oprot.writeFieldEnd();
+    if (this.compression != null) {
+      field.name = "compression";
+      field.type = TType.STRING;
+      field.id = 3;
+      oprot.writeFieldBegin(field);
+      oprot.writeString(this.compression);
+      oprot.writeFieldEnd();
+    }
+    field.name = "inMemory";
+    field.type = TType.BOOL;
+    field.id = 4;
+    oprot.writeFieldBegin(field);
+    oprot.writeBool(this.inMemory);
+    oprot.writeFieldEnd();
+    field.name = "maxValueLength";
+    field.type = TType.I32;
+    field.id = 5;
+    oprot.writeFieldBegin(field);
+    oprot.writeI32(this.maxValueLength);
+    oprot.writeFieldEnd();
+    if (this.bloomFilterType != null) {
+      field.name = "bloomFilterType";
+      field.type = TType.STRING;
+      field.id = 6;
+      oprot.writeFieldBegin(field);
+      oprot.writeString(this.bloomFilterType);
+      oprot.writeFieldEnd();
+    }
+    field.name = "bloomFilterVectorSize";
+    field.type = TType.I32;
+    field.id = 7;
+    oprot.writeFieldBegin(field);
+    oprot.writeI32(this.bloomFilterVectorSize);
+    oprot.writeFieldEnd();
+    field.name = "bloomFilterNbHashes";
+    field.type = TType.I32;
+    field.id = 8;
+    oprot.writeFieldBegin(field);
+    oprot.writeI32(this.bloomFilterNbHashes);
+    oprot.writeFieldEnd();
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ColumnDescriptor(");
+    sb.append("name:");
+    sb.append(this.name);
+    sb.append(",maxVersions:");
+    sb.append(this.maxVersions);
+    sb.append(",compression:");
+    sb.append(this.compression);
+    sb.append(",inMemory:");
+    sb.append(this.inMemory);
+    sb.append(",maxValueLength:");
+    sb.append(this.maxValueLength);
+    sb.append(",bloomFilterType:");
+    sb.append(this.bloomFilterType);
+    sb.append(",bloomFilterVectorSize:");
+    sb.append(this.bloomFilterVectorSize);
+    sb.append(",bloomFilterNbHashes:");
+    sb.append(this.bloomFilterNbHashes);
+    sb.append(")");
+    return sb.toString();
+  }
+
+}
+

+ 34 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/Constants.java

@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+public class Constants {
+
+}

+ 6873 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java

@@ -0,0 +1,6873 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+public class Hbase {
+
+  public interface Iface {
+
+    /**
+     * List all the userspace tables.
+     * @return - returns a list of names
+     */
+    public ArrayList<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;
+
+    /**
+     * 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;
+
+    /**
+     * Create a table with the specified column families.  The name
+     * field for each ColumnDescriptor must be set and must end in a
+     * colon (:).  All other fields are optional and will get default
+     * values if not explicitly specified.
+     * 
+     * @param tableName name of table to create
+     * @param columnFamilies list of column family descriptors
+     * 
+     * @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;
+
+    /**
+     * Deletes a table
+     * @param tableName name of table to delete
+     * @throws NotFound if table doesn't exist on server
+     */
+    public void deleteTable(byte[] tableName) throws IOError, NotFound, TException;
+
+    /**
+     * Get a single value for the specified table, row, and column at the
+     * latest timestamp.
+     * 
+     * @param tableName name of table
+     * @param row row key
+     * @param column column name
+     * @return value for specified row/column
+     */
+    public byte[] get(byte[] tableName, byte[] row, byte[] column) throws IOError, NotFound, TException;
+
+    /**
+     * Get the specified number of versions for the specified table,
+     * row, and column.
+     * 
+     * @param tableName name of table
+     * @param row row key
+     * @param column column name
+     * @param numVersions number of versions to retrieve
+     * @return list of values for specified row/column
+     */
+    public ArrayList<byte[]> getVer(byte[] tableName, byte[] row, byte[] column, int numVersions) throws IOError, NotFound, TException;
+
+    /**
+     * Get the specified number of versions for the specified table,
+     * row, and column.  Only versions less than or equal to the specified
+     * timestamp will be returned.
+     * 
+     * @param tableName name of table
+     * @param row row key
+     * @param column column name
+     * @param timestamp timestamp
+     * @param numVersions number of versions to retrieve
+     * @return list of values for specified row/column
+     */
+    public ArrayList<byte[]> 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
+     * timestamp.
+     * 
+     * @param tableName name of table
+     * @param row row key
+     * @return Map of columns to values.  Map is empty if row does not exist.
+     */
+    public AbstractMap<byte[],byte[]> getRow(byte[] tableName, byte[] row) throws IOError, TException;
+
+    /**
+     * Get all the data for the specified table and row at the specified
+     * timestamp.
+     * 
+     * @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
+     */
+    public void put(byte[] tableName, byte[] row, byte[] column, byte[] value) throws IOError, TException;
+
+    /**
+     * Apply a series of mutations (updates/deletes) to a row in a
+     * single transaction.  If an exception is thrown, then the
+     * transaction is aborted.  Default current timestamp is used, and
+     * all entries will have an identical timestamp.
+     * 
+     * @param tableName name of table
+     * @param row row key
+     * @param mutations list of mutation commands
+     */
+    public void mutateRow(byte[] tableName, byte[] row, ArrayList<Mutation> mutations) throws IOError, TException;
+
+    /**
+     * Apply a series of mutations (updates/deletes) to a row in a
+     * single transaction.  If an exception is thrown, then the
+     * transaction is aborted.  The specified timestamp is used, and
+     * all entries will have an identical timestamp.
+     * 
+     * @param tableName name of table
+     * @param row row key
+     * @param mutations list of mutation commands
+     * @param timestamp timestamp
+     */
+    public void mutateRowTs(byte[] tableName, byte[] row, ArrayList<Mutation> mutations, long timestamp) throws IOError, TException;
+
+    /**
+     * Delete all cells that match the passed row and column.
+     * 
+     * @param tableName name of table
+     * @param row Row to update
+     * @param column name of column whose value is to be deleted
+     */
+    public void deleteAll(byte[] tableName, byte[] row, byte[] column) throws IOError, TException;
+
+    /**
+     * Delete all cells that match the passed row and column and whose
+     * timestamp is equal-to or older than the passed timestamp.
+     * 
+     * @param tableName name of table
+     * @param row Row to update
+     * @param column name of column whose value is to be deleted
+     * @param timestamp timestamp
+     */
+    public void deleteAllTs(byte[] tableName, byte[] row, byte[] column, long timestamp) throws IOError, TException;
+
+    /**
+     * Completely delete the row's cells.
+     * 
+     * @param tableName name of table
+     * @param row key of the row to be completely deleted.
+     */
+    public void deleteAllRow(byte[] tableName, byte[] row) throws IOError, TException;
+
+    /**
+     * Completely delete the row's cells marked with a timestamp
+     * equal-to or older than the passed timestamp.
+     * 
+     * @param tableName name of table
+     * @param row key of the row to be completely deleted.
+     * @param timestamp timestamp
+     */
+    public void deleteAllRowTs(byte[] tableName, byte[] row, long timestamp) throws IOError, TException;
+
+    /**
+     * Get a scanner on the current table starting at the specified row and
+     * ending at the last row in the table.  Return the specified columns.
+     * 
+     * @param columns columns to scan. If column name is a column family, all
+     * columns of the specified column family are returned.  Its also possible
+     * to pass a regex in the column qualifier.
+     * @param tableName name of table
+     * @param startRow starting row in table to scan.  send "" (empty string) to
+     *                 start at the first row.
+     * 
+     * @return scanner id to be used with other scanner procedures
+     */
+    public int scannerOpen(byte[] tableName, byte[] startRow, ArrayList<byte[]> columns) throws IOError, TException;
+
+    /**
+     * Get a scanner on the current table starting and stopping at the
+     * specified rows.  ending at the last row in the table.  Return the
+     * specified columns.
+     * 
+     * @param columns columns to scan. If column name is a column family, all
+     * columns of the specified column family are returned.  Its also possible
+     * to pass a regex in the column qualifier.
+     * @param tableName name of table
+     * @param startRow starting row in table to scan.  send "" (empty string) to
+     *                 start at the first row.
+     * @param stopRow row to stop scanning on.  This row is *not* included in the
+     *                scanner's results
+     * 
+     * @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;
+
+    /**
+     * Get a scanner on the current table starting at the specified row and
+     * ending at the last row in the table.  Return the specified columns.
+     * Only values with the specified timestamp are returned.
+     * 
+     * @param columns columns to scan. If column name is a column family, all
+     * columns of the specified column family are returned.  Its also possible
+     * to pass a regex in the column qualifier.
+     * @param tableName name of table
+     * @param startRow starting row in table to scan.  send "" (empty string) to
+     *                 start at the first row.
+     * @param timestamp timestamp
+     * 
+     * @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;
+
+    /**
+     * Get a scanner on the current table starting and stopping at the
+     * specified rows.  ending at the last row in the table.  Return the
+     * specified columns.  Only values with the specified timestamp are
+     * returned.
+     * 
+     * @param columns columns to scan. If column name is a column family, all
+     * columns of the specified column family are returned.  Its also possible
+     * to pass a regex in the column qualifier.
+     * @param tableName name of table
+     * @param startRow starting row in table to scan.  send "" (empty string) to
+     *                 start at the first row.
+     * @param stopRow row to stop scanning on.  This row is *not* included
+     *                in the scanner's results
+     * @param timestamp timestamp
+     * 
+     * @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;
+
+    /**
+     * Returns the scanner's current row value and advances to the next
+     * row in the table.  When there are no more rows in the table, or a key
+     * greater-than-or-equal-to the scanner's specified stopRow is reached,
+     * a NotFound exception is returned.
+     * 
+     * @param id id of a scanner returned by scannerOpen
+     * @return a ScanEntry object representing the current row's values
+     * @throws IllegalArgument if ScannerID is invalid
+     * @throws NotFound when the scanner reaches the end
+     */
+    public ScanEntry scannerGet(int id) throws IOError, IllegalArgument, NotFound, TException;
+
+    /**
+     * Closes the server-state associated with an open scanner.
+     * 
+     * @param id id of a scanner returned by scannerOpen
+     * @throws IllegalArgument if ScannerID is invalid
+     */
+    public void scannerClose(int id) throws IOError, IllegalArgument, TException;
+
+  }
+
+  public static class Client implements Iface {
+    public Client(TProtocol prot)
+    {
+      this(prot, prot);
+    }
+
+    public Client(TProtocol iprot, TProtocol oprot)
+    {
+      iprot_ = iprot;
+      oprot_ = oprot;
+    }
+
+    protected TProtocol iprot_;
+    protected TProtocol oprot_;
+
+    protected int seqid_;
+
+    public ArrayList<byte[]> getTableNames() throws IOError, TException
+    {
+      send_getTableNames();
+      return recv_getTableNames();
+    }
+
+    public void send_getTableNames() throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getTableNames", TMessageType.CALL, seqid_));
+      getTableNames_args args = new getTableNames_args();
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public ArrayList<byte[]> recv_getTableNames() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      getTableNames_result result = new getTableNames_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, "getTableNames failed: unknown result");
+    }
+
+    public AbstractMap<byte[],ColumnDescriptor> getColumnDescriptors(byte[] tableName) throws IOError, TException
+    {
+      send_getColumnDescriptors(tableName);
+      return recv_getColumnDescriptors();
+    }
+
+    public void send_getColumnDescriptors(byte[] tableName) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getColumnDescriptors", TMessageType.CALL, seqid_));
+      getColumnDescriptors_args args = new getColumnDescriptors_args();
+      args.tableName = tableName;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public AbstractMap<byte[],ColumnDescriptor> recv_getColumnDescriptors() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      getColumnDescriptors_result result = new getColumnDescriptors_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, "getColumnDescriptors failed: unknown result");
+    }
+
+    public ArrayList<RegionDescriptor> getTableRegions(byte[] tableName) throws IOError, TException
+    {
+      send_getTableRegions(tableName);
+      return recv_getTableRegions();
+    }
+
+    public void send_getTableRegions(byte[] tableName) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getTableRegions", TMessageType.CALL, seqid_));
+      getTableRegions_args args = new getTableRegions_args();
+      args.tableName = tableName;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public ArrayList<RegionDescriptor> recv_getTableRegions() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      getTableRegions_result result = new getTableRegions_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, "getTableRegions failed: unknown result");
+    }
+
+    public void createTable(byte[] tableName, ArrayList<ColumnDescriptor> columnFamilies) throws IOError, IllegalArgument, AlreadyExists, TException
+    {
+      send_createTable(tableName, columnFamilies);
+      recv_createTable();
+    }
+
+    public void send_createTable(byte[] tableName, ArrayList<ColumnDescriptor> columnFamilies) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("createTable", TMessageType.CALL, seqid_));
+      createTable_args args = new createTable_args();
+      args.tableName = tableName;
+      args.columnFamilies = columnFamilies;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_createTable() throws IOError, IllegalArgument, AlreadyExists, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      createTable_result result = new createTable_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      if (result.__isset.ia) {
+        throw result.ia;
+      }
+      if (result.__isset.exist) {
+        throw result.exist;
+      }
+      return;
+    }
+
+    public void deleteTable(byte[] tableName) throws IOError, NotFound, TException
+    {
+      send_deleteTable(tableName);
+      recv_deleteTable();
+    }
+
+    public void send_deleteTable(byte[] tableName) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("deleteTable", TMessageType.CALL, seqid_));
+      deleteTable_args args = new deleteTable_args();
+      args.tableName = tableName;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_deleteTable() throws IOError, NotFound, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      deleteTable_result result = new deleteTable_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      if (result.__isset.nf) {
+        throw result.nf;
+      }
+      return;
+    }
+
+    public byte[] get(byte[] tableName, byte[] row, byte[] column) throws IOError, NotFound, TException
+    {
+      send_get(tableName, row, column);
+      return recv_get();
+    }
+
+    public void send_get(byte[] tableName, byte[] row, byte[] column) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("get", TMessageType.CALL, seqid_));
+      get_args args = new get_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.column = column;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public byte[] recv_get() throws IOError, NotFound, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      get_result result = new get_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.success) {
+        return result.success;
+      }
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      if (result.__isset.nf) {
+        throw result.nf;
+      }
+      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
+    {
+      send_getVer(tableName, row, column, numVersions);
+      return recv_getVer();
+    }
+
+    public void send_getVer(byte[] tableName, byte[] row, byte[] column, int numVersions) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getVer", TMessageType.CALL, seqid_));
+      getVer_args args = new getVer_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.column = column;
+      args.numVersions = numVersions;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public ArrayList<byte[]> recv_getVer() throws IOError, NotFound, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      getVer_result result = new getVer_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.success) {
+        return result.success;
+      }
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      if (result.__isset.nf) {
+        throw result.nf;
+      }
+      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
+    {
+      send_getVerTs(tableName, row, column, timestamp, numVersions);
+      return recv_getVerTs();
+    }
+
+    public void send_getVerTs(byte[] tableName, byte[] row, byte[] column, long timestamp, int numVersions) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getVerTs", TMessageType.CALL, seqid_));
+      getVerTs_args args = new getVerTs_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.column = column;
+      args.timestamp = timestamp;
+      args.numVersions = numVersions;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public ArrayList<byte[]> recv_getVerTs() throws IOError, NotFound, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      getVerTs_result result = new getVerTs_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.success) {
+        return result.success;
+      }
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      if (result.__isset.nf) {
+        throw result.nf;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "getVerTs failed: unknown result");
+    }
+
+    public AbstractMap<byte[],byte[]> getRow(byte[] tableName, byte[] row) throws IOError, TException
+    {
+      send_getRow(tableName, row);
+      return recv_getRow();
+    }
+
+    public void send_getRow(byte[] tableName, byte[] row) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getRow", TMessageType.CALL, seqid_));
+      getRow_args args = new getRow_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public AbstractMap<byte[],byte[]> recv_getRow() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      getRow_result result = new getRow_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, "getRow failed: unknown result");
+    }
+
+    public AbstractMap<byte[],byte[]> getRowTs(byte[] tableName, byte[] row, long timestamp) throws IOError, TException
+    {
+      send_getRowTs(tableName, row, timestamp);
+      return recv_getRowTs();
+    }
+
+    public void send_getRowTs(byte[] tableName, byte[] row, long timestamp) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getRowTs", TMessageType.CALL, seqid_));
+      getRowTs_args args = new getRowTs_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.timestamp = timestamp;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public AbstractMap<byte[],byte[]> recv_getRowTs() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      getRowTs_result result = new getRowTs_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, "getRowTs failed: unknown result");
+    }
+
+    public void put(byte[] tableName, byte[] row, byte[] column, byte[] value) throws IOError, 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, 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;
+      }
+      return;
+    }
+
+    public void mutateRow(byte[] tableName, byte[] row, ArrayList<Mutation> mutations) throws IOError, TException
+    {
+      send_mutateRow(tableName, row, mutations);
+      recv_mutateRow();
+    }
+
+    public void send_mutateRow(byte[] tableName, byte[] row, ArrayList<Mutation> mutations) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("mutateRow", TMessageType.CALL, seqid_));
+      mutateRow_args args = new mutateRow_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.mutations = mutations;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_mutateRow() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      mutateRow_result result = new mutateRow_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void mutateRowTs(byte[] tableName, byte[] row, ArrayList<Mutation> mutations, long timestamp) throws IOError, TException
+    {
+      send_mutateRowTs(tableName, row, mutations, timestamp);
+      recv_mutateRowTs();
+    }
+
+    public void send_mutateRowTs(byte[] tableName, byte[] row, ArrayList<Mutation> mutations, long timestamp) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("mutateRowTs", TMessageType.CALL, seqid_));
+      mutateRowTs_args args = new mutateRowTs_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.mutations = mutations;
+      args.timestamp = timestamp;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_mutateRowTs() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      mutateRowTs_result result = new mutateRowTs_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void deleteAll(byte[] tableName, byte[] row, byte[] column) throws IOError, TException
+    {
+      send_deleteAll(tableName, row, column);
+      recv_deleteAll();
+    }
+
+    public void send_deleteAll(byte[] tableName, byte[] row, byte[] column) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("deleteAll", TMessageType.CALL, seqid_));
+      deleteAll_args args = new deleteAll_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.column = column;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_deleteAll() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      deleteAll_result result = new deleteAll_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void deleteAllTs(byte[] tableName, byte[] row, byte[] column, long timestamp) throws IOError, TException
+    {
+      send_deleteAllTs(tableName, row, column, timestamp);
+      recv_deleteAllTs();
+    }
+
+    public void send_deleteAllTs(byte[] tableName, byte[] row, byte[] column, long timestamp) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("deleteAllTs", TMessageType.CALL, seqid_));
+      deleteAllTs_args args = new deleteAllTs_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.column = column;
+      args.timestamp = timestamp;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_deleteAllTs() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      deleteAllTs_result result = new deleteAllTs_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void deleteAllRow(byte[] tableName, byte[] row) throws IOError, TException
+    {
+      send_deleteAllRow(tableName, row);
+      recv_deleteAllRow();
+    }
+
+    public void send_deleteAllRow(byte[] tableName, byte[] row) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("deleteAllRow", TMessageType.CALL, seqid_));
+      deleteAllRow_args args = new deleteAllRow_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_deleteAllRow() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      deleteAllRow_result result = new deleteAllRow_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public void deleteAllRowTs(byte[] tableName, byte[] row, long timestamp) throws IOError, TException
+    {
+      send_deleteAllRowTs(tableName, row, timestamp);
+      recv_deleteAllRowTs();
+    }
+
+    public void send_deleteAllRowTs(byte[] tableName, byte[] row, long timestamp) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("deleteAllRowTs", TMessageType.CALL, seqid_));
+      deleteAllRowTs_args args = new deleteAllRowTs_args();
+      args.tableName = tableName;
+      args.row = row;
+      args.timestamp = timestamp;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_deleteAllRowTs() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      deleteAllRowTs_result result = new deleteAllRowTs_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      return;
+    }
+
+    public int scannerOpen(byte[] tableName, byte[] startRow, ArrayList<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
+    {
+      oprot_.writeMessageBegin(new TMessage("scannerOpen", TMessageType.CALL, seqid_));
+      scannerOpen_args args = new scannerOpen_args();
+      args.tableName = tableName;
+      args.startRow = startRow;
+      args.columns = columns;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public int recv_scannerOpen() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      scannerOpen_result result = new scannerOpen_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, "scannerOpen failed: unknown result");
+    }
+
+    public int scannerOpenWithStop(byte[] tableName, byte[] startRow, byte[] stopRow, ArrayList<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
+    {
+      oprot_.writeMessageBegin(new TMessage("scannerOpenWithStop", TMessageType.CALL, seqid_));
+      scannerOpenWithStop_args args = new scannerOpenWithStop_args();
+      args.tableName = tableName;
+      args.startRow = startRow;
+      args.stopRow = stopRow;
+      args.columns = columns;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public int recv_scannerOpenWithStop() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      scannerOpenWithStop_result result = new scannerOpenWithStop_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, "scannerOpenWithStop failed: unknown result");
+    }
+
+    public int scannerOpenTs(byte[] tableName, byte[] startRow, ArrayList<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
+    {
+      oprot_.writeMessageBegin(new TMessage("scannerOpenTs", TMessageType.CALL, seqid_));
+      scannerOpenTs_args args = new scannerOpenTs_args();
+      args.tableName = tableName;
+      args.startRow = startRow;
+      args.columns = columns;
+      args.timestamp = timestamp;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public int recv_scannerOpenTs() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      scannerOpenTs_result result = new scannerOpenTs_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, "scannerOpenTs failed: unknown result");
+    }
+
+    public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow, byte[] stopRow, ArrayList<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
+    {
+      oprot_.writeMessageBegin(new TMessage("scannerOpenWithStopTs", TMessageType.CALL, seqid_));
+      scannerOpenWithStopTs_args args = new scannerOpenWithStopTs_args();
+      args.tableName = tableName;
+      args.startRow = startRow;
+      args.stopRow = stopRow;
+      args.columns = columns;
+      args.timestamp = timestamp;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public int recv_scannerOpenWithStopTs() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      scannerOpenWithStopTs_result result = new scannerOpenWithStopTs_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, "scannerOpenWithStopTs failed: unknown result");
+    }
+
+    public ScanEntry scannerGet(int id) throws IOError, IllegalArgument, NotFound, TException
+    {
+      send_scannerGet(id);
+      return recv_scannerGet();
+    }
+
+    public void send_scannerGet(int id) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("scannerGet", TMessageType.CALL, seqid_));
+      scannerGet_args args = new scannerGet_args();
+      args.id = id;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public ScanEntry recv_scannerGet() throws IOError, IllegalArgument, NotFound, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      scannerGet_result result = new scannerGet_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.success) {
+        return result.success;
+      }
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      if (result.__isset.ia) {
+        throw result.ia;
+      }
+      if (result.__isset.nf) {
+        throw result.nf;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "scannerGet failed: unknown result");
+    }
+
+    public void scannerClose(int id) throws IOError, IllegalArgument, TException
+    {
+      send_scannerClose(id);
+      recv_scannerClose();
+    }
+
+    public void send_scannerClose(int id) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("scannerClose", TMessageType.CALL, seqid_));
+      scannerClose_args args = new scannerClose_args();
+      args.id = id;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_scannerClose() throws IOError, IllegalArgument, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      scannerClose_result result = new scannerClose_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.__isset.io) {
+        throw result.io;
+      }
+      if (result.__isset.ia) {
+        throw result.ia;
+      }
+      return;
+    }
+
+  }
+  public static class Processor implements TProcessor {
+    public Processor(Iface iface)
+    {
+      iface_ = iface;
+      processMap_.put("getTableNames", new getTableNames());
+      processMap_.put("getColumnDescriptors", new getColumnDescriptors());
+      processMap_.put("getTableRegions", new getTableRegions());
+      processMap_.put("createTable", new createTable());
+      processMap_.put("deleteTable", new deleteTable());
+      processMap_.put("get", new get());
+      processMap_.put("getVer", new getVer());
+      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("deleteAll", new deleteAll());
+      processMap_.put("deleteAllTs", new deleteAllTs());
+      processMap_.put("deleteAllRow", new deleteAllRow());
+      processMap_.put("deleteAllRowTs", new deleteAllRowTs());
+      processMap_.put("scannerOpen", new scannerOpen());
+      processMap_.put("scannerOpenWithStop", new scannerOpenWithStop());
+      processMap_.put("scannerOpenTs", new scannerOpenTs());
+      processMap_.put("scannerOpenWithStopTs", new scannerOpenWithStopTs());
+      processMap_.put("scannerGet", new scannerGet());
+      processMap_.put("scannerClose", new scannerClose());
+    }
+
+    protected static interface ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException;
+    }
+
+    private Iface iface_;
+    protected final HashMap<String,ProcessFunction> processMap_ = new HashMap<String,ProcessFunction>();
+
+    public boolean process(TProtocol iprot, TProtocol oprot) throws TException
+    {
+      TMessage msg = iprot.readMessageBegin();
+      ProcessFunction fn = processMap_.get(msg.name);
+      if (fn == null) {
+        TProtocolUtil.skip(iprot, TType.STRUCT);
+        iprot.readMessageEnd();
+        TApplicationException x = new TApplicationException(TApplicationException.UNKNOWN_METHOD, "Invalid method name: '"+msg.name+"'");
+        oprot.writeMessageBegin(new TMessage(msg.name, TMessageType.EXCEPTION, msg.seqid));
+        x.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+        return true;
+      }
+      fn.process(msg.seqid, iprot, oprot);
+      return true;
+    }
+
+    private class getTableNames implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getTableNames_args args = new getTableNames_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        getTableNames_result result = new getTableNames_result();
+        try {
+          result.success = iface_.getTableNames();
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("getTableNames", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getColumnDescriptors implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getColumnDescriptors_args args = new getColumnDescriptors_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        getColumnDescriptors_result result = new getColumnDescriptors_result();
+        try {
+          result.success = iface_.getColumnDescriptors(args.tableName);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("getColumnDescriptors", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getTableRegions implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getTableRegions_args args = new getTableRegions_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        getTableRegions_result result = new getTableRegions_result();
+        try {
+          result.success = iface_.getTableRegions(args.tableName);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("getTableRegions", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class createTable implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        createTable_args args = new createTable_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        createTable_result result = new createTable_result();
+        try {
+          iface_.createTable(args.tableName, args.columnFamilies);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        } catch (IllegalArgument ia) {
+          result.ia = ia;
+          result.__isset.ia = true;
+        } catch (AlreadyExists exist) {
+          result.exist = exist;
+          result.__isset.exist = true;
+        }
+        oprot.writeMessageBegin(new TMessage("createTable", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class deleteTable implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        deleteTable_args args = new deleteTable_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        deleteTable_result result = new deleteTable_result();
+        try {
+          iface_.deleteTable(args.tableName);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        } catch (NotFound nf) {
+          result.nf = nf;
+          result.__isset.nf = true;
+        }
+        oprot.writeMessageBegin(new TMessage("deleteTable", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class get implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        get_args args = new get_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        get_result result = new get_result();
+        try {
+          result.success = iface_.get(args.tableName, args.row, args.column);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        } catch (NotFound nf) {
+          result.nf = nf;
+          result.__isset.nf = true;
+        }
+        oprot.writeMessageBegin(new TMessage("get", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getVer implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getVer_args args = new getVer_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        getVer_result result = new getVer_result();
+        try {
+          result.success = iface_.getVer(args.tableName, args.row, args.column, args.numVersions);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        } catch (NotFound nf) {
+          result.nf = nf;
+          result.__isset.nf = true;
+        }
+        oprot.writeMessageBegin(new TMessage("getVer", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getVerTs implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getVerTs_args args = new getVerTs_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        getVerTs_result result = new getVerTs_result();
+        try {
+          result.success = iface_.getVerTs(args.tableName, args.row, args.column, args.timestamp, args.numVersions);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        } catch (NotFound nf) {
+          result.nf = nf;
+          result.__isset.nf = true;
+        }
+        oprot.writeMessageBegin(new TMessage("getVerTs", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getRow implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getRow_args args = new getRow_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        getRow_result result = new getRow_result();
+        try {
+          result.success = iface_.getRow(args.tableName, args.row);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("getRow", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getRowTs implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getRowTs_args args = new getRowTs_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        getRowTs_result result = new getRowTs_result();
+        try {
+          result.success = iface_.getRowTs(args.tableName, args.row, args.timestamp);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("getRowTs", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    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;
+        }
+        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
+      {
+        mutateRow_args args = new mutateRow_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        mutateRow_result result = new mutateRow_result();
+        try {
+          iface_.mutateRow(args.tableName, args.row, args.mutations);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("mutateRow", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class mutateRowTs implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        mutateRowTs_args args = new mutateRowTs_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        mutateRowTs_result result = new mutateRowTs_result();
+        try {
+          iface_.mutateRowTs(args.tableName, args.row, args.mutations, args.timestamp);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("mutateRowTs", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class deleteAll implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        deleteAll_args args = new deleteAll_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        deleteAll_result result = new deleteAll_result();
+        try {
+          iface_.deleteAll(args.tableName, args.row, args.column);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("deleteAll", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class deleteAllTs implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        deleteAllTs_args args = new deleteAllTs_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        deleteAllTs_result result = new deleteAllTs_result();
+        try {
+          iface_.deleteAllTs(args.tableName, args.row, args.column, args.timestamp);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("deleteAllTs", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class deleteAllRow implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        deleteAllRow_args args = new deleteAllRow_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        deleteAllRow_result result = new deleteAllRow_result();
+        try {
+          iface_.deleteAllRow(args.tableName, args.row);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("deleteAllRow", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class deleteAllRowTs implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        deleteAllRowTs_args args = new deleteAllRowTs_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        deleteAllRowTs_result result = new deleteAllRowTs_result();
+        try {
+          iface_.deleteAllRowTs(args.tableName, args.row, args.timestamp);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("deleteAllRowTs", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class scannerOpen implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        scannerOpen_args args = new scannerOpen_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        scannerOpen_result result = new scannerOpen_result();
+        try {
+          result.success = iface_.scannerOpen(args.tableName, args.startRow, args.columns);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("scannerOpen", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class scannerOpenWithStop implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        scannerOpenWithStop_args args = new scannerOpenWithStop_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        scannerOpenWithStop_result result = new scannerOpenWithStop_result();
+        try {
+          result.success = iface_.scannerOpenWithStop(args.tableName, args.startRow, args.stopRow, args.columns);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("scannerOpenWithStop", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class scannerOpenTs implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        scannerOpenTs_args args = new scannerOpenTs_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        scannerOpenTs_result result = new scannerOpenTs_result();
+        try {
+          result.success = iface_.scannerOpenTs(args.tableName, args.startRow, args.columns, args.timestamp);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("scannerOpenTs", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class scannerOpenWithStopTs implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        scannerOpenWithStopTs_args args = new scannerOpenWithStopTs_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        scannerOpenWithStopTs_result result = new scannerOpenWithStopTs_result();
+        try {
+          result.success = iface_.scannerOpenWithStopTs(args.tableName, args.startRow, args.stopRow, args.columns, args.timestamp);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        }
+        oprot.writeMessageBegin(new TMessage("scannerOpenWithStopTs", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class scannerGet implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        scannerGet_args args = new scannerGet_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        scannerGet_result result = new scannerGet_result();
+        try {
+          result.success = iface_.scannerGet(args.id);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        } catch (IllegalArgument ia) {
+          result.ia = ia;
+          result.__isset.ia = true;
+        } catch (NotFound nf) {
+          result.nf = nf;
+          result.__isset.nf = true;
+        }
+        oprot.writeMessageBegin(new TMessage("scannerGet", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class scannerClose implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        scannerClose_args args = new scannerClose_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        scannerClose_result result = new scannerClose_result();
+        try {
+          iface_.scannerClose(args.id);
+        } catch (IOError io) {
+          result.io = io;
+          result.__isset.io = true;
+        } catch (IllegalArgument ia) {
+          result.ia = ia;
+          result.__isset.ia = true;
+        }
+        oprot.writeMessageBegin(new TMessage("scannerClose", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+  }
+
+  public static class getTableNames_args implements TBase, java.io.Serializable   {
+    public getTableNames_args() {
+    }
+
+    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 ArrayList<byte[]> success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public getTableNames_result() {
+    }
+
+    public getTableNames_result(
+      ArrayList<byte[]> success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = 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 0:
+            if (field.type == TType.LIST) {
+              {
+                TList _list5 = iprot.readListBegin();
+                this.success = new ArrayList<byte[]>(_list5.size);
+                for (int _i6 = 0; _i6 < _list5.size; ++_i6)
+                {
+                  byte[] _elem7 = null;
+                  _elem7 = iprot.readBinary();
+                  this.success.add(_elem7);
+                }
+                iprot.readListEnd();
+              }
+              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("getTableNames_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        if (this.success != null) {
+          field.name = "success";
+          field.type = TType.LIST;
+          field.id = 0;
+          oprot.writeFieldBegin(field);
+          {
+            oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
+            for (byte[] _iter8 : this.success)            {
+              oprot.writeBinary(_iter8);
+            }
+            oprot.writeListEnd();
+          }
+          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("getTableNames_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getColumnDescriptors_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+    }
+
+    public getColumnDescriptors_args() {
+    }
+
+    public getColumnDescriptors_args(
+      byte[] tableName)
+    {
+      this();
+      this.tableName = tableName;
+      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("getColumnDescriptors_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("getColumnDescriptors_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getColumnDescriptors_result implements TBase, java.io.Serializable   {
+    public AbstractMap<byte[],ColumnDescriptor> success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public getColumnDescriptors_result() {
+    }
+
+    public getColumnDescriptors_result(
+      AbstractMap<byte[],ColumnDescriptor> success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = 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 0:
+            if (field.type == TType.MAP) {
+              {
+                TMap _map9 = iprot.readMapBegin();
+                this.success = new HashMap<byte[],ColumnDescriptor>(2*_map9.size);
+                for (int _i10 = 0; _i10 < _map9.size; ++_i10)
+                {
+                  byte[] _key11;
+                  ColumnDescriptor _val12;
+                  _key11 = iprot.readBinary();
+                  _val12 = new ColumnDescriptor();
+                  _val12.read(iprot);
+                  this.success.put(_key11, _val12);
+                }
+                iprot.readMapEnd();
+              }
+              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("getColumnDescriptors_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        if (this.success != null) {
+          field.name = "success";
+          field.type = TType.MAP;
+          field.id = 0;
+          oprot.writeFieldBegin(field);
+          {
+            oprot.writeMapBegin(new TMap(TType.STRING, TType.STRUCT, this.success.size()));
+            for (byte[] _iter13 : this.success.keySet())            {
+              oprot.writeBinary(_iter13);
+              this.success.get(_iter13).write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          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("getColumnDescriptors_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getTableRegions_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+    }
+
+    public getTableRegions_args() {
+    }
+
+    public getTableRegions_args(
+      byte[] tableName)
+    {
+      this();
+      this.tableName = tableName;
+      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("getTableRegions_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("getTableRegions_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getTableRegions_result implements TBase, java.io.Serializable   {
+    public ArrayList<RegionDescriptor> success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public getTableRegions_result() {
+    }
+
+    public getTableRegions_result(
+      ArrayList<RegionDescriptor> success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = 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 0:
+            if (field.type == TType.LIST) {
+              {
+                TList _list14 = iprot.readListBegin();
+                this.success = new ArrayList<RegionDescriptor>(_list14.size);
+                for (int _i15 = 0; _i15 < _list14.size; ++_i15)
+                {
+                  RegionDescriptor _elem16 = new RegionDescriptor();
+                  _elem16 = new RegionDescriptor();
+                  _elem16.read(iprot);
+                  this.success.add(_elem16);
+                }
+                iprot.readListEnd();
+              }
+              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("getTableRegions_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        if (this.success != null) {
+          field.name = "success";
+          field.type = TType.LIST;
+          field.id = 0;
+          oprot.writeFieldBegin(field);
+          {
+            oprot.writeListBegin(new TList(TType.STRUCT, this.success.size()));
+            for (RegionDescriptor _iter17 : this.success)            {
+              _iter17.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          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("getTableRegions_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class createTable_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public ArrayList<ColumnDescriptor> columnFamilies;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean columnFamilies = false;
+    }
+
+    public createTable_args() {
+    }
+
+    public createTable_args(
+      byte[] tableName,
+      ArrayList<ColumnDescriptor> columnFamilies)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.columnFamilies = columnFamilies;
+      this.__isset.columnFamilies = 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;
+          case 2:
+            if (field.type == TType.LIST) {
+              {
+                TList _list18 = iprot.readListBegin();
+                this.columnFamilies = new ArrayList<ColumnDescriptor>(_list18.size);
+                for (int _i19 = 0; _i19 < _list18.size; ++_i19)
+                {
+                  ColumnDescriptor _elem20 = new ColumnDescriptor();
+                  _elem20 = new ColumnDescriptor();
+                  _elem20.read(iprot);
+                  this.columnFamilies.add(_elem20);
+                }
+                iprot.readListEnd();
+              }
+              this.__isset.columnFamilies = 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("createTable_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();
+      }
+      if (this.columnFamilies != null) {
+        field.name = "columnFamilies";
+        field.type = TType.LIST;
+        field.id = 2;
+        oprot.writeFieldBegin(field);
+        {
+          oprot.writeListBegin(new TList(TType.STRUCT, this.columnFamilies.size()));
+          for (ColumnDescriptor _iter21 : this.columnFamilies)          {
+            _iter21.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("createTable_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",columnFamilies:");
+      sb.append(this.columnFamilies);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class createTable_result implements TBase, java.io.Serializable   {
+    public IOError io;
+    public IllegalArgument ia;
+    public AlreadyExists exist;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean io = false;
+      public boolean ia = false;
+      public boolean exist = false;
+    }
+
+    public createTable_result() {
+    }
+
+    public createTable_result(
+      IOError io,
+      IllegalArgument ia,
+      AlreadyExists exist)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = true;
+      this.ia = ia;
+      this.__isset.ia = true;
+      this.exist = exist;
+      this.__isset.exist = 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.ia = new IllegalArgument();
+              this.ia.read(iprot);
+              this.__isset.ia = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 3:
+            if (field.type == TType.STRUCT) {
+              this.exist = new AlreadyExists();
+              this.exist.read(iprot);
+              this.__isset.exist = 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("createTable_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.ia) {
+        if (this.ia != null) {
+          field.name = "ia";
+          field.type = TType.STRUCT;
+          field.id = 2;
+          oprot.writeFieldBegin(field);
+          this.ia.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      } else if (this.__isset.exist) {
+        if (this.exist != null) {
+          field.name = "exist";
+          field.type = TType.STRUCT;
+          field.id = 3;
+          oprot.writeFieldBegin(field);
+          this.exist.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("createTable_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(",ia:");
+      sb.append(this.ia.toString());
+      sb.append(",exist:");
+      sb.append(this.exist.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class deleteTable_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+    }
+
+    public deleteTable_args() {
+    }
+
+    public deleteTable_args(
+      byte[] tableName)
+    {
+      this();
+      this.tableName = tableName;
+      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();
+        }
+      }
+      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() {
+    }
+
+    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 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;
+          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;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("get_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();
+      }
+      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(");
+      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 IOError io;
+    public NotFound nf;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+      public boolean nf = false;
+    }
+
+    public get_result() {
+    }
+
+    public get_result(
+      byte[] success,
+      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 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.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();
+              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("get_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.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();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("get_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(",nf:");
+      sb.append(this.nf.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getVer_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 boolean tableName = false;
+      public boolean row = false;
+      public boolean column = false;
+      public boolean numVersions = false;
+    }
+
+    public getVer_args() {
+    }
+
+    public getVer_args(
+      byte[] tableName,
+      byte[] row,
+      byte[] column,
+      int numVersions)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = true;
+      this.column = column;
+      this.__isset.column = true;
+      this.numVersions = numVersions;
+      this.__isset.numVersions = 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;
+          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;
+          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;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("getVer_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();
+      }
+      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();
+      }
+      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(");
+      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 IOError io;
+    public NotFound nf;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+      public boolean nf = false;
+    }
+
+    public getVer_result() {
+    }
+
+    public getVer_result(
+      ArrayList<byte[]> success,
+      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 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 _list22 = iprot.readListBegin();
+                this.success = new ArrayList<byte[]>(_list22.size);
+                for (int _i23 = 0; _i23 < _list22.size; ++_i23)
+                {
+                  byte[] _elem24 = null;
+                  _elem24 = iprot.readBinary();
+                  this.success.add(_elem24);
+                }
+                iprot.readListEnd();
+              }
+              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;
+          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("getVer_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        if (this.success != null) {
+          field.name = "success";
+          field.type = TType.LIST;
+          field.id = 0;
+          oprot.writeFieldBegin(field);
+          {
+            oprot.writeListBegin(new TList(TType.STRING, this.success.size()));
+            for (byte[] _iter25 : this.success)            {
+              oprot.writeBinary(_iter25);
+            }
+            oprot.writeListEnd();
+          }
+          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();
+        }
+      } 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();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getVer_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(",nf:");
+      sb.append(this.nf.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getVerTs_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 boolean tableName = false;
+      public boolean row = false;
+      public boolean column = false;
+      public boolean timestamp = false;
+      public boolean numVersions = false;
+    }
+
+    public getVerTs_args() {
+    }
+
+    public getVerTs_args(
+      byte[] tableName,
+      byte[] row,
+      byte[] column,
+      long timestamp,
+      int numVersions)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = true;
+      this.column = column;
+      this.__isset.column = true;
+      this.timestamp = timestamp;
+      this.__isset.timestamp = true;
+      this.numVersions = numVersions;
+      this.__isset.numVersions = 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;
+          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;
+          case 4:
+            if (field.type == TType.I64) {
+              this.timestamp = iprot.readI64();
+              this.__isset.timestamp = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 5:
+            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;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("getVerTs_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();
+      }
+      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();
+      }
+      field.name = "timestamp";
+      field.type = TType.I64;
+      field.id = 4;
+      oprot.writeFieldBegin(field);
+      oprot.writeI64(this.timestamp);
+      oprot.writeFieldEnd();
+      field.name = "numVersions";
+      field.type = TType.I32;
+      field.id = 5;
+      oprot.writeFieldBegin(field);
+      oprot.writeI32(this.numVersions);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getVerTs_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",row:");
+      sb.append(this.row);
+      sb.append(",column:");
+      sb.append(this.column);
+      sb.append(",timestamp:");
+      sb.append(this.timestamp);
+      sb.append(",numVersions:");
+      sb.append(this.numVersions);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getVerTs_result implements TBase, java.io.Serializable   {
+    public ArrayList<byte[]> success;
+    public IOError io;
+    public NotFound nf;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+      public boolean nf = false;
+    }
+
+    public getVerTs_result() {
+    }
+
+    public getVerTs_result(
+      ArrayList<byte[]> success,
+      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 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 _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();
+              }
+              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;
+          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("getVerTs_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        if (this.success != null) {
+          field.name = "success";
+          field.type = TType.LIST;
+          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();
+          }
+          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();
+        }
+      } 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();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getVerTs_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(",nf:");
+      sb.append(this.nf.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getRow_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] row;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean row = false;
+    }
+
+    public getRow_args() {
+    }
+
+    public getRow_args(
+      byte[] tableName,
+      byte[] row)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = 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;
+          case 2:
+            if (field.type == TType.STRING) {
+              this.row = iprot.readBinary();
+              this.__isset.row = 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("getRow_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();
+      }
+      if (this.row != null) {
+        field.name = "row";
+        field.type = TType.STRING;
+        field.id = 2;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.row);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getRow_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",row:");
+      sb.append(this.row);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getRow_result implements TBase, java.io.Serializable   {
+    public AbstractMap<byte[],byte[]> success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public getRow_result() {
+    }
+
+    public getRow_result(
+      AbstractMap<byte[],byte[]> success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = 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 0:
+            if (field.type == TType.MAP) {
+              {
+                TMap _map30 = iprot.readMapBegin();
+                this.success = new HashMap<byte[],byte[]>(2*_map30.size);
+                for (int _i31 = 0; _i31 < _map30.size; ++_i31)
+                {
+                  byte[] _key32;
+                  byte[] _val33;
+                  _key32 = iprot.readBinary();
+                  _val33 = iprot.readBinary();
+                  this.success.put(_key32, _val33);
+                }
+                iprot.readMapEnd();
+              }
+              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("getRow_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        if (this.success != null) {
+          field.name = "success";
+          field.type = TType.MAP;
+          field.id = 0;
+          oprot.writeFieldBegin(field);
+          {
+            oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.success.size()));
+            for (byte[] _iter34 : this.success.keySet())            {
+              oprot.writeBinary(_iter34);
+              oprot.writeBinary(this.success.get(_iter34));
+            }
+            oprot.writeMapEnd();
+          }
+          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("getRow_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getRowTs_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] row;
+    public long timestamp;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean row = false;
+      public boolean timestamp = false;
+    }
+
+    public getRowTs_args() {
+    }
+
+    public getRowTs_args(
+      byte[] tableName,
+      byte[] row,
+      long timestamp)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = true;
+      this.timestamp = timestamp;
+      this.__isset.timestamp = 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;
+          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.I64) {
+              this.timestamp = iprot.readI64();
+              this.__isset.timestamp = 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("getRowTs_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();
+      }
+      if (this.row != null) {
+        field.name = "row";
+        field.type = TType.STRING;
+        field.id = 2;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.row);
+        oprot.writeFieldEnd();
+      }
+      field.name = "timestamp";
+      field.type = TType.I64;
+      field.id = 3;
+      oprot.writeFieldBegin(field);
+      oprot.writeI64(this.timestamp);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getRowTs_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",row:");
+      sb.append(this.row);
+      sb.append(",timestamp:");
+      sb.append(this.timestamp);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class getRowTs_result implements TBase, java.io.Serializable   {
+    public AbstractMap<byte[],byte[]> success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public getRowTs_result() {
+    }
+
+    public getRowTs_result(
+      AbstractMap<byte[],byte[]> success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = 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 0:
+            if (field.type == TType.MAP) {
+              {
+                TMap _map35 = iprot.readMapBegin();
+                this.success = new HashMap<byte[],byte[]>(2*_map35.size);
+                for (int _i36 = 0; _i36 < _map35.size; ++_i36)
+                {
+                  byte[] _key37;
+                  byte[] _val38;
+                  _key37 = iprot.readBinary();
+                  _val38 = iprot.readBinary();
+                  this.success.put(_key37, _val38);
+                }
+                iprot.readMapEnd();
+              }
+              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("getRowTs_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        if (this.success != null) {
+          field.name = "success";
+          field.type = TType.MAP;
+          field.id = 0;
+          oprot.writeFieldBegin(field);
+          {
+            oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.success.size()));
+            for (byte[] _iter39 : this.success.keySet())            {
+              oprot.writeBinary(_iter39);
+              oprot.writeBinary(this.success.get(_iter39));
+            }
+            oprot.writeMapEnd();
+          }
+          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("getRowTs_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class put_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] row;
+    public byte[] column;
+    public byte[] value;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean row = false;
+      public boolean column = false;
+      public boolean value = false;
+    }
+
+    public put_args() {
+    }
+
+    public put_args(
+      byte[] tableName,
+      byte[] row,
+      byte[] column,
+      byte[] value)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = true;
+      this.column = column;
+      this.__isset.column = true;
+      this.value = value;
+      this.__isset.value = 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;
+          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;
+          case 4:
+            if (field.type == TType.STRING) {
+              this.value = iprot.readBinary();
+              this.__isset.value = 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("put_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();
+      }
+      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();
+      }
+      if (this.value != null) {
+        field.name = "value";
+        field.type = TType.STRING;
+        field.id = 4;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.value);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("put_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",row:");
+      sb.append(this.row);
+      sb.append(",column:");
+      sb.append(this.column);
+      sb.append(",value:");
+      sb.append(this.value);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class put_result implements TBase, java.io.Serializable   {
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean io = false;
+    }
+
+    public put_result() {
+    }
+
+    public put_result(
+      IOError io)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = 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;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("put_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("put_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class mutateRow_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] row;
+    public ArrayList<Mutation> mutations;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean row = false;
+      public boolean mutations = false;
+    }
+
+    public mutateRow_args() {
+    }
+
+    public mutateRow_args(
+      byte[] tableName,
+      byte[] row,
+      ArrayList<Mutation> mutations)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = true;
+      this.mutations = mutations;
+      this.__isset.mutations = 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;
+          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.LIST) {
+              {
+                TList _list40 = iprot.readListBegin();
+                this.mutations = new ArrayList<Mutation>(_list40.size);
+                for (int _i41 = 0; _i41 < _list40.size; ++_i41)
+                {
+                  Mutation _elem42 = new Mutation();
+                  _elem42 = new Mutation();
+                  _elem42.read(iprot);
+                  this.mutations.add(_elem42);
+                }
+                iprot.readListEnd();
+              }
+              this.__isset.mutations = 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("mutateRow_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();
+      }
+      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.mutations != null) {
+        field.name = "mutations";
+        field.type = TType.LIST;
+        field.id = 3;
+        oprot.writeFieldBegin(field);
+        {
+          oprot.writeListBegin(new TList(TType.STRUCT, this.mutations.size()));
+          for (Mutation _iter43 : this.mutations)          {
+            _iter43.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("mutateRow_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",row:");
+      sb.append(this.row);
+      sb.append(",mutations:");
+      sb.append(this.mutations);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class mutateRow_result implements TBase, java.io.Serializable   {
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean io = false;
+    }
+
+    public mutateRow_result() {
+    }
+
+    public mutateRow_result(
+      IOError io)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = 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;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("mutateRow_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("mutateRow_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class mutateRowTs_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] row;
+    public ArrayList<Mutation> mutations;
+    public long timestamp;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean row = false;
+      public boolean mutations = false;
+      public boolean timestamp = false;
+    }
+
+    public mutateRowTs_args() {
+    }
+
+    public mutateRowTs_args(
+      byte[] tableName,
+      byte[] row,
+      ArrayList<Mutation> mutations,
+      long timestamp)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = true;
+      this.mutations = mutations;
+      this.__isset.mutations = true;
+      this.timestamp = timestamp;
+      this.__isset.timestamp = 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;
+          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.LIST) {
+              {
+                TList _list44 = iprot.readListBegin();
+                this.mutations = new ArrayList<Mutation>(_list44.size);
+                for (int _i45 = 0; _i45 < _list44.size; ++_i45)
+                {
+                  Mutation _elem46 = new Mutation();
+                  _elem46 = new Mutation();
+                  _elem46.read(iprot);
+                  this.mutations.add(_elem46);
+                }
+                iprot.readListEnd();
+              }
+              this.__isset.mutations = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 4:
+            if (field.type == TType.I64) {
+              this.timestamp = iprot.readI64();
+              this.__isset.timestamp = 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("mutateRowTs_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();
+      }
+      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.mutations != null) {
+        field.name = "mutations";
+        field.type = TType.LIST;
+        field.id = 3;
+        oprot.writeFieldBegin(field);
+        {
+          oprot.writeListBegin(new TList(TType.STRUCT, this.mutations.size()));
+          for (Mutation _iter47 : this.mutations)          {
+            _iter47.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      field.name = "timestamp";
+      field.type = TType.I64;
+      field.id = 4;
+      oprot.writeFieldBegin(field);
+      oprot.writeI64(this.timestamp);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("mutateRowTs_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",row:");
+      sb.append(this.row);
+      sb.append(",mutations:");
+      sb.append(this.mutations);
+      sb.append(",timestamp:");
+      sb.append(this.timestamp);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class mutateRowTs_result implements TBase, java.io.Serializable   {
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean io = false;
+    }
+
+    public mutateRowTs_result() {
+    }
+
+    public mutateRowTs_result(
+      IOError io)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = 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;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("mutateRowTs_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("mutateRowTs_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class deleteAll_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 deleteAll_args() {
+    }
+
+    public deleteAll_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 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;
+          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;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("deleteAll_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();
+      }
+      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("deleteAll_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 deleteAll_result implements TBase, java.io.Serializable   {
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean io = false;
+    }
+
+    public deleteAll_result() {
+    }
+
+    public deleteAll_result(
+      IOError io)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = 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;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("deleteAll_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("deleteAll_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class deleteAllTs_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] row;
+    public byte[] column;
+    public long timestamp;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean row = false;
+      public boolean column = false;
+      public boolean timestamp = false;
+    }
+
+    public deleteAllTs_args() {
+    }
+
+    public deleteAllTs_args(
+      byte[] tableName,
+      byte[] row,
+      byte[] column,
+      long timestamp)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = true;
+      this.column = column;
+      this.__isset.column = true;
+      this.timestamp = timestamp;
+      this.__isset.timestamp = 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;
+          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;
+          case 4:
+            if (field.type == TType.I64) {
+              this.timestamp = iprot.readI64();
+              this.__isset.timestamp = 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("deleteAllTs_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();
+      }
+      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();
+      }
+      field.name = "timestamp";
+      field.type = TType.I64;
+      field.id = 4;
+      oprot.writeFieldBegin(field);
+      oprot.writeI64(this.timestamp);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("deleteAllTs_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",row:");
+      sb.append(this.row);
+      sb.append(",column:");
+      sb.append(this.column);
+      sb.append(",timestamp:");
+      sb.append(this.timestamp);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class deleteAllTs_result implements TBase, java.io.Serializable   {
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean io = false;
+    }
+
+    public deleteAllTs_result() {
+    }
+
+    public deleteAllTs_result(
+      IOError io)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = 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;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("deleteAllTs_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("deleteAllTs_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class deleteAllRow_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] row;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean row = false;
+    }
+
+    public deleteAllRow_args() {
+    }
+
+    public deleteAllRow_args(
+      byte[] tableName,
+      byte[] row)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = 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;
+          case 2:
+            if (field.type == TType.STRING) {
+              this.row = iprot.readBinary();
+              this.__isset.row = 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("deleteAllRow_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();
+      }
+      if (this.row != null) {
+        field.name = "row";
+        field.type = TType.STRING;
+        field.id = 2;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.row);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("deleteAllRow_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",row:");
+      sb.append(this.row);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class deleteAllRow_result implements TBase, java.io.Serializable   {
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean io = false;
+    }
+
+    public deleteAllRow_result() {
+    }
+
+    public deleteAllRow_result(
+      IOError io)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = 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;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("deleteAllRow_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("deleteAllRow_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class deleteAllRowTs_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] row;
+    public long timestamp;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean row = false;
+      public boolean timestamp = false;
+    }
+
+    public deleteAllRowTs_args() {
+    }
+
+    public deleteAllRowTs_args(
+      byte[] tableName,
+      byte[] row,
+      long timestamp)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.row = row;
+      this.__isset.row = true;
+      this.timestamp = timestamp;
+      this.__isset.timestamp = 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;
+          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.I64) {
+              this.timestamp = iprot.readI64();
+              this.__isset.timestamp = 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("deleteAllRowTs_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();
+      }
+      if (this.row != null) {
+        field.name = "row";
+        field.type = TType.STRING;
+        field.id = 2;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.row);
+        oprot.writeFieldEnd();
+      }
+      field.name = "timestamp";
+      field.type = TType.I64;
+      field.id = 3;
+      oprot.writeFieldBegin(field);
+      oprot.writeI64(this.timestamp);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("deleteAllRowTs_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",row:");
+      sb.append(this.row);
+      sb.append(",timestamp:");
+      sb.append(this.timestamp);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class deleteAllRowTs_result implements TBase, java.io.Serializable   {
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean io = false;
+    }
+
+    public deleteAllRowTs_result() {
+    }
+
+    public deleteAllRowTs_result(
+      IOError io)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = 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;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      TStruct struct = new TStruct("deleteAllRowTs_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("deleteAllRowTs_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerOpen_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] startRow;
+    public ArrayList<byte[]> columns;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean startRow = false;
+      public boolean columns = false;
+    }
+
+    public scannerOpen_args() {
+    }
+
+    public scannerOpen_args(
+      byte[] tableName,
+      byte[] startRow,
+      ArrayList<byte[]> columns)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.startRow = startRow;
+      this.__isset.startRow = true;
+      this.columns = columns;
+      this.__isset.columns = 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;
+          case 2:
+            if (field.type == TType.STRING) {
+              this.startRow = iprot.readBinary();
+              this.__isset.startRow = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 3:
+            if (field.type == TType.LIST) {
+              {
+                TList _list48 = iprot.readListBegin();
+                this.columns = new ArrayList<byte[]>(_list48.size);
+                for (int _i49 = 0; _i49 < _list48.size; ++_i49)
+                {
+                  byte[] _elem50 = null;
+                  _elem50 = iprot.readBinary();
+                  this.columns.add(_elem50);
+                }
+                iprot.readListEnd();
+              }
+              this.__isset.columns = 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("scannerOpen_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();
+      }
+      if (this.startRow != null) {
+        field.name = "startRow";
+        field.type = TType.STRING;
+        field.id = 2;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.startRow);
+        oprot.writeFieldEnd();
+      }
+      if (this.columns != null) {
+        field.name = "columns";
+        field.type = TType.LIST;
+        field.id = 3;
+        oprot.writeFieldBegin(field);
+        {
+          oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
+          for (byte[] _iter51 : this.columns)          {
+            oprot.writeBinary(_iter51);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerOpen_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",startRow:");
+      sb.append(this.startRow);
+      sb.append(",columns:");
+      sb.append(this.columns);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerOpen_result implements TBase, java.io.Serializable   {
+    public int success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public scannerOpen_result() {
+    }
+
+    public scannerOpen_result(
+      int success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = 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 0:
+            if (field.type == TType.I32) {
+              this.success = iprot.readI32();
+              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("scannerOpen_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        field.name = "success";
+        field.type = TType.I32;
+        field.id = 0;
+        oprot.writeFieldBegin(field);
+        oprot.writeI32(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("scannerOpen_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerOpenWithStop_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] startRow;
+    public byte[] stopRow;
+    public ArrayList<byte[]> columns;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean startRow = false;
+      public boolean stopRow = false;
+      public boolean columns = false;
+    }
+
+    public scannerOpenWithStop_args() {
+    }
+
+    public scannerOpenWithStop_args(
+      byte[] tableName,
+      byte[] startRow,
+      byte[] stopRow,
+      ArrayList<byte[]> columns)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.startRow = startRow;
+      this.__isset.startRow = true;
+      this.stopRow = stopRow;
+      this.__isset.stopRow = true;
+      this.columns = columns;
+      this.__isset.columns = 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;
+          case 2:
+            if (field.type == TType.STRING) {
+              this.startRow = iprot.readBinary();
+              this.__isset.startRow = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 3:
+            if (field.type == TType.STRING) {
+              this.stopRow = iprot.readBinary();
+              this.__isset.stopRow = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 4:
+            if (field.type == TType.LIST) {
+              {
+                TList _list52 = iprot.readListBegin();
+                this.columns = new ArrayList<byte[]>(_list52.size);
+                for (int _i53 = 0; _i53 < _list52.size; ++_i53)
+                {
+                  byte[] _elem54 = null;
+                  _elem54 = iprot.readBinary();
+                  this.columns.add(_elem54);
+                }
+                iprot.readListEnd();
+              }
+              this.__isset.columns = 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("scannerOpenWithStop_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();
+      }
+      if (this.startRow != null) {
+        field.name = "startRow";
+        field.type = TType.STRING;
+        field.id = 2;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.startRow);
+        oprot.writeFieldEnd();
+      }
+      if (this.stopRow != null) {
+        field.name = "stopRow";
+        field.type = TType.STRING;
+        field.id = 3;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.stopRow);
+        oprot.writeFieldEnd();
+      }
+      if (this.columns != null) {
+        field.name = "columns";
+        field.type = TType.LIST;
+        field.id = 4;
+        oprot.writeFieldBegin(field);
+        {
+          oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
+          for (byte[] _iter55 : this.columns)          {
+            oprot.writeBinary(_iter55);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerOpenWithStop_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",startRow:");
+      sb.append(this.startRow);
+      sb.append(",stopRow:");
+      sb.append(this.stopRow);
+      sb.append(",columns:");
+      sb.append(this.columns);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerOpenWithStop_result implements TBase, java.io.Serializable   {
+    public int success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public scannerOpenWithStop_result() {
+    }
+
+    public scannerOpenWithStop_result(
+      int success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = 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 0:
+            if (field.type == TType.I32) {
+              this.success = iprot.readI32();
+              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("scannerOpenWithStop_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        field.name = "success";
+        field.type = TType.I32;
+        field.id = 0;
+        oprot.writeFieldBegin(field);
+        oprot.writeI32(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("scannerOpenWithStop_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerOpenTs_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] startRow;
+    public ArrayList<byte[]> columns;
+    public long timestamp;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean startRow = false;
+      public boolean columns = false;
+      public boolean timestamp = false;
+    }
+
+    public scannerOpenTs_args() {
+    }
+
+    public scannerOpenTs_args(
+      byte[] tableName,
+      byte[] startRow,
+      ArrayList<byte[]> columns,
+      long timestamp)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.startRow = startRow;
+      this.__isset.startRow = true;
+      this.columns = columns;
+      this.__isset.columns = true;
+      this.timestamp = timestamp;
+      this.__isset.timestamp = 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;
+          case 2:
+            if (field.type == TType.STRING) {
+              this.startRow = iprot.readBinary();
+              this.__isset.startRow = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 3:
+            if (field.type == TType.LIST) {
+              {
+                TList _list56 = iprot.readListBegin();
+                this.columns = new ArrayList<byte[]>(_list56.size);
+                for (int _i57 = 0; _i57 < _list56.size; ++_i57)
+                {
+                  byte[] _elem58 = null;
+                  _elem58 = iprot.readBinary();
+                  this.columns.add(_elem58);
+                }
+                iprot.readListEnd();
+              }
+              this.__isset.columns = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 4:
+            if (field.type == TType.I64) {
+              this.timestamp = iprot.readI64();
+              this.__isset.timestamp = 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("scannerOpenTs_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();
+      }
+      if (this.startRow != null) {
+        field.name = "startRow";
+        field.type = TType.STRING;
+        field.id = 2;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.startRow);
+        oprot.writeFieldEnd();
+      }
+      if (this.columns != null) {
+        field.name = "columns";
+        field.type = TType.LIST;
+        field.id = 3;
+        oprot.writeFieldBegin(field);
+        {
+          oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
+          for (byte[] _iter59 : this.columns)          {
+            oprot.writeBinary(_iter59);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      field.name = "timestamp";
+      field.type = TType.I64;
+      field.id = 4;
+      oprot.writeFieldBegin(field);
+      oprot.writeI64(this.timestamp);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerOpenTs_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",startRow:");
+      sb.append(this.startRow);
+      sb.append(",columns:");
+      sb.append(this.columns);
+      sb.append(",timestamp:");
+      sb.append(this.timestamp);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerOpenTs_result implements TBase, java.io.Serializable   {
+    public int success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public scannerOpenTs_result() {
+    }
+
+    public scannerOpenTs_result(
+      int success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = 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 0:
+            if (field.type == TType.I32) {
+              this.success = iprot.readI32();
+              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("scannerOpenTs_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        field.name = "success";
+        field.type = TType.I32;
+        field.id = 0;
+        oprot.writeFieldBegin(field);
+        oprot.writeI32(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("scannerOpenTs_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerOpenWithStopTs_args implements TBase, java.io.Serializable   {
+    public byte[] tableName;
+    public byte[] startRow;
+    public byte[] stopRow;
+    public ArrayList<byte[]> columns;
+    public long timestamp;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean tableName = false;
+      public boolean startRow = false;
+      public boolean stopRow = false;
+      public boolean columns = false;
+      public boolean timestamp = false;
+    }
+
+    public scannerOpenWithStopTs_args() {
+    }
+
+    public scannerOpenWithStopTs_args(
+      byte[] tableName,
+      byte[] startRow,
+      byte[] stopRow,
+      ArrayList<byte[]> columns,
+      long timestamp)
+    {
+      this();
+      this.tableName = tableName;
+      this.__isset.tableName = true;
+      this.startRow = startRow;
+      this.__isset.startRow = true;
+      this.stopRow = stopRow;
+      this.__isset.stopRow = true;
+      this.columns = columns;
+      this.__isset.columns = true;
+      this.timestamp = timestamp;
+      this.__isset.timestamp = 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;
+          case 2:
+            if (field.type == TType.STRING) {
+              this.startRow = iprot.readBinary();
+              this.__isset.startRow = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 3:
+            if (field.type == TType.STRING) {
+              this.stopRow = iprot.readBinary();
+              this.__isset.stopRow = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 4:
+            if (field.type == TType.LIST) {
+              {
+                TList _list60 = iprot.readListBegin();
+                this.columns = new ArrayList<byte[]>(_list60.size);
+                for (int _i61 = 0; _i61 < _list60.size; ++_i61)
+                {
+                  byte[] _elem62 = null;
+                  _elem62 = iprot.readBinary();
+                  this.columns.add(_elem62);
+                }
+                iprot.readListEnd();
+              }
+              this.__isset.columns = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 5:
+            if (field.type == TType.I64) {
+              this.timestamp = iprot.readI64();
+              this.__isset.timestamp = 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("scannerOpenWithStopTs_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();
+      }
+      if (this.startRow != null) {
+        field.name = "startRow";
+        field.type = TType.STRING;
+        field.id = 2;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.startRow);
+        oprot.writeFieldEnd();
+      }
+      if (this.stopRow != null) {
+        field.name = "stopRow";
+        field.type = TType.STRING;
+        field.id = 3;
+        oprot.writeFieldBegin(field);
+        oprot.writeBinary(this.stopRow);
+        oprot.writeFieldEnd();
+      }
+      if (this.columns != null) {
+        field.name = "columns";
+        field.type = TType.LIST;
+        field.id = 4;
+        oprot.writeFieldBegin(field);
+        {
+          oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
+          for (byte[] _iter63 : this.columns)          {
+            oprot.writeBinary(_iter63);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      field.name = "timestamp";
+      field.type = TType.I64;
+      field.id = 5;
+      oprot.writeFieldBegin(field);
+      oprot.writeI64(this.timestamp);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerOpenWithStopTs_args(");
+      sb.append("tableName:");
+      sb.append(this.tableName);
+      sb.append(",startRow:");
+      sb.append(this.startRow);
+      sb.append(",stopRow:");
+      sb.append(this.stopRow);
+      sb.append(",columns:");
+      sb.append(this.columns);
+      sb.append(",timestamp:");
+      sb.append(this.timestamp);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerOpenWithStopTs_result implements TBase, java.io.Serializable   {
+    public int success;
+    public IOError io;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+    }
+
+    public scannerOpenWithStopTs_result() {
+    }
+
+    public scannerOpenWithStopTs_result(
+      int success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = 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 0:
+            if (field.type == TType.I32) {
+              this.success = iprot.readI32();
+              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("scannerOpenWithStopTs_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        field.name = "success";
+        field.type = TType.I32;
+        field.id = 0;
+        oprot.writeFieldBegin(field);
+        oprot.writeI32(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("scannerOpenWithStopTs_result(");
+      sb.append("success:");
+      sb.append(this.success);
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerGet_args implements TBase, java.io.Serializable   {
+    public int id;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean id = false;
+    }
+
+    public scannerGet_args() {
+    }
+
+    public scannerGet_args(
+      int id)
+    {
+      this();
+      this.id = id;
+      this.__isset.id = 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.I32) {
+              this.id = iprot.readI32();
+              this.__isset.id = 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("scannerGet_args");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+      field.name = "id";
+      field.type = TType.I32;
+      field.id = 1;
+      oprot.writeFieldBegin(field);
+      oprot.writeI32(this.id);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerGet_args(");
+      sb.append("id:");
+      sb.append(this.id);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerGet_result implements TBase, java.io.Serializable   {
+    public ScanEntry success;
+    public IOError io;
+    public IllegalArgument ia;
+    public NotFound nf;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean success = false;
+      public boolean io = false;
+      public boolean ia = false;
+      public boolean nf = false;
+    }
+
+    public scannerGet_result() {
+    }
+
+    public scannerGet_result(
+      ScanEntry success,
+      IOError io,
+      IllegalArgument ia,
+      NotFound nf)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+      this.__isset.io = true;
+      this.ia = ia;
+      this.__isset.ia = 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 0:
+            if (field.type == TType.STRUCT) {
+              this.success = new ScanEntry();
+              this.success.read(iprot);
+              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;
+          case 2:
+            if (field.type == TType.STRUCT) {
+              this.ia = new IllegalArgument();
+              this.ia.read(iprot);
+              this.__isset.ia = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 3:
+            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("scannerGet_result");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+
+      if (this.__isset.success) {
+        if (this.success != null) {
+          field.name = "success";
+          field.type = TType.STRUCT;
+          field.id = 0;
+          oprot.writeFieldBegin(field);
+          this.success.write(oprot);
+          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();
+        }
+      } else if (this.__isset.ia) {
+        if (this.ia != null) {
+          field.name = "ia";
+          field.type = TType.STRUCT;
+          field.id = 2;
+          oprot.writeFieldBegin(field);
+          this.ia.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      } else if (this.__isset.nf) {
+        if (this.nf != null) {
+          field.name = "nf";
+          field.type = TType.STRUCT;
+          field.id = 3;
+          oprot.writeFieldBegin(field);
+          this.nf.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerGet_result(");
+      sb.append("success:");
+      sb.append(this.success.toString());
+      sb.append(",io:");
+      sb.append(this.io.toString());
+      sb.append(",ia:");
+      sb.append(this.ia.toString());
+      sb.append(",nf:");
+      sb.append(this.nf.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerClose_args implements TBase, java.io.Serializable   {
+    public int id;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean id = false;
+    }
+
+    public scannerClose_args() {
+    }
+
+    public scannerClose_args(
+      int id)
+    {
+      this();
+      this.id = id;
+      this.__isset.id = 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.I32) {
+              this.id = iprot.readI32();
+              this.__isset.id = 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("scannerClose_args");
+      oprot.writeStructBegin(struct);
+      TField field = new TField();
+      field.name = "id";
+      field.type = TType.I32;
+      field.id = 1;
+      oprot.writeFieldBegin(field);
+      oprot.writeI32(this.id);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerClose_args(");
+      sb.append("id:");
+      sb.append(this.id);
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+  public static class scannerClose_result implements TBase, java.io.Serializable   {
+    public IOError io;
+    public IllegalArgument ia;
+
+    public final Isset __isset = new Isset();
+    public static final class Isset {
+      public boolean io = false;
+      public boolean ia = false;
+    }
+
+    public scannerClose_result() {
+    }
+
+    public scannerClose_result(
+      IOError io,
+      IllegalArgument ia)
+    {
+      this();
+      this.io = io;
+      this.__isset.io = true;
+      this.ia = ia;
+      this.__isset.ia = 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.ia = new IllegalArgument();
+              this.ia.read(iprot);
+              this.__isset.ia = 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("scannerClose_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.ia) {
+        if (this.ia != null) {
+          field.name = "ia";
+          field.type = TType.STRUCT;
+          field.id = 2;
+          oprot.writeFieldBegin(field);
+          this.ia.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerClose_result(");
+      sb.append("io:");
+      sb.append(this.io.toString());
+      sb.append(",ia:");
+      sb.append(this.ia.toString());
+      sb.append(")");
+      return sb.toString();
+    }
+
+  }
+
+}

+ 112 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/IOError.java

@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+/**
+ * An IOError exception signals that an error occurred communicating
+ * to the Hbase master or an Hbase region server.  Also used to return
+ * more general Hbase error conditions.
+ */
+public class IOError extends Exception implements TBase, java.io.Serializable {
+  public String message;
+
+  public final Isset __isset = new Isset();
+  public static final class Isset {
+    public boolean message = false;
+  }
+
+  public IOError() {
+  }
+
+  public IOError(
+    String message)
+  {
+    this();
+    this.message = message;
+    this.__isset.message = 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.message = iprot.readString();
+            this.__isset.message = 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("IOError");
+    oprot.writeStructBegin(struct);
+    TField field = new TField();
+    if (this.message != null) {
+      field.name = "message";
+      field.type = TType.STRING;
+      field.id = 1;
+      oprot.writeFieldBegin(field);
+      oprot.writeString(this.message);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("IOError(");
+    sb.append("message:");
+    sb.append(this.message);
+    sb.append(")");
+    return sb.toString();
+  }
+
+}
+

+ 111 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java

@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+/**
+ * An IllegalArgument exception indicates an illegal or invalid
+ * argument was passed into a procedure.
+ */
+public class IllegalArgument extends Exception implements TBase, java.io.Serializable {
+  public String message;
+
+  public final Isset __isset = new Isset();
+  public static final class Isset {
+    public boolean message = false;
+  }
+
+  public IllegalArgument() {
+  }
+
+  public IllegalArgument(
+    String message)
+  {
+    this();
+    this.message = message;
+    this.__isset.message = 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.message = iprot.readString();
+            this.__isset.message = 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("IllegalArgument");
+    oprot.writeStructBegin(struct);
+    TField field = new TField();
+    if (this.message != null) {
+      field.name = "message";
+      field.type = TType.STRING;
+      field.id = 1;
+      oprot.writeFieldBegin(field);
+      oprot.writeString(this.message);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("IllegalArgument(");
+    sb.append("message:");
+    sb.append(this.message);
+    sb.append(")");
+    return sb.toString();
+  }
+
+}
+

+ 156 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java

@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+/**
+ * A Mutation object is used to either update or delete a column-value.
+ */
+public class Mutation implements TBase, java.io.Serializable {
+  public boolean isDelete;
+  public byte[] column;
+  public byte[] value;
+
+  public final Isset __isset = new Isset();
+  public static final class Isset {
+    public boolean isDelete = false;
+    public boolean column = false;
+    public boolean value = false;
+  }
+
+  public Mutation() {
+    this.isDelete = false;
+
+  }
+
+  public Mutation(
+    boolean isDelete,
+    byte[] column,
+    byte[] value)
+  {
+    this();
+    this.isDelete = isDelete;
+    this.__isset.isDelete = true;
+    this.column = column;
+    this.__isset.column = true;
+    this.value = value;
+    this.__isset.value = 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.BOOL) {
+            this.isDelete = iprot.readBool();
+            this.__isset.isDelete = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2:
+          if (field.type == TType.STRING) {
+            this.column = iprot.readBinary();
+            this.__isset.column = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3:
+          if (field.type == TType.STRING) {
+            this.value = iprot.readBinary();
+            this.__isset.value = 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("Mutation");
+    oprot.writeStructBegin(struct);
+    TField field = new TField();
+    field.name = "isDelete";
+    field.type = TType.BOOL;
+    field.id = 1;
+    oprot.writeFieldBegin(field);
+    oprot.writeBool(this.isDelete);
+    oprot.writeFieldEnd();
+    if (this.column != null) {
+      field.name = "column";
+      field.type = TType.STRING;
+      field.id = 2;
+      oprot.writeFieldBegin(field);
+      oprot.writeBinary(this.column);
+      oprot.writeFieldEnd();
+    }
+    if (this.value != null) {
+      field.name = "value";
+      field.type = TType.STRING;
+      field.id = 3;
+      oprot.writeFieldBegin(field);
+      oprot.writeBinary(this.value);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Mutation(");
+    sb.append("isDelete:");
+    sb.append(this.isDelete);
+    sb.append(",column:");
+    sb.append(this.column);
+    sb.append(",value:");
+    sb.append(this.value);
+    sb.append(")");
+    return sb.toString();
+  }
+
+}
+

+ 111 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/NotFound.java

@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+/**
+ * A NotFound exception is used to indicate that no value was found
+ * for a query, or that a scanner has reached it's end.
+ */
+public class NotFound extends Exception implements TBase, java.io.Serializable {
+  public String message;
+
+  public final Isset __isset = new Isset();
+  public static final class Isset {
+    public boolean message = false;
+  }
+
+  public NotFound() {
+  }
+
+  public NotFound(
+    String message)
+  {
+    this();
+    this.message = message;
+    this.__isset.message = 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.message = iprot.readString();
+            this.__isset.message = 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("NotFound");
+    oprot.writeStructBegin(struct);
+    TField field = new TField();
+    if (this.message != null) {
+      field.name = "message";
+      field.type = TType.STRING;
+      field.id = 1;
+      oprot.writeFieldBegin(field);
+      oprot.writeString(this.message);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NotFound(");
+    sb.append("message:");
+    sb.append(this.message);
+    sb.append(")");
+    return sb.toString();
+  }
+
+}
+

+ 111 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/RegionDescriptor.java

@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+/**
+ * A RegionDescriptor contains informationa about an HTable region.
+ * Currently, this is just the startKey of the region.
+ */
+public class RegionDescriptor implements TBase, java.io.Serializable {
+  public byte[] startKey;
+
+  public final Isset __isset = new Isset();
+  public static final class Isset {
+    public boolean startKey = false;
+  }
+
+  public RegionDescriptor() {
+  }
+
+  public RegionDescriptor(
+    byte[] startKey)
+  {
+    this();
+    this.startKey = startKey;
+    this.__isset.startKey = 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.startKey = iprot.readBinary();
+            this.__isset.startKey = 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("RegionDescriptor");
+    oprot.writeStructBegin(struct);
+    TField field = new TField();
+    if (this.startKey != null) {
+      field.name = "startKey";
+      field.type = TType.STRING;
+      field.id = 1;
+      oprot.writeFieldBegin(field);
+      oprot.writeBinary(this.startKey);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("RegionDescriptor(");
+    sb.append("startKey:");
+    sb.append(this.startKey);
+    sb.append(")");
+    return sb.toString();
+  }
+
+}
+

+ 153 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/generated/ScanEntry.java

@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package org.apache.hadoop.hbase.thrift.generated;
+
+import java.util.ArrayList;
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.HashSet;
+import com.facebook.thrift.*;
+
+import com.facebook.thrift.protocol.*;
+import com.facebook.thrift.transport.*;
+
+/**
+ * A ScanEntry contains the row, column, and value information for a scanner's
+ * current location.
+ */
+public class ScanEntry implements TBase, java.io.Serializable {
+  public byte[] row;
+  public AbstractMap<byte[],byte[]> columns;
+
+  public final Isset __isset = new Isset();
+  public static final class Isset {
+    public boolean row = false;
+    public boolean columns = false;
+  }
+
+  public ScanEntry() {
+  }
+
+  public ScanEntry(
+    byte[] row,
+    AbstractMap<byte[],byte[]> columns)
+  {
+    this();
+    this.row = row;
+    this.__isset.row = true;
+    this.columns = columns;
+    this.__isset.columns = 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.row = iprot.readBinary();
+            this.__isset.row = true;
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2:
+          if (field.type == TType.MAP) {
+            {
+              TMap _map0 = iprot.readMapBegin();
+              this.columns = new HashMap<byte[],byte[]>(2*_map0.size);
+              for (int _i1 = 0; _i1 < _map0.size; ++_i1)
+              {
+                byte[] _key2;
+                byte[] _val3;
+                _key2 = iprot.readBinary();
+                _val3 = iprot.readBinary();
+                this.columns.put(_key2, _val3);
+              }
+              iprot.readMapEnd();
+            }
+            this.__isset.columns = 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("ScanEntry");
+    oprot.writeStructBegin(struct);
+    TField field = new TField();
+    if (this.row != null) {
+      field.name = "row";
+      field.type = TType.STRING;
+      field.id = 1;
+      oprot.writeFieldBegin(field);
+      oprot.writeBinary(this.row);
+      oprot.writeFieldEnd();
+    }
+    if (this.columns != null) {
+      field.name = "columns";
+      field.type = TType.MAP;
+      field.id = 2;
+      oprot.writeFieldBegin(field);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.columns.size()));
+        for (byte[] _iter4 : this.columns.keySet())        {
+          oprot.writeBinary(_iter4);
+          oprot.writeBinary(this.columns.get(_iter4));
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ScanEntry(");
+    sb.append("row:");
+    sb.append(this.row);
+    sb.append(",columns:");
+    sb.append(this.columns);
+    sb.append(")");
+    return sb.toString();
+  }
+
+}
+

+ 58 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/thrift/package.html

@@ -0,0 +1,58 @@
+<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
+<html>
+<head />
+<body bgcolor="white">
+Provides an HBase <a href="http://developers.facebook.com/thrift/">Thrift</a>
+service.
+
+This directory contains a Thrift interface definition file for an Hbase RPC
+service and a Java server implementation.
+
+<h2><a name="whatisthrift">What is Thrift?</a></h2> 
+
+<p>"Thrift is a software framework for scalable cross-language services
+development. It combines a powerful software stack with a code generation
+engine to build services that work efficiently and seamlessly between C++,
+Java, Python, PHP, and Ruby. Thrift was developed at Facebook, and we are now
+releasing it as open source."  For additional information, see
+http://developers.facebook.com/thrift/.  Facebook has announced their intent
+to migrate Thrift into Apache Incubator.
+</p>
+
+<h2><a name="description">Description</a></h2>
+
+<p>The <a href="generated/Hbase.Iface.html">Hbase API</a> is defined in the
+file Hbase.thrift.  A server-side implementation of the API is in {@link
+org.apache.hadoop.hbase.thrift.ThriftServer}.  The generated interfaces,
+types, and RPC utility files are checked into SVN under the {@link
+org.apache.hadoop.hbase.thrift.generated} directory.
+
+</p>
+
+<p>The files were generated by running the commands:
+<pre>
+  thrift -strict -java Hbase.thrift
+  mv gen-java/org/apache/hadoop/hbase/thrift/generated .
+  rm -rf gen-java
+</pre>
+</p>
+
+<p>The 'thrift' binary is the Thrift compiler, and it is distributed as a part
+of
+the Thrift package.  Additionally, specific language runtime libraries are a
+part of the Thrift package.  A version of the Java runtime is checked into SVN
+under the hbase/lib directory.
+</p>
+
+<p>The version of Thrift used to generate the Java files is revision 746 from
+the <a href="http://svn.facebook.com/svnroot/thrift/">SVN repository</a>.</p>
+
+<p>The ThriftServer is run like:
+<pre>
+
+  ./bin/hbase thrift [-h|--help] [-p|--port PORT]
+</pre>
+The default port is 9090.
+</p>
+</body>
+</html>