HBASE-1367 Get rid of Thrift exception 'NotFound'

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@771604 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Andrew Kyle Purtell 2009-05-05 07:26:49 +00:00
parent fc856b6a24
commit ca0e255aff
22 changed files with 14555 additions and 3643 deletions

View File

@ -16,6 +16,7 @@ Release 0.20.0 - Unreleased
(Ryan Rawson and Stack)
HBASE-1342 Add to filesystem info needed to rebuild .META.
HBASE-1361 Disable bloom filters
HBASE-1367 Get rid of Thrift exception 'NotFound'
BUG FIXES
HBASE-1140 "ant clean test" fails (Nitay Joffe via Stack)

Binary file not shown.

BIN
lib/libthrift-r771587.jar Normal file

Binary file not shown.

View File

@ -140,14 +140,6 @@ 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
@ -230,22 +222,23 @@ service Hbase {
/**
* Deletes a table
* @param tableName name of table to delete
* @throws NotFound if table doesn't exist on server
* @throws IOError if table doesn't exist on server or there was some other
* problem
*/
void deleteTable(1:Text tableName)
throws (1:IOError io, 2:NotFound nf)
throws (1:IOError io)
/**
* Get a single TCell for the specified table, row, and column at the
* latest timestamp.
* latest timestamp. Returns an empty list if no such value exists.
*
* @param tableName name of table
* @param row row key
* @param column column name
* @return value for specified row/column
*/
TCell get(1:Text tableName, 2:Text row, 3:Text column)
throws (1:IOError io, 2:NotFound nf)
list<TCell> get(1:Text tableName, 2:Text row, 3:Text column)
throws (1:IOError io)
/**
* Get the specified number of versions for the specified table,
@ -257,8 +250,9 @@ service Hbase {
* @param numVersions number of versions to retrieve
* @return list of cells for specified row/column
*/
list<TCell> getVer(1:Text tableName, 2:Text row, 3:Text column, 4:i32 numVersions)
throws (1:IOError io, 2:NotFound nf)
list<TCell> getVer(1:Text tableName, 2:Text row, 3:Text column,
4:i32 numVersions)
throws (1:IOError io)
/**
* Get the specified number of versions for the specified table,
@ -272,59 +266,58 @@ service Hbase {
* @param numVersions number of versions to retrieve
* @return list of cells for specified row/column
*/
list<TCell> getVerTs(1:Text tableName, 2:Text row, 3:Text column, 4:i64 timestamp, 5:i32 numVersions)
throws (1:IOError io, 2:NotFound nf)
list<TCell> getVerTs(1:Text tableName, 2:Text row, 3:Text column,
4:i64 timestamp, 5:i32 numVersions)
throws (1:IOError io)
/**
* Get all the data for the specified table and row at the latest
* timestamp.
* timestamp. Returns an empty list if the row does not exist.
*
* @param tableName name of table
* @param row row key
* @return TRowResult containing the row and map of columns to TCells
* @throws NotFound if the row does not exist
*/
TRowResult getRow(1:Text tableName, 2:Text row)
throws (1:IOError io, 2:NotFound nf)
list<TRowResult> getRow(1:Text tableName, 2:Text row)
throws (1:IOError io)
/**
* Get the specified columns for the specified table and row at the latest
* timestamp.
* timestamp. Returns an empty list if the row does not exist.
*
* @param tableName name of table
* @param row row key
* @param columns List of columns to return, null for all columns
* @return TRowResult containing the row and map of columns to TCells
* @throws NotFound if the row does not exist
*/
TRowResult getRowWithColumns(1:Text tableName, 2:Text row, 3:list<Text> columns)
throws (1:IOError io, 2:NotFound nf)
list<TRowResult> getRowWithColumns(1:Text tableName, 2:Text row,
3:list<Text> columns)
throws (1:IOError io)
/**
* Get all the data for the specified table and row at the specified
* timestamp.
* timestamp. Returns an empty list if the row does not exist.
*
* @param tableName of table
* @param row row key
* @param timestamp timestamp
* @return TRowResult containing the row and map of columns to TCells
* @throws NotFound if the row does not exist
*/
TRowResult getRowTs(1:Text tableName, 2:Text row, 3:i64 timestamp)
throws (1:IOError io, 2:NotFound nf)
list<TRowResult> getRowTs(1:Text tableName, 2:Text row, 3:i64 timestamp)
throws (1:IOError io)
/**
* Get the specified columns for the specified table and row at the specified
* timestamp.
* timestamp. Returns an empty list if the row does not exist.
*
* @param tableName name of table
* @param row row key
* @param columns List of columns to return, null for all columns
* @return TRowResult containing the row and map of columns to TCells
* @throws NotFound if the row does not exist
*/
TRowResult getRowWithColumnsTs(1:Text tableName, 2:Text row, 3:list<Text> columns, 4:i64 timestamp)
throws (1:IOError io, 2:NotFound nf)
list<TRowResult> getRowWithColumnsTs(1:Text tableName, 2:Text row,
3:list<Text> columns, 4:i64 timestamp)
throws (1:IOError io)
/**
* Apply a series of mutations (updates/deletes) to a row in a
@ -520,15 +513,15 @@ service Hbase {
* 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.
* an empty list is returned.
*
* @param id id of a scanner returned by scannerOpen
* @return a TRowResult containing the current row and a map of the columns to TCells.
* @throws IllegalArgument if ScannerID is invalid
* @throws NotFound when the scanner reaches the end
*/
TRowResult scannerGet(1:ScannerID id)
throws (1:IOError io, 2:IllegalArgument ia, 3:NotFound nf)
list<TRowResult> scannerGet(1:ScannerID id)
throws (1:IOError io, 2:IllegalArgument ia)
/**
* Closes the server-state associated with an open scanner.

View File

@ -47,19 +47,18 @@ 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.TRegionInfo;
import org.apache.hadoop.hbase.thrift.generated.TCell;
import org.apache.hadoop.hbase.thrift.generated.TRowResult;
import org.apache.hadoop.hbase.util.Bytes;
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;
import org.apache.thrift.TException;
import org.apache.thrift.protocol.TCompactProtocol;
import org.apache.thrift.protocol.TProtocolFactory;
import org.apache.thrift.server.THsHaServer;
import org.apache.thrift.server.TServer;
import org.apache.thrift.transport.TNonblockingServerSocket;
import org.apache.thrift.transport.TNonblockingServerTransport;
/**
* ThriftServer - this class starts up a Thrift server which implements the
@ -179,7 +178,7 @@ public class ThriftServer {
}
}
public void compact(byte[] tableNameOrRegionName) throws IOError, TException {
public void compact(byte[] tableNameOrRegionName) throws IOError {
try{
admin.compact(tableNameOrRegionName);
} catch (IOException e) {
@ -187,7 +186,7 @@ public class ThriftServer {
}
}
public void majorCompact(byte[] tableNameOrRegionName) throws IOError, TException {
public void majorCompact(byte[] tableNameOrRegionName) throws IOError {
try{
admin.majorCompact(tableNameOrRegionName);
} catch (IOException e) {
@ -230,14 +229,11 @@ public class ThriftServer {
}
}
public TCell get(byte[] tableName, byte[] row, byte[] column)
throws NotFound, IOError {
public List<TCell> get(byte[] tableName, byte[] row, byte[] column)
throws IOError {
try {
HTable table = getTable(tableName);
Cell cell = table.get(row, column);
if (cell == null) {
throw new NotFound();
}
return ThriftUtilities.cellFromHBase(cell);
} catch (IOException e) {
throw new IOError(e.getMessage());
@ -245,63 +241,48 @@ public class ThriftServer {
}
public List<TCell> getVer(byte[] tableName, byte[] row,
byte[] column, int numVersions) throws IOError, NotFound {
byte[] column, int numVersions) throws IOError {
try {
HTable table = getTable(tableName);
Cell[] cells =
table.get(row, column, numVersions);
if (cells == null) {
throw new NotFound();
}
List<TCell> list = new ArrayList<TCell>();
for (int i = 0; i < cells.length; i++) {
list.add(ThriftUtilities.cellFromHBase(cells[i]));
}
return list;
return ThriftUtilities.cellFromHBase(cells);
} catch (IOException e) {
throw new IOError(e.getMessage());
}
}
public List<TCell> getVerTs(byte[] tableName, byte[] row,
byte[] column, long timestamp, int numVersions) throws IOError,
NotFound {
byte[] column, long timestamp, int numVersions) throws IOError {
try {
HTable table = getTable(tableName);
Cell[] cells = table.get(row, column, timestamp, numVersions);
if (cells == null) {
throw new NotFound();
}
List<TCell> list = new ArrayList<TCell>();
for (int i = 0; i < cells.length; i++) {
list.add(ThriftUtilities.cellFromHBase(cells[i]));
}
return list;
return ThriftUtilities.cellFromHBase(cells);
} catch (IOException e) {
throw new IOError(e.getMessage());
}
}
public TRowResult getRow(byte[] tableName, byte[] row)
throws IOError, NotFound {
public List<TRowResult> getRow(byte[] tableName, byte[] row)
throws IOError {
return getRowWithColumnsTs(tableName, row, null,
HConstants.LATEST_TIMESTAMP);
}
public TRowResult getRowWithColumns(byte[] tableName, byte[] row,
List<byte[]> columns) throws IOError, NotFound {
public List<TRowResult> getRowWithColumns(byte[] tableName, byte[] row,
List<byte[]> columns) throws IOError {
return getRowWithColumnsTs(tableName, row, columns,
HConstants.LATEST_TIMESTAMP);
}
public TRowResult getRowTs(byte[] tableName, byte[] row,
long timestamp) throws IOError, NotFound {
public List<TRowResult> getRowTs(byte[] tableName, byte[] row,
long timestamp) throws IOError {
return getRowWithColumnsTs(tableName, row, null,
timestamp);
}
public TRowResult getRowWithColumnsTs(byte[] tableName, byte[] row,
List<byte[]> columns, long timestamp) throws IOError, NotFound {
public List<TRowResult> getRowWithColumnsTs(byte[] tableName, byte[] row,
List<byte[]> columns, long timestamp) throws IOError {
try {
HTable table = getTable(tableName);
if (columns == null) {
@ -365,13 +346,13 @@ public class ThriftServer {
}
}
public void deleteTable(byte[] tableName) throws IOError, NotFound {
public void deleteTable(byte[] tableName) throws IOError {
if (LOG.isDebugEnabled()) {
LOG.debug("deleteTable: table=" + new String(tableName));
}
try {
if (!admin.tableExists(tableName)) {
throw new NotFound();
throw new IOError("table does not exist");
}
admin.deleteTable(tableName);
} catch (IOException e) {
@ -459,7 +440,7 @@ public class ThriftServer {
removeScanner(id);
}
public TRowResult scannerGet(int id) throws IllegalArgument, NotFound,
public List<TRowResult> scannerGet(int id) throws IllegalArgument,
IOError {
LOG.debug("scannerGet: id=" + id);
Scanner scanner = getScanner(id);
@ -472,7 +453,7 @@ public class ThriftServer {
try {
results = scanner.next();
if (results == null) {
throw new NotFound("end of scanner reached");
return new ArrayList<TRowResult>();
}
} catch (IOException e) {
throw new IOError(e.getMessage());
@ -632,9 +613,10 @@ public class ThriftServer {
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,
TNonblockingServerTransport serverTransport =
new TNonblockingServerSocket(port);
TProtocolFactory protFactory = new TCompactProtocol.Factory();
TServer server = new THsHaServer(processor, serverTransport,
protFactory);
server.serve();
}

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hbase.thrift;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
@ -27,7 +29,6 @@ import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
import org.apache.hadoop.hbase.thrift.generated.NotFound;
import org.apache.hadoop.hbase.thrift.generated.TCell;
import org.apache.hadoop.hbase.thrift.generated.TRowResult;
import org.apache.hadoop.hbase.util.Bytes;
@ -82,38 +83,61 @@ public class ThriftUtilities {
}
/**
* This utility method creates a new Thrift TCell "struct" based on
* an Hbase Cell object.
* This utility method creates a list of Thrift TCell "struct" based on
* an Hbase Cell object. The empty list is returned if the input is null.
*
* @param in
* Hbase Cell object
* @return Thrift TCell
* @return Thrift TCell array
*/
static public TCell cellFromHBase(Cell in) {
return new TCell(in.getValue(), in.getTimestamp());
static public List<TCell> cellFromHBase(Cell in) {
List<TCell> list = new ArrayList<TCell>(1);
if (in != null) {
list.add(new TCell(in.getValue(), in.getTimestamp()));
}
return list;
}
/**
* This utility method creates a new Thrift TRowResult "struct" based on
* an Hbase RowResult object.
* This utility method creates a list of Thrift TCell "struct" based on
* an Hbase Cell array. The empty list is returned if the input is null.
* @param in Hbase Cell array
* @return Thrift TCell array
*/
static public List<TCell> cellFromHBase(Cell[] in) {
List<TCell> list = new ArrayList<TCell>(in.length);
if (in != null) {
for (int i = 0; i < in.length; i++) {
list.add(new TCell(in[i].getValue(), in[i].getTimestamp()));
}
}
return list;
}
/**
* This utility method creates a list of Thrift TRowResult "struct" based on
* an Hbase RowResult object. The empty list is returned if the input is
* null.
*
* @param in
* Hbase RowResult object
* @return Thrift TRowResult
* @throws NotFound
* @return Thrift TRowResult array
*/
static public TRowResult rowResultFromHBase(RowResult in)
throws NotFound {
static public List<TRowResult> rowResultFromHBase(RowResult in) {
List<TRowResult> list = new ArrayList<TRowResult>();
if(in == null) {
throw new NotFound();
return list;
}
TRowResult result = new TRowResult();
result.row = in.getRow();
result.columns = new TreeMap<byte[], TCell>(Bytes.BYTES_COMPARATOR);
for (Map.Entry<byte[], Cell> entry : in.entrySet()){
result.columns.put(entry.getKey(), ThriftUtilities.cellFromHBase(entry.getValue()));
Cell cell = entry.getValue();
result.columns.put(entry.getKey(),
new TCell(cell.getValue(), cell.getTimestamp()));
}
return result;
list.add(result);
return list;
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Autogenerated by Thrift
*
@ -29,21 +28,34 @@ import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
import java.util.Collections;
import com.facebook.thrift.protocol.*;
import com.facebook.thrift.transport.*;
import org.apache.thrift.*;
import org.apache.thrift.meta_data.*;
import org.apache.thrift.protocol.*;
/**
* 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 class AlreadyExists extends Exception implements TBase, java.io.Serializable, Cloneable {
private static final TStruct STRUCT_DESC = new TStruct("AlreadyExists");
private static final TField MESSAGE_FIELD_DESC = new TField("message", TType.STRING, (short)1);
public final Isset __isset = new Isset();
public static final class Isset implements java.io.Serializable {
public boolean message = false;
public String message;
public static final int MESSAGE = 1;
private final Isset __isset = new Isset();
private static final class Isset implements java.io.Serializable {
}
public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
put(MESSAGE, new FieldMetaData("message", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
}});
static {
FieldMetaData.addStructMetaDataMap(AlreadyExists.class, metaDataMap);
}
public AlreadyExists() {
@ -54,9 +66,81 @@ public class AlreadyExists extends Exception implements TBase, java.io.Serializa
{
this();
this.message = message;
this.__isset.message = true;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public AlreadyExists(AlreadyExists other) {
if (other.isSetMessage()) {
this.message = other.message;
}
}
@Override
public AlreadyExists clone() {
return new AlreadyExists(this);
}
public String getMessage() {
return this.message;
}
public void setMessage(String message) {
this.message = message;
}
public void unsetMessage() {
this.message = null;
}
// Returns true if field message is set (has been asigned a value) and false otherwise
public boolean isSetMessage() {
return this.message != null;
}
public void setMessageIsSet(boolean value) {
if (!value) {
this.message = null;
}
}
public void setFieldValue(int fieldID, Object value) {
switch (fieldID) {
case MESSAGE:
if (value == null) {
unsetMessage();
} else {
setMessage((String)value);
}
break;
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
public Object getFieldValue(int fieldID) {
switch (fieldID) {
case MESSAGE:
return getMessage();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
// Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
public boolean isSet(int fieldID) {
switch (fieldID) {
case MESSAGE:
return isSetMessage();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
@ -69,8 +153,8 @@ public class AlreadyExists extends Exception implements TBase, java.io.Serializa
if (that == null)
return false;
boolean this_present_message = true && (this.message != null);
boolean that_present_message = true && (that.message != null);
boolean this_present_message = true && this.isSetMessage();
boolean that_present_message = true && that.isSetMessage();
if (this_present_message || that_present_message) {
if (!(this_present_message && that_present_message))
return false;
@ -81,6 +165,7 @@ public class AlreadyExists extends Exception implements TBase, java.io.Serializa
return true;
}
@Override
public int hashCode() {
return 0;
}
@ -96,10 +181,9 @@ public class AlreadyExists extends Exception implements TBase, java.io.Serializa
}
switch (field.id)
{
case 1:
case MESSAGE:
if (field.type == TType.STRING) {
this.message = iprot.readString();
this.__isset.message = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
@ -111,17 +195,18 @@ public class AlreadyExists extends Exception implements TBase, java.io.Serializa
iprot.readFieldEnd();
}
iprot.readStructEnd();
// check for required fields of primitive type, which can't be checked in the validate method
validate();
}
public void write(TProtocol oprot) throws TException {
TStruct struct = new TStruct("AlreadyExists");
oprot.writeStructBegin(struct);
TField field = new TField();
validate();
oprot.writeStructBegin(STRUCT_DESC);
if (this.message != null) {
field.name = "message";
field.type = TType.STRING;
field.id = 1;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
oprot.writeString(this.message);
oprot.writeFieldEnd();
}
@ -129,13 +214,26 @@ public class AlreadyExists extends Exception implements TBase, java.io.Serializa
oprot.writeStructEnd();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("AlreadyExists(");
boolean first = true;
sb.append("message:");
sb.append(this.message);
if (this.message == null) {
sb.append("null");
} else {
sb.append(this.message);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws TException {
// check for required fields
// check that fields of type enum have valid values
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Autogenerated by Thrift
*
@ -29,22 +28,39 @@ import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
import java.util.Collections;
import com.facebook.thrift.protocol.*;
import com.facebook.thrift.transport.*;
import org.apache.thrift.*;
import org.apache.thrift.meta_data.*;
import org.apache.thrift.protocol.*;
/**
* A BatchMutation object is used to apply a number of Mutations to a single row.
*/
public class BatchMutation implements TBase, java.io.Serializable {
public byte[] row;
public List<Mutation> mutations;
public class BatchMutation implements TBase, java.io.Serializable, Cloneable {
private static final TStruct STRUCT_DESC = new TStruct("BatchMutation");
private static final TField ROW_FIELD_DESC = new TField("row", TType.STRING, (short)1);
private static final TField MUTATIONS_FIELD_DESC = new TField("mutations", TType.LIST, (short)2);
public final Isset __isset = new Isset();
public static final class Isset implements java.io.Serializable {
public boolean row = false;
public boolean mutations = false;
public byte[] row;
public static final int ROW = 1;
public List<Mutation> mutations;
public static final int MUTATIONS = 2;
private final Isset __isset = new Isset();
private static final class Isset implements java.io.Serializable {
}
public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
put(ROW, new FieldMetaData("row", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(MUTATIONS, new FieldMetaData("mutations", TFieldRequirementType.DEFAULT,
new ListMetaData(TType.LIST,
new StructMetaData(TType.STRUCT, Mutation.class))));
}});
static {
FieldMetaData.addStructMetaDataMap(BatchMutation.class, metaDataMap);
}
public BatchMutation() {
@ -56,11 +72,140 @@ public class BatchMutation implements TBase, java.io.Serializable {
{
this();
this.row = row;
this.__isset.row = true;
this.mutations = mutations;
this.__isset.mutations = true;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public BatchMutation(BatchMutation other) {
if (other.isSetRow()) {
this.row = other.row;
}
if (other.isSetMutations()) {
List<Mutation> __this__mutations = new ArrayList<Mutation>();
for (Mutation other_element : other.mutations) {
__this__mutations.add(new Mutation(other_element));
}
this.mutations = __this__mutations;
}
}
@Override
public BatchMutation clone() {
return new BatchMutation(this);
}
public byte[] getRow() {
return this.row;
}
public void setRow(byte[] row) {
this.row = row;
}
public void unsetRow() {
this.row = null;
}
// Returns true if field row is set (has been asigned a value) and false otherwise
public boolean isSetRow() {
return this.row != null;
}
public void setRowIsSet(boolean value) {
if (!value) {
this.row = null;
}
}
public int getMutationsSize() {
return (this.mutations == null) ? 0 : this.mutations.size();
}
public java.util.Iterator<Mutation> getMutationsIterator() {
return (this.mutations == null) ? null : this.mutations.iterator();
}
public void addToMutations(Mutation elem) {
if (this.mutations == null) {
this.mutations = new ArrayList<Mutation>();
}
this.mutations.add(elem);
}
public List<Mutation> getMutations() {
return this.mutations;
}
public void setMutations(List<Mutation> mutations) {
this.mutations = mutations;
}
public void unsetMutations() {
this.mutations = null;
}
// Returns true if field mutations is set (has been asigned a value) and false otherwise
public boolean isSetMutations() {
return this.mutations != null;
}
public void setMutationsIsSet(boolean value) {
if (!value) {
this.mutations = null;
}
}
public void setFieldValue(int fieldID, Object value) {
switch (fieldID) {
case ROW:
if (value == null) {
unsetRow();
} else {
setRow((byte[])value);
}
break;
case MUTATIONS:
if (value == null) {
unsetMutations();
} else {
setMutations((List<Mutation>)value);
}
break;
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
public Object getFieldValue(int fieldID) {
switch (fieldID) {
case ROW:
return getRow();
case MUTATIONS:
return getMutations();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
// Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
public boolean isSet(int fieldID) {
switch (fieldID) {
case ROW:
return isSetRow();
case MUTATIONS:
return isSetMutations();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
@ -73,8 +218,8 @@ public class BatchMutation implements TBase, java.io.Serializable {
if (that == null)
return false;
boolean this_present_row = true && (this.row != null);
boolean that_present_row = true && (that.row != null);
boolean this_present_row = true && this.isSetRow();
boolean that_present_row = true && that.isSetRow();
if (this_present_row || that_present_row) {
if (!(this_present_row && that_present_row))
return false;
@ -82,8 +227,8 @@ public class BatchMutation implements TBase, java.io.Serializable {
return false;
}
boolean this_present_mutations = true && (this.mutations != null);
boolean that_present_mutations = true && (that.mutations != null);
boolean this_present_mutations = true && this.isSetMutations();
boolean that_present_mutations = true && that.isSetMutations();
if (this_present_mutations || that_present_mutations) {
if (!(this_present_mutations && that_present_mutations))
return false;
@ -94,6 +239,7 @@ public class BatchMutation implements TBase, java.io.Serializable {
return true;
}
@Override
public int hashCode() {
return 0;
}
@ -109,29 +255,27 @@ public class BatchMutation implements TBase, java.io.Serializable {
}
switch (field.id)
{
case 1:
case ROW:
if (field.type == TType.STRING) {
this.row = iprot.readBinary();
this.__isset.row = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 2:
case MUTATIONS:
if (field.type == TType.LIST) {
{
TList _list0 = iprot.readListBegin();
this.mutations = new ArrayList<Mutation>(_list0.size);
for (int _i1 = 0; _i1 < _list0.size; ++_i1)
{
Mutation _elem2 = new Mutation();
Mutation _elem2;
_elem2 = new Mutation();
_elem2.read(iprot);
this.mutations.add(_elem2);
}
iprot.readListEnd();
}
this.__isset.mutations = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
@ -143,25 +287,23 @@ public class BatchMutation implements TBase, java.io.Serializable {
iprot.readFieldEnd();
}
iprot.readStructEnd();
// check for required fields of primitive type, which can't be checked in the validate method
validate();
}
public void write(TProtocol oprot) throws TException {
TStruct struct = new TStruct("BatchMutation");
oprot.writeStructBegin(struct);
TField field = new TField();
validate();
oprot.writeStructBegin(STRUCT_DESC);
if (this.row != null) {
field.name = "row";
field.type = TType.STRING;
field.id = 1;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(ROW_FIELD_DESC);
oprot.writeBinary(this.row);
oprot.writeFieldEnd();
}
if (this.mutations != null) {
field.name = "mutations";
field.type = TType.LIST;
field.id = 2;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
{
oprot.writeListBegin(new TList(TType.STRUCT, this.mutations.size()));
for (Mutation _iter3 : this.mutations) {
@ -175,15 +317,34 @@ public class BatchMutation implements TBase, java.io.Serializable {
oprot.writeStructEnd();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("BatchMutation(");
boolean first = true;
sb.append("row:");
sb.append(this.row);
sb.append(",mutations:");
sb.append(this.mutations);
if (this.row == null) {
sb.append("null");
} else {
sb.append(this.row);
}
first = false;
if (!first) sb.append(", ");
sb.append("mutations:");
if (this.mutations == null) {
sb.append("null");
} else {
sb.append(this.mutations);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws TException {
// check for required fields
// check that fields of type enum have valid values
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Autogenerated by Thrift
*
@ -29,42 +28,89 @@ import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
import java.util.Collections;
import com.facebook.thrift.protocol.*;
import com.facebook.thrift.transport.*;
import org.apache.thrift.*;
import org.apache.thrift.meta_data.*;
import org.apache.thrift.protocol.*;
/**
* 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 boolean blockCacheEnabled;
public int timeToLive;
public class ColumnDescriptor implements TBase, java.io.Serializable, Cloneable {
private static final TStruct STRUCT_DESC = new TStruct("ColumnDescriptor");
private static final TField NAME_FIELD_DESC = new TField("name", TType.STRING, (short)1);
private static final TField MAX_VERSIONS_FIELD_DESC = new TField("maxVersions", TType.I32, (short)2);
private static final TField COMPRESSION_FIELD_DESC = new TField("compression", TType.STRING, (short)3);
private static final TField IN_MEMORY_FIELD_DESC = new TField("inMemory", TType.BOOL, (short)4);
private static final TField MAX_VALUE_LENGTH_FIELD_DESC = new TField("maxValueLength", TType.I32, (short)5);
private static final TField BLOOM_FILTER_TYPE_FIELD_DESC = new TField("bloomFilterType", TType.STRING, (short)6);
private static final TField BLOOM_FILTER_VECTOR_SIZE_FIELD_DESC = new TField("bloomFilterVectorSize", TType.I32, (short)7);
private static final TField BLOOM_FILTER_NB_HASHES_FIELD_DESC = new TField("bloomFilterNbHashes", TType.I32, (short)8);
private static final TField BLOCK_CACHE_ENABLED_FIELD_DESC = new TField("blockCacheEnabled", TType.BOOL, (short)9);
private static final TField TIME_TO_LIVE_FIELD_DESC = new TField("timeToLive", TType.I32, (short)10);
public final Isset __isset = new Isset();
public static final class Isset implements java.io.Serializable {
public boolean name = false;
public byte[] name;
public static final int NAME = 1;
public int maxVersions;
public static final int MAXVERSIONS = 2;
public String compression;
public static final int COMPRESSION = 3;
public boolean inMemory;
public static final int INMEMORY = 4;
public int maxValueLength;
public static final int MAXVALUELENGTH = 5;
public String bloomFilterType;
public static final int BLOOMFILTERTYPE = 6;
public int bloomFilterVectorSize;
public static final int BLOOMFILTERVECTORSIZE = 7;
public int bloomFilterNbHashes;
public static final int BLOOMFILTERNBHASHES = 8;
public boolean blockCacheEnabled;
public static final int BLOCKCACHEENABLED = 9;
public int timeToLive;
public static final int TIMETOLIVE = 10;
private final Isset __isset = new Isset();
private static final class Isset implements java.io.Serializable {
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 boolean blockCacheEnabled = false;
public boolean timeToLive = false;
}
public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
put(NAME, new FieldMetaData("name", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(MAXVERSIONS, new FieldMetaData("maxVersions", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.I32)));
put(COMPRESSION, new FieldMetaData("compression", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(INMEMORY, new FieldMetaData("inMemory", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.BOOL)));
put(MAXVALUELENGTH, new FieldMetaData("maxValueLength", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.I32)));
put(BLOOMFILTERTYPE, new FieldMetaData("bloomFilterType", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(BLOOMFILTERVECTORSIZE, new FieldMetaData("bloomFilterVectorSize", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.I32)));
put(BLOOMFILTERNBHASHES, new FieldMetaData("bloomFilterNbHashes", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.I32)));
put(BLOCKCACHEENABLED, new FieldMetaData("blockCacheEnabled", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.BOOL)));
put(TIMETOLIVE, new FieldMetaData("timeToLive", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.I32)));
}});
static {
FieldMetaData.addStructMetaDataMap(ColumnDescriptor.class, metaDataMap);
}
public ColumnDescriptor() {
this.maxVersions = 3;
@ -100,17 +146,14 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
{
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;
@ -121,6 +164,416 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
this.__isset.timeToLive = true;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public ColumnDescriptor(ColumnDescriptor other) {
if (other.isSetName()) {
this.name = other.name;
}
__isset.maxVersions = other.__isset.maxVersions;
this.maxVersions = other.maxVersions;
if (other.isSetCompression()) {
this.compression = other.compression;
}
__isset.inMemory = other.__isset.inMemory;
this.inMemory = other.inMemory;
__isset.maxValueLength = other.__isset.maxValueLength;
this.maxValueLength = other.maxValueLength;
if (other.isSetBloomFilterType()) {
this.bloomFilterType = other.bloomFilterType;
}
__isset.bloomFilterVectorSize = other.__isset.bloomFilterVectorSize;
this.bloomFilterVectorSize = other.bloomFilterVectorSize;
__isset.bloomFilterNbHashes = other.__isset.bloomFilterNbHashes;
this.bloomFilterNbHashes = other.bloomFilterNbHashes;
__isset.blockCacheEnabled = other.__isset.blockCacheEnabled;
this.blockCacheEnabled = other.blockCacheEnabled;
__isset.timeToLive = other.__isset.timeToLive;
this.timeToLive = other.timeToLive;
}
@Override
public ColumnDescriptor clone() {
return new ColumnDescriptor(this);
}
public byte[] getName() {
return this.name;
}
public void setName(byte[] name) {
this.name = name;
}
public void unsetName() {
this.name = null;
}
// Returns true if field name is set (has been asigned a value) and false otherwise
public boolean isSetName() {
return this.name != null;
}
public void setNameIsSet(boolean value) {
if (!value) {
this.name = null;
}
}
public int getMaxVersions() {
return this.maxVersions;
}
public void setMaxVersions(int maxVersions) {
this.maxVersions = maxVersions;
this.__isset.maxVersions = true;
}
public void unsetMaxVersions() {
this.__isset.maxVersions = false;
}
// Returns true if field maxVersions is set (has been asigned a value) and false otherwise
public boolean isSetMaxVersions() {
return this.__isset.maxVersions;
}
public void setMaxVersionsIsSet(boolean value) {
this.__isset.maxVersions = value;
}
public String getCompression() {
return this.compression;
}
public void setCompression(String compression) {
this.compression = compression;
}
public void unsetCompression() {
this.compression = null;
}
// Returns true if field compression is set (has been asigned a value) and false otherwise
public boolean isSetCompression() {
return this.compression != null;
}
public void setCompressionIsSet(boolean value) {
if (!value) {
this.compression = null;
}
}
public boolean isInMemory() {
return this.inMemory;
}
public void setInMemory(boolean inMemory) {
this.inMemory = inMemory;
this.__isset.inMemory = true;
}
public void unsetInMemory() {
this.__isset.inMemory = false;
}
// Returns true if field inMemory is set (has been asigned a value) and false otherwise
public boolean isSetInMemory() {
return this.__isset.inMemory;
}
public void setInMemoryIsSet(boolean value) {
this.__isset.inMemory = value;
}
public int getMaxValueLength() {
return this.maxValueLength;
}
public void setMaxValueLength(int maxValueLength) {
this.maxValueLength = maxValueLength;
this.__isset.maxValueLength = true;
}
public void unsetMaxValueLength() {
this.__isset.maxValueLength = false;
}
// Returns true if field maxValueLength is set (has been asigned a value) and false otherwise
public boolean isSetMaxValueLength() {
return this.__isset.maxValueLength;
}
public void setMaxValueLengthIsSet(boolean value) {
this.__isset.maxValueLength = value;
}
public String getBloomFilterType() {
return this.bloomFilterType;
}
public void setBloomFilterType(String bloomFilterType) {
this.bloomFilterType = bloomFilterType;
}
public void unsetBloomFilterType() {
this.bloomFilterType = null;
}
// Returns true if field bloomFilterType is set (has been asigned a value) and false otherwise
public boolean isSetBloomFilterType() {
return this.bloomFilterType != null;
}
public void setBloomFilterTypeIsSet(boolean value) {
if (!value) {
this.bloomFilterType = null;
}
}
public int getBloomFilterVectorSize() {
return this.bloomFilterVectorSize;
}
public void setBloomFilterVectorSize(int bloomFilterVectorSize) {
this.bloomFilterVectorSize = bloomFilterVectorSize;
this.__isset.bloomFilterVectorSize = true;
}
public void unsetBloomFilterVectorSize() {
this.__isset.bloomFilterVectorSize = false;
}
// Returns true if field bloomFilterVectorSize is set (has been asigned a value) and false otherwise
public boolean isSetBloomFilterVectorSize() {
return this.__isset.bloomFilterVectorSize;
}
public void setBloomFilterVectorSizeIsSet(boolean value) {
this.__isset.bloomFilterVectorSize = value;
}
public int getBloomFilterNbHashes() {
return this.bloomFilterNbHashes;
}
public void setBloomFilterNbHashes(int bloomFilterNbHashes) {
this.bloomFilterNbHashes = bloomFilterNbHashes;
this.__isset.bloomFilterNbHashes = true;
}
public void unsetBloomFilterNbHashes() {
this.__isset.bloomFilterNbHashes = false;
}
// Returns true if field bloomFilterNbHashes is set (has been asigned a value) and false otherwise
public boolean isSetBloomFilterNbHashes() {
return this.__isset.bloomFilterNbHashes;
}
public void setBloomFilterNbHashesIsSet(boolean value) {
this.__isset.bloomFilterNbHashes = value;
}
public boolean isBlockCacheEnabled() {
return this.blockCacheEnabled;
}
public void setBlockCacheEnabled(boolean blockCacheEnabled) {
this.blockCacheEnabled = blockCacheEnabled;
this.__isset.blockCacheEnabled = true;
}
public void unsetBlockCacheEnabled() {
this.__isset.blockCacheEnabled = false;
}
// Returns true if field blockCacheEnabled is set (has been asigned a value) and false otherwise
public boolean isSetBlockCacheEnabled() {
return this.__isset.blockCacheEnabled;
}
public void setBlockCacheEnabledIsSet(boolean value) {
this.__isset.blockCacheEnabled = value;
}
public int getTimeToLive() {
return this.timeToLive;
}
public void setTimeToLive(int timeToLive) {
this.timeToLive = timeToLive;
this.__isset.timeToLive = true;
}
public void unsetTimeToLive() {
this.__isset.timeToLive = false;
}
// Returns true if field timeToLive is set (has been asigned a value) and false otherwise
public boolean isSetTimeToLive() {
return this.__isset.timeToLive;
}
public void setTimeToLiveIsSet(boolean value) {
this.__isset.timeToLive = value;
}
public void setFieldValue(int fieldID, Object value) {
switch (fieldID) {
case NAME:
if (value == null) {
unsetName();
} else {
setName((byte[])value);
}
break;
case MAXVERSIONS:
if (value == null) {
unsetMaxVersions();
} else {
setMaxVersions((Integer)value);
}
break;
case COMPRESSION:
if (value == null) {
unsetCompression();
} else {
setCompression((String)value);
}
break;
case INMEMORY:
if (value == null) {
unsetInMemory();
} else {
setInMemory((Boolean)value);
}
break;
case MAXVALUELENGTH:
if (value == null) {
unsetMaxValueLength();
} else {
setMaxValueLength((Integer)value);
}
break;
case BLOOMFILTERTYPE:
if (value == null) {
unsetBloomFilterType();
} else {
setBloomFilterType((String)value);
}
break;
case BLOOMFILTERVECTORSIZE:
if (value == null) {
unsetBloomFilterVectorSize();
} else {
setBloomFilterVectorSize((Integer)value);
}
break;
case BLOOMFILTERNBHASHES:
if (value == null) {
unsetBloomFilterNbHashes();
} else {
setBloomFilterNbHashes((Integer)value);
}
break;
case BLOCKCACHEENABLED:
if (value == null) {
unsetBlockCacheEnabled();
} else {
setBlockCacheEnabled((Boolean)value);
}
break;
case TIMETOLIVE:
if (value == null) {
unsetTimeToLive();
} else {
setTimeToLive((Integer)value);
}
break;
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
public Object getFieldValue(int fieldID) {
switch (fieldID) {
case NAME:
return getName();
case MAXVERSIONS:
return new Integer(getMaxVersions());
case COMPRESSION:
return getCompression();
case INMEMORY:
return new Boolean(isInMemory());
case MAXVALUELENGTH:
return new Integer(getMaxValueLength());
case BLOOMFILTERTYPE:
return getBloomFilterType();
case BLOOMFILTERVECTORSIZE:
return new Integer(getBloomFilterVectorSize());
case BLOOMFILTERNBHASHES:
return new Integer(getBloomFilterNbHashes());
case BLOCKCACHEENABLED:
return new Boolean(isBlockCacheEnabled());
case TIMETOLIVE:
return new Integer(getTimeToLive());
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
// Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
public boolean isSet(int fieldID) {
switch (fieldID) {
case NAME:
return isSetName();
case MAXVERSIONS:
return isSetMaxVersions();
case COMPRESSION:
return isSetCompression();
case INMEMORY:
return isSetInMemory();
case MAXVALUELENGTH:
return isSetMaxValueLength();
case BLOOMFILTERTYPE:
return isSetBloomFilterType();
case BLOOMFILTERVECTORSIZE:
return isSetBloomFilterVectorSize();
case BLOOMFILTERNBHASHES:
return isSetBloomFilterNbHashes();
case BLOCKCACHEENABLED:
return isSetBlockCacheEnabled();
case TIMETOLIVE:
return isSetTimeToLive();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
@ -133,8 +586,8 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
if (that == null)
return false;
boolean this_present_name = true && (this.name != null);
boolean that_present_name = true && (that.name != null);
boolean this_present_name = true && this.isSetName();
boolean that_present_name = true && that.isSetName();
if (this_present_name || that_present_name) {
if (!(this_present_name && that_present_name))
return false;
@ -151,8 +604,8 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
return false;
}
boolean this_present_compression = true && (this.compression != null);
boolean that_present_compression = true && (that.compression != null);
boolean this_present_compression = true && this.isSetCompression();
boolean that_present_compression = true && that.isSetCompression();
if (this_present_compression || that_present_compression) {
if (!(this_present_compression && that_present_compression))
return false;
@ -178,8 +631,8 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
return false;
}
boolean this_present_bloomFilterType = true && (this.bloomFilterType != null);
boolean that_present_bloomFilterType = true && (that.bloomFilterType != null);
boolean this_present_bloomFilterType = true && this.isSetBloomFilterType();
boolean that_present_bloomFilterType = true && that.isSetBloomFilterType();
if (this_present_bloomFilterType || that_present_bloomFilterType) {
if (!(this_present_bloomFilterType && that_present_bloomFilterType))
return false;
@ -226,6 +679,7 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
return true;
}
@Override
public int hashCode() {
return 0;
}
@ -241,15 +695,14 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
}
switch (field.id)
{
case 1:
case NAME:
if (field.type == TType.STRING) {
this.name = iprot.readBinary();
this.__isset.name = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 2:
case MAXVERSIONS:
if (field.type == TType.I32) {
this.maxVersions = iprot.readI32();
this.__isset.maxVersions = true;
@ -257,15 +710,14 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 3:
case COMPRESSION:
if (field.type == TType.STRING) {
this.compression = iprot.readString();
this.__isset.compression = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 4:
case INMEMORY:
if (field.type == TType.BOOL) {
this.inMemory = iprot.readBool();
this.__isset.inMemory = true;
@ -273,7 +725,7 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 5:
case MAXVALUELENGTH:
if (field.type == TType.I32) {
this.maxValueLength = iprot.readI32();
this.__isset.maxValueLength = true;
@ -281,15 +733,14 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 6:
case BLOOMFILTERTYPE:
if (field.type == TType.STRING) {
this.bloomFilterType = iprot.readString();
this.__isset.bloomFilterType = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 7:
case BLOOMFILTERVECTORSIZE:
if (field.type == TType.I32) {
this.bloomFilterVectorSize = iprot.readI32();
this.__isset.bloomFilterVectorSize = true;
@ -297,7 +748,7 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 8:
case BLOOMFILTERNBHASHES:
if (field.type == TType.I32) {
this.bloomFilterNbHashes = iprot.readI32();
this.__isset.bloomFilterNbHashes = true;
@ -305,7 +756,7 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 9:
case BLOCKCACHEENABLED:
if (field.type == TType.BOOL) {
this.blockCacheEnabled = iprot.readBool();
this.__isset.blockCacheEnabled = true;
@ -313,7 +764,7 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 10:
case TIMETOLIVE:
if (field.type == TType.I32) {
this.timeToLive = iprot.readI32();
this.__isset.timeToLive = true;
@ -328,107 +779,120 @@ public class ColumnDescriptor implements TBase, java.io.Serializable {
iprot.readFieldEnd();
}
iprot.readStructEnd();
// check for required fields of primitive type, which can't be checked in the validate method
validate();
}
public void write(TProtocol oprot) throws TException {
TStruct struct = new TStruct("ColumnDescriptor");
oprot.writeStructBegin(struct);
TField field = new TField();
validate();
oprot.writeStructBegin(STRUCT_DESC);
if (this.name != null) {
field.name = "name";
field.type = TType.STRING;
field.id = 1;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(NAME_FIELD_DESC);
oprot.writeBinary(this.name);
oprot.writeFieldEnd();
}
field.name = "maxVersions";
field.type = TType.I32;
field.id = 2;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(MAX_VERSIONS_FIELD_DESC);
oprot.writeI32(this.maxVersions);
oprot.writeFieldEnd();
if (this.compression != null) {
field.name = "compression";
field.type = TType.STRING;
field.id = 3;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(COMPRESSION_FIELD_DESC);
oprot.writeString(this.compression);
oprot.writeFieldEnd();
}
field.name = "inMemory";
field.type = TType.BOOL;
field.id = 4;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(IN_MEMORY_FIELD_DESC);
oprot.writeBool(this.inMemory);
oprot.writeFieldEnd();
field.name = "maxValueLength";
field.type = TType.I32;
field.id = 5;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(MAX_VALUE_LENGTH_FIELD_DESC);
oprot.writeI32(this.maxValueLength);
oprot.writeFieldEnd();
if (this.bloomFilterType != null) {
field.name = "bloomFilterType";
field.type = TType.STRING;
field.id = 6;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(BLOOM_FILTER_TYPE_FIELD_DESC);
oprot.writeString(this.bloomFilterType);
oprot.writeFieldEnd();
}
field.name = "bloomFilterVectorSize";
field.type = TType.I32;
field.id = 7;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(BLOOM_FILTER_VECTOR_SIZE_FIELD_DESC);
oprot.writeI32(this.bloomFilterVectorSize);
oprot.writeFieldEnd();
field.name = "bloomFilterNbHashes";
field.type = TType.I32;
field.id = 8;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(BLOOM_FILTER_NB_HASHES_FIELD_DESC);
oprot.writeI32(this.bloomFilterNbHashes);
oprot.writeFieldEnd();
field.name = "blockCacheEnabled";
field.type = TType.BOOL;
field.id = 9;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(BLOCK_CACHE_ENABLED_FIELD_DESC);
oprot.writeBool(this.blockCacheEnabled);
oprot.writeFieldEnd();
field.name = "timeToLive";
field.type = TType.I32;
field.id = 10;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(TIME_TO_LIVE_FIELD_DESC);
oprot.writeI32(this.timeToLive);
oprot.writeFieldEnd();
oprot.writeFieldStop();
oprot.writeStructEnd();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("ColumnDescriptor(");
boolean first = true;
sb.append("name:");
sb.append(this.name);
sb.append(",maxVersions:");
if (this.name == null) {
sb.append("null");
} else {
sb.append(this.name);
}
first = false;
if (!first) sb.append(", ");
sb.append("maxVersions:");
sb.append(this.maxVersions);
sb.append(",compression:");
sb.append(this.compression);
sb.append(",inMemory:");
first = false;
if (!first) sb.append(", ");
sb.append("compression:");
if (this.compression == null) {
sb.append("null");
} else {
sb.append(this.compression);
}
first = false;
if (!first) sb.append(", ");
sb.append("inMemory:");
sb.append(this.inMemory);
sb.append(",maxValueLength:");
first = false;
if (!first) sb.append(", ");
sb.append("maxValueLength:");
sb.append(this.maxValueLength);
sb.append(",bloomFilterType:");
sb.append(this.bloomFilterType);
sb.append(",bloomFilterVectorSize:");
first = false;
if (!first) sb.append(", ");
sb.append("bloomFilterType:");
if (this.bloomFilterType == null) {
sb.append("null");
} else {
sb.append(this.bloomFilterType);
}
first = false;
if (!first) sb.append(", ");
sb.append("bloomFilterVectorSize:");
sb.append(this.bloomFilterVectorSize);
sb.append(",bloomFilterNbHashes:");
first = false;
if (!first) sb.append(", ");
sb.append("bloomFilterNbHashes:");
sb.append(this.bloomFilterNbHashes);
sb.append(",blockCacheEnabled:");
first = false;
if (!first) sb.append(", ");
sb.append("blockCacheEnabled:");
sb.append(this.blockCacheEnabled);
sb.append(",timeToLive:");
first = false;
if (!first) sb.append(", ");
sb.append("timeToLive:");
sb.append(this.timeToLive);
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws TException {
// check for required fields
// check that fields of type enum have valid values
}
}

View File

@ -1,36 +0,0 @@
/**
* 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.List;
import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
public class Constants {
}

File diff suppressed because it is too large Load Diff

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Autogenerated by Thrift
*
@ -29,22 +28,35 @@ import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
import java.util.Collections;
import com.facebook.thrift.protocol.*;
import com.facebook.thrift.transport.*;
import org.apache.thrift.*;
import org.apache.thrift.meta_data.*;
import org.apache.thrift.protocol.*;
/**
* 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 class IOError extends Exception implements TBase, java.io.Serializable, Cloneable {
private static final TStruct STRUCT_DESC = new TStruct("IOError");
private static final TField MESSAGE_FIELD_DESC = new TField("message", TType.STRING, (short)1);
public final Isset __isset = new Isset();
public static final class Isset implements java.io.Serializable {
public boolean message = false;
public String message;
public static final int MESSAGE = 1;
private final Isset __isset = new Isset();
private static final class Isset implements java.io.Serializable {
}
public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
put(MESSAGE, new FieldMetaData("message", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
}});
static {
FieldMetaData.addStructMetaDataMap(IOError.class, metaDataMap);
}
public IOError() {
@ -55,9 +67,81 @@ public class IOError extends Exception implements TBase, java.io.Serializable {
{
this();
this.message = message;
this.__isset.message = true;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public IOError(IOError other) {
if (other.isSetMessage()) {
this.message = other.message;
}
}
@Override
public IOError clone() {
return new IOError(this);
}
public String getMessage() {
return this.message;
}
public void setMessage(String message) {
this.message = message;
}
public void unsetMessage() {
this.message = null;
}
// Returns true if field message is set (has been asigned a value) and false otherwise
public boolean isSetMessage() {
return this.message != null;
}
public void setMessageIsSet(boolean value) {
if (!value) {
this.message = null;
}
}
public void setFieldValue(int fieldID, Object value) {
switch (fieldID) {
case MESSAGE:
if (value == null) {
unsetMessage();
} else {
setMessage((String)value);
}
break;
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
public Object getFieldValue(int fieldID) {
switch (fieldID) {
case MESSAGE:
return getMessage();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
// Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
public boolean isSet(int fieldID) {
switch (fieldID) {
case MESSAGE:
return isSetMessage();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
@ -70,8 +154,8 @@ public class IOError extends Exception implements TBase, java.io.Serializable {
if (that == null)
return false;
boolean this_present_message = true && (this.message != null);
boolean that_present_message = true && (that.message != null);
boolean this_present_message = true && this.isSetMessage();
boolean that_present_message = true && that.isSetMessage();
if (this_present_message || that_present_message) {
if (!(this_present_message && that_present_message))
return false;
@ -82,6 +166,7 @@ public class IOError extends Exception implements TBase, java.io.Serializable {
return true;
}
@Override
public int hashCode() {
return 0;
}
@ -97,10 +182,9 @@ public class IOError extends Exception implements TBase, java.io.Serializable {
}
switch (field.id)
{
case 1:
case MESSAGE:
if (field.type == TType.STRING) {
this.message = iprot.readString();
this.__isset.message = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
@ -112,17 +196,18 @@ public class IOError extends Exception implements TBase, java.io.Serializable {
iprot.readFieldEnd();
}
iprot.readStructEnd();
// check for required fields of primitive type, which can't be checked in the validate method
validate();
}
public void write(TProtocol oprot) throws TException {
TStruct struct = new TStruct("IOError");
oprot.writeStructBegin(struct);
TField field = new TField();
validate();
oprot.writeStructBegin(STRUCT_DESC);
if (this.message != null) {
field.name = "message";
field.type = TType.STRING;
field.id = 1;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
oprot.writeString(this.message);
oprot.writeFieldEnd();
}
@ -130,13 +215,26 @@ public class IOError extends Exception implements TBase, java.io.Serializable {
oprot.writeStructEnd();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("IOError(");
boolean first = true;
sb.append("message:");
sb.append(this.message);
if (this.message == null) {
sb.append("null");
} else {
sb.append(this.message);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws TException {
// check for required fields
// check that fields of type enum have valid values
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Autogenerated by Thrift
*
@ -29,21 +28,34 @@ import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
import java.util.Collections;
import com.facebook.thrift.protocol.*;
import com.facebook.thrift.transport.*;
import org.apache.thrift.*;
import org.apache.thrift.meta_data.*;
import org.apache.thrift.protocol.*;
/**
* 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 class IllegalArgument extends Exception implements TBase, java.io.Serializable, Cloneable {
private static final TStruct STRUCT_DESC = new TStruct("IllegalArgument");
private static final TField MESSAGE_FIELD_DESC = new TField("message", TType.STRING, (short)1);
public final Isset __isset = new Isset();
public static final class Isset implements java.io.Serializable {
public boolean message = false;
public String message;
public static final int MESSAGE = 1;
private final Isset __isset = new Isset();
private static final class Isset implements java.io.Serializable {
}
public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
put(MESSAGE, new FieldMetaData("message", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
}});
static {
FieldMetaData.addStructMetaDataMap(IllegalArgument.class, metaDataMap);
}
public IllegalArgument() {
@ -54,9 +66,81 @@ public class IllegalArgument extends Exception implements TBase, java.io.Seriali
{
this();
this.message = message;
this.__isset.message = true;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public IllegalArgument(IllegalArgument other) {
if (other.isSetMessage()) {
this.message = other.message;
}
}
@Override
public IllegalArgument clone() {
return new IllegalArgument(this);
}
public String getMessage() {
return this.message;
}
public void setMessage(String message) {
this.message = message;
}
public void unsetMessage() {
this.message = null;
}
// Returns true if field message is set (has been asigned a value) and false otherwise
public boolean isSetMessage() {
return this.message != null;
}
public void setMessageIsSet(boolean value) {
if (!value) {
this.message = null;
}
}
public void setFieldValue(int fieldID, Object value) {
switch (fieldID) {
case MESSAGE:
if (value == null) {
unsetMessage();
} else {
setMessage((String)value);
}
break;
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
public Object getFieldValue(int fieldID) {
switch (fieldID) {
case MESSAGE:
return getMessage();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
// Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
public boolean isSet(int fieldID) {
switch (fieldID) {
case MESSAGE:
return isSetMessage();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
@ -69,8 +153,8 @@ public class IllegalArgument extends Exception implements TBase, java.io.Seriali
if (that == null)
return false;
boolean this_present_message = true && (this.message != null);
boolean that_present_message = true && (that.message != null);
boolean this_present_message = true && this.isSetMessage();
boolean that_present_message = true && that.isSetMessage();
if (this_present_message || that_present_message) {
if (!(this_present_message && that_present_message))
return false;
@ -81,6 +165,7 @@ public class IllegalArgument extends Exception implements TBase, java.io.Seriali
return true;
}
@Override
public int hashCode() {
return 0;
}
@ -96,10 +181,9 @@ public class IllegalArgument extends Exception implements TBase, java.io.Seriali
}
switch (field.id)
{
case 1:
case MESSAGE:
if (field.type == TType.STRING) {
this.message = iprot.readString();
this.__isset.message = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
@ -111,17 +195,18 @@ public class IllegalArgument extends Exception implements TBase, java.io.Seriali
iprot.readFieldEnd();
}
iprot.readStructEnd();
// check for required fields of primitive type, which can't be checked in the validate method
validate();
}
public void write(TProtocol oprot) throws TException {
TStruct struct = new TStruct("IllegalArgument");
oprot.writeStructBegin(struct);
TField field = new TField();
validate();
oprot.writeStructBegin(STRUCT_DESC);
if (this.message != null) {
field.name = "message";
field.type = TType.STRING;
field.id = 1;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
oprot.writeString(this.message);
oprot.writeFieldEnd();
}
@ -129,13 +214,26 @@ public class IllegalArgument extends Exception implements TBase, java.io.Seriali
oprot.writeStructEnd();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("IllegalArgument(");
boolean first = true;
sb.append("message:");
sb.append(this.message);
if (this.message == null) {
sb.append("null");
} else {
sb.append(this.message);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws TException {
// check for required fields
// check that fields of type enum have valid values
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Autogenerated by Thrift
*
@ -29,24 +28,44 @@ import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
import java.util.Collections;
import com.facebook.thrift.protocol.*;
import com.facebook.thrift.transport.*;
import org.apache.thrift.*;
import org.apache.thrift.meta_data.*;
import org.apache.thrift.protocol.*;
/**
* 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 class Mutation implements TBase, java.io.Serializable, Cloneable {
private static final TStruct STRUCT_DESC = new TStruct("Mutation");
private static final TField IS_DELETE_FIELD_DESC = new TField("isDelete", TType.BOOL, (short)1);
private static final TField COLUMN_FIELD_DESC = new TField("column", TType.STRING, (short)2);
private static final TField VALUE_FIELD_DESC = new TField("value", TType.STRING, (short)3);
public final Isset __isset = new Isset();
public static final class Isset implements java.io.Serializable {
public boolean isDelete;
public static final int ISDELETE = 1;
public byte[] column;
public static final int COLUMN = 2;
public byte[] value;
public static final int VALUE = 3;
private final Isset __isset = new Isset();
private static final class Isset implements java.io.Serializable {
public boolean isDelete = false;
public boolean column = false;
public boolean value = false;
}
public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
put(ISDELETE, new FieldMetaData("isDelete", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.BOOL)));
put(COLUMN, new FieldMetaData("column", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(VALUE, new FieldMetaData("value", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
}});
static {
FieldMetaData.addStructMetaDataMap(Mutation.class, metaDataMap);
}
public Mutation() {
@ -63,11 +82,158 @@ public class Mutation implements TBase, java.io.Serializable {
this.isDelete = isDelete;
this.__isset.isDelete = true;
this.column = column;
this.__isset.column = true;
this.value = value;
this.__isset.value = true;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public Mutation(Mutation other) {
__isset.isDelete = other.__isset.isDelete;
this.isDelete = other.isDelete;
if (other.isSetColumn()) {
this.column = other.column;
}
if (other.isSetValue()) {
this.value = other.value;
}
}
@Override
public Mutation clone() {
return new Mutation(this);
}
public boolean isIsDelete() {
return this.isDelete;
}
public void setIsDelete(boolean isDelete) {
this.isDelete = isDelete;
this.__isset.isDelete = true;
}
public void unsetIsDelete() {
this.__isset.isDelete = false;
}
// Returns true if field isDelete is set (has been asigned a value) and false otherwise
public boolean isSetIsDelete() {
return this.__isset.isDelete;
}
public void setIsDeleteIsSet(boolean value) {
this.__isset.isDelete = value;
}
public byte[] getColumn() {
return this.column;
}
public void setColumn(byte[] column) {
this.column = column;
}
public void unsetColumn() {
this.column = null;
}
// Returns true if field column is set (has been asigned a value) and false otherwise
public boolean isSetColumn() {
return this.column != null;
}
public void setColumnIsSet(boolean value) {
if (!value) {
this.column = null;
}
}
public byte[] getValue() {
return this.value;
}
public void setValue(byte[] value) {
this.value = value;
}
public void unsetValue() {
this.value = null;
}
// Returns true if field value is set (has been asigned a value) and false otherwise
public boolean isSetValue() {
return this.value != null;
}
public void setValueIsSet(boolean value) {
if (!value) {
this.value = null;
}
}
public void setFieldValue(int fieldID, Object value) {
switch (fieldID) {
case ISDELETE:
if (value == null) {
unsetIsDelete();
} else {
setIsDelete((Boolean)value);
}
break;
case COLUMN:
if (value == null) {
unsetColumn();
} else {
setColumn((byte[])value);
}
break;
case VALUE:
if (value == null) {
unsetValue();
} else {
setValue((byte[])value);
}
break;
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
public Object getFieldValue(int fieldID) {
switch (fieldID) {
case ISDELETE:
return new Boolean(isIsDelete());
case COLUMN:
return getColumn();
case VALUE:
return getValue();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
// Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
public boolean isSet(int fieldID) {
switch (fieldID) {
case ISDELETE:
return isSetIsDelete();
case COLUMN:
return isSetColumn();
case VALUE:
return isSetValue();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
@ -89,8 +255,8 @@ public class Mutation implements TBase, java.io.Serializable {
return false;
}
boolean this_present_column = true && (this.column != null);
boolean that_present_column = true && (that.column != null);
boolean this_present_column = true && this.isSetColumn();
boolean that_present_column = true && that.isSetColumn();
if (this_present_column || that_present_column) {
if (!(this_present_column && that_present_column))
return false;
@ -98,8 +264,8 @@ public class Mutation implements TBase, java.io.Serializable {
return false;
}
boolean this_present_value = true && (this.value != null);
boolean that_present_value = true && (that.value != null);
boolean this_present_value = true && this.isSetValue();
boolean that_present_value = true && that.isSetValue();
if (this_present_value || that_present_value) {
if (!(this_present_value && that_present_value))
return false;
@ -110,6 +276,7 @@ public class Mutation implements TBase, java.io.Serializable {
return true;
}
@Override
public int hashCode() {
return 0;
}
@ -125,7 +292,7 @@ public class Mutation implements TBase, java.io.Serializable {
}
switch (field.id)
{
case 1:
case ISDELETE:
if (field.type == TType.BOOL) {
this.isDelete = iprot.readBool();
this.__isset.isDelete = true;
@ -133,18 +300,16 @@ public class Mutation implements TBase, java.io.Serializable {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 2:
case COLUMN:
if (field.type == TType.STRING) {
this.column = iprot.readBinary();
this.__isset.column = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 3:
case VALUE:
if (field.type == TType.STRING) {
this.value = iprot.readBinary();
this.__isset.value = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
@ -156,31 +321,26 @@ public class Mutation implements TBase, java.io.Serializable {
iprot.readFieldEnd();
}
iprot.readStructEnd();
// check for required fields of primitive type, which can't be checked in the validate method
validate();
}
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);
validate();
oprot.writeStructBegin(STRUCT_DESC);
oprot.writeFieldBegin(IS_DELETE_FIELD_DESC);
oprot.writeBool(this.isDelete);
oprot.writeFieldEnd();
if (this.column != null) {
field.name = "column";
field.type = TType.STRING;
field.id = 2;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(COLUMN_FIELD_DESC);
oprot.writeBinary(this.column);
oprot.writeFieldEnd();
}
if (this.value != null) {
field.name = "value";
field.type = TType.STRING;
field.id = 3;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(VALUE_FIELD_DESC);
oprot.writeBinary(this.value);
oprot.writeFieldEnd();
}
@ -188,17 +348,38 @@ public class Mutation implements TBase, java.io.Serializable {
oprot.writeStructEnd();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("Mutation(");
boolean first = true;
sb.append("isDelete:");
sb.append(this.isDelete);
sb.append(",column:");
sb.append(this.column);
sb.append(",value:");
sb.append(this.value);
first = false;
if (!first) sb.append(", ");
sb.append("column:");
if (this.column == null) {
sb.append("null");
} else {
sb.append(this.column);
}
first = false;
if (!first) sb.append(", ");
sb.append("value:");
if (this.value == null) {
sb.append("null");
} else {
sb.append(this.value);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws TException {
// check for required fields
// check that fields of type enum have valid values
}
}

View File

@ -1,141 +0,0 @@
/**
* 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.List;
import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.Set;
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 implements java.io.Serializable {
public boolean message = false;
}
public NotFound() {
}
public NotFound(
String message)
{
this();
this.message = message;
this.__isset.message = true;
}
public boolean equals(Object that) {
if (that == null)
return false;
if (that instanceof NotFound)
return this.equals((NotFound)that);
return false;
}
public boolean equals(NotFound that) {
if (that == null)
return false;
boolean this_present_message = true && (this.message != null);
boolean that_present_message = true && (that.message != null);
if (this_present_message || that_present_message) {
if (!(this_present_message && that_present_message))
return false;
if (!this.message.equals(that.message))
return false;
}
return true;
}
public int hashCode() {
return 0;
}
public void read(TProtocol iprot) throws TException {
TField field;
iprot.readStructBegin();
while (true)
{
field = iprot.readFieldBegin();
if (field.type == TType.STOP) {
break;
}
switch (field.id)
{
case 1:
if (field.type == TType.STRING) {
this.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();
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Autogenerated by Thrift
*
@ -29,10 +28,11 @@ import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
import java.util.Collections;
import com.facebook.thrift.protocol.*;
import com.facebook.thrift.transport.*;
import org.apache.thrift.*;
import org.apache.thrift.meta_data.*;
import org.apache.thrift.protocol.*;
/**
* TCell - Used to transport a cell value (byte[]) and the timestamp it was
@ -40,16 +40,32 @@ import com.facebook.thrift.transport.*;
* the timestamp of a cell to a first-class value, making it easy to take
* note of temporal data. Cell is used all the way from HStore up to HTable.
*/
public class TCell implements TBase, java.io.Serializable {
public byte[] value;
public long timestamp;
public class TCell implements TBase, java.io.Serializable, Cloneable {
private static final TStruct STRUCT_DESC = new TStruct("TCell");
private static final TField VALUE_FIELD_DESC = new TField("value", TType.STRING, (short)1);
private static final TField TIMESTAMP_FIELD_DESC = new TField("timestamp", TType.I64, (short)2);
public final Isset __isset = new Isset();
public static final class Isset implements java.io.Serializable {
public boolean value = false;
public byte[] value;
public static final int VALUE = 1;
public long timestamp;
public static final int TIMESTAMP = 2;
private final Isset __isset = new Isset();
private static final class Isset implements java.io.Serializable {
public boolean timestamp = false;
}
public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
put(VALUE, new FieldMetaData("value", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(TIMESTAMP, new FieldMetaData("timestamp", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.I64)));
}});
static {
FieldMetaData.addStructMetaDataMap(TCell.class, metaDataMap);
}
public TCell() {
}
@ -59,11 +75,120 @@ public class TCell implements TBase, java.io.Serializable {
{
this();
this.value = value;
this.__isset.value = true;
this.timestamp = timestamp;
this.__isset.timestamp = true;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TCell(TCell other) {
if (other.isSetValue()) {
this.value = other.value;
}
__isset.timestamp = other.__isset.timestamp;
this.timestamp = other.timestamp;
}
@Override
public TCell clone() {
return new TCell(this);
}
public byte[] getValue() {
return this.value;
}
public void setValue(byte[] value) {
this.value = value;
}
public void unsetValue() {
this.value = null;
}
// Returns true if field value is set (has been asigned a value) and false otherwise
public boolean isSetValue() {
return this.value != null;
}
public void setValueIsSet(boolean value) {
if (!value) {
this.value = null;
}
}
public long getTimestamp() {
return this.timestamp;
}
public void setTimestamp(long timestamp) {
this.timestamp = timestamp;
this.__isset.timestamp = true;
}
public void unsetTimestamp() {
this.__isset.timestamp = false;
}
// Returns true if field timestamp is set (has been asigned a value) and false otherwise
public boolean isSetTimestamp() {
return this.__isset.timestamp;
}
public void setTimestampIsSet(boolean value) {
this.__isset.timestamp = value;
}
public void setFieldValue(int fieldID, Object value) {
switch (fieldID) {
case VALUE:
if (value == null) {
unsetValue();
} else {
setValue((byte[])value);
}
break;
case TIMESTAMP:
if (value == null) {
unsetTimestamp();
} else {
setTimestamp((Long)value);
}
break;
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
public Object getFieldValue(int fieldID) {
switch (fieldID) {
case VALUE:
return getValue();
case TIMESTAMP:
return new Long(getTimestamp());
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
// Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
public boolean isSet(int fieldID) {
switch (fieldID) {
case VALUE:
return isSetValue();
case TIMESTAMP:
return isSetTimestamp();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
@ -76,8 +201,8 @@ public class TCell implements TBase, java.io.Serializable {
if (that == null)
return false;
boolean this_present_value = true && (this.value != null);
boolean that_present_value = true && (that.value != null);
boolean this_present_value = true && this.isSetValue();
boolean that_present_value = true && that.isSetValue();
if (this_present_value || that_present_value) {
if (!(this_present_value && that_present_value))
return false;
@ -97,6 +222,7 @@ public class TCell implements TBase, java.io.Serializable {
return true;
}
@Override
public int hashCode() {
return 0;
}
@ -112,15 +238,14 @@ public class TCell implements TBase, java.io.Serializable {
}
switch (field.id)
{
case 1:
case VALUE:
if (field.type == TType.STRING) {
this.value = iprot.readBinary();
this.__isset.value = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 2:
case TIMESTAMP:
if (field.type == TType.I64) {
this.timestamp = iprot.readI64();
this.__isset.timestamp = true;
@ -135,39 +260,52 @@ public class TCell implements TBase, java.io.Serializable {
iprot.readFieldEnd();
}
iprot.readStructEnd();
// check for required fields of primitive type, which can't be checked in the validate method
validate();
}
public void write(TProtocol oprot) throws TException {
TStruct struct = new TStruct("TCell");
oprot.writeStructBegin(struct);
TField field = new TField();
validate();
oprot.writeStructBegin(STRUCT_DESC);
if (this.value != null) {
field.name = "value";
field.type = TType.STRING;
field.id = 1;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(VALUE_FIELD_DESC);
oprot.writeBinary(this.value);
oprot.writeFieldEnd();
}
field.name = "timestamp";
field.type = TType.I64;
field.id = 2;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
oprot.writeI64(this.timestamp);
oprot.writeFieldEnd();
oprot.writeFieldStop();
oprot.writeStructEnd();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("TCell(");
boolean first = true;
sb.append("value:");
sb.append(this.value);
sb.append(",timestamp:");
if (this.value == null) {
sb.append("null");
} else {
sb.append(this.value);
}
first = false;
if (!first) sb.append(", ");
sb.append("timestamp:");
sb.append(this.timestamp);
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws TException {
// check for required fields
// check that fields of type enum have valid values
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Autogenerated by Thrift
*
@ -29,30 +28,57 @@ import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
import java.util.Collections;
import com.facebook.thrift.protocol.*;
import com.facebook.thrift.transport.*;
import org.apache.thrift.*;
import org.apache.thrift.meta_data.*;
import org.apache.thrift.protocol.*;
/**
* A TRegionInfo contains information about an HTable region.
*/
public class TRegionInfo implements TBase, java.io.Serializable {
public byte[] startKey;
public byte[] endKey;
public long id;
public byte[] name;
public byte version;
public class TRegionInfo implements TBase, java.io.Serializable, Cloneable {
private static final TStruct STRUCT_DESC = new TStruct("TRegionInfo");
private static final TField START_KEY_FIELD_DESC = new TField("startKey", TType.STRING, (short)1);
private static final TField END_KEY_FIELD_DESC = new TField("endKey", TType.STRING, (short)2);
private static final TField ID_FIELD_DESC = new TField("id", TType.I64, (short)3);
private static final TField NAME_FIELD_DESC = new TField("name", TType.STRING, (short)4);
private static final TField VERSION_FIELD_DESC = new TField("version", TType.BYTE, (short)5);
public final Isset __isset = new Isset();
public static final class Isset implements java.io.Serializable {
public boolean startKey = false;
public boolean endKey = false;
public byte[] startKey;
public static final int STARTKEY = 1;
public byte[] endKey;
public static final int ENDKEY = 2;
public long id;
public static final int ID = 3;
public byte[] name;
public static final int NAME = 4;
public byte version;
public static final int VERSION = 5;
private final Isset __isset = new Isset();
private static final class Isset implements java.io.Serializable {
public boolean id = false;
public boolean name = false;
public boolean version = false;
}
public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
put(STARTKEY, new FieldMetaData("startKey", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(ENDKEY, new FieldMetaData("endKey", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(ID, new FieldMetaData("id", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.I64)));
put(NAME, new FieldMetaData("name", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(VERSION, new FieldMetaData("version", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.BYTE)));
}});
static {
FieldMetaData.addStructMetaDataMap(TRegionInfo.class, metaDataMap);
}
public TRegionInfo() {
}
@ -65,17 +91,239 @@ public class TRegionInfo implements TBase, java.io.Serializable {
{
this();
this.startKey = startKey;
this.__isset.startKey = true;
this.endKey = endKey;
this.__isset.endKey = true;
this.id = id;
this.__isset.id = true;
this.name = name;
this.__isset.name = true;
this.version = version;
this.__isset.version = true;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TRegionInfo(TRegionInfo other) {
if (other.isSetStartKey()) {
this.startKey = other.startKey;
}
if (other.isSetEndKey()) {
this.endKey = other.endKey;
}
__isset.id = other.__isset.id;
this.id = other.id;
if (other.isSetName()) {
this.name = other.name;
}
__isset.version = other.__isset.version;
this.version = other.version;
}
@Override
public TRegionInfo clone() {
return new TRegionInfo(this);
}
public byte[] getStartKey() {
return this.startKey;
}
public void setStartKey(byte[] startKey) {
this.startKey = startKey;
}
public void unsetStartKey() {
this.startKey = null;
}
// Returns true if field startKey is set (has been asigned a value) and false otherwise
public boolean isSetStartKey() {
return this.startKey != null;
}
public void setStartKeyIsSet(boolean value) {
if (!value) {
this.startKey = null;
}
}
public byte[] getEndKey() {
return this.endKey;
}
public void setEndKey(byte[] endKey) {
this.endKey = endKey;
}
public void unsetEndKey() {
this.endKey = null;
}
// Returns true if field endKey is set (has been asigned a value) and false otherwise
public boolean isSetEndKey() {
return this.endKey != null;
}
public void setEndKeyIsSet(boolean value) {
if (!value) {
this.endKey = null;
}
}
public long getId() {
return this.id;
}
public void setId(long id) {
this.id = id;
this.__isset.id = true;
}
public void unsetId() {
this.__isset.id = false;
}
// Returns true if field id is set (has been asigned a value) and false otherwise
public boolean isSetId() {
return this.__isset.id;
}
public void setIdIsSet(boolean value) {
this.__isset.id = value;
}
public byte[] getName() {
return this.name;
}
public void setName(byte[] name) {
this.name = name;
}
public void unsetName() {
this.name = null;
}
// Returns true if field name is set (has been asigned a value) and false otherwise
public boolean isSetName() {
return this.name != null;
}
public void setNameIsSet(boolean value) {
if (!value) {
this.name = null;
}
}
public byte getVersion() {
return this.version;
}
public void setVersion(byte version) {
this.version = version;
this.__isset.version = true;
}
public void unsetVersion() {
this.__isset.version = false;
}
// Returns true if field version is set (has been asigned a value) and false otherwise
public boolean isSetVersion() {
return this.__isset.version;
}
public void setVersionIsSet(boolean value) {
this.__isset.version = value;
}
public void setFieldValue(int fieldID, Object value) {
switch (fieldID) {
case STARTKEY:
if (value == null) {
unsetStartKey();
} else {
setStartKey((byte[])value);
}
break;
case ENDKEY:
if (value == null) {
unsetEndKey();
} else {
setEndKey((byte[])value);
}
break;
case ID:
if (value == null) {
unsetId();
} else {
setId((Long)value);
}
break;
case NAME:
if (value == null) {
unsetName();
} else {
setName((byte[])value);
}
break;
case VERSION:
if (value == null) {
unsetVersion();
} else {
setVersion((Byte)value);
}
break;
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
public Object getFieldValue(int fieldID) {
switch (fieldID) {
case STARTKEY:
return getStartKey();
case ENDKEY:
return getEndKey();
case ID:
return new Long(getId());
case NAME:
return getName();
case VERSION:
return new Byte(getVersion());
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
// Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
public boolean isSet(int fieldID) {
switch (fieldID) {
case STARTKEY:
return isSetStartKey();
case ENDKEY:
return isSetEndKey();
case ID:
return isSetId();
case NAME:
return isSetName();
case VERSION:
return isSetVersion();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
@ -88,8 +336,8 @@ public class TRegionInfo implements TBase, java.io.Serializable {
if (that == null)
return false;
boolean this_present_startKey = true && (this.startKey != null);
boolean that_present_startKey = true && (that.startKey != null);
boolean this_present_startKey = true && this.isSetStartKey();
boolean that_present_startKey = true && that.isSetStartKey();
if (this_present_startKey || that_present_startKey) {
if (!(this_present_startKey && that_present_startKey))
return false;
@ -97,8 +345,8 @@ public class TRegionInfo implements TBase, java.io.Serializable {
return false;
}
boolean this_present_endKey = true && (this.endKey != null);
boolean that_present_endKey = true && (that.endKey != null);
boolean this_present_endKey = true && this.isSetEndKey();
boolean that_present_endKey = true && that.isSetEndKey();
if (this_present_endKey || that_present_endKey) {
if (!(this_present_endKey && that_present_endKey))
return false;
@ -115,8 +363,8 @@ public class TRegionInfo implements TBase, java.io.Serializable {
return false;
}
boolean this_present_name = true && (this.name != null);
boolean that_present_name = true && (that.name != null);
boolean this_present_name = true && this.isSetName();
boolean that_present_name = true && that.isSetName();
if (this_present_name || that_present_name) {
if (!(this_present_name && that_present_name))
return false;
@ -136,6 +384,7 @@ public class TRegionInfo implements TBase, java.io.Serializable {
return true;
}
@Override
public int hashCode() {
return 0;
}
@ -151,23 +400,21 @@ public class TRegionInfo implements TBase, java.io.Serializable {
}
switch (field.id)
{
case 1:
case STARTKEY:
if (field.type == TType.STRING) {
this.startKey = iprot.readBinary();
this.__isset.startKey = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 2:
case ENDKEY:
if (field.type == TType.STRING) {
this.endKey = iprot.readBinary();
this.__isset.endKey = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 3:
case ID:
if (field.type == TType.I64) {
this.id = iprot.readI64();
this.__isset.id = true;
@ -175,15 +422,14 @@ public class TRegionInfo implements TBase, java.io.Serializable {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 4:
case NAME:
if (field.type == TType.STRING) {
this.name = iprot.readBinary();
this.__isset.name = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 5:
case VERSION:
if (field.type == TType.BYTE) {
this.version = iprot.readByte();
this.__isset.version = true;
@ -198,67 +444,85 @@ public class TRegionInfo implements TBase, java.io.Serializable {
iprot.readFieldEnd();
}
iprot.readStructEnd();
// check for required fields of primitive type, which can't be checked in the validate method
validate();
}
public void write(TProtocol oprot) throws TException {
TStruct struct = new TStruct("TRegionInfo");
oprot.writeStructBegin(struct);
TField field = new TField();
validate();
oprot.writeStructBegin(STRUCT_DESC);
if (this.startKey != null) {
field.name = "startKey";
field.type = TType.STRING;
field.id = 1;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(START_KEY_FIELD_DESC);
oprot.writeBinary(this.startKey);
oprot.writeFieldEnd();
}
if (this.endKey != null) {
field.name = "endKey";
field.type = TType.STRING;
field.id = 2;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(END_KEY_FIELD_DESC);
oprot.writeBinary(this.endKey);
oprot.writeFieldEnd();
}
field.name = "id";
field.type = TType.I64;
field.id = 3;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(ID_FIELD_DESC);
oprot.writeI64(this.id);
oprot.writeFieldEnd();
if (this.name != null) {
field.name = "name";
field.type = TType.STRING;
field.id = 4;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(NAME_FIELD_DESC);
oprot.writeBinary(this.name);
oprot.writeFieldEnd();
}
field.name = "version";
field.type = TType.BYTE;
field.id = 5;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(VERSION_FIELD_DESC);
oprot.writeByte(this.version);
oprot.writeFieldEnd();
oprot.writeFieldStop();
oprot.writeStructEnd();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("TRegionInfo(");
boolean first = true;
sb.append("startKey:");
sb.append(this.startKey);
sb.append(",endKey:");
sb.append(this.endKey);
sb.append(",id:");
if (this.startKey == null) {
sb.append("null");
} else {
sb.append(this.startKey);
}
first = false;
if (!first) sb.append(", ");
sb.append("endKey:");
if (this.endKey == null) {
sb.append("null");
} else {
sb.append(this.endKey);
}
first = false;
if (!first) sb.append(", ");
sb.append("id:");
sb.append(this.id);
sb.append(",name:");
sb.append(this.name);
sb.append(",version:");
first = false;
if (!first) sb.append(", ");
sb.append("name:");
if (this.name == null) {
sb.append("null");
} else {
sb.append(this.name);
}
first = false;
if (!first) sb.append(", ");
sb.append("version:");
sb.append(this.version);
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws TException {
// check for required fields
// check that fields of type enum have valid values
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Autogenerated by Thrift
*
@ -29,22 +28,40 @@ import java.util.Map;
import java.util.HashMap;
import java.util.Set;
import java.util.HashSet;
import com.facebook.thrift.*;
import java.util.Collections;
import com.facebook.thrift.protocol.*;
import com.facebook.thrift.transport.*;
import org.apache.thrift.*;
import org.apache.thrift.meta_data.*;
import org.apache.thrift.protocol.*;
/**
* Holds row name and then a map of columns to cells.
*/
public class TRowResult implements TBase, java.io.Serializable {
public byte[] row;
public Map<byte[],TCell> columns;
public class TRowResult implements TBase, java.io.Serializable, Cloneable {
private static final TStruct STRUCT_DESC = new TStruct("TRowResult");
private static final TField ROW_FIELD_DESC = new TField("row", TType.STRING, (short)1);
private static final TField COLUMNS_FIELD_DESC = new TField("columns", TType.MAP, (short)2);
public final Isset __isset = new Isset();
public static final class Isset implements java.io.Serializable {
public boolean row = false;
public boolean columns = false;
public byte[] row;
public static final int ROW = 1;
public Map<byte[],TCell> columns;
public static final int COLUMNS = 2;
private final Isset __isset = new Isset();
private static final class Isset implements java.io.Serializable {
}
public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
put(ROW, new FieldMetaData("row", TFieldRequirementType.DEFAULT,
new FieldValueMetaData(TType.STRING)));
put(COLUMNS, new FieldMetaData("columns", TFieldRequirementType.DEFAULT,
new MapMetaData(TType.MAP,
new FieldValueMetaData(TType.STRING),
new StructMetaData(TType.STRUCT, TCell.class))));
}});
static {
FieldMetaData.addStructMetaDataMap(TRowResult.class, metaDataMap);
}
public TRowResult() {
@ -56,11 +73,144 @@ public class TRowResult implements TBase, java.io.Serializable {
{
this();
this.row = row;
this.__isset.row = true;
this.columns = columns;
this.__isset.columns = true;
}
/**
* Performs a deep copy on <i>other</i>.
*/
public TRowResult(TRowResult other) {
if (other.isSetRow()) {
this.row = other.row;
}
if (other.isSetColumns()) {
Map<byte[],TCell> __this__columns = new HashMap<byte[],TCell>();
for (Map.Entry<byte[], TCell> other_element : other.columns.entrySet()) {
byte[] other_element_key = other_element.getKey();
TCell other_element_value = other_element.getValue();
byte[] __this__columns_copy_key = other_element_key;
TCell __this__columns_copy_value = new TCell(other_element_value);
__this__columns.put(__this__columns_copy_key, __this__columns_copy_value);
}
this.columns = __this__columns;
}
}
@Override
public TRowResult clone() {
return new TRowResult(this);
}
public byte[] getRow() {
return this.row;
}
public void setRow(byte[] row) {
this.row = row;
}
public void unsetRow() {
this.row = null;
}
// Returns true if field row is set (has been asigned a value) and false otherwise
public boolean isSetRow() {
return this.row != null;
}
public void setRowIsSet(boolean value) {
if (!value) {
this.row = null;
}
}
public int getColumnsSize() {
return (this.columns == null) ? 0 : this.columns.size();
}
public void putToColumns(byte[] key, TCell val) {
if (this.columns == null) {
this.columns = new HashMap<byte[],TCell>();
}
this.columns.put(key, val);
}
public Map<byte[],TCell> getColumns() {
return this.columns;
}
public void setColumns(Map<byte[],TCell> columns) {
this.columns = columns;
}
public void unsetColumns() {
this.columns = null;
}
// Returns true if field columns is set (has been asigned a value) and false otherwise
public boolean isSetColumns() {
return this.columns != null;
}
public void setColumnsIsSet(boolean value) {
if (!value) {
this.columns = null;
}
}
public void setFieldValue(int fieldID, Object value) {
switch (fieldID) {
case ROW:
if (value == null) {
unsetRow();
} else {
setRow((byte[])value);
}
break;
case COLUMNS:
if (value == null) {
unsetColumns();
} else {
setColumns((Map<byte[],TCell>)value);
}
break;
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
public Object getFieldValue(int fieldID) {
switch (fieldID) {
case ROW:
return getRow();
case COLUMNS:
return getColumns();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
// Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
public boolean isSet(int fieldID) {
switch (fieldID) {
case ROW:
return isSetRow();
case COLUMNS:
return isSetColumns();
default:
throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
}
}
@Override
public boolean equals(Object that) {
if (that == null)
return false;
@ -73,8 +223,8 @@ public class TRowResult implements TBase, java.io.Serializable {
if (that == null)
return false;
boolean this_present_row = true && (this.row != null);
boolean that_present_row = true && (that.row != null);
boolean this_present_row = true && this.isSetRow();
boolean that_present_row = true && that.isSetRow();
if (this_present_row || that_present_row) {
if (!(this_present_row && that_present_row))
return false;
@ -82,8 +232,8 @@ public class TRowResult implements TBase, java.io.Serializable {
return false;
}
boolean this_present_columns = true && (this.columns != null);
boolean that_present_columns = true && (that.columns != null);
boolean this_present_columns = true && this.isSetColumns();
boolean that_present_columns = true && that.isSetColumns();
if (this_present_columns || that_present_columns) {
if (!(this_present_columns && that_present_columns))
return false;
@ -94,6 +244,7 @@ public class TRowResult implements TBase, java.io.Serializable {
return true;
}
@Override
public int hashCode() {
return 0;
}
@ -109,15 +260,14 @@ public class TRowResult implements TBase, java.io.Serializable {
}
switch (field.id)
{
case 1:
case ROW:
if (field.type == TType.STRING) {
this.row = iprot.readBinary();
this.__isset.row = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
break;
case 2:
case COLUMNS:
if (field.type == TType.MAP) {
{
TMap _map4 = iprot.readMapBegin();
@ -133,7 +283,6 @@ public class TRowResult implements TBase, java.io.Serializable {
}
iprot.readMapEnd();
}
this.__isset.columns = true;
} else {
TProtocolUtil.skip(iprot, field.type);
}
@ -145,30 +294,28 @@ public class TRowResult implements TBase, java.io.Serializable {
iprot.readFieldEnd();
}
iprot.readStructEnd();
// check for required fields of primitive type, which can't be checked in the validate method
validate();
}
public void write(TProtocol oprot) throws TException {
TStruct struct = new TStruct("TRowResult");
oprot.writeStructBegin(struct);
TField field = new TField();
validate();
oprot.writeStructBegin(STRUCT_DESC);
if (this.row != null) {
field.name = "row";
field.type = TType.STRING;
field.id = 1;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(ROW_FIELD_DESC);
oprot.writeBinary(this.row);
oprot.writeFieldEnd();
}
if (this.columns != null) {
field.name = "columns";
field.type = TType.MAP;
field.id = 2;
oprot.writeFieldBegin(field);
oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
{
oprot.writeMapBegin(new TMap(TType.STRING, TType.STRUCT, this.columns.size()));
for (byte[] _iter8 : this.columns.keySet()) {
oprot.writeBinary(_iter8);
this.columns.get(_iter8).write(oprot);
for (Map.Entry<byte[], TCell> _iter8 : this.columns.entrySet()) {
oprot.writeBinary(_iter8.getKey());
_iter8.getValue().write(oprot);
}
oprot.writeMapEnd();
}
@ -178,15 +325,34 @@ public class TRowResult implements TBase, java.io.Serializable {
oprot.writeStructEnd();
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("TRowResult(");
boolean first = true;
sb.append("row:");
sb.append(this.row);
sb.append(",columns:");
sb.append(this.columns);
if (this.row == null) {
sb.append("null");
} else {
sb.append(this.row);
}
first = false;
if (!first) sb.append(", ");
sb.append("columns:");
if (this.columns == null) {
sb.append("null");
} else {
sb.append(this.columns);
}
first = false;
sb.append(")");
return sb.toString();
}
public void validate() throws TException {
// check for required fields
// check that fields of type enum have valid values
}
}

View File

@ -62,9 +62,6 @@ 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 release 20080411p1 from
the <a href="http://developers.facebook.com/thrift/">thrift homepage</a>.</p>
<p>To start ThriftServer, use:
<pre>
./bin/hbase-daemon.sh start thrift [--port=PORT]

View File

@ -1,5 +1,5 @@
/**
* Copyright 2008 The Apache Software Foundation
* Copyright 2008-2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
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.TCell;
import org.apache.hadoop.hbase.thrift.generated.TRowResult;
import org.apache.hadoop.hbase.util.Bytes;
@ -127,8 +126,8 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
handler.mutateRow(tableAname, rowAname, getMutations());
// Assert that the changes were made
assertTrue(Bytes.equals(valueAname, handler.get(tableAname, rowAname, columnAname).value));
TRowResult rowResult1 = handler.getRow(tableAname, rowAname);
assertTrue(Bytes.equals(valueAname, handler.get(tableAname, rowAname, columnAname).get(0).value));
TRowResult rowResult1 = handler.getRow(tableAname, rowAname).get(0);
assertTrue(Bytes.equals(rowAname, rowResult1.row));
assertTrue(Bytes.equals(valueBname, rowResult1.columns.get(columnBname).value));
@ -136,20 +135,14 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
handler.mutateRows(tableAname, getBatchMutations());
// Assert that changes were made to rowA
boolean failed1 = false;
try {
handler.get(tableAname, rowAname, columnAname);
} catch (NotFound nf) {
failed1 = true;
}
assertTrue(failed1);
assertTrue(Bytes.equals(valueCname, handler.get(tableAname, rowAname, columnBname).value));
assertFalse(handler.get(tableAname, rowAname, columnAname).size() > 0);
assertTrue(Bytes.equals(valueCname, handler.get(tableAname, rowAname, columnBname).get(0).value));
List<TCell> versions = handler.getVer(tableAname, rowAname, columnBname, MAXVERSIONS);
assertTrue(Bytes.equals(valueCname, versions.get(0).value));
assertTrue(Bytes.equals(valueBname, versions.get(1).value));
// Assert that changes were made to rowB
TRowResult rowResult2 = handler.getRow(tableAname, rowBname);
TRowResult rowResult2 = handler.getRow(tableAname, rowBname).get(0);
assertTrue(Bytes.equals(rowBname, rowResult2.row));
assertTrue(Bytes.equals(valueCname, rowResult2.columns.get(columnAname).value));
assertTrue(Bytes.equals(valueDname, rowResult2.columns.get(columnBname).value));
@ -159,20 +152,8 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
handler.deleteAllRow(tableAname, rowBname);
// Assert that the deletes were applied
boolean failed2 = false;
try {
handler.get(tableAname, rowAname, columnBname);
} catch (NotFound nf) {
failed2 = true;
}
assertTrue(failed2);
boolean failed3 = false;
try {
handler.getRow(tableAname, rowBname);
} catch (NotFound nf) {
failed3 = true;
}
assertTrue(failed3);
assertFalse(handler.get(tableAname, rowAname, columnBname).size() == 0);
assertFalse(handler.getRow(tableAname, rowBname).size() == 0);
// Teardown
handler.disableTable(tableAname);
@ -210,8 +191,8 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
assertEquals(handler.getVerTs(tableAname, rowAname, columnBname, time2, MAXVERSIONS).size(), 2);
assertEquals(handler.getVerTs(tableAname, rowAname, columnBname, time1, MAXVERSIONS).size(), 1);
TRowResult rowResult1 = handler.getRowTs(tableAname, rowAname, time1);
TRowResult rowResult2 = handler.getRowTs(tableAname, rowAname, time2);
TRowResult rowResult1 = handler.getRowTs(tableAname, rowAname, time1).get(0);
TRowResult rowResult2 = handler.getRowTs(tableAname, rowAname, time2).get(0);
assertTrue(Bytes.equals(rowResult1.columns.get(columnAname).value, valueAname));
assertTrue(Bytes.equals(rowResult1.columns.get(columnBname).value, valueBname));
assertTrue(Bytes.equals(rowResult2.columns.get(columnBname).value, valueCname));
@ -221,11 +202,11 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
List<byte[]> columns = new ArrayList<byte[]>();
columns.add(columnBname);
rowResult1 = handler.getRowWithColumns(tableAname, rowAname, columns);
rowResult1 = handler.getRowWithColumns(tableAname, rowAname, columns).get(0);
assertTrue(Bytes.equals(rowResult1.columns.get(columnBname).value, valueCname));
assertFalse(rowResult1.columns.containsKey(columnAname));
rowResult1 = handler.getRowWithColumnsTs(tableAname, rowAname, columns, time1);
rowResult1 = handler.getRowWithColumnsTs(tableAname, rowAname, columns, time1).get(0);
assertTrue(Bytes.equals(rowResult1.columns.get(columnBname).value, valueBname));
assertFalse(rowResult1.columns.containsKey(columnAname));
@ -234,21 +215,9 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
handler.deleteAllRowTs(tableAname, rowBname, time2);
// Assert that the timestamp-related methods retrieve the correct data
boolean failed = false;
try {
handler.getVerTs(tableAname, rowAname, columnBname, time1, MAXVERSIONS);
} catch (NotFound nf) {
failed = true;
}
assertTrue(failed);
assertTrue(Bytes.equals(handler.get(tableAname, rowAname, columnBname).value, valueCname));
boolean failed2 = false;
try {
handler.getRow(tableAname, rowBname);
} catch (NotFound nf) {
failed2 = true;
}
assertTrue(failed2);
assertFalse(handler.getVerTs(tableAname, rowAname, columnBname, time1, MAXVERSIONS).size() > 0);
assertTrue(Bytes.equals(handler.get(tableAname, rowAname, columnBname).get(0).value, valueCname));
assertFalse(handler.getRow(tableAname, rowBname).size() > 0);
// Teardown
handler.disableTable(tableAname);
@ -280,11 +249,11 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
// Test a scanner on all rows and all columns, no timestamp
int scanner1 = handler.scannerOpen(tableAname, rowAname, getColumnList(true, true));
TRowResult rowResult1a = handler.scannerGet(scanner1);
TRowResult rowResult1a = handler.scannerGet(scanner1).get(0);
assertTrue(Bytes.equals(rowResult1a.row, rowAname));
assertEquals(rowResult1a.columns.size(), 1);
assertTrue(Bytes.equals(rowResult1a.columns.get(columnBname).value, valueCname));
TRowResult rowResult1b = handler.scannerGet(scanner1);
TRowResult rowResult1b = handler.scannerGet(scanner1).get(0);
assertTrue(Bytes.equals(rowResult1b.row, rowBname));
assertEquals(rowResult1b.columns.size(), 2);
assertTrue(Bytes.equals(rowResult1b.columns.get(columnAname).value, valueCname));
@ -293,7 +262,7 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
// Test a scanner on all rows and all columns, with timestamp
int scanner2 = handler.scannerOpenTs(tableAname, rowAname, getColumnList(true, true), time1);
TRowResult rowResult2a = handler.scannerGet(scanner2);
TRowResult rowResult2a = handler.scannerGet(scanner2).get(0);
assertEquals(rowResult2a.columns.size(), 2);
assertTrue(Bytes.equals(rowResult2a.columns.get(columnAname).value, valueAname));
assertTrue(Bytes.equals(rowResult2a.columns.get(columnBname).value, valueBname));
@ -307,7 +276,7 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
// Test a scanner on the first row and second column only, with timestamp
int scanner4 = handler.scannerOpenWithStopTs(tableAname, rowAname, rowBname,
getColumnList(false, true), time1);
TRowResult rowResult4a = handler.scannerGet(scanner4);
TRowResult rowResult4a = handler.scannerGet(scanner4).get(0);
assertEquals(rowResult4a.columns.size(), 1);
assertTrue(Bytes.equals(rowResult4a.columns.get(columnBname).value, valueBname));
@ -394,13 +363,7 @@ public class DisabledTestThriftServer extends HBaseClusterTestCase {
* @throws Exception
*/
private void closeScanner(int scannerId, ThriftServer.HBaseHandler handler) throws Exception {
boolean failed = false;
try {
handler.scannerGet(scannerId);
} catch (NotFound nf) {
failed = true;
}
assertTrue(failed);
handler.scannerGet(scannerId);
handler.scannerClose(scannerId);
}
}