diff --git a/CHANGES.txt b/CHANGES.txt
index c7a2bfdf17f..8c6b05a3dd3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -26,6 +26,7 @@ Trunk (unreleased changes)
(Bryan Duxbury via Stack)
HADOOP-2240 Truncate for hbase (Edward Yoon via Stack)
HADOOP-2389 Provide multiple language bindings for HBase (Thrift)
+ (David Simpson via Stack)
OPTIMIZATIONS
HADOOP-2479 Save on number of Text object creations
diff --git a/bin/hbase b/bin/hbase
index fec72d54e18..62a9299d97c 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -58,9 +58,11 @@ esac
if [ $# = 0 ]; then
echo "Usage: hbase [--hadoop=hadoopdir] "
echo "where 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 " CLASSNAME run the class named CLASSNAME"
echo "Most commands print help when invoked w/o parameters."
@@ -213,6 +215,10 @@ elif [ "$COMMAND" = "master" ] ; then
CLASS='org.apache.hadoop.hbase.HMaster'
elif [ "$COMMAND" = "regionserver" ] ; then
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
CLASS=$COMMAND
fi
diff --git a/src/examples/thrift/DemoClient.cpp b/src/examples/thrift/DemoClient.cpp
new file mode 100644
index 00000000000..8cf5ffd3bcb
--- /dev/null
+++ b/src/examples/thrift/DemoClient.cpp
@@ -0,0 +1,238 @@
+#include
+#include
+#include
+#include
+
+#include
+
+#include
+#include
+#include
+
+#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 StrVec;
+typedef std::map StrMap;
+typedef std::vector ColVec;
+typedef std::map 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 socket(new TSocket("localhost", 9090));
+ boost::shared_ptr transport(new TBufferedTransport(socket));
+ boost::shared_ptr 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 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(i));
+ client.put(t, row, "entry:sqr", boost::lexical_cast(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());
+ }
+
+}
diff --git a/src/examples/thrift/DemoClient.java b/src/examples/thrift/DemoClient.java
new file mode 100644
index 00000000000..c77002d86c1
--- /dev/null
+++ b/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 columns = new ArrayList();
+ 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 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 columnNames = new ArrayList();
+ 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 mutations = new ArrayList();
+ 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 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 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 values) {
+ // copy values into a TreeMap to get them in sorted order
+
+ TreeMap sorted = new TreeMap();
+ for (AbstractMap.Entry entry : values.entrySet()) {
+ sorted.put(utf8(entry.getKey()), entry.getValue());
+ }
+
+ StringBuilder rowStr = new StringBuilder();
+ for (SortedMap.Entry entry : sorted.entrySet()) {
+ rowStr.append(entry.getKey());
+ rowStr.append(" => ");
+ rowStr.append(utf8(entry.getValue()));
+ rowStr.append("; ");
+ }
+ System.out.println("row: " + utf8(row) + ", cols: " + rowStr);
+ }
+}
diff --git a/src/examples/thrift/DemoClient.rb b/src/examples/thrift/DemoClient.rb
new file mode 100644
index 00000000000..b86d9e7f1b0
--- /dev/null
+++ b/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()
diff --git a/src/examples/thrift/Makefile b/src/examples/thrift/Makefile
new file mode 100644
index 00000000000..4cc9d4231d4
--- /dev/null
+++ b/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
diff --git a/src/examples/thrift/README.txt b/src/examples/thrift/README.txt
new file mode 100644
index 00000000000..2edbeaf5744
--- /dev/null
+++ b/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
+
+
diff --git a/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java b/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
index 5d7611ab49f..ea80e1eb043 100644
--- a/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
+++ b/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
@@ -152,6 +152,18 @@ public class BloomFilterDescriptor implements WritableComparable {
return value.toString();
}
+ public BloomFilterType getType() {
+ return filterType;
+ }
+
+ public int getVectorSize() {
+ return vectorSize;
+ }
+
+ public int getNbHash() {
+ return nbHash;
+ }
+
/** {@inheritDoc} */
@Override
public boolean equals(Object obj) {
diff --git a/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift b/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift
new file mode 100644
index 00000000000..c34070919cd
--- /dev/null
+++ b/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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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)
+}
diff --git a/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java b/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
new file mode 100644
index 00000000000..f217840b86f
--- /dev/null
+++ b/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 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();
+ }
+
+ /**
+ * 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 getTableNames() throws IOError {
+ LOG.debug("getTableNames");
+ try {
+ HTableDescriptor[] tables = this.admin.listTables();
+ ArrayList list = new ArrayList(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 getTableRegions(byte[] tableName)
+ throws IOError {
+ LOG.debug("getTableRegions: " + new String(tableName));
+ try {
+ HTable table = getTable(tableName);
+ Text[] startKeys = table.getStartKeys();
+ ArrayList regions = new ArrayList();
+ 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 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(Arrays.asList(values));
+ } catch (IOException e) {
+ throw new IOError(e.getMessage());
+ }
+ }
+
+ public ArrayList 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(Arrays.asList(values));
+ } catch (IOException e) {
+ throw new IOError(e.getMessage());
+ }
+ }
+
+ public AbstractMap getRow(byte[] tableName, byte[] row)
+ throws IOError {
+ return getRowTs(tableName, row, HConstants.LATEST_TIMESTAMP);
+ }
+
+ public AbstractMap 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 values = table.getRow(getText(row), timestamp);
+ // copy the map from type to
+ HashMap returnValues = new HashMap();
+ for (Entry 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 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 mutations) throws IOError {
+ mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP);
+ }
+
+ public void mutateRowTs(byte[] tableName, byte[] row,
+ ArrayList 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 results = new TreeMap();
+
+ 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(results.size());
+
+ for (SortedMap.Entry e : results.entrySet()) {
+ retval.columns.put(e.getKey().getBytes(), e.getValue());
+ }
+ return retval;
+ }
+
+ public int scannerOpen(byte[] tableName, byte[] startRow,
+ ArrayList 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 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 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 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 getColumnDescriptors(
+ byte[] tableName) throws IOError, TException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("getColumnDescriptors: table=" + new String(tableName));
+ }
+ try {
+ HashMap columns = new HashMap();
+
+ HTable table = getTable(tableName);
+ HTableDescriptor desc = table.getMetadata();
+
+ for (Entry 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.");
+ }
+}
diff --git a/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java b/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
new file mode 100644
index 00000000000..9d1bfb32993
--- /dev/null
+++ b/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;
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java b/src/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
new file mode 100644
index 00000000000..c1be39f4c9f
--- /dev/null
+++ b/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();
+ }
+
+}
+
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java b/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
new file mode 100644
index 00000000000..a77d2c0bf3d
--- /dev/null
+++ b/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();
+ }
+
+}
+
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/Constants.java b/src/java/org/apache/hadoop/hbase/thrift/generated/Constants.java
new file mode 100644
index 00000000000..bee2177e556
--- /dev/null
+++ b/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 {
+
+}
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java b/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
new file mode 100644
index 00000000000..3368f2d18e0
--- /dev/null
+++ b/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 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 getColumnDescriptors(byte[] tableName) throws IOError, TException;
+
+ /**
+ * List the regions associated with a table.
+ * @param tableName table name
+ * @return list of region descriptors
+ */
+ public ArrayList 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 columnFamilies) throws IOError, IllegalArgument, AlreadyExists, TException
+ {
+ send_createTable(tableName, columnFamilies);
+ recv_createTable();
+ }
+
+ public void send_createTable(byte[] tableName, ArrayList 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 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 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 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 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 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 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 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 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 mutations) throws IOError, TException
+ {
+ send_mutateRow(tableName, row, mutations);
+ recv_mutateRow();
+ }
+
+ public void send_mutateRow(byte[] tableName, byte[] row, ArrayList 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 mutations, long timestamp) throws IOError, TException
+ {
+ send_mutateRowTs(tableName, row, mutations, timestamp);
+ recv_mutateRowTs();
+ }
+
+ public void send_mutateRowTs(byte[] tableName, byte[] row, ArrayList 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 columns) throws IOError, TException
+ {
+ send_scannerOpen(tableName, startRow, columns);
+ return recv_scannerOpen();
+ }
+
+ public void send_scannerOpen(byte[] tableName, byte[] startRow, ArrayList 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 columns) throws IOError, TException
+ {
+ send_scannerOpenWithStop(tableName, startRow, stopRow, columns);
+ return recv_scannerOpenWithStop();
+ }
+
+ public void send_scannerOpenWithStop(byte[] tableName, byte[] startRow, byte[] stopRow, ArrayList 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 columns, long timestamp) throws IOError, TException
+ {
+ send_scannerOpenTs(tableName, startRow, columns, timestamp);
+ return recv_scannerOpenTs();
+ }
+
+ public void send_scannerOpenTs(byte[] tableName, byte[] startRow, ArrayList 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 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 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 processMap_ = new HashMap();
+
+ 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 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 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(_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 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 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(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 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 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(_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 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 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(_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 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 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(_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 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 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(_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 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 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(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 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 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(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 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 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(_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 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 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(_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 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 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(_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 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 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(_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 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 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(_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 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 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(_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();
+ }
+
+ }
+
+}
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/IOError.java b/src/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
new file mode 100644
index 00000000000..d1c97b5c100
--- /dev/null
+++ b/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();
+ }
+
+}
+
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java b/src/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
new file mode 100644
index 00000000000..d3e23c8d97a
--- /dev/null
+++ b/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();
+ }
+
+}
+
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java b/src/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
new file mode 100644
index 00000000000..2ac03b323dc
--- /dev/null
+++ b/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();
+ }
+
+}
+
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/NotFound.java b/src/java/org/apache/hadoop/hbase/thrift/generated/NotFound.java
new file mode 100644
index 00000000000..cc03bb5abf4
--- /dev/null
+++ b/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();
+ }
+
+}
+
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/RegionDescriptor.java b/src/java/org/apache/hadoop/hbase/thrift/generated/RegionDescriptor.java
new file mode 100644
index 00000000000..1e5831dafd0
--- /dev/null
+++ b/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();
+ }
+
+}
+
diff --git a/src/java/org/apache/hadoop/hbase/thrift/generated/ScanEntry.java b/src/java/org/apache/hadoop/hbase/thrift/generated/ScanEntry.java
new file mode 100644
index 00000000000..78ed0a3e9ad
--- /dev/null
+++ b/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 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 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(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();
+ }
+
+}
+
diff --git a/src/java/org/apache/hadoop/hbase/thrift/package.html b/src/java/org/apache/hadoop/hbase/thrift/package.html
new file mode 100644
index 00000000000..54045a3fcaf
--- /dev/null
+++ b/src/java/org/apache/hadoop/hbase/thrift/package.html
@@ -0,0 +1,58 @@
+
+
+
+
+Provides an HBase Thrift
+service.
+
+This directory contains a Thrift interface definition file for an Hbase RPC
+service and a Java server implementation.
+
+
+
+"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.
+
+
+
+
+The Hbase API 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.
+
+
+
+The files were generated by running the commands:
+
+ thrift -strict -java Hbase.thrift
+ mv gen-java/org/apache/hadoop/hbase/thrift/generated .
+ rm -rf gen-java
+
+
+
+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.
+
+
+The version of Thrift used to generate the Java files is revision 746 from
+the SVN repository.
+
+The ThriftServer is run like:
+
+
+ ./bin/hbase thrift [-h|--help] [-p|--port PORT]
+
+The default port is 9090.
+
+
+