HADOOP-2389 Provide multiple language bindings for HBase (Thrift)
Actual code (first commit added thrift lib so this patch would build). git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@611535 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9c142a76a1
commit
470b3a3200
|
@ -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
|
||||
|
|
12
bin/hbase
12
bin/hbase
|
@ -58,9 +58,11 @@ esac
|
|||
if [ $# = 0 ]; then
|
||||
echo "Usage: hbase [--hadoop=hadoopdir] <command>"
|
||||
echo "where <command> is one of:"
|
||||
echo " shell run the hbase shell"
|
||||
echo " master run a hbase HMaster node"
|
||||
echo " regionserver run a hbase HRegionServer node"
|
||||
echo " shell run the Hbase shell"
|
||||
echo " master run an Hbase HMaster node"
|
||||
echo " regionserver run an Hbase HRegionServer node"
|
||||
echo " rest run an Hbase REST server"
|
||||
echo " thrift run an Hbase Thrift server"
|
||||
echo " or"
|
||||
echo " 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
|
||||
|
|
|
@ -0,0 +1,238 @@
|
|||
#include <stdio.h>
|
||||
#include <unistd.h>
|
||||
#include <sys/time.h>
|
||||
#include <poll.h>
|
||||
|
||||
#include <iostream>
|
||||
|
||||
#include <protocol/TBinaryProtocol.h>
|
||||
#include <transport/TSocket.h>
|
||||
#include <transport/TTransportUtils.h>
|
||||
|
||||
#include "Hbase.h"
|
||||
|
||||
using namespace facebook::thrift;
|
||||
using namespace facebook::thrift::protocol;
|
||||
using namespace facebook::thrift::transport;
|
||||
|
||||
using namespace apache::hadoop::hbase::thrift;
|
||||
|
||||
typedef std::vector<std::string> StrVec;
|
||||
typedef std::map<std::string,std::string> StrMap;
|
||||
typedef std::vector<ColumnDescriptor> ColVec;
|
||||
typedef std::map<std::string,ColumnDescriptor> ColMap;
|
||||
|
||||
|
||||
static void
|
||||
printRow(const std::string &row, const StrMap &columns)
|
||||
{
|
||||
std::cout << "row: " << row << ", cols: ";
|
||||
for (StrMap::const_iterator it = columns.begin(); it != columns.end(); ++it) {
|
||||
std::cout << it->first << " => " << it->second << "; ";
|
||||
}
|
||||
std::cout << std::endl;
|
||||
}
|
||||
|
||||
static void
|
||||
printEntry(const ScanEntry &entry)
|
||||
{
|
||||
printRow(entry.row, entry.columns);
|
||||
}
|
||||
|
||||
static void
|
||||
printVersions(const std::string &row, const StrVec &versions)
|
||||
{
|
||||
std::cout << "row: " << row << ", values: ";
|
||||
for (StrVec::const_iterator it = versions.begin(); it != versions.end(); ++it) {
|
||||
std::cout << *it << "; ";
|
||||
}
|
||||
std::cout << std::endl;
|
||||
}
|
||||
|
||||
int
|
||||
main(int argc, char** argv)
|
||||
{
|
||||
boost::shared_ptr<TTransport> socket(new TSocket("localhost", 9090));
|
||||
boost::shared_ptr<TTransport> transport(new TBufferedTransport(socket));
|
||||
boost::shared_ptr<TProtocol> protocol(new TBinaryProtocol(transport));
|
||||
HbaseClient client(protocol);
|
||||
|
||||
try {
|
||||
transport->open();
|
||||
|
||||
std::string t("demo_table");
|
||||
|
||||
//
|
||||
// Scan all tables, look for the demo table and delete it.
|
||||
//
|
||||
std::cout << "scanning tables..." << std::endl;
|
||||
StrVec tables;
|
||||
client.getTableNames(tables);
|
||||
for (StrVec::const_iterator it = tables.begin(); it != tables.end(); ++it) {
|
||||
std::cout << " found: " << *it << std::endl;
|
||||
if (t == *it) {
|
||||
std::cout << " deleting table: " << *it << std::endl;
|
||||
client.deleteTable(*it);
|
||||
}
|
||||
}
|
||||
|
||||
//
|
||||
// Create the demo table with two column families, entry: and unused:
|
||||
//
|
||||
ColVec columns;
|
||||
columns.push_back(ColumnDescriptor());
|
||||
columns.back().name = "entry:";
|
||||
columns.back().maxVersions = 10;
|
||||
columns.push_back(ColumnDescriptor());
|
||||
columns.back().name = "unused:";
|
||||
|
||||
std::cout << "creating table: " << t << std::endl;
|
||||
try {
|
||||
client.createTable(t, columns);
|
||||
} catch (AlreadyExists &ae) {
|
||||
std::cout << "WARN: " << ae.message << std::endl;
|
||||
}
|
||||
|
||||
ColMap columnMap;
|
||||
client.getColumnDescriptors(columnMap, t);
|
||||
std::cout << "column families in " << t << ": " << std::endl;
|
||||
for (ColMap::const_iterator it = columnMap.begin(); it != columnMap.end(); ++it) {
|
||||
std::cout << " column: " << it->second.name << ", maxVer: " << it->second.maxVersions << std::endl;
|
||||
}
|
||||
|
||||
//
|
||||
// Test UTF-8 handling
|
||||
//
|
||||
std::string invalid("foo-\xfc\xa1\xa1\xa1\xa1\xa1");
|
||||
std::string valid("foo-\xE7\x94\x9F\xE3\x83\x93\xE3\x83\xBC\xE3\x83\xAB");
|
||||
|
||||
// non-utf8 is fine for data
|
||||
client.put(t, "foo", "entry:foo", invalid);
|
||||
|
||||
// try empty strings
|
||||
client.put(t, "", "entry:", "");
|
||||
|
||||
// this row name is valid utf8
|
||||
client.put(t, valid, "entry:foo", valid);
|
||||
|
||||
// non-utf8 is not allowed in row names
|
||||
try {
|
||||
client.put(t, invalid, "entry:foo", invalid);
|
||||
std::cout << "FATAL: shouldn't get here!" << std::endl;
|
||||
exit(-1);
|
||||
} catch (IOError e) {
|
||||
std::cout << "expected error: " << e.message << std::endl;
|
||||
}
|
||||
|
||||
// Run a scanner on the rows we just created
|
||||
StrVec columnNames;
|
||||
columnNames.push_back("entry:");
|
||||
|
||||
std::cout << "Starting scanner..." << std::endl;
|
||||
int scanner = client.scannerOpen(t, "", columnNames);
|
||||
try {
|
||||
while (true) {
|
||||
ScanEntry value;
|
||||
client.scannerGet(value, scanner);
|
||||
printEntry(value);
|
||||
}
|
||||
} catch (NotFound &nf) {
|
||||
client.scannerClose(scanner);
|
||||
std::cout << "Scanner finished" << std::endl;
|
||||
}
|
||||
|
||||
//
|
||||
// Run some operations on a bunch of rows.
|
||||
//
|
||||
for (int i = 100; i >= 0; --i) {
|
||||
// format row keys as "00000" to "00100"
|
||||
char buf[32];
|
||||
sprintf(buf, "%0.5d", i);
|
||||
std::string row(buf);
|
||||
|
||||
StrMap values;
|
||||
|
||||
client.put(t, row, "unused:", "DELETE_ME");
|
||||
client.getRow(values, t, row);
|
||||
printRow(row, values);
|
||||
client.deleteAllRow(t, row);
|
||||
|
||||
client.put(t, row, "entry:num", "0");
|
||||
client.put(t, row, "entry:foo", "FOO");
|
||||
client.getRow(values, t, row);
|
||||
printRow(row, values);
|
||||
|
||||
// sleep to force later timestamp
|
||||
poll(0, 0, 50);
|
||||
|
||||
std::vector<Mutation> mutations;
|
||||
mutations.push_back(Mutation());
|
||||
mutations.back().column = "entry:foo";
|
||||
mutations.back().isDelete = true;
|
||||
mutations.push_back(Mutation());
|
||||
mutations.back().column = "entry:num";
|
||||
mutations.back().value = "-1";
|
||||
client.mutateRow(t, row, mutations);
|
||||
client.getRow(values, t, row);
|
||||
printRow(row, values);
|
||||
|
||||
client.put(t, row, "entry:num", boost::lexical_cast<std::string>(i));
|
||||
client.put(t, row, "entry:sqr", boost::lexical_cast<std::string>(i*i));
|
||||
client.getRow(values, t, row);
|
||||
printRow(row, values);
|
||||
|
||||
mutations.clear();
|
||||
mutations.push_back(Mutation());
|
||||
mutations.back().column = "entry:num";
|
||||
mutations.back().value = "-999";
|
||||
mutations.push_back(Mutation());
|
||||
mutations.back().column = "entry:sqr";
|
||||
mutations.back().isDelete = true;
|
||||
client.mutateRowTs(t, row, mutations, 1); // shouldn't override latest
|
||||
client.getRow(values, t, row);
|
||||
printRow(row, values);
|
||||
|
||||
StrVec versions;
|
||||
client.getVer(versions, t, row, "entry:num", 10);
|
||||
printVersions(row, versions);
|
||||
assert(versions.size() == 4);
|
||||
std::cout << std::endl;
|
||||
|
||||
try {
|
||||
std::string value;
|
||||
client.get(value, t, row, "entry:foo");
|
||||
std::cout << "FATAL: shouldn't get here!" << std::endl;
|
||||
exit(-1);
|
||||
} catch (NotFound &nf) {
|
||||
// blank
|
||||
}
|
||||
}
|
||||
|
||||
// scan all rows/columns
|
||||
|
||||
columnNames.clear();
|
||||
client.getColumnDescriptors(columnMap, t);
|
||||
for (ColMap::const_iterator it = columnMap.begin(); it != columnMap.end(); ++it) {
|
||||
columnNames.push_back(it->first);
|
||||
}
|
||||
|
||||
std::cout << "Starting scanner..." << std::endl;
|
||||
scanner = client.scannerOpenWithStop(t, "00020", "00040", columnNames);
|
||||
try {
|
||||
while (true) {
|
||||
ScanEntry value;
|
||||
client.scannerGet(value, scanner);
|
||||
printEntry(value);
|
||||
}
|
||||
} catch (NotFound &nf) {
|
||||
client.scannerClose(scanner);
|
||||
std::cout << "Scanner finished" << std::endl;
|
||||
}
|
||||
|
||||
transport->close();
|
||||
}
|
||||
catch (TException &tx) {
|
||||
printf("ERROR: %s\n", tx.what());
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,276 @@
|
|||
package org.apache.hadoop.hbase.thrift;
|
||||
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.CharacterCodingException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.CharsetDecoder;
|
||||
import java.text.NumberFormat;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.ArrayList;
|
||||
import java.util.TreeMap;
|
||||
import java.util.SortedMap;
|
||||
|
||||
import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
|
||||
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift.generated.Hbase;
|
||||
import org.apache.hadoop.hbase.thrift.generated.IOError;
|
||||
import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
|
||||
import org.apache.hadoop.hbase.thrift.generated.Mutation;
|
||||
import org.apache.hadoop.hbase.thrift.generated.NotFound;
|
||||
import org.apache.hadoop.hbase.thrift.generated.ScanEntry;
|
||||
import com.facebook.thrift.TException;
|
||||
import com.facebook.thrift.protocol.TBinaryProtocol;
|
||||
import com.facebook.thrift.protocol.TProtocol;
|
||||
import com.facebook.thrift.transport.TSocket;
|
||||
import com.facebook.thrift.transport.TTransport;
|
||||
|
||||
public class DemoClient {
|
||||
|
||||
protected int port = 9090;
|
||||
CharsetDecoder decoder = null;
|
||||
|
||||
public static void main(String[] args)
|
||||
throws IOError, TException, NotFound, UnsupportedEncodingException, IllegalArgument, AlreadyExists
|
||||
{
|
||||
DemoClient client = new DemoClient();
|
||||
client.run();
|
||||
}
|
||||
|
||||
DemoClient() {
|
||||
decoder = Charset.forName("UTF-8").newDecoder();
|
||||
}
|
||||
|
||||
// Helper to translate byte[]'s to UTF8 strings
|
||||
private String utf8(byte[] buf) {
|
||||
try {
|
||||
return decoder.decode(ByteBuffer.wrap(buf)).toString();
|
||||
} catch (CharacterCodingException e) {
|
||||
return "[INVALID UTF-8]";
|
||||
}
|
||||
}
|
||||
|
||||
// Helper to translate strings to UTF8 bytes
|
||||
private byte[] bytes(String s) {
|
||||
try {
|
||||
return s.getBytes("UTF-8");
|
||||
} catch (UnsupportedEncodingException e) {
|
||||
e.printStackTrace();
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private void run() throws IOError, TException, NotFound, IllegalArgument,
|
||||
AlreadyExists {
|
||||
TTransport transport = new TSocket("localhost", port);
|
||||
TProtocol protocol = new TBinaryProtocol(transport, true, true);
|
||||
Hbase.Client client = new Hbase.Client(protocol);
|
||||
|
||||
transport.open();
|
||||
|
||||
byte[] t = bytes("demo_table");
|
||||
|
||||
//
|
||||
// Scan all tables, look for the demo table and delete it.
|
||||
//
|
||||
System.out.println("scanning tables...");
|
||||
for (byte[] name : client.getTableNames()) {
|
||||
System.out.println(" found: " + utf8(name));
|
||||
if (utf8(name).equals(utf8(t))) {
|
||||
System.out.println(" deleting table: " + utf8(name));
|
||||
client.deleteTable(name);
|
||||
}
|
||||
}
|
||||
|
||||
//
|
||||
// Create the demo table with two column families, entry: and unused:
|
||||
//
|
||||
ArrayList<ColumnDescriptor> columns = new ArrayList<ColumnDescriptor>();
|
||||
ColumnDescriptor col = null;
|
||||
col = new ColumnDescriptor();
|
||||
col.name = bytes("entry:");
|
||||
col.maxVersions = 10;
|
||||
columns.add(col);
|
||||
col = new ColumnDescriptor();
|
||||
col.name = bytes("unused:");
|
||||
columns.add(col);
|
||||
|
||||
System.out.println("creating table: " + utf8(t));
|
||||
try {
|
||||
client.createTable(t, columns);
|
||||
} catch (AlreadyExists ae) {
|
||||
System.out.println("WARN: " + ae.message);
|
||||
}
|
||||
|
||||
System.out.println("column families in " + utf8(t) + ": ");
|
||||
AbstractMap<byte[], ColumnDescriptor> columnMap = client.getColumnDescriptors(t);
|
||||
for (ColumnDescriptor col2 : columnMap.values()) {
|
||||
System.out.println(" column: " + utf8(col2.name) + ", maxVer: " + Integer.toString(col2.maxVersions));
|
||||
}
|
||||
|
||||
//
|
||||
// Test UTF-8 handling
|
||||
//
|
||||
byte[] invalid = { (byte) 'f', (byte) 'o', (byte) 'o', (byte) '-', (byte) 0xfc, (byte) 0xa1, (byte) 0xa1, (byte) 0xa1, (byte) 0xa1 };
|
||||
byte[] valid = { (byte) 'f', (byte) 'o', (byte) 'o', (byte) '-', (byte) 0xE7, (byte) 0x94, (byte) 0x9F, (byte) 0xE3, (byte) 0x83, (byte) 0x93, (byte) 0xE3, (byte) 0x83, (byte) 0xBC, (byte) 0xE3, (byte) 0x83, (byte) 0xAB};
|
||||
|
||||
// non-utf8 is fine for data
|
||||
client.put(t, bytes("foo"), bytes("entry:foo"), invalid);
|
||||
|
||||
// try empty strings
|
||||
client.put(t, bytes(""), bytes("entry:"), bytes(""));
|
||||
|
||||
// this row name is valid utf8
|
||||
client.put(t, valid, bytes("entry:foo"), valid);
|
||||
|
||||
// non-utf8 is not allowed in row names
|
||||
try {
|
||||
client.put(t, invalid, bytes("entry:foo"), invalid);
|
||||
System.out.println("FATAL: shouldn't get here");
|
||||
System.exit(-1);
|
||||
} catch (IOError e) {
|
||||
System.out.println("expected error: " + e.message);
|
||||
}
|
||||
|
||||
// Run a scanner on the rows we just created
|
||||
ArrayList<byte[]> columnNames = new ArrayList<byte[]>();
|
||||
columnNames.add(bytes("entry:"));
|
||||
|
||||
System.out.println("Starting scanner...");
|
||||
int scanner = client
|
||||
.scannerOpen(t, bytes(""), columnNames);
|
||||
try {
|
||||
while (true) {
|
||||
ScanEntry value = client.scannerGet(scanner);
|
||||
printEntry(value);
|
||||
}
|
||||
} catch (NotFound nf) {
|
||||
client.scannerClose(scanner);
|
||||
System.out.println("Scanner finished");
|
||||
}
|
||||
|
||||
//
|
||||
// Run some operations on a bunch of rows
|
||||
//
|
||||
for (int i = 100; i >= 0; --i) {
|
||||
// format row keys as "00000" to "00100"
|
||||
NumberFormat nf = NumberFormat.getInstance();
|
||||
nf.setMinimumIntegerDigits(5);
|
||||
nf.setGroupingUsed(false);
|
||||
byte[] row = bytes(nf.format(i));
|
||||
|
||||
client.put(t, row, bytes("unused:"), bytes("DELETE_ME"));
|
||||
printRow(row, client.getRow(t, row));
|
||||
client.deleteAllRow(t, row);
|
||||
|
||||
client.put(t, row, bytes("entry:num"), bytes("0"));
|
||||
client.put(t, row, bytes("entry:foo"), bytes("FOO"));
|
||||
printRow(row, client.getRow(t, row));
|
||||
|
||||
Mutation m = null;
|
||||
ArrayList<Mutation> mutations = new ArrayList<Mutation>();
|
||||
m = new Mutation();
|
||||
m.column = bytes("entry:foo");
|
||||
m.isDelete = true;
|
||||
mutations.add(m);
|
||||
m = new Mutation();
|
||||
m.column = bytes("entry:num");
|
||||
m.value = bytes("-1");
|
||||
mutations.add(m);
|
||||
client.mutateRow(t, row, mutations);
|
||||
printRow(row, client.getRow(t, row));
|
||||
|
||||
client.put(t, row, bytes("entry:num"), bytes(Integer.toString(i)));
|
||||
client.put(t, row, bytes("entry:sqr"), bytes(Integer.toString(i * i)));
|
||||
printRow(row, client.getRow(t, row));
|
||||
|
||||
// sleep to force later timestamp
|
||||
try {
|
||||
Thread.sleep(50);
|
||||
} catch (InterruptedException e) {
|
||||
// no-op
|
||||
}
|
||||
|
||||
mutations.clear();
|
||||
m = new Mutation();
|
||||
m.column = bytes("entry:num");
|
||||
m.value = bytes("-999");
|
||||
mutations.add(m);
|
||||
m = new Mutation();
|
||||
m.column = bytes("entry:sqr");
|
||||
m.isDelete = true;
|
||||
client.mutateRowTs(t, row, mutations, 1); // shouldn't override latest
|
||||
printRow(row, client.getRow(t, row));
|
||||
|
||||
ArrayList<byte[]> versions = client.getVer(t, row, bytes("entry:num"), 10);
|
||||
printVersions(row, versions);
|
||||
if (versions.size() != 4) {
|
||||
System.out.println("FATAL: wrong # of versions");
|
||||
System.exit(-1);
|
||||
}
|
||||
|
||||
try {
|
||||
client.get(t, row, bytes("entry:foo"));
|
||||
System.out.println("FATAL: shouldn't get here");
|
||||
System.exit(-1);
|
||||
} catch (NotFound nf2) {
|
||||
// blank
|
||||
}
|
||||
|
||||
System.out.println("");
|
||||
}
|
||||
|
||||
// scan all rows/columnNames
|
||||
|
||||
columnNames.clear();
|
||||
for (ColumnDescriptor col2 : client.getColumnDescriptors(t).values()) {
|
||||
columnNames.add(col2.name);
|
||||
}
|
||||
|
||||
System.out.println("Starting scanner...");
|
||||
scanner = client.scannerOpenWithStop(t, bytes("00020"), bytes("00040"),
|
||||
columnNames);
|
||||
try {
|
||||
while (true) {
|
||||
ScanEntry value = client.scannerGet(scanner);
|
||||
printEntry(value);
|
||||
}
|
||||
} catch (NotFound nf) {
|
||||
client.scannerClose(scanner);
|
||||
System.out.println("Scanner finished");
|
||||
}
|
||||
|
||||
transport.close();
|
||||
}
|
||||
|
||||
private final void printVersions(byte[] row, ArrayList<byte[]> values) {
|
||||
StringBuilder rowStr = new StringBuilder();
|
||||
for (byte[] value : values) {
|
||||
rowStr.append(utf8(value));
|
||||
rowStr.append("; ");
|
||||
}
|
||||
System.out.println("row: " + utf8(row) + ", values: " + rowStr);
|
||||
}
|
||||
|
||||
private final void printEntry(ScanEntry entry) {
|
||||
printRow(entry.row, entry.columns);
|
||||
}
|
||||
|
||||
private final void printRow(byte[] row, AbstractMap<byte[], byte[]> values) {
|
||||
// copy values into a TreeMap to get them in sorted order
|
||||
|
||||
TreeMap<String,byte[]> sorted = new TreeMap<String,byte[]>();
|
||||
for (AbstractMap.Entry<byte[], byte[]> entry : values.entrySet()) {
|
||||
sorted.put(utf8(entry.getKey()), entry.getValue());
|
||||
}
|
||||
|
||||
StringBuilder rowStr = new StringBuilder();
|
||||
for (SortedMap.Entry<String, byte[]> entry : sorted.entrySet()) {
|
||||
rowStr.append(entry.getKey());
|
||||
rowStr.append(" => ");
|
||||
rowStr.append(utf8(entry.getValue()));
|
||||
rowStr.append("; ");
|
||||
}
|
||||
System.out.println("row: " + utf8(row) + ", cols: " + rowStr);
|
||||
}
|
||||
}
|
|
@ -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()
|
|
@ -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
|
|
@ -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
|
||||
|
||||
|
|
@ -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) {
|
||||
|
|
|
@ -0,0 +1,437 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
// ----------------------------------------------------------------
|
||||
// HBase.thrift -
|
||||
//
|
||||
// This is a Thrift interface definition file for the Hbase service.
|
||||
// Target language libraries for C++, Java, Ruby, PHP, (and more) are
|
||||
// generated by running this file through the Thrift compiler with the
|
||||
// appropriate flags. The Thrift compiler binary and runtime
|
||||
// libraries for various languages is currently available from
|
||||
// Facebook (http://developers.facebook.com/thrift/). The intent is
|
||||
// for the Thrift project to migrate to Apache Incubator.
|
||||
//
|
||||
// See the package.html file for information on the version of Thrift
|
||||
// used to generate the *.java files checked into the Hbase project.
|
||||
// ----------------------------------------------------------------
|
||||
|
||||
java_package org.apache.hadoop.hbase.thrift.generated
|
||||
cpp_namespace apache.hadoop.hbase.thrift
|
||||
ruby_namespace Apache.Hadoop.Hbase.Thrift
|
||||
// note: other language namespaces tbd...
|
||||
|
||||
//
|
||||
// Types
|
||||
//
|
||||
|
||||
// NOTE: all variables with the Text type are assumed to be correctly
|
||||
// formatted UTF-8 strings. This is a programming language and locale
|
||||
// dependent property that the client application is repsonsible for
|
||||
// maintaining. If strings with an invalid encoding are sent, an
|
||||
// IOError will be thrown.
|
||||
|
||||
typedef binary Text
|
||||
typedef binary Bytes
|
||||
typedef i32 ScannerID
|
||||
|
||||
|
||||
/**
|
||||
* An HColumnDescriptor contains information about a column family
|
||||
* such as the number of versions, compression settings, etc. It is
|
||||
* used as input when creating a table or adding a column.
|
||||
*/
|
||||
struct ColumnDescriptor {
|
||||
1:Text name,
|
||||
2:i32 maxVersions = 3,
|
||||
3:string compression = "NONE",
|
||||
4:bool inMemory = 0,
|
||||
5:i32 maxValueLength = 2147483647,
|
||||
6:string bloomFilterType = "NONE",
|
||||
7:i32 bloomFilterVectorSize = 0,
|
||||
8:i32 bloomFilterNbHashes = 0
|
||||
}
|
||||
|
||||
/**
|
||||
* A RegionDescriptor contains informationa about an HTable region.
|
||||
* Currently, this is just the startKey of the region.
|
||||
*/
|
||||
struct RegionDescriptor {
|
||||
1:Text startKey,
|
||||
}
|
||||
|
||||
/**
|
||||
* A Mutation object is used to either update or delete a column-value.
|
||||
*/
|
||||
struct Mutation {
|
||||
1:bool isDelete = 0,
|
||||
2:Text column,
|
||||
3:Text value
|
||||
}
|
||||
|
||||
/**
|
||||
* A ScanEntry contains the row, column, and value information for a scanner's
|
||||
* current location.
|
||||
*/
|
||||
struct ScanEntry {
|
||||
1:Text row,
|
||||
2:map<Text, Bytes> columns
|
||||
}
|
||||
|
||||
//
|
||||
// Exceptions
|
||||
//
|
||||
/**
|
||||
* An IOError exception signals that an error occurred communicating
|
||||
* to the Hbase master or an Hbase region server. Also used to return
|
||||
* more general Hbase error conditions.
|
||||
*/
|
||||
exception IOError {
|
||||
1:string message
|
||||
}
|
||||
|
||||
/**
|
||||
* An IllegalArgument exception indicates an illegal or invalid
|
||||
* argument was passed into a procedure.
|
||||
*/
|
||||
exception IllegalArgument {
|
||||
1:string message
|
||||
}
|
||||
|
||||
/**
|
||||
* A NotFound exception is used to indicate that no value was found
|
||||
* for a query, or that a scanner has reached it's end.
|
||||
*/
|
||||
exception NotFound {
|
||||
1:string message
|
||||
}
|
||||
|
||||
/**
|
||||
* An AlreadyExists exceptions signals that a table with the specified
|
||||
* name already exists
|
||||
*/
|
||||
exception AlreadyExists {
|
||||
1:string message
|
||||
}
|
||||
|
||||
//
|
||||
// Service
|
||||
//
|
||||
|
||||
service Hbase {
|
||||
|
||||
/**
|
||||
* List all the userspace tables.
|
||||
* @return - returns a list of names
|
||||
*/
|
||||
list<Text> getTableNames()
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* List all the column families assoicated with a table.
|
||||
* @param tableName table name
|
||||
* @return list of column family descriptors
|
||||
*/
|
||||
map<Text,ColumnDescriptor> getColumnDescriptors (1:Text tableName)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* List the regions associated with a table.
|
||||
* @param tableName table name
|
||||
* @return list of region descriptors
|
||||
*/
|
||||
list<RegionDescriptor> getTableRegions(1:Text tableName)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Create a table with the specified column families. The name
|
||||
* field for each ColumnDescriptor must be set and must end in a
|
||||
* colon (:). All other fields are optional and will get default
|
||||
* values if not explicitly specified.
|
||||
*
|
||||
* @param tableName name of table to create
|
||||
* @param columnFamilies list of column family descriptors
|
||||
*
|
||||
* @throws IllegalArgument if an input parameter is invalid
|
||||
* @throws AlreadyExists if the table name already exists
|
||||
*/
|
||||
void createTable(1:Text tableName, 2:list<ColumnDescriptor> columnFamilies)
|
||||
throws (1:IOError io, 2:IllegalArgument ia, 3:AlreadyExists exist)
|
||||
|
||||
/**
|
||||
* Deletes a table
|
||||
* @param tableName name of table to delete
|
||||
* @throws NotFound if table doesn't exist on server
|
||||
*/
|
||||
void deleteTable(1:Text tableName)
|
||||
throws (1:IOError io, 2:NotFound nf)
|
||||
|
||||
/**
|
||||
* Get a single value for the specified table, row, and column at the
|
||||
* latest timestamp.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row row key
|
||||
* @param column column name
|
||||
* @return value for specified row/column
|
||||
*/
|
||||
Bytes get(1:Text tableName, 2:Text row, 3:Text column)
|
||||
throws (1:IOError io, 2:NotFound nf)
|
||||
|
||||
/**
|
||||
* Get the specified number of versions for the specified table,
|
||||
* row, and column.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row row key
|
||||
* @param column column name
|
||||
* @param numVersions number of versions to retrieve
|
||||
* @return list of values for specified row/column
|
||||
*/
|
||||
list<Bytes> getVer(1:Text tableName, 2:Text row, 3:Text column, 4:i32 numVersions)
|
||||
throws (1:IOError io, 2:NotFound nf)
|
||||
|
||||
/**
|
||||
* Get the specified number of versions for the specified table,
|
||||
* row, and column. Only versions less than or equal to the specified
|
||||
* timestamp will be returned.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row row key
|
||||
* @param column column name
|
||||
* @param timestamp timestamp
|
||||
* @param numVersions number of versions to retrieve
|
||||
* @return list of values for specified row/column
|
||||
*/
|
||||
list<Bytes> getVerTs(1:Text tableName, 2:Text row, 3:Text column, 4:i64 timestamp, 5:i32 numVersions)
|
||||
throws (1:IOError io, 2:NotFound nf)
|
||||
|
||||
/**
|
||||
* Get all the data for the specified table and row at the latest
|
||||
* timestamp.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row row key
|
||||
* @return Map of columns to values. Map is empty if row does not exist.
|
||||
*/
|
||||
map<Text, Bytes> getRow(1:Text tableName, 2:Text row)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Get all the data for the specified table and row at the specified
|
||||
* timestamp.
|
||||
*
|
||||
* @param tableName of table
|
||||
* @param row row key
|
||||
* @param timestamp timestamp
|
||||
* @return Map of columns to values. Map is empty if row does not exist.
|
||||
*/
|
||||
map<Text, Bytes> getRowTs(1:Text tableName, 2:Text row, 3:i64 timestamp)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Put a single value at the specified table, row, and column.
|
||||
* To put muliple values in a single transaction, or to specify
|
||||
* a non-default timestamp, use {@link #mutateRow} and/or
|
||||
* {@link #mutateRowTs}
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row row key
|
||||
* @param column column name
|
||||
*/
|
||||
void put(1:Text tableName, 2:Text row, 3:Text column, 4:Bytes value)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Apply a series of mutations (updates/deletes) to a row in a
|
||||
* single transaction. If an exception is thrown, then the
|
||||
* transaction is aborted. Default current timestamp is used, and
|
||||
* all entries will have an identical timestamp.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row row key
|
||||
* @param mutations list of mutation commands
|
||||
*/
|
||||
void mutateRow(1:Text tableName, 2:Text row, 3:list<Mutation> mutations)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Apply a series of mutations (updates/deletes) to a row in a
|
||||
* single transaction. If an exception is thrown, then the
|
||||
* transaction is aborted. The specified timestamp is used, and
|
||||
* all entries will have an identical timestamp.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row row key
|
||||
* @param mutations list of mutation commands
|
||||
* @param timestamp timestamp
|
||||
*/
|
||||
void mutateRowTs(1:Text tableName, 2:Text row, 3:list<Mutation> mutations, 4:i64 timestamp)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Delete all cells that match the passed row and column.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row Row to update
|
||||
* @param column name of column whose value is to be deleted
|
||||
*/
|
||||
void deleteAll(1:Text tableName, 2:Text row, 3:Text column)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Delete all cells that match the passed row and column and whose
|
||||
* timestamp is equal-to or older than the passed timestamp.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row Row to update
|
||||
* @param column name of column whose value is to be deleted
|
||||
* @param timestamp timestamp
|
||||
*/
|
||||
void deleteAllTs(1:Text tableName, 2:Text row, 3:Text column, 4:i64 timestamp)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Completely delete the row's cells.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row key of the row to be completely deleted.
|
||||
*/
|
||||
void deleteAllRow(1:Text tableName, 2:Text row)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Completely delete the row's cells marked with a timestamp
|
||||
* equal-to or older than the passed timestamp.
|
||||
*
|
||||
* @param tableName name of table
|
||||
* @param row key of the row to be completely deleted.
|
||||
* @param timestamp timestamp
|
||||
*/
|
||||
void deleteAllRowTs(1:Text tableName, 2:Text row, 3:i64 timestamp)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Get a scanner on the current table starting at the specified row and
|
||||
* ending at the last row in the table. Return the specified columns.
|
||||
*
|
||||
* @param columns columns to scan. If column name is a column family, all
|
||||
* columns of the specified column family are returned. Its also possible
|
||||
* to pass a regex in the column qualifier.
|
||||
* @param tableName name of table
|
||||
* @param startRow starting row in table to scan. send "" (empty string) to
|
||||
* start at the first row.
|
||||
*
|
||||
* @return scanner id to be used with other scanner procedures
|
||||
*/
|
||||
ScannerID scannerOpen(1:Text tableName,
|
||||
2:Text startRow,
|
||||
3:list<Text> columns)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Get a scanner on the current table starting and stopping at the
|
||||
* specified rows. ending at the last row in the table. Return the
|
||||
* specified columns.
|
||||
*
|
||||
* @param columns columns to scan. If column name is a column family, all
|
||||
* columns of the specified column family are returned. Its also possible
|
||||
* to pass a regex in the column qualifier.
|
||||
* @param tableName name of table
|
||||
* @param startRow starting row in table to scan. send "" (empty string) to
|
||||
* start at the first row.
|
||||
* @param stopRow row to stop scanning on. This row is *not* included in the
|
||||
* scanner's results
|
||||
*
|
||||
* @return scanner id to be used with other scanner procedures
|
||||
*/
|
||||
ScannerID scannerOpenWithStop(1:Text tableName,
|
||||
2:Text startRow,
|
||||
3:Text stopRow,
|
||||
4:list<Text> columns)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Get a scanner on the current table starting at the specified row and
|
||||
* ending at the last row in the table. Return the specified columns.
|
||||
* Only values with the specified timestamp are returned.
|
||||
*
|
||||
* @param columns columns to scan. If column name is a column family, all
|
||||
* columns of the specified column family are returned. Its also possible
|
||||
* to pass a regex in the column qualifier.
|
||||
* @param tableName name of table
|
||||
* @param startRow starting row in table to scan. send "" (empty string) to
|
||||
* start at the first row.
|
||||
* @param timestamp timestamp
|
||||
*
|
||||
* @return scanner id to be used with other scanner procedures
|
||||
*/
|
||||
ScannerID scannerOpenTs(1:Text tableName,
|
||||
2:Text startRow,
|
||||
3:list<Text> columns,
|
||||
4:i64 timestamp)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Get a scanner on the current table starting and stopping at the
|
||||
* specified rows. ending at the last row in the table. Return the
|
||||
* specified columns. Only values with the specified timestamp are
|
||||
* returned.
|
||||
*
|
||||
* @param columns columns to scan. If column name is a column family, all
|
||||
* columns of the specified column family are returned. Its also possible
|
||||
* to pass a regex in the column qualifier.
|
||||
* @param tableName name of table
|
||||
* @param startRow starting row in table to scan. send "" (empty string) to
|
||||
* start at the first row.
|
||||
* @param stopRow row to stop scanning on. This row is *not* included
|
||||
* in the scanner's results
|
||||
* @param timestamp timestamp
|
||||
*
|
||||
* @return scanner id to be used with other scanner procedures
|
||||
*/
|
||||
ScannerID scannerOpenWithStopTs(1:Text tableName,
|
||||
2:Text startRow,
|
||||
3:Text stopRow,
|
||||
4:list<Text> columns,
|
||||
5:i64 timestamp)
|
||||
throws (1:IOError io)
|
||||
|
||||
/**
|
||||
* Returns the scanner's current row value and advances to the next
|
||||
* row in the table. When there are no more rows in the table, or a key
|
||||
* greater-than-or-equal-to the scanner's specified stopRow is reached,
|
||||
* a NotFound exception is returned.
|
||||
*
|
||||
* @param id id of a scanner returned by scannerOpen
|
||||
* @return a ScanEntry object representing the current row's values
|
||||
* @throws IllegalArgument if ScannerID is invalid
|
||||
* @throws NotFound when the scanner reaches the end
|
||||
*/
|
||||
ScanEntry scannerGet(1:ScannerID id)
|
||||
throws (1:IOError io, 2:IllegalArgument ia, 3:NotFound nf)
|
||||
|
||||
/**
|
||||
* Closes the server-state associated with an open scanner.
|
||||
*
|
||||
* @param id id of a scanner returned by scannerOpen
|
||||
* @throws IllegalArgument if ScannerID is invalid
|
||||
*/
|
||||
void scannerClose(1:ScannerID id)
|
||||
throws (1:IOError io, 2:IllegalArgument ia)
|
||||
}
|
|
@ -0,0 +1,624 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.thrift;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.MalformedInputException;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.CommandLineParser;
|
||||
import org.apache.commons.cli.GnuParser;
|
||||
import org.apache.commons.cli.HelpFormatter;
|
||||
import org.apache.commons.cli.Options;
|
||||
import org.apache.commons.cli.ParseException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HBaseAdmin;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HScannerInterface;
|
||||
import org.apache.hadoop.hbase.HStoreKey;
|
||||
import org.apache.hadoop.hbase.HTable;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
|
||||
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift.generated.Hbase;
|
||||
import org.apache.hadoop.hbase.thrift.generated.IOError;
|
||||
import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
|
||||
import org.apache.hadoop.hbase.thrift.generated.Mutation;
|
||||
import org.apache.hadoop.hbase.thrift.generated.NotFound;
|
||||
import org.apache.hadoop.hbase.thrift.generated.RegionDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift.generated.ScanEntry;
|
||||
|
||||
import com.facebook.thrift.TException;
|
||||
import com.facebook.thrift.protocol.TBinaryProtocol;
|
||||
import com.facebook.thrift.protocol.TProtocolFactory;
|
||||
import com.facebook.thrift.server.TServer;
|
||||
import com.facebook.thrift.server.TThreadPoolServer;
|
||||
import com.facebook.thrift.transport.TServerSocket;
|
||||
import com.facebook.thrift.transport.TServerTransport;
|
||||
|
||||
/**
|
||||
* ThriftServer - this class starts up a Thrift server which implements the
|
||||
* Hbase API specified in the Hbase.thrift IDL file.
|
||||
*/
|
||||
public class ThriftServer {
|
||||
|
||||
/**
|
||||
* The HBaseHandler is a glue object that connects Thrift RPC calls to the
|
||||
* HBase client API primarily defined in the HBaseAdmin and HTable objects.
|
||||
*/
|
||||
public static class HBaseHandler implements Hbase.Iface {
|
||||
protected HBaseConfiguration conf = new HBaseConfiguration();
|
||||
protected HBaseAdmin admin = null;
|
||||
protected final Log LOG = LogFactory.getLog(this.getClass().getName());
|
||||
|
||||
// nextScannerId and scannerMap are used to manage scanner state
|
||||
protected int nextScannerId = 0;
|
||||
protected HashMap<Integer, HScannerInterface> scannerMap = null;
|
||||
|
||||
/**
|
||||
* Creates and returns an HTable instance from a given table name.
|
||||
*
|
||||
* @param tableName
|
||||
* name of table
|
||||
* @return HTable object
|
||||
* @throws IOException
|
||||
* @throws IOException
|
||||
*/
|
||||
protected HTable getTable(final byte[] tableName) throws IOError,
|
||||
IOException {
|
||||
return new HTable(this.conf, getText(tableName));
|
||||
}
|
||||
|
||||
/**
|
||||
* Assigns a unique ID to the scanner and adds the mapping to an internal
|
||||
* hash-map.
|
||||
*
|
||||
* @param scanner
|
||||
* @return integer scanner id
|
||||
*/
|
||||
protected synchronized int addScanner(HScannerInterface scanner) {
|
||||
int id = nextScannerId++;
|
||||
scannerMap.put(id, scanner);
|
||||
return id;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the scanner associated with the specified ID.
|
||||
*
|
||||
* @param id
|
||||
* @return a HScannerInterface, or null if ID was invalid.
|
||||
*/
|
||||
protected synchronized HScannerInterface getScanner(int id) {
|
||||
return scannerMap.get(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the scanner associated with the specified ID from the internal
|
||||
* id->scanner hash-map.
|
||||
*
|
||||
* @param id
|
||||
* @return a HScannerInterface, or null if ID was invalid.
|
||||
*/
|
||||
protected synchronized HScannerInterface removeScanner(int id) {
|
||||
return scannerMap.remove(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs an HBaseHandler object.
|
||||
*
|
||||
* @throws MasterNotRunningException
|
||||
*/
|
||||
HBaseHandler() throws MasterNotRunningException {
|
||||
conf = new HBaseConfiguration();
|
||||
admin = new HBaseAdmin(conf);
|
||||
scannerMap = new HashMap<Integer, HScannerInterface>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a byte array to a Text object after validating the UTF-8
|
||||
* encoding.
|
||||
*
|
||||
* @param buf
|
||||
* UTF-8 encoded bytes
|
||||
* @return Text object
|
||||
* @throws IllegalArgument
|
||||
*/
|
||||
Text getText(byte[] buf) throws IOError {
|
||||
try {
|
||||
Text.validateUTF8(buf);
|
||||
} catch (MalformedInputException e) {
|
||||
throw new IOError("invalid UTF-8 encoding in row or column name");
|
||||
}
|
||||
return new Text(buf);
|
||||
}
|
||||
|
||||
//
|
||||
// The Thrift Hbase.Iface interface is implemented below.
|
||||
// Documentation for the methods and datastructures is the Hbase.thrift file
|
||||
// used to generate the interface.
|
||||
//
|
||||
|
||||
public ArrayList<byte[]> getTableNames() throws IOError {
|
||||
LOG.debug("getTableNames");
|
||||
try {
|
||||
HTableDescriptor[] tables = this.admin.listTables();
|
||||
ArrayList<byte[]> list = new ArrayList<byte[]>(tables.length);
|
||||
for (int i = 0; i < tables.length; i++) {
|
||||
list.add(tables[i].getName().toString().getBytes());
|
||||
}
|
||||
return list;
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public ArrayList<RegionDescriptor> getTableRegions(byte[] tableName)
|
||||
throws IOError {
|
||||
LOG.debug("getTableRegions: " + new String(tableName));
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
Text[] startKeys = table.getStartKeys();
|
||||
ArrayList<RegionDescriptor> regions = new ArrayList<RegionDescriptor>();
|
||||
for (int i = 0; i < startKeys.length; i++) {
|
||||
RegionDescriptor region = new RegionDescriptor();
|
||||
region.startKey = startKeys[i].toString().getBytes();
|
||||
regions.add(region);
|
||||
}
|
||||
return regions;
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public byte[] get(byte[] tableName, byte[] row, byte[] column)
|
||||
throws NotFound, IOError {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("get: table=" + new String(tableName) + ", row="
|
||||
+ new String(row) + ", col=" + new String(column));
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
byte[] value = table.get(getText(row), getText(column));
|
||||
if (value == null) {
|
||||
throw new NotFound();
|
||||
}
|
||||
return value;
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public ArrayList<byte[]> getVer(byte[] tableName, byte[] row,
|
||||
byte[] column, int numVersions) throws IOError, NotFound {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("getVer: table=" + new String(tableName) + ", row="
|
||||
+ new String(row) + ", col=" + new String(column) + ", numVers="
|
||||
+ numVersions);
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
byte[][] values = table.get(getText(row), getText(column), numVersions);
|
||||
if (values == null) {
|
||||
throw new NotFound();
|
||||
}
|
||||
return new ArrayList<byte[]>(Arrays.asList(values));
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public ArrayList<byte[]> getVerTs(byte[] tableName, byte[] row,
|
||||
byte[] column, long timestamp, int numVersions) throws IOError,
|
||||
NotFound {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("getVerTs: table=" + new String(tableName) + ", row="
|
||||
+ new String(row) + ", col=" + new String(column) + ", ts="
|
||||
+ timestamp + ", numVers=" + numVersions);
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
byte[][] values = table.get(getText(row), getText(column), timestamp,
|
||||
numVersions);
|
||||
if (values == null) {
|
||||
throw new NotFound();
|
||||
}
|
||||
return new ArrayList<byte[]>(Arrays.asList(values));
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public AbstractMap<byte[], byte[]> getRow(byte[] tableName, byte[] row)
|
||||
throws IOError {
|
||||
return getRowTs(tableName, row, HConstants.LATEST_TIMESTAMP);
|
||||
}
|
||||
|
||||
public AbstractMap<byte[], byte[]> getRowTs(byte[] tableName, byte[] row,
|
||||
long timestamp) throws IOError {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("getRowTs: table=" + new String(tableName) + ", row="
|
||||
+ new String(row) + ", ts=" + timestamp);
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
SortedMap<Text, byte[]> values = table.getRow(getText(row), timestamp);
|
||||
// copy the map from type <Text, byte[]> to <byte[], byte[]>
|
||||
HashMap<byte[], byte[]> returnValues = new HashMap<byte[], byte[]>();
|
||||
for (Entry<Text, byte[]> e : values.entrySet()) {
|
||||
returnValues.put(e.getKey().getBytes(), e.getValue());
|
||||
}
|
||||
return returnValues;
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void put(byte[] tableName, byte[] row, byte[] column, byte[] value)
|
||||
throws IOError {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("put: table=" + new String(tableName) + ", row="
|
||||
+ new String(row) + ", col=" + new String(column)
|
||||
+ ", value.length=" + value.length);
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
long lockid = table.startUpdate(getText(row));
|
||||
table.put(lockid, getText(column), value);
|
||||
table.commit(lockid);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void deleteAll(byte[] tableName, byte[] row, byte[] column)
|
||||
throws IOError {
|
||||
deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP);
|
||||
}
|
||||
|
||||
public void deleteAllTs(byte[] tableName, byte[] row, byte[] column,
|
||||
long timestamp) throws IOError {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("deleteAllTs: table=" + new String(tableName) + ", row="
|
||||
+ new String(row) + ", col=" + new String(column) + ", ts="
|
||||
+ timestamp);
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
table.deleteAll(getText(row), getText(column), timestamp);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void deleteAllRow(byte[] tableName, byte[] row) throws IOError {
|
||||
deleteAllRowTs(tableName, row, HConstants.LATEST_TIMESTAMP);
|
||||
}
|
||||
|
||||
public void deleteAllRowTs(byte[] tableName, byte[] row, long timestamp)
|
||||
throws IOError {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("deleteAllRowTs: table=" + new String(tableName) + ", row="
|
||||
+ new String(row) + ", ts=" + timestamp);
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
table.deleteAll(getText(row), timestamp);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void createTable(byte[] tableName,
|
||||
ArrayList<ColumnDescriptor> columnFamilies) throws IOError,
|
||||
IllegalArgument, AlreadyExists {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("createTable: table=" + new String(tableName));
|
||||
}
|
||||
try {
|
||||
Text tableStr = getText(tableName);
|
||||
if (admin.tableExists(tableStr)) {
|
||||
throw new AlreadyExists("table name already in use");
|
||||
}
|
||||
HTableDescriptor desc = new HTableDescriptor(tableStr.toString());
|
||||
for (ColumnDescriptor col : columnFamilies) {
|
||||
HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col);
|
||||
desc.addFamily(colDesc);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("createTable: col=" + colDesc.getName());
|
||||
}
|
||||
}
|
||||
admin.createTable(desc);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new IllegalArgument(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void deleteTable(byte[] tableName) throws IOError, NotFound {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("deleteTable: table=" + new String(tableName));
|
||||
}
|
||||
try {
|
||||
Text tableStr = getText(tableName);
|
||||
if (!admin.tableExists(tableStr)) {
|
||||
throw new NotFound();
|
||||
}
|
||||
admin.deleteTable(tableStr);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void mutateRow(byte[] tableName, byte[] row,
|
||||
ArrayList<Mutation> mutations) throws IOError {
|
||||
mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP);
|
||||
}
|
||||
|
||||
public void mutateRowTs(byte[] tableName, byte[] row,
|
||||
ArrayList<Mutation> mutations, long timestamp) throws IOError {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("mutateRowTs: table=" + new String(tableName) + ", row="
|
||||
+ new String(row) + ", ts=" + timestamp + " mutations="
|
||||
+ mutations.size());
|
||||
for (Mutation m : mutations) {
|
||||
if (m.isDelete) {
|
||||
LOG.debug("mutateRowTs: : delete - " + getText(m.column));
|
||||
} else {
|
||||
LOG.debug("mutateRowTs: : put - " + getText(m.column) + " => "
|
||||
+ m.value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Long lockid = null;
|
||||
HTable table = null;
|
||||
|
||||
try {
|
||||
table = getTable(tableName);
|
||||
lockid = table.startUpdate(getText(row));
|
||||
for (Mutation m : mutations) {
|
||||
if (m.isDelete) {
|
||||
table.delete(lockid, getText(m.column));
|
||||
} else {
|
||||
table.put(lockid, getText(m.column), m.value);
|
||||
}
|
||||
}
|
||||
table.commit(lockid, timestamp);
|
||||
} catch (IOException e) {
|
||||
if (lockid != null) {
|
||||
table.abort(lockid);
|
||||
}
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void scannerClose(int id) throws IOError, IllegalArgument {
|
||||
LOG.debug("scannerClose: id=" + id);
|
||||
HScannerInterface scanner = getScanner(id);
|
||||
if (scanner == null) {
|
||||
throw new IllegalArgument("scanner ID is invalid");
|
||||
}
|
||||
try {
|
||||
scanner.close();
|
||||
removeScanner(id);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public ScanEntry scannerGet(int id) throws IllegalArgument, NotFound,
|
||||
IOError {
|
||||
LOG.debug("scannerGet: id=" + id);
|
||||
HScannerInterface scanner = getScanner(id);
|
||||
if (scanner == null) {
|
||||
throw new IllegalArgument("scanner ID is invalid");
|
||||
}
|
||||
|
||||
HStoreKey key = new HStoreKey();
|
||||
TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||
|
||||
try {
|
||||
if (scanner.next(key, results) == false) {
|
||||
throw new NotFound("end of scanner reached");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
|
||||
ScanEntry retval = new ScanEntry();
|
||||
retval.row = key.getRow().getBytes();
|
||||
retval.columns = new HashMap<byte[], byte[]>(results.size());
|
||||
|
||||
for (SortedMap.Entry<Text, byte[]> e : results.entrySet()) {
|
||||
retval.columns.put(e.getKey().getBytes(), e.getValue());
|
||||
}
|
||||
return retval;
|
||||
}
|
||||
|
||||
public int scannerOpen(byte[] tableName, byte[] startRow,
|
||||
ArrayList<byte[]> columns) throws IOError {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("scannerOpen: table=" + getText(tableName) + ", start="
|
||||
+ getText(startRow) + ", columns=" + columns.toString());
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
Text[] columnsText = new Text[columns.size()];
|
||||
for (int i = 0; i < columns.size(); ++i) {
|
||||
columnsText[i] = getText(columns.get(i));
|
||||
}
|
||||
HScannerInterface scanner = table.obtainScanner(columnsText,
|
||||
getText(startRow));
|
||||
return addScanner(scanner);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public int scannerOpenWithStop(byte[] tableName, byte[] startRow,
|
||||
byte[] stopRow, ArrayList<byte[]> columns) throws IOError, TException {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("scannerOpen: table=" + getText(tableName) + ", start="
|
||||
+ getText(startRow) + ", stop=" + getText(stopRow) + ", columns="
|
||||
+ columns.toString());
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
Text[] columnsText = new Text[columns.size()];
|
||||
for (int i = 0; i < columns.size(); ++i) {
|
||||
columnsText[i] = getText(columns.get(i));
|
||||
}
|
||||
HScannerInterface scanner = table.obtainScanner(columnsText,
|
||||
getText(startRow), getText(stopRow));
|
||||
return addScanner(scanner);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public int scannerOpenTs(byte[] tableName, byte[] startRow,
|
||||
ArrayList<byte[]> columns, long timestamp) throws IOError, TException {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("scannerOpen: table=" + getText(tableName) + ", start="
|
||||
+ getText(startRow) + ", columns=" + columns.toString()
|
||||
+ ", timestamp=" + timestamp);
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
Text[] columnsText = new Text[columns.size()];
|
||||
for (int i = 0; i < columns.size(); ++i) {
|
||||
columnsText[i] = getText(columns.get(i));
|
||||
}
|
||||
HScannerInterface scanner = table.obtainScanner(columnsText,
|
||||
getText(startRow), timestamp);
|
||||
return addScanner(scanner);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow,
|
||||
byte[] stopRow, ArrayList<byte[]> columns, long timestamp)
|
||||
throws IOError, TException {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("scannerOpen: table=" + getText(tableName) + ", start="
|
||||
+ getText(startRow) + ", stop=" + getText(stopRow) + ", columns="
|
||||
+ columns.toString() + ", timestamp=" + timestamp);
|
||||
}
|
||||
try {
|
||||
HTable table = getTable(tableName);
|
||||
Text[] columnsText = new Text[columns.size()];
|
||||
for (int i = 0; i < columns.size(); ++i) {
|
||||
columnsText[i] = getText(columns.get(i));
|
||||
}
|
||||
HScannerInterface scanner = table.obtainScanner(columnsText,
|
||||
getText(startRow), getText(stopRow), timestamp);
|
||||
return addScanner(scanner);
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public AbstractMap<byte[], ColumnDescriptor> getColumnDescriptors(
|
||||
byte[] tableName) throws IOError, TException {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("getColumnDescriptors: table=" + new String(tableName));
|
||||
}
|
||||
try {
|
||||
HashMap<byte[], ColumnDescriptor> columns = new HashMap<byte[], ColumnDescriptor>();
|
||||
|
||||
HTable table = getTable(tableName);
|
||||
HTableDescriptor desc = table.getMetadata();
|
||||
|
||||
for (Entry<Text, HColumnDescriptor> e : desc.families().entrySet()) {
|
||||
ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e.getValue());
|
||||
columns.put(col.name, col);
|
||||
}
|
||||
return columns;
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) {
|
||||
Log LOG = LogFactory.getLog("ThriftServer");
|
||||
|
||||
// Parse command-line
|
||||
//
|
||||
Options options = new Options();
|
||||
options.addOption("h", "help", false, "print this message");
|
||||
options.addOption("p", "port", true,
|
||||
"server listening port (default: 9090)");
|
||||
CommandLineParser parser = new GnuParser();
|
||||
CommandLine line;
|
||||
|
||||
try {
|
||||
line = parser.parse(options, args);
|
||||
} catch (ParseException e) {
|
||||
System.out.println("ERROR: " + e.getMessage());
|
||||
HelpFormatter formatter = new HelpFormatter();
|
||||
formatter.printHelp("ThriftServer [options]", options);
|
||||
return;
|
||||
}
|
||||
|
||||
if (line.hasOption("h")) {
|
||||
HelpFormatter formatter = new HelpFormatter();
|
||||
formatter.printHelp("ThriftServer [options]", options);
|
||||
return;
|
||||
}
|
||||
|
||||
int port = Integer.parseInt(line.getOptionValue("p", "9090"));
|
||||
|
||||
// Launch Thrift Server
|
||||
//
|
||||
try {
|
||||
LOG
|
||||
.info("starting HBase Thrift server on port "
|
||||
+ Integer.toString(port));
|
||||
HBaseHandler handler = new HBaseHandler();
|
||||
Hbase.Processor processor = new Hbase.Processor(handler);
|
||||
TServerTransport serverTransport = new TServerSocket(port);
|
||||
TProtocolFactory protFactory = new TBinaryProtocol.Factory(true, true);
|
||||
TServer server = new TThreadPoolServer(processor, serverTransport,
|
||||
protFactory);
|
||||
|
||||
LOG.info("Starting the server...");
|
||||
server.serve();
|
||||
|
||||
} catch (Exception x) {
|
||||
x.printStackTrace();
|
||||
}
|
||||
LOG.info("done.");
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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 {
|
||||
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,153 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Autogenerated by Thrift
|
||||
*
|
||||
* DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
|
||||
*/
|
||||
package org.apache.hadoop.hbase.thrift.generated;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import com.facebook.thrift.*;
|
||||
|
||||
import com.facebook.thrift.protocol.*;
|
||||
import com.facebook.thrift.transport.*;
|
||||
|
||||
/**
|
||||
* A ScanEntry contains the row, column, and value information for a scanner's
|
||||
* current location.
|
||||
*/
|
||||
public class ScanEntry implements TBase, java.io.Serializable {
|
||||
public byte[] row;
|
||||
public AbstractMap<byte[],byte[]> columns;
|
||||
|
||||
public final Isset __isset = new Isset();
|
||||
public static final class Isset {
|
||||
public boolean row = false;
|
||||
public boolean columns = false;
|
||||
}
|
||||
|
||||
public ScanEntry() {
|
||||
}
|
||||
|
||||
public ScanEntry(
|
||||
byte[] row,
|
||||
AbstractMap<byte[],byte[]> columns)
|
||||
{
|
||||
this();
|
||||
this.row = row;
|
||||
this.__isset.row = true;
|
||||
this.columns = columns;
|
||||
this.__isset.columns = true;
|
||||
}
|
||||
|
||||
public void read(TProtocol iprot) throws TException {
|
||||
TField field;
|
||||
iprot.readStructBegin();
|
||||
while (true)
|
||||
{
|
||||
field = iprot.readFieldBegin();
|
||||
if (field.type == TType.STOP) {
|
||||
break;
|
||||
}
|
||||
switch (field.id)
|
||||
{
|
||||
case 1:
|
||||
if (field.type == TType.STRING) {
|
||||
this.row = iprot.readBinary();
|
||||
this.__isset.row = true;
|
||||
} else {
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
}
|
||||
break;
|
||||
case 2:
|
||||
if (field.type == TType.MAP) {
|
||||
{
|
||||
TMap _map0 = iprot.readMapBegin();
|
||||
this.columns = new HashMap<byte[],byte[]>(2*_map0.size);
|
||||
for (int _i1 = 0; _i1 < _map0.size; ++_i1)
|
||||
{
|
||||
byte[] _key2;
|
||||
byte[] _val3;
|
||||
_key2 = iprot.readBinary();
|
||||
_val3 = iprot.readBinary();
|
||||
this.columns.put(_key2, _val3);
|
||||
}
|
||||
iprot.readMapEnd();
|
||||
}
|
||||
this.__isset.columns = true;
|
||||
} else {
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
break;
|
||||
}
|
||||
iprot.readFieldEnd();
|
||||
}
|
||||
iprot.readStructEnd();
|
||||
}
|
||||
|
||||
public void write(TProtocol oprot) throws TException {
|
||||
TStruct struct = new TStruct("ScanEntry");
|
||||
oprot.writeStructBegin(struct);
|
||||
TField field = new TField();
|
||||
if (this.row != null) {
|
||||
field.name = "row";
|
||||
field.type = TType.STRING;
|
||||
field.id = 1;
|
||||
oprot.writeFieldBegin(field);
|
||||
oprot.writeBinary(this.row);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
if (this.columns != null) {
|
||||
field.name = "columns";
|
||||
field.type = TType.MAP;
|
||||
field.id = 2;
|
||||
oprot.writeFieldBegin(field);
|
||||
{
|
||||
oprot.writeMapBegin(new TMap(TType.STRING, TType.STRING, this.columns.size()));
|
||||
for (byte[] _iter4 : this.columns.keySet()) {
|
||||
oprot.writeBinary(_iter4);
|
||||
oprot.writeBinary(this.columns.get(_iter4));
|
||||
}
|
||||
oprot.writeMapEnd();
|
||||
}
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
oprot.writeFieldStop();
|
||||
oprot.writeStructEnd();
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder("ScanEntry(");
|
||||
sb.append("row:");
|
||||
sb.append(this.row);
|
||||
sb.append(",columns:");
|
||||
sb.append(this.columns);
|
||||
sb.append(")");
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,58 @@
|
|||
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 3.2 Final//EN">
|
||||
<html>
|
||||
<head />
|
||||
<body bgcolor="white">
|
||||
Provides an HBase <a href="http://developers.facebook.com/thrift/">Thrift</a>
|
||||
service.
|
||||
|
||||
This directory contains a Thrift interface definition file for an Hbase RPC
|
||||
service and a Java server implementation.
|
||||
|
||||
<h2><a name="whatisthrift">What is Thrift?</a></h2>
|
||||
|
||||
<p>"Thrift is a software framework for scalable cross-language services
|
||||
development. It combines a powerful software stack with a code generation
|
||||
engine to build services that work efficiently and seamlessly between C++,
|
||||
Java, Python, PHP, and Ruby. Thrift was developed at Facebook, and we are now
|
||||
releasing it as open source." For additional information, see
|
||||
http://developers.facebook.com/thrift/. Facebook has announced their intent
|
||||
to migrate Thrift into Apache Incubator.
|
||||
</p>
|
||||
|
||||
<h2><a name="description">Description</a></h2>
|
||||
|
||||
<p>The <a href="generated/Hbase.Iface.html">Hbase API</a> is defined in the
|
||||
file Hbase.thrift. A server-side implementation of the API is in {@link
|
||||
org.apache.hadoop.hbase.thrift.ThriftServer}. The generated interfaces,
|
||||
types, and RPC utility files are checked into SVN under the {@link
|
||||
org.apache.hadoop.hbase.thrift.generated} directory.
|
||||
|
||||
</p>
|
||||
|
||||
<p>The files were generated by running the commands:
|
||||
<pre>
|
||||
thrift -strict -java Hbase.thrift
|
||||
mv gen-java/org/apache/hadoop/hbase/thrift/generated .
|
||||
rm -rf gen-java
|
||||
</pre>
|
||||
</p>
|
||||
|
||||
<p>The 'thrift' binary is the Thrift compiler, and it is distributed as a part
|
||||
of
|
||||
the Thrift package. Additionally, specific language runtime libraries are a
|
||||
part of the Thrift package. A version of the Java runtime is checked into SVN
|
||||
under the hbase/lib directory.
|
||||
</p>
|
||||
|
||||
<p>The version of Thrift used to generate the Java files is revision 746 from
|
||||
the <a href="http://svn.facebook.com/svnroot/thrift/">SVN repository</a>.</p>
|
||||
|
||||
<p>The ThriftServer is run like:
|
||||
<pre>
|
||||
|
||||
./bin/hbase thrift [-h|--help] [-p|--port PORT]
|
||||
</pre>
|
||||
The default port is 9090.
|
||||
</p>
|
||||
</body>
|
||||
</html>
|
Loading…
Reference in New Issue