HBASE-21661 Provide Thrift2 implementation of Table/Admin
This commit is contained in:
parent
5c902b48e5
commit
f053003ce7
|
@ -144,8 +144,16 @@ public final class Constants {
|
|||
public static final String THRIFT_READONLY_ENABLED = "hbase.thrift.readonly";
|
||||
public static final boolean THRIFT_READONLY_ENABLED_DEFAULT = false;
|
||||
|
||||
public static final String HBASE_THRIFT_CLIENT_SCANNER_CACHING =
|
||||
"hbase.thrift.client.scanner.caching";
|
||||
|
||||
public static final int HBASE_THRIFT_CLIENT_SCANNER_CACHING_DEFAULT = 20;
|
||||
|
||||
public static final String HBASE_THRIFT_SERVER_NAME = "hbase.thrift.server.name";
|
||||
public static final String HBASE_THRIFT_SERVER_PORT = "hbase.thrift.server.port";
|
||||
|
||||
public static final String HBASE_THRIFT_CLIENT_BUIDLER_CLASS =
|
||||
"hbase.thrift.client.builder.class";
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -141,6 +141,7 @@ import org.eclipse.jetty.util.thread.QueuedThreadPool;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
|
||||
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
@ -277,6 +278,16 @@ public class ThriftServer extends Configured implements Tool {
|
|||
HbaseHandlerMetricsProxy.newInstance((Hbase.Iface) hbaseServiceHandler, metrics, conf));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public TServer getTserver() {
|
||||
return tserver;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Server getHttpServer() {
|
||||
return httpServer;
|
||||
}
|
||||
|
||||
protected void printUsageAndExit(Options options, int exitCode)
|
||||
throws ExitCodeException {
|
||||
HelpFormatter formatter = new HelpFormatter();
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
|
|||
* An AlreadyExists exceptions signals that a table with the specified
|
||||
* name already exists
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class AlreadyExists extends TException implements org.apache.thrift.TBase<AlreadyExists, AlreadyExists._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExists> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyExists");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* A BatchMutation object is used to apply a number of Mutations to a single row.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class BatchMutation implements org.apache.thrift.TBase<BatchMutation, BatchMutation._Fields>, java.io.Serializable, Cloneable, Comparable<BatchMutation> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BatchMutation");
|
||||
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
|
|||
* such as the number of versions, compression settings, etc. It is
|
||||
* used as input when creating a table or adding a column.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class ColumnDescriptor implements org.apache.thrift.TBase<ColumnDescriptor, ColumnDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ColumnDescriptor");
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class Hbase {
|
||||
|
||||
public interface Iface {
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
|
|||
* to the Hbase master or an Hbase region server. Also used to return
|
||||
* more general Hbase error conditions.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class IOError extends TException implements org.apache.thrift.TBase<IOError, IOError._Fields>, java.io.Serializable, Cloneable, Comparable<IOError> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IOError");
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
|
|||
* An IllegalArgument exception indicates an illegal or invalid
|
||||
* argument was passed into a procedure.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class IllegalArgument extends TException implements org.apache.thrift.TBase<IllegalArgument, IllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<IllegalArgument> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("IllegalArgument");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* A Mutation object is used to either update or delete a column-value.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class Mutation implements org.apache.thrift.TBase<Mutation, Mutation._Fields>, java.io.Serializable, Cloneable, Comparable<Mutation> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Mutation");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* An Append object is used to specify the parameters for performing the append operation.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
|
||||
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.slf4j.LoggerFactory;
|
|||
* 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.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TCell implements org.apache.thrift.TBase<TCell, TCell._Fields>, java.io.Serializable, Cloneable, Comparable<TCell> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCell");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* Holds column name and the cell.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
|
|||
* For increments that are not incrementColumnValue
|
||||
* equivalents.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* A TRegionInfo contains information about an HTable region.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TRegionInfo implements org.apache.thrift.TBase<TRegionInfo, TRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TRegionInfo> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRegionInfo");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* Holds row name and then a map of columns to cells.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TRowResult implements org.apache.thrift.TBase<TRowResult, TRowResult._Fields>, java.io.Serializable, Cloneable, Comparable<TRowResult> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowResult");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* A Scan object is used to specify scanner parameters when opening a scanner.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
|
||||
|
||||
|
|
|
@ -575,7 +575,7 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements TH
|
|||
public List<TTableDescriptor> getTableDescriptorsByPattern(String regex, boolean includeSysTables)
|
||||
throws TIOError, TException {
|
||||
try {
|
||||
Pattern pattern = Pattern.compile(regex);
|
||||
Pattern pattern = (regex == null ? null : Pattern.compile(regex));
|
||||
List<TableDescriptor> tableDescriptors = connectionCache.getAdmin()
|
||||
.listTableDescriptors(pattern, includeSysTables);
|
||||
return tableDescriptorsFromHBase(tableDescriptors);
|
||||
|
@ -600,7 +600,7 @@ public class ThriftHBaseServiceHandler extends HBaseServiceHandler implements TH
|
|||
public List<TTableName> getTableNamesByPattern(String regex, boolean includeSysTables)
|
||||
throws TIOError, TException {
|
||||
try {
|
||||
Pattern pattern = Pattern.compile(regex);
|
||||
Pattern pattern = (regex == null ? null : Pattern.compile(regex));
|
||||
TableName[] tableNames = connectionCache.getAdmin()
|
||||
.listTableNames(pattern, includeSysTables);
|
||||
return tableNamesFromHBase(tableNames);
|
||||
|
|
|
@ -25,15 +25,19 @@ import java.nio.ByteBuffer;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableSet;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellBuilderFactory;
|
||||
import org.apache.hadoop.hbase.CellBuilderType;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
import org.apache.hadoop.hbase.ExtendedCellBuilder;
|
||||
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeepDeletedCells;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
|
@ -47,6 +51,7 @@ import org.apache.hadoop.hbase.client.Delete;
|
|||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.OperationWithAttributes;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
|
@ -55,14 +60,20 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.client.Scan.ReadType;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.filter.ParseFilter;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.security.visibility.Authorizations;
|
||||
import org.apache.hadoop.hbase.security.visibility.CellVisibility;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TAppend;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TAuthorization;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TBloomFilterType;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TCellVisibility;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TColumn;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TColumnIncrement;
|
||||
|
@ -179,6 +190,14 @@ public final class ThriftUtilities {
|
|||
out.addFamily(column.getFamily());
|
||||
}
|
||||
}
|
||||
if (in.isSetFilterBytes()) {
|
||||
try {
|
||||
Filter filter = FilterBase.parseFrom(in.getFilterBytes());
|
||||
out.setFilter(filter);
|
||||
} catch (DeserializationException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
return out;
|
||||
}
|
||||
|
@ -579,6 +598,80 @@ public final class ThriftUtilities {
|
|||
out.setReplicaId(in.getTargetReplicaId());
|
||||
}
|
||||
|
||||
if (in.isSetFilterBytes()) {
|
||||
try {
|
||||
Filter filter = FilterBase.parseFrom(in.getFilterBytes());
|
||||
out.setFilter(filter);
|
||||
} catch (DeserializationException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
return out;
|
||||
}
|
||||
|
||||
public static TScan scanFromHBase(Scan in) throws IOException {
|
||||
TScan out = new TScan();
|
||||
out.setStartRow(in.getStartRow());
|
||||
out.setStopRow(in.getStopRow());
|
||||
out.setCaching(in.getCaching());
|
||||
out.setMaxVersions(in.getMaxVersions());
|
||||
for (Map.Entry<byte[], NavigableSet<byte[]>> family : in.getFamilyMap().entrySet()) {
|
||||
|
||||
if (family.getValue() != null && !family.getValue().isEmpty()) {
|
||||
for (byte[] qualifier : family.getValue()) {
|
||||
TColumn column = new TColumn();
|
||||
column.setFamily(family.getKey());
|
||||
column.setQualifier(qualifier);
|
||||
out.addToColumns(column);
|
||||
}
|
||||
} else {
|
||||
TColumn column = new TColumn();
|
||||
column.setFamily(family.getKey());
|
||||
out.addToColumns(column);
|
||||
}
|
||||
}
|
||||
TTimeRange tTimeRange = new TTimeRange();
|
||||
tTimeRange.setMinStamp(in.getTimeRange().getMin()).setMaxStamp(in.getTimeRange().getMax());
|
||||
out.setTimeRange(tTimeRange);
|
||||
out.setBatchSize(in.getBatch());
|
||||
|
||||
for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
|
||||
out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
|
||||
ByteBuffer.wrap(attribute.getValue()));
|
||||
}
|
||||
|
||||
try {
|
||||
Authorizations authorizations = in.getAuthorizations();
|
||||
if (authorizations != null) {
|
||||
TAuthorization tAuthorization = new TAuthorization();
|
||||
tAuthorization.setLabels(authorizations.getLabels());
|
||||
out.setAuthorizations(tAuthorization);
|
||||
}
|
||||
} catch (DeserializationException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
out.setReversed(in.isReversed());
|
||||
out.setCacheBlocks(in.getCacheBlocks());
|
||||
out.setReadType(readTypeFromHBase(in.getReadType()));
|
||||
out.setLimit(in.getLimit());
|
||||
out.setConsistency(consistencyFromHBase(in.getConsistency()));
|
||||
out.setTargetReplicaId(in.getReplicaId());
|
||||
for (Map.Entry<byte[], TimeRange> entry : in.getColumnFamilyTimeRange().entrySet()) {
|
||||
if (entry.getValue() != null) {
|
||||
TTimeRange timeRange = new TTimeRange();
|
||||
timeRange.setMinStamp(entry.getValue().getMin()).setMaxStamp(entry.getValue().getMax());
|
||||
out.putToColFamTimeRangeMap(ByteBuffer.wrap(entry.getKey()), timeRange);
|
||||
}
|
||||
}
|
||||
if (in.getFilter() != null) {
|
||||
try {
|
||||
out.setFilterBytes(in.getFilter().toByteArray());
|
||||
} catch (IOException ioE) {
|
||||
throw new RuntimeException(ioE);
|
||||
}
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
|
@ -713,6 +806,15 @@ public final class ThriftUtilities {
|
|||
}
|
||||
}
|
||||
|
||||
private static TReadType readTypeFromHBase(ReadType readType) {
|
||||
switch (readType) {
|
||||
case DEFAULT: return TReadType.DEFAULT;
|
||||
case STREAM: return TReadType.STREAM;
|
||||
case PREAD: return TReadType.PREAD;
|
||||
default: return TReadType.DEFAULT;
|
||||
}
|
||||
}
|
||||
|
||||
private static Consistency consistencyFromThrift(TConsistency tConsistency) {
|
||||
switch (tConsistency.getValue()) {
|
||||
case 1: return Consistency.STRONG;
|
||||
|
@ -725,6 +827,15 @@ public final class ThriftUtilities {
|
|||
return TableName.valueOf(tableName.getNs(), tableName.getQualifier());
|
||||
}
|
||||
|
||||
public static TableName[] tableNamesArrayFromThrift(List<TTableName> tableNames) {
|
||||
TableName[] out = new TableName[tableNames.size()];
|
||||
int index = 0;
|
||||
for (TTableName tableName : tableNames) {
|
||||
out[index++] = tableNameFromThrift(tableName);
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
public static List<TableName> tableNamesFromThrift(List<TTableName> tableNames) {
|
||||
List<TableName> out = new ArrayList<>(tableNames.size());
|
||||
for (TTableName tableName : tableNames) {
|
||||
|
@ -740,6 +851,14 @@ public final class ThriftUtilities {
|
|||
return tableName;
|
||||
}
|
||||
|
||||
public static List<TTableName> tableNamesFromHBase(List<TableName> in) {
|
||||
List<TTableName> out = new ArrayList<>(in.size());
|
||||
for (TableName tableName : in) {
|
||||
out.add(tableNameFromHBase(tableName));
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
public static List<TTableName> tableNamesFromHBase(TableName[] in) {
|
||||
List<TTableName> out = new ArrayList<>(in.length);
|
||||
for (TableName tableName : in) {
|
||||
|
@ -920,6 +1039,28 @@ public final class ThriftUtilities {
|
|||
return builder.build();
|
||||
}
|
||||
|
||||
public static HTableDescriptor hTableDescriptorFromThrift(TTableDescriptor in) {
|
||||
return new HTableDescriptor(tableDescriptorFromThrift(in));
|
||||
}
|
||||
|
||||
public static HTableDescriptor[] hTableDescriptorsFromThrift(List<TTableDescriptor> in) {
|
||||
HTableDescriptor[] out = new HTableDescriptor[in.size()];
|
||||
int index = 0;
|
||||
for (TTableDescriptor tTableDescriptor : in) {
|
||||
out[index++] = hTableDescriptorFromThrift(tTableDescriptor);
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
|
||||
public static List<TableDescriptor> tableDescriptorsFromThrift(List<TTableDescriptor> in) {
|
||||
List<TableDescriptor> out = new ArrayList<>();
|
||||
for (TTableDescriptor tableDescriptor : in) {
|
||||
out.add(tableDescriptorFromThrift(tableDescriptor));
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
private static TDurability durabilityFromHBase(Durability durability) {
|
||||
switch (durability) {
|
||||
case USE_DEFAULT: return TDurability.USE_DEFAULT;
|
||||
|
@ -1038,4 +1179,300 @@ public final class ThriftUtilities {
|
|||
return out;
|
||||
}
|
||||
|
||||
|
||||
private static TConsistency consistencyFromHBase(Consistency consistency) {
|
||||
switch (consistency) {
|
||||
case STRONG: return TConsistency.STRONG;
|
||||
case TIMELINE: return TConsistency.TIMELINE;
|
||||
default: return TConsistency.STRONG;
|
||||
}
|
||||
}
|
||||
|
||||
public static TGet getFromHBase(Get in) {
|
||||
TGet out = new TGet();
|
||||
out.setRow(in.getRow());
|
||||
|
||||
TTimeRange tTimeRange = new TTimeRange();
|
||||
tTimeRange.setMaxStamp(in.getTimeRange().getMax()).setMinStamp(in.getTimeRange().getMin());
|
||||
out.setTimeRange(tTimeRange);
|
||||
out.setMaxVersions(in.getMaxVersions());
|
||||
|
||||
for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
|
||||
out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
|
||||
ByteBuffer.wrap(attribute.getValue()));
|
||||
}
|
||||
try {
|
||||
Authorizations authorizations = in.getAuthorizations();
|
||||
if (authorizations != null) {
|
||||
TAuthorization tAuthorization = new TAuthorization();
|
||||
tAuthorization.setLabels(authorizations.getLabels());
|
||||
out.setAuthorizations(tAuthorization);
|
||||
}
|
||||
} catch (DeserializationException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
out.setConsistency(consistencyFromHBase(in.getConsistency()));
|
||||
out.setTargetReplicaId(in.getReplicaId());
|
||||
out.setCacheBlocks(in.getCacheBlocks());
|
||||
out.setStoreLimit(in.getMaxResultsPerColumnFamily());
|
||||
out.setStoreOffset(in.getRowOffsetPerColumnFamily());
|
||||
out.setExistence_only(in.isCheckExistenceOnly());
|
||||
for (Map.Entry<byte[], NavigableSet<byte[]>> family : in.getFamilyMap().entrySet()) {
|
||||
|
||||
if (family.getValue() != null && !family.getValue().isEmpty()) {
|
||||
for (byte[] qualifier : family.getValue()) {
|
||||
TColumn column = new TColumn();
|
||||
column.setFamily(family.getKey());
|
||||
column.setQualifier(qualifier);
|
||||
out.addToColumns(column);
|
||||
}
|
||||
} else {
|
||||
TColumn column = new TColumn();
|
||||
column.setFamily(family.getKey());
|
||||
out.addToColumns(column);
|
||||
}
|
||||
}
|
||||
if (in.getFilter() != null) {
|
||||
try {
|
||||
out.setFilterBytes(in.getFilter().toByteArray());
|
||||
} catch (IOException ioE) {
|
||||
throw new RuntimeException(ioE);
|
||||
}
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
public static Cell toCell(ExtendedCellBuilder cellBuilder, byte[] row, TColumnValue columnValue) {
|
||||
return cellBuilder.clear()
|
||||
.setRow(row)
|
||||
.setFamily(columnValue.getFamily())
|
||||
.setQualifier(columnValue.getQualifier())
|
||||
.setTimestamp(columnValue.getTimestamp())
|
||||
.setType(columnValue.getType())
|
||||
.setValue(columnValue.getValue())
|
||||
.setTags(columnValue.getTags())
|
||||
.build();
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
public static Result resultFromThrift(TResult in) {
|
||||
if (in == null) {
|
||||
return null;
|
||||
}
|
||||
if (!in.isSetColumnValues() || in.getColumnValues().isEmpty()){
|
||||
return in.isStale() ? EMPTY_RESULT_STALE : EMPTY_RESULT;
|
||||
}
|
||||
List<Cell> cells = new ArrayList<>(in.getColumnValues().size());
|
||||
ExtendedCellBuilder builder = ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
|
||||
for (TColumnValue columnValue : in.getColumnValues()) {
|
||||
cells.add(toCell(builder, in.getRow(), columnValue));
|
||||
}
|
||||
return Result.create(cells, null, in.isStale(), in.isPartial());
|
||||
}
|
||||
|
||||
public static TPut putFromHBase(Put in) {
|
||||
TPut out = new TPut();
|
||||
out.setRow(in.getRow());
|
||||
if (in.getTimestamp() != HConstants.LATEST_TIMESTAMP) {
|
||||
out.setTimestamp(in.getTimestamp());
|
||||
}
|
||||
if (in.getDurability() != Durability.USE_DEFAULT) {
|
||||
out.setDurability(durabilityFromHBase(in.getDurability()));
|
||||
}
|
||||
for (Map.Entry<byte [], List<Cell>> entry : in.getFamilyCellMap().entrySet()) {
|
||||
byte[] family = entry.getKey();
|
||||
for (Cell cell : entry.getValue()) {
|
||||
TColumnValue columnValue = new TColumnValue();
|
||||
columnValue.setFamily(family)
|
||||
.setQualifier(CellUtil.cloneQualifier(cell))
|
||||
.setType(cell.getType().getCode())
|
||||
.setTimestamp(cell.getTimestamp())
|
||||
.setValue(CellUtil.cloneValue(cell));
|
||||
if (cell.getTagsLength() != 0) {
|
||||
columnValue.setTags(CellUtil.cloneTags(cell));
|
||||
}
|
||||
out.addToColumnValues(columnValue);
|
||||
}
|
||||
}
|
||||
for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
|
||||
out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
|
||||
ByteBuffer.wrap(attribute.getValue()));
|
||||
}
|
||||
try {
|
||||
CellVisibility cellVisibility = in.getCellVisibility();
|
||||
if (cellVisibility != null) {
|
||||
TCellVisibility tCellVisibility = new TCellVisibility();
|
||||
tCellVisibility.setExpression(cellVisibility.getExpression());
|
||||
out.setCellVisibility(tCellVisibility);
|
||||
}
|
||||
} catch (DeserializationException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
public static List<TPut> putsFromHBase(List<Put> in) {
|
||||
List<TPut> out = new ArrayList<>(in.size());
|
||||
for (Put put : in) {
|
||||
out.add(putFromHBase(put));
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
public static NamespaceDescriptor[] namespaceDescriptorsFromThrift(
|
||||
List<TNamespaceDescriptor> in) {
|
||||
NamespaceDescriptor[] out = new NamespaceDescriptor[in.size()];
|
||||
int index = 0;
|
||||
for (TNamespaceDescriptor descriptor : in) {
|
||||
out[index++] = namespaceDescriptorFromThrift(descriptor);
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
public static List<TDelete> deletesFromHBase(List<Delete> in) {
|
||||
List<TDelete> out = new ArrayList<>(in.size());
|
||||
for (Delete delete : in) {
|
||||
out.add(deleteFromHBase(delete));
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
public static TAppend appendFromHBase(Append in) throws IOException {
|
||||
TAppend out = new TAppend();
|
||||
out.setRow(in.getRow());
|
||||
|
||||
if (in.getDurability() != Durability.USE_DEFAULT) {
|
||||
out.setDurability(durabilityFromHBase(in.getDurability()));
|
||||
}
|
||||
for (Map.Entry<byte [], List<Cell>> entry : in.getFamilyCellMap().entrySet()) {
|
||||
byte[] family = entry.getKey();
|
||||
for (Cell cell : entry.getValue()) {
|
||||
TColumnValue columnValue = new TColumnValue();
|
||||
columnValue.setFamily(family)
|
||||
.setQualifier(CellUtil.cloneQualifier(cell))
|
||||
.setType(cell.getType().getCode())
|
||||
.setTimestamp(cell.getTimestamp())
|
||||
.setValue(CellUtil.cloneValue(cell));
|
||||
if (cell.getTagsLength() != 0) {
|
||||
columnValue.setTags(CellUtil.cloneTags(cell));
|
||||
}
|
||||
out.addToColumns(columnValue);
|
||||
}
|
||||
}
|
||||
for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
|
||||
out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
|
||||
ByteBuffer.wrap(attribute.getValue()));
|
||||
}
|
||||
try {
|
||||
CellVisibility cellVisibility = in.getCellVisibility();
|
||||
if (cellVisibility != null) {
|
||||
TCellVisibility tCellVisibility = new TCellVisibility();
|
||||
tCellVisibility.setExpression(cellVisibility.getExpression());
|
||||
out.setCellVisibility(tCellVisibility);
|
||||
}
|
||||
} catch (DeserializationException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
out.setReturnResults(in.isReturnResults());
|
||||
return out;
|
||||
}
|
||||
|
||||
public static TIncrement incrementFromHBase(Increment in) throws IOException {
|
||||
TIncrement out = new TIncrement();
|
||||
out.setRow(in.getRow());
|
||||
|
||||
if (in.getDurability() != Durability.USE_DEFAULT) {
|
||||
out.setDurability(durabilityFromHBase(in.getDurability()));
|
||||
}
|
||||
for (Map.Entry<byte [], List<Cell>> entry : in.getFamilyCellMap().entrySet()) {
|
||||
byte[] family = entry.getKey();
|
||||
for (Cell cell : entry.getValue()) {
|
||||
TColumnIncrement columnValue = new TColumnIncrement();
|
||||
columnValue.setFamily(family).setQualifier(CellUtil.cloneQualifier(cell));
|
||||
columnValue.setAmount(
|
||||
Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
|
||||
out.addToColumns(columnValue);
|
||||
}
|
||||
}
|
||||
for (Map.Entry<String, byte[]> attribute : in.getAttributesMap().entrySet()) {
|
||||
out.putToAttributes(ByteBuffer.wrap(Bytes.toBytes(attribute.getKey())),
|
||||
ByteBuffer.wrap(attribute.getValue()));
|
||||
}
|
||||
try {
|
||||
CellVisibility cellVisibility = in.getCellVisibility();
|
||||
if (cellVisibility != null) {
|
||||
TCellVisibility tCellVisibility = new TCellVisibility();
|
||||
tCellVisibility.setExpression(cellVisibility.getExpression());
|
||||
out.setCellVisibility(tCellVisibility);
|
||||
}
|
||||
} catch (DeserializationException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
out.setReturnResults(in.isReturnResults());
|
||||
return out;
|
||||
}
|
||||
|
||||
public static TRowMutations rowMutationsFromHBase(RowMutations in) {
|
||||
TRowMutations tRowMutations = new TRowMutations();
|
||||
tRowMutations.setRow(in.getRow());
|
||||
for (Mutation mutation : in.getMutations()) {
|
||||
TMutation tMutation = new TMutation();
|
||||
if (mutation instanceof Put) {
|
||||
tMutation.setPut(ThriftUtilities.putFromHBase((Put)mutation));
|
||||
} else if (mutation instanceof Delete) {
|
||||
tMutation.setDeleteSingle(ThriftUtilities.deleteFromHBase((Delete)mutation));
|
||||
} else {
|
||||
throw new IllegalArgumentException(
|
||||
"Only Put and Delete is supported in mutateRow, but muation=" + mutation);
|
||||
}
|
||||
tRowMutations.addToMutations(tMutation);
|
||||
}
|
||||
return tRowMutations;
|
||||
}
|
||||
|
||||
public static TCompareOp compareOpFromHBase(CompareOperator compareOp) {
|
||||
switch (compareOp) {
|
||||
case LESS: return TCompareOp.LESS;
|
||||
case LESS_OR_EQUAL: return TCompareOp.LESS_OR_EQUAL;
|
||||
case EQUAL: return TCompareOp.EQUAL;
|
||||
case NOT_EQUAL: return TCompareOp.NOT_EQUAL;
|
||||
case GREATER_OR_EQUAL: return TCompareOp.GREATER_OR_EQUAL;
|
||||
case GREATER: return TCompareOp.GREATER;
|
||||
case NO_OP: return TCompareOp.NO_OP;
|
||||
default: return null;
|
||||
}
|
||||
}
|
||||
public static List<ByteBuffer> splitKeyFromHBase(byte[][] in) {
|
||||
if (in == null || in.length == 0) {
|
||||
return null;
|
||||
}
|
||||
List<ByteBuffer> out = new ArrayList<>(in.length);
|
||||
for (byte[] key : in) {
|
||||
out.add(ByteBuffer.wrap(key));
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
public static Result[] resultsFromThrift(List<TResult> in) {
|
||||
Result[] out = new Result[in.size()];
|
||||
int index = 0;
|
||||
for (TResult tResult : in) {
|
||||
out[index++] = resultFromThrift(tResult);
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
public static List<TGet> getsFromHBase(List<Get> in) {
|
||||
List<TGet> out = new ArrayList<>(in.size());
|
||||
for (Get get : in) {
|
||||
out.add(getFromHBase(get));
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,37 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.thrift2.client;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.thrift.transport.TTransport;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public abstract class ThriftClientBuilder {
|
||||
protected ThriftConnection connection;
|
||||
|
||||
public ThriftClientBuilder(ThriftConnection connection) {
|
||||
this.connection = connection;
|
||||
}
|
||||
|
||||
public abstract Pair<THBaseService.Client, TTransport> getClient() throws IOException;
|
||||
}
|
|
@ -0,0 +1,322 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.thrift2.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.ipc.RpcClient.DEFAULT_SOCKET_TIMEOUT_CONNECT;
|
||||
import static org.apache.hadoop.hbase.ipc.RpcClient.SOCKET_TIMEOUT_CONNECT;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import org.apache.commons.lang3.NotImplementedException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutatorParams;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableBuilder;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.thrift.Constants;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.http.client.HttpClient;
|
||||
import org.apache.http.client.config.RequestConfig;
|
||||
import org.apache.http.client.utils.HttpClientUtils;
|
||||
import org.apache.http.impl.client.HttpClientBuilder;
|
||||
import org.apache.thrift.protocol.TBinaryProtocol;
|
||||
import org.apache.thrift.protocol.TCompactProtocol;
|
||||
import org.apache.thrift.protocol.TProtocol;
|
||||
import org.apache.thrift.transport.TFramedTransport;
|
||||
import org.apache.thrift.transport.THttpClient;
|
||||
import org.apache.thrift.transport.TSocket;
|
||||
import org.apache.thrift.transport.TTransport;
|
||||
import org.apache.thrift.transport.TTransportException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class ThriftConnection implements Connection {
|
||||
private Configuration conf;
|
||||
private User user;
|
||||
// For HTTP protocol
|
||||
private HttpClient httpClient;
|
||||
private boolean httpClientCreated = false;
|
||||
private boolean isClosed = false;
|
||||
|
||||
private String host;
|
||||
private int port;
|
||||
private boolean isFramed = false;
|
||||
private boolean isCompact = false;
|
||||
|
||||
private ThriftClientBuilder clientBuilder;
|
||||
|
||||
private int operationTimeout;
|
||||
private int connectTimeout;
|
||||
|
||||
public ThriftConnection(Configuration conf, ExecutorService pool, final User user)
|
||||
throws IOException {
|
||||
this.conf = conf;
|
||||
this.user = user;
|
||||
this.host = conf.get(Constants.HBASE_THRIFT_SERVER_NAME);
|
||||
this.port = conf.getInt(Constants.HBASE_THRIFT_SERVER_PORT, -1);
|
||||
Preconditions.checkArgument(port > 0);
|
||||
Preconditions.checkArgument(host != null);
|
||||
this.isFramed = conf.getBoolean(Constants.FRAMED_CONF_KEY, Constants.FRAMED_CONF_DEFAULT);
|
||||
this.isCompact = conf.getBoolean(Constants.COMPACT_CONF_KEY, Constants.COMPACT_CONF_DEFAULT);
|
||||
this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
|
||||
this.connectTimeout = conf.getInt(SOCKET_TIMEOUT_CONNECT, DEFAULT_SOCKET_TIMEOUT_CONNECT);
|
||||
|
||||
String className = conf.get(Constants.HBASE_THRIFT_CLIENT_BUIDLER_CLASS,
|
||||
DefaultThriftClientBuilder.class.getName());
|
||||
try {
|
||||
Class<?> clazz = Class.forName(className);
|
||||
Constructor<?> constructor = clazz
|
||||
.getDeclaredConstructor(ThriftConnection.class);
|
||||
constructor.setAccessible(true);
|
||||
clientBuilder = (ThriftClientBuilder) constructor.newInstance(this);
|
||||
}catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized void setHttpClient(HttpClient httpClient) {
|
||||
this.httpClient = httpClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
public String getHost() {
|
||||
return host;
|
||||
}
|
||||
|
||||
public int getPort() {
|
||||
return port;
|
||||
}
|
||||
|
||||
public boolean isFramed() {
|
||||
return isFramed;
|
||||
}
|
||||
|
||||
public boolean isCompact() {
|
||||
return isCompact;
|
||||
}
|
||||
|
||||
public int getOperationTimeout() {
|
||||
return operationTimeout;
|
||||
}
|
||||
|
||||
public int getConnectTimeout() {
|
||||
return connectTimeout;
|
||||
}
|
||||
|
||||
public ThriftClientBuilder getClientBuilder() {
|
||||
return clientBuilder;
|
||||
}
|
||||
|
||||
/**
|
||||
* the default thrift client builder.
|
||||
* One can extend the ThriftClientBuilder to builder custom client, implement
|
||||
* features like authentication(hbase-examples/thrift/DemoClient)
|
||||
*
|
||||
*/
|
||||
public static class DefaultThriftClientBuilder extends ThriftClientBuilder {
|
||||
|
||||
@Override
|
||||
public Pair<THBaseService.Client, TTransport> getClient() throws IOException {
|
||||
TSocket sock = new TSocket(connection.getHost(), connection.getPort());
|
||||
sock.setSocketTimeout(connection.getOperationTimeout());
|
||||
sock.setConnectTimeout(connection.getConnectTimeout());
|
||||
TTransport tTransport = sock;
|
||||
if (connection.isFramed()) {
|
||||
tTransport = new TFramedTransport(tTransport);
|
||||
}
|
||||
try {
|
||||
sock.open();
|
||||
} catch (TTransportException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
TProtocol prot;
|
||||
if (connection.isCompact()) {
|
||||
prot = new TCompactProtocol(tTransport);
|
||||
} else {
|
||||
prot = new TBinaryProtocol(tTransport);
|
||||
}
|
||||
THBaseService.Client client = new THBaseService.Client(prot);
|
||||
return new Pair<>(client, tTransport);
|
||||
}
|
||||
|
||||
public DefaultThriftClientBuilder(ThriftConnection connection) {
|
||||
super(connection);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* the default thrift http client builder.
|
||||
* One can extend the ThriftClientBuilder to builder custom http client, implement
|
||||
* features like authentication or 'DoAs'(hbase-examples/thrift/HttpDoAsClient)
|
||||
*
|
||||
*/
|
||||
public static class HTTPThriftClientBuilder extends ThriftClientBuilder {
|
||||
Map<String,String> customHeader = new HashMap<>();
|
||||
|
||||
public HTTPThriftClientBuilder(ThriftConnection connection) {
|
||||
super(connection);
|
||||
}
|
||||
|
||||
public void addCostumHeader(String key, String value) {
|
||||
customHeader.put(key, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Pair<THBaseService.Client, TTransport> getClient() throws IOException {
|
||||
Preconditions.checkArgument(connection.getHost().startsWith("http"),
|
||||
"http client host must start with http or https");
|
||||
String url = connection.getHost() + ":" + connection.getPort();
|
||||
try {
|
||||
THttpClient httpClient = new THttpClient(url, connection.getHttpClient());
|
||||
for (Map.Entry<String, String> header : customHeader.entrySet()) {
|
||||
httpClient.setCustomHeader(header.getKey(), header.getValue());
|
||||
}
|
||||
httpClient.open();
|
||||
TProtocol prot = new TBinaryProtocol(httpClient);
|
||||
THBaseService.Client client = new THBaseService.Client(prot);
|
||||
return new Pair<>(client, httpClient);
|
||||
} catch (TTransportException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a ThriftAdmin, ThriftAdmin is NOT thread safe
|
||||
* @return a ThriftAdmin
|
||||
* @throws IOException IOException
|
||||
*/
|
||||
@Override
|
||||
public Admin getAdmin() throws IOException {
|
||||
Pair<THBaseService.Client, TTransport> client = clientBuilder.getClient();
|
||||
return new ThriftAdmin(client.getFirst(), client.getSecond(), conf);
|
||||
}
|
||||
|
||||
public synchronized HttpClient getHttpClient() {
|
||||
if (httpClient != null) {
|
||||
return httpClient;
|
||||
}
|
||||
HttpClientBuilder builder = HttpClientBuilder.create();
|
||||
RequestConfig.Builder requestBuilder = RequestConfig.custom();
|
||||
requestBuilder = requestBuilder.setConnectTimeout(getConnectTimeout());
|
||||
requestBuilder = requestBuilder.setConnectionRequestTimeout(getOperationTimeout());
|
||||
builder.setDefaultRequestConfig(requestBuilder.build());
|
||||
httpClient = builder.build();
|
||||
httpClientCreated = true;
|
||||
return httpClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
if (httpClient != null && httpClientCreated) {
|
||||
HttpClientUtils.closeQuietly(httpClient);
|
||||
}
|
||||
isClosed = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isClosed() {
|
||||
return isClosed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a TableBuider to build ThriftTable, ThriftTable is NOT thread safe
|
||||
* @return a TableBuilder
|
||||
* @throws IOException IOException
|
||||
*/
|
||||
@Override
|
||||
public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) {
|
||||
return new TableBuilder() {
|
||||
@Override
|
||||
public TableBuilder setOperationTimeout(int timeout) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableBuilder setRpcTimeout(int timeout) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableBuilder setReadRpcTimeout(int timeout) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableBuilder setWriteRpcTimeout(int timeout) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Table build() {
|
||||
try {
|
||||
Pair<THBaseService.Client, TTransport> client = clientBuilder.getClient();
|
||||
return new ThriftTable(tableName, client.getFirst(), client.getSecond(), conf);
|
||||
} catch (IOException ioE) {
|
||||
throw new RuntimeException(ioE);
|
||||
}
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(String why, Throwable e) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAborted() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferedMutator getBufferedMutator(TableName tableName) throws IOException {
|
||||
throw new NotImplementedException("batchCoprocessorService not supported in ThriftTable");
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {
|
||||
throw new NotImplementedException("batchCoprocessorService not supported in ThriftTable");
|
||||
}
|
||||
|
||||
@Override
|
||||
public RegionLocator getRegionLocator(TableName tableName) throws IOException {
|
||||
throw new NotImplementedException("batchCoprocessorService not supported in ThriftTable");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,492 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.thrift2.client;
|
||||
|
||||
import static org.apache.hadoop.hbase.thrift.Constants.HBASE_THRIFT_CLIENT_SCANNER_CACHING;
|
||||
import static org.apache.hadoop.hbase.thrift.Constants.HBASE_THRIFT_CLIENT_SCANNER_CACHING_DEFAULT;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.lang3.NotImplementedException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Row;
|
||||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.thrift2.ThriftUtilities;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TAppend;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TDelete;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TGet;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TPut;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TResult;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TRowMutations;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TScan;
|
||||
import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.thrift.TException;
|
||||
import org.apache.thrift.transport.TTransport;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.common.primitives.Booleans;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class ThriftTable implements Table {
|
||||
|
||||
private TableName tableName;
|
||||
private Configuration conf;
|
||||
private TTransport tTransport;
|
||||
private THBaseService.Client client;
|
||||
private ByteBuffer tableNameInBytes;
|
||||
private int operationTimeout;
|
||||
|
||||
private final int scannerCaching;
|
||||
|
||||
public ThriftTable(TableName tableName, THBaseService.Client client, TTransport tTransport,
|
||||
Configuration conf) {
|
||||
this.tableName = tableName;
|
||||
this.tableNameInBytes = ByteBuffer.wrap(tableName.toBytes());
|
||||
this.conf = conf;
|
||||
this.tTransport = tTransport;
|
||||
this.client = client;
|
||||
this.scannerCaching = conf.getInt(HBASE_THRIFT_CLIENT_SCANNER_CACHING,
|
||||
HBASE_THRIFT_CLIENT_SCANNER_CACHING_DEFAULT);
|
||||
this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableName getName() {
|
||||
return tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableDescriptor getDescriptor() throws IOException {
|
||||
try {
|
||||
TTableDescriptor tableDescriptor = client
|
||||
.getTableDescriptor(ThriftUtilities.tableNameFromHBase(tableName));
|
||||
return ThriftUtilities.tableDescriptorFromThrift(tableDescriptor);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean exists(Get get) throws IOException {
|
||||
TGet tGet = ThriftUtilities.getFromHBase(get);
|
||||
try {
|
||||
return client.exists(tableNameInBytes, tGet);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean[] exists(List<Get> gets) throws IOException {
|
||||
List<TGet> tGets = new ArrayList<>();
|
||||
for (Get get: gets) {
|
||||
tGets.add(ThriftUtilities.getFromHBase(get));
|
||||
}
|
||||
try {
|
||||
List<Boolean> results = client.existsAll(tableNameInBytes, tGets);
|
||||
return Booleans.toArray(results);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void batch(List<? extends Row> actions, Object[] results)
|
||||
throws IOException {
|
||||
throw new IOException("Batch not supported in ThriftTable, use put(List<Put> puts), "
|
||||
+ "get(List<Get> gets) or delete(List<Delete> deletes) respectively");
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public <R> void batchCallback(List<? extends Row> actions, Object[] results,
|
||||
Batch.Callback<R> callback) throws IOException {
|
||||
throw new IOException("BatchCallback not supported in ThriftTable, use put(List<Put> puts), "
|
||||
+ "get(List<Get> gets) or delete(List<Delete> deletes) respectively");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result get(Get get) throws IOException {
|
||||
TGet tGet = ThriftUtilities.getFromHBase(get);
|
||||
try {
|
||||
TResult tResult = client.get(tableNameInBytes, tGet);
|
||||
return ThriftUtilities.resultFromThrift(tResult);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result[] get(List<Get> gets) throws IOException {
|
||||
List<TGet> tGets = ThriftUtilities.getsFromHBase(gets);
|
||||
try {
|
||||
List<TResult> results = client.getMultiple(tableNameInBytes, tGets);
|
||||
return ThriftUtilities.resultsFromThrift(results);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A scanner to perform scan from thrift server
|
||||
* getScannerResults is used in this scanner
|
||||
*/
|
||||
private class Scanner implements ResultScanner {
|
||||
protected TScan scan;
|
||||
protected Result lastResult = null;
|
||||
protected final Queue<Result> cache = new ArrayDeque<>();;
|
||||
|
||||
|
||||
public Scanner(Scan scan) throws IOException {
|
||||
if (scan.getBatch() > 0) {
|
||||
throw new IOException("Batch is not supported in Scanner");
|
||||
}
|
||||
if (scan.getCaching() <= 0) {
|
||||
scan.setCaching(scannerCaching);
|
||||
} else if (scan.getCaching() == 1 && scan.isReversed()){
|
||||
// for reverse scan, we need to pass the last row to the next scanner
|
||||
// we need caching number bigger than 1
|
||||
scan.setCaching(scan.getCaching() + 1);
|
||||
}
|
||||
this.scan = ThriftUtilities.scanFromHBase(scan);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Result next() throws IOException {
|
||||
if (cache.size() == 0) {
|
||||
setupNextScanner();
|
||||
try {
|
||||
List<TResult> tResults = client
|
||||
.getScannerResults(tableNameInBytes, scan, scan.getCaching());
|
||||
Result[] results = ThriftUtilities.resultsFromThrift(tResults);
|
||||
boolean firstKey = true;
|
||||
for (Result result : results) {
|
||||
// If it is a reverse scan, we use the last result's key as the startkey, since there is
|
||||
// no way to construct a closet rowkey smaller than the last result
|
||||
// So when the results return, we must rule out the first result, since it has already
|
||||
// returned to user.
|
||||
if (firstKey) {
|
||||
firstKey = false;
|
||||
if (scan.isReversed() && lastResult != null) {
|
||||
if (Bytes.equals(lastResult.getRow(), result.getRow())) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
cache.add(result);
|
||||
lastResult = result;
|
||||
}
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
if (cache.size() > 0) {
|
||||
return cache.poll();
|
||||
} else {
|
||||
//scan finished
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean renewLease() {
|
||||
throw new RuntimeException("renewLease() not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScanMetrics getScanMetrics() {
|
||||
throw new RuntimeException("getScanMetrics() not supported");
|
||||
}
|
||||
|
||||
private void setupNextScanner() {
|
||||
//if lastResult is null null, it means it is not the fist scan
|
||||
if (lastResult!= null) {
|
||||
byte[] lastRow = lastResult.getRow();
|
||||
if (scan.isReversed()) {
|
||||
//for reverse scan, we can't find the closet row before this row
|
||||
scan.setStartRow(lastRow);
|
||||
} else {
|
||||
scan.setStartRow(createClosestRowAfter(lastRow));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create the closest row after the specified row
|
||||
*/
|
||||
protected byte[] createClosestRowAfter(byte[] row) {
|
||||
if (row == null) {
|
||||
throw new RuntimeException("The passed row is null");
|
||||
}
|
||||
return Arrays.copyOf(row, row.length + 1);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResultScanner getScanner(Scan scan) throws IOException {
|
||||
return new Scanner(scan);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResultScanner getScanner(byte[] family) throws IOException {
|
||||
Scan scan = new Scan();
|
||||
scan.addFamily(family);
|
||||
return getScanner(scan);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException {
|
||||
Scan scan = new Scan();
|
||||
scan.addColumn(family, qualifier);
|
||||
return getScanner(scan);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(Put put) throws IOException {
|
||||
TPut tPut = ThriftUtilities.putFromHBase(put);
|
||||
try {
|
||||
client.put(tableNameInBytes, tPut);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void put(List<Put> puts) throws IOException {
|
||||
List<TPut> tPuts = ThriftUtilities.putsFromHBase(puts);
|
||||
try {
|
||||
client.putMultiple(tableNameInBytes, tPuts);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(Delete delete) throws IOException {
|
||||
TDelete tDelete = ThriftUtilities.deleteFromHBase(delete);
|
||||
try {
|
||||
client.deleteSingle(tableNameInBytes, tDelete);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(List<Delete> deletes) throws IOException {
|
||||
List<TDelete> tDeletes = ThriftUtilities.deletesFromHBase(deletes);
|
||||
try {
|
||||
client.deleteMultiple(tableNameInBytes, tDeletes);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private class CheckAndMutateBuilderImpl implements CheckAndMutateBuilder {
|
||||
|
||||
private final byte[] row;
|
||||
private final byte[] family;
|
||||
private byte[] qualifier;
|
||||
private CompareOperator op;
|
||||
private byte[] value;
|
||||
|
||||
CheckAndMutateBuilderImpl(byte[] row, byte[] family) {
|
||||
this.row = Preconditions.checkNotNull(row, "row is null");
|
||||
this.family = Preconditions.checkNotNull(family, "family is null");
|
||||
}
|
||||
|
||||
@Override
|
||||
public CheckAndMutateBuilder qualifier(byte[] qualifier) {
|
||||
this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using" +
|
||||
" an empty byte array, or just do not call this method if you want a null qualifier");
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CheckAndMutateBuilder timeRange(TimeRange timeRange) {
|
||||
throw new NotImplementedException("timeRange not supported in ThriftTable");
|
||||
}
|
||||
|
||||
@Override
|
||||
public CheckAndMutateBuilder ifNotExists() {
|
||||
this.op = CompareOperator.EQUAL;
|
||||
this.value = null;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value) {
|
||||
this.op = Preconditions.checkNotNull(compareOp, "compareOp is null");
|
||||
this.value = Preconditions.checkNotNull(value, "value is null");
|
||||
return this;
|
||||
}
|
||||
|
||||
private void preCheck() {
|
||||
Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by" +
|
||||
" calling ifNotExists/ifEquals/ifMatches before executing the request");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean thenPut(Put put) throws IOException {
|
||||
preCheck();
|
||||
RowMutations rowMutations = new RowMutations(put.getRow());
|
||||
rowMutations.add(put);
|
||||
return checkAndMutate(row, family, qualifier, op, value, rowMutations);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean thenDelete(Delete delete) throws IOException {
|
||||
preCheck();
|
||||
RowMutations rowMutations = new RowMutations(delete.getRow());
|
||||
rowMutations.add(delete);
|
||||
return checkAndMutate(row, family, qualifier, op, value, rowMutations);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean thenMutate(RowMutations mutation) throws IOException {
|
||||
preCheck();
|
||||
return checkAndMutate(row, family, qualifier, op, value, mutation);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
|
||||
byte[] value, RowMutations mutation) throws IOException {
|
||||
try {
|
||||
ByteBuffer valueBuffer = value == null? null : ByteBuffer.wrap(value);
|
||||
return client.checkAndMutate(tableNameInBytes, ByteBuffer.wrap(row), ByteBuffer.wrap(family),
|
||||
ByteBuffer.wrap(qualifier), ThriftUtilities.compareOpFromHBase(op), valueBuffer,
|
||||
ThriftUtilities.rowMutationsFromHBase(mutation));
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
|
||||
return new CheckAndMutateBuilderImpl(row, family);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void mutateRow(RowMutations rm) throws IOException {
|
||||
TRowMutations tRowMutations = ThriftUtilities.rowMutationsFromHBase(rm);
|
||||
try {
|
||||
client.mutateRow(tableNameInBytes, tRowMutations);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result append(Append append) throws IOException {
|
||||
TAppend tAppend = ThriftUtilities.appendFromHBase(append);
|
||||
try {
|
||||
TResult tResult = client.append(tableNameInBytes, tAppend);
|
||||
return ThriftUtilities.resultFromThrift(tResult);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Result increment(Increment increment) throws IOException {
|
||||
TIncrement tIncrement = ThriftUtilities.incrementFromHBase(increment);
|
||||
try {
|
||||
TResult tResult = client.increment(tableNameInBytes, tIncrement);
|
||||
return ThriftUtilities.resultFromThrift(tResult);
|
||||
} catch (TException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
tTransport.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getRpcTimeout(TimeUnit unit) {
|
||||
return unit.convert(operationTimeout, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getReadRpcTimeout(TimeUnit unit) {
|
||||
return unit.convert(operationTimeout, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getWriteRpcTimeout(TimeUnit unit) {
|
||||
return unit.convert(operationTimeout, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getOperationTimeout(TimeUnit unit) {
|
||||
return unit.convert(operationTimeout, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CoprocessorRpcChannel coprocessorService(byte[] row) {
|
||||
throw new NotImplementedException("coprocessorService not supported in ThriftTable");
|
||||
}
|
||||
|
||||
}
|
|
@ -34,7 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TAppend implements org.apache.thrift.TBase<TAppend, TAppend._Fields>, java.io.Serializable, Cloneable, Comparable<TAppend> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAppend");
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TAuthorization implements org.apache.thrift.TBase<TAuthorization, TAuthorization._Fields>, java.io.Serializable, Cloneable, Comparable<TAuthorization> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TAuthorization");
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TCellVisibility implements org.apache.thrift.TBase<TCellVisibility, TCellVisibility._Fields>, java.io.Serializable, Cloneable, Comparable<TCellVisibility> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCellVisibility");
|
||||
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
|
|||
* in a HBase table by column family and optionally
|
||||
* a column qualifier and timestamp
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable, Comparable<TColumn> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.ColumnFamilyDescriptor
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TColumnFamilyDescriptor implements org.apache.thrift.TBase<TColumnFamilyDescriptor, TColumnFamilyDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnFamilyDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnFamilyDescriptor");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* Represents a single cell and the amount to increment it by
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TColumnIncrement implements org.apache.thrift.TBase<TColumnIncrement, TColumnIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnIncrement> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnIncrement");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* Represents a single cell and its value.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TColumnValue implements org.apache.thrift.TBase<TColumnValue, TColumnValue._Fields>, java.io.Serializable, Cloneable, Comparable<TColumnValue> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue");
|
||||
|
||||
|
|
|
@ -60,7 +60,7 @@ import org.slf4j.LoggerFactory;
|
|||
* by changing the durability. If you don't provide durability, it defaults to
|
||||
* column family's default setting for durability.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TDelete implements org.apache.thrift.TBase<TDelete, TDelete._Fields>, java.io.Serializable, Cloneable, Comparable<TDelete> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDelete");
|
||||
|
||||
|
|
|
@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
|
|||
* If you specify a time range and a timestamp the range is ignored.
|
||||
* Timestamps on TColumns are ignored.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.io.Serializable, Cloneable, Comparable<TGet> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGet");
|
||||
|
||||
|
@ -65,6 +65,7 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
private static final org.apache.thrift.protocol.TField STORE_LIMIT_FIELD_DESC = new org.apache.thrift.protocol.TField("storeLimit", org.apache.thrift.protocol.TType.I32, (short)12);
|
||||
private static final org.apache.thrift.protocol.TField STORE_OFFSET_FIELD_DESC = new org.apache.thrift.protocol.TField("storeOffset", org.apache.thrift.protocol.TType.I32, (short)13);
|
||||
private static final org.apache.thrift.protocol.TField EXISTENCE_ONLY_FIELD_DESC = new org.apache.thrift.protocol.TField("existence_only", org.apache.thrift.protocol.TType.BOOL, (short)14);
|
||||
private static final org.apache.thrift.protocol.TField FILTER_BYTES_FIELD_DESC = new org.apache.thrift.protocol.TField("filterBytes", org.apache.thrift.protocol.TType.STRING, (short)15);
|
||||
|
||||
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
|
||||
static {
|
||||
|
@ -90,6 +91,7 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
public int storeLimit; // optional
|
||||
public int storeOffset; // optional
|
||||
public boolean existence_only; // optional
|
||||
public ByteBuffer filterBytes; // optional
|
||||
|
||||
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
|
||||
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
|
||||
|
@ -110,7 +112,8 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
CACHE_BLOCKS((short)11, "cacheBlocks"),
|
||||
STORE_LIMIT((short)12, "storeLimit"),
|
||||
STORE_OFFSET((short)13, "storeOffset"),
|
||||
EXISTENCE_ONLY((short)14, "existence_only");
|
||||
EXISTENCE_ONLY((short)14, "existence_only"),
|
||||
FILTER_BYTES((short)15, "filterBytes");
|
||||
|
||||
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
|
||||
|
||||
|
@ -153,6 +156,8 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
return STORE_OFFSET;
|
||||
case 14: // EXISTENCE_ONLY
|
||||
return EXISTENCE_ONLY;
|
||||
case 15: // FILTER_BYTES
|
||||
return FILTER_BYTES;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
|
@ -201,7 +206,7 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
private static final int __STOREOFFSET_ISSET_ID = 5;
|
||||
private static final int __EXISTENCE_ONLY_ISSET_ID = 6;
|
||||
private byte __isset_bitfield = 0;
|
||||
private static final _Fields optionals[] = {_Fields.COLUMNS,_Fields.TIMESTAMP,_Fields.TIME_RANGE,_Fields.MAX_VERSIONS,_Fields.FILTER_STRING,_Fields.ATTRIBUTES,_Fields.AUTHORIZATIONS,_Fields.CONSISTENCY,_Fields.TARGET_REPLICA_ID,_Fields.CACHE_BLOCKS,_Fields.STORE_LIMIT,_Fields.STORE_OFFSET,_Fields.EXISTENCE_ONLY};
|
||||
private static final _Fields optionals[] = {_Fields.COLUMNS,_Fields.TIMESTAMP,_Fields.TIME_RANGE,_Fields.MAX_VERSIONS,_Fields.FILTER_STRING,_Fields.ATTRIBUTES,_Fields.AUTHORIZATIONS,_Fields.CONSISTENCY,_Fields.TARGET_REPLICA_ID,_Fields.CACHE_BLOCKS,_Fields.STORE_LIMIT,_Fields.STORE_OFFSET,_Fields.EXISTENCE_ONLY,_Fields.FILTER_BYTES};
|
||||
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
|
||||
static {
|
||||
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
|
@ -236,6 +241,8 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
|
||||
tmpMap.put(_Fields.EXISTENCE_ONLY, new org.apache.thrift.meta_data.FieldMetaData("existence_only", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
|
||||
tmpMap.put(_Fields.FILTER_BYTES, new org.apache.thrift.meta_data.FieldMetaData("filterBytes", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
|
||||
metaDataMap = Collections.unmodifiableMap(tmpMap);
|
||||
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGet.class, metaDataMap);
|
||||
}
|
||||
|
@ -288,6 +295,9 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
this.storeLimit = other.storeLimit;
|
||||
this.storeOffset = other.storeOffset;
|
||||
this.existence_only = other.existence_only;
|
||||
if (other.isSetFilterBytes()) {
|
||||
this.filterBytes = org.apache.thrift.TBaseHelper.copyBinary(other.filterBytes);
|
||||
}
|
||||
}
|
||||
|
||||
public TGet deepCopy() {
|
||||
|
@ -317,6 +327,7 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
this.storeOffset = 0;
|
||||
setExistence_onlyIsSet(false);
|
||||
this.existence_only = false;
|
||||
this.filterBytes = null;
|
||||
}
|
||||
|
||||
public byte[] getRow() {
|
||||
|
@ -702,6 +713,40 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EXISTENCE_ONLY_ISSET_ID, value);
|
||||
}
|
||||
|
||||
public byte[] getFilterBytes() {
|
||||
setFilterBytes(org.apache.thrift.TBaseHelper.rightSize(filterBytes));
|
||||
return filterBytes == null ? null : filterBytes.array();
|
||||
}
|
||||
|
||||
public ByteBuffer bufferForFilterBytes() {
|
||||
return org.apache.thrift.TBaseHelper.copyBinary(filterBytes);
|
||||
}
|
||||
|
||||
public TGet setFilterBytes(byte[] filterBytes) {
|
||||
this.filterBytes = filterBytes == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(filterBytes, filterBytes.length));
|
||||
return this;
|
||||
}
|
||||
|
||||
public TGet setFilterBytes(ByteBuffer filterBytes) {
|
||||
this.filterBytes = org.apache.thrift.TBaseHelper.copyBinary(filterBytes);
|
||||
return this;
|
||||
}
|
||||
|
||||
public void unsetFilterBytes() {
|
||||
this.filterBytes = null;
|
||||
}
|
||||
|
||||
/** Returns true if field filterBytes is set (has been assigned a value) and false otherwise */
|
||||
public boolean isSetFilterBytes() {
|
||||
return this.filterBytes != null;
|
||||
}
|
||||
|
||||
public void setFilterBytesIsSet(boolean value) {
|
||||
if (!value) {
|
||||
this.filterBytes = null;
|
||||
}
|
||||
}
|
||||
|
||||
public void setFieldValue(_Fields field, Object value) {
|
||||
switch (field) {
|
||||
case ROW:
|
||||
|
@ -816,6 +861,14 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
}
|
||||
break;
|
||||
|
||||
case FILTER_BYTES:
|
||||
if (value == null) {
|
||||
unsetFilterBytes();
|
||||
} else {
|
||||
setFilterBytes((ByteBuffer)value);
|
||||
}
|
||||
break;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -863,6 +916,9 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
case EXISTENCE_ONLY:
|
||||
return isExistence_only();
|
||||
|
||||
case FILTER_BYTES:
|
||||
return getFilterBytes();
|
||||
|
||||
}
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
@ -902,6 +958,8 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
return isSetStoreOffset();
|
||||
case EXISTENCE_ONLY:
|
||||
return isSetExistence_only();
|
||||
case FILTER_BYTES:
|
||||
return isSetFilterBytes();
|
||||
}
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
@ -1045,6 +1103,15 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
return false;
|
||||
}
|
||||
|
||||
boolean this_present_filterBytes = true && this.isSetFilterBytes();
|
||||
boolean that_present_filterBytes = true && that.isSetFilterBytes();
|
||||
if (this_present_filterBytes || that_present_filterBytes) {
|
||||
if (!(this_present_filterBytes && that_present_filterBytes))
|
||||
return false;
|
||||
if (!this.filterBytes.equals(that.filterBytes))
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -1122,6 +1189,11 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
if (present_existence_only)
|
||||
list.add(existence_only);
|
||||
|
||||
boolean present_filterBytes = true && (isSetFilterBytes());
|
||||
list.add(present_filterBytes);
|
||||
if (present_filterBytes)
|
||||
list.add(filterBytes);
|
||||
|
||||
return list.hashCode();
|
||||
}
|
||||
|
||||
|
@ -1273,6 +1345,16 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
return lastComparison;
|
||||
}
|
||||
}
|
||||
lastComparison = Boolean.valueOf(isSetFilterBytes()).compareTo(other.isSetFilterBytes());
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
if (isSetFilterBytes()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterBytes, other.filterBytes);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
@ -1402,6 +1484,16 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
sb.append(this.existence_only);
|
||||
first = false;
|
||||
}
|
||||
if (isSetFilterBytes()) {
|
||||
if (!first) sb.append(", ");
|
||||
sb.append("filterBytes:");
|
||||
if (this.filterBytes == null) {
|
||||
sb.append("null");
|
||||
} else {
|
||||
org.apache.thrift.TBaseHelper.toString(this.filterBytes, sb);
|
||||
}
|
||||
first = false;
|
||||
}
|
||||
sb.append(")");
|
||||
return sb.toString();
|
||||
}
|
||||
|
@ -1593,6 +1685,14 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
break;
|
||||
case 15: // FILTER_BYTES
|
||||
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
|
||||
struct.filterBytes = iprot.readBinary();
|
||||
struct.setFilterBytesIsSet(true);
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
|
@ -1705,6 +1805,13 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
oprot.writeBool(struct.existence_only);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
if (struct.filterBytes != null) {
|
||||
if (struct.isSetFilterBytes()) {
|
||||
oprot.writeFieldBegin(FILTER_BYTES_FIELD_DESC);
|
||||
oprot.writeBinary(struct.filterBytes);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
}
|
||||
oprot.writeFieldStop();
|
||||
oprot.writeStructEnd();
|
||||
}
|
||||
|
@ -1763,7 +1870,10 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
if (struct.isSetExistence_only()) {
|
||||
optionals.set(12);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 13);
|
||||
if (struct.isSetFilterBytes()) {
|
||||
optionals.set(13);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 14);
|
||||
if (struct.isSetColumns()) {
|
||||
{
|
||||
oprot.writeI32(struct.columns.size());
|
||||
|
@ -1816,6 +1926,9 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
if (struct.isSetExistence_only()) {
|
||||
oprot.writeBool(struct.existence_only);
|
||||
}
|
||||
if (struct.isSetFilterBytes()) {
|
||||
oprot.writeBinary(struct.filterBytes);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1823,7 +1936,7 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
TTupleProtocol iprot = (TTupleProtocol) prot;
|
||||
struct.row = iprot.readBinary();
|
||||
struct.setRowIsSet(true);
|
||||
BitSet incoming = iprot.readBitSet(13);
|
||||
BitSet incoming = iprot.readBitSet(14);
|
||||
if (incoming.get(0)) {
|
||||
{
|
||||
org.apache.thrift.protocol.TList _list27 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
|
||||
|
@ -1899,6 +2012,10 @@ public class TGet implements org.apache.thrift.TBase<TGet, TGet._Fields>, java.i
|
|||
struct.existence_only = iprot.readBool();
|
||||
struct.setExistence_onlyIsSet(true);
|
||||
}
|
||||
if (incoming.get(13)) {
|
||||
struct.filterBytes = iprot.readBinary();
|
||||
struct.setFilterBytesIsSet(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class THBaseService {
|
||||
|
||||
public interface Iface {
|
||||
|
@ -30957,7 +30957,7 @@ public class THBaseService {
|
|||
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
|
||||
static {
|
||||
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
tmpMap.put(_Fields.REGEX, new org.apache.thrift.meta_data.FieldMetaData("regex", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
tmpMap.put(_Fields.REGEX, new org.apache.thrift.meta_data.FieldMetaData("regex", org.apache.thrift.TFieldRequirementType.DEFAULT,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
|
||||
tmpMap.put(_Fields.INCLUDE_SYS_TABLES, new org.apache.thrift.meta_data.FieldMetaData("includeSysTables", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
|
||||
|
@ -31223,9 +31223,6 @@ public class THBaseService {
|
|||
|
||||
public void validate() throws org.apache.thrift.TException {
|
||||
// check for required fields
|
||||
if (regex == null) {
|
||||
throw new org.apache.thrift.protocol.TProtocolException("Required field 'regex' was not present! Struct: " + toString());
|
||||
}
|
||||
// alas, we cannot check 'includeSysTables' because it's a primitive and you chose the non-beans generator.
|
||||
// check for sub-struct validity
|
||||
}
|
||||
|
@ -31325,17 +31322,27 @@ public class THBaseService {
|
|||
@Override
|
||||
public void write(org.apache.thrift.protocol.TProtocol prot, getTableDescriptorsByPattern_args struct) throws org.apache.thrift.TException {
|
||||
TTupleProtocol oprot = (TTupleProtocol) prot;
|
||||
oprot.writeString(struct.regex);
|
||||
oprot.writeBool(struct.includeSysTables);
|
||||
BitSet optionals = new BitSet();
|
||||
if (struct.isSetRegex()) {
|
||||
optionals.set(0);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 1);
|
||||
if (struct.isSetRegex()) {
|
||||
oprot.writeString(struct.regex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void read(org.apache.thrift.protocol.TProtocol prot, getTableDescriptorsByPattern_args struct) throws org.apache.thrift.TException {
|
||||
TTupleProtocol iprot = (TTupleProtocol) prot;
|
||||
struct.regex = iprot.readString();
|
||||
struct.setRegexIsSet(true);
|
||||
struct.includeSysTables = iprot.readBool();
|
||||
struct.setIncludeSysTablesIsSet(true);
|
||||
BitSet incoming = iprot.readBitSet(1);
|
||||
if (incoming.get(0)) {
|
||||
struct.regex = iprot.readString();
|
||||
struct.setRegexIsSet(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -32845,7 +32852,7 @@ public class THBaseService {
|
|||
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
|
||||
static {
|
||||
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
tmpMap.put(_Fields.REGEX, new org.apache.thrift.meta_data.FieldMetaData("regex", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
tmpMap.put(_Fields.REGEX, new org.apache.thrift.meta_data.FieldMetaData("regex", org.apache.thrift.TFieldRequirementType.DEFAULT,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
|
||||
tmpMap.put(_Fields.INCLUDE_SYS_TABLES, new org.apache.thrift.meta_data.FieldMetaData("includeSysTables", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
|
||||
|
@ -33111,9 +33118,6 @@ public class THBaseService {
|
|||
|
||||
public void validate() throws org.apache.thrift.TException {
|
||||
// check for required fields
|
||||
if (regex == null) {
|
||||
throw new org.apache.thrift.protocol.TProtocolException("Required field 'regex' was not present! Struct: " + toString());
|
||||
}
|
||||
// alas, we cannot check 'includeSysTables' because it's a primitive and you chose the non-beans generator.
|
||||
// check for sub-struct validity
|
||||
}
|
||||
|
@ -33213,17 +33217,27 @@ public class THBaseService {
|
|||
@Override
|
||||
public void write(org.apache.thrift.protocol.TProtocol prot, getTableNamesByPattern_args struct) throws org.apache.thrift.TException {
|
||||
TTupleProtocol oprot = (TTupleProtocol) prot;
|
||||
oprot.writeString(struct.regex);
|
||||
oprot.writeBool(struct.includeSysTables);
|
||||
BitSet optionals = new BitSet();
|
||||
if (struct.isSetRegex()) {
|
||||
optionals.set(0);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 1);
|
||||
if (struct.isSetRegex()) {
|
||||
oprot.writeString(struct.regex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void read(org.apache.thrift.protocol.TProtocol prot, getTableNamesByPattern_args struct) throws org.apache.thrift.TException {
|
||||
TTupleProtocol iprot = (TTupleProtocol) prot;
|
||||
struct.regex = iprot.readString();
|
||||
struct.setRegexIsSet(true);
|
||||
struct.includeSysTables = iprot.readBool();
|
||||
struct.setIncludeSysTablesIsSet(true);
|
||||
BitSet incoming = iprot.readBitSet(1);
|
||||
if (incoming.get(0)) {
|
||||
struct.regex = iprot.readString();
|
||||
struct.setRegexIsSet(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -34733,7 +34747,7 @@ public class THBaseService {
|
|||
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
tmpMap.put(_Fields.DESC, new org.apache.thrift.meta_data.FieldMetaData("desc", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableDescriptor.class)));
|
||||
tmpMap.put(_Fields.SPLIT_KEYS, new org.apache.thrift.meta_data.FieldMetaData("splitKeys", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
tmpMap.put(_Fields.SPLIT_KEYS, new org.apache.thrift.meta_data.FieldMetaData("splitKeys", org.apache.thrift.TFieldRequirementType.DEFAULT,
|
||||
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))));
|
||||
metaDataMap = Collections.unmodifiableMap(tmpMap);
|
||||
|
@ -35021,9 +35035,6 @@ public class THBaseService {
|
|||
if (desc == null) {
|
||||
throw new org.apache.thrift.protocol.TProtocolException("Required field 'desc' was not present! Struct: " + toString());
|
||||
}
|
||||
if (splitKeys == null) {
|
||||
throw new org.apache.thrift.protocol.TProtocolException("Required field 'splitKeys' was not present! Struct: " + toString());
|
||||
}
|
||||
// check for sub-struct validity
|
||||
if (desc != null) {
|
||||
desc.validate();
|
||||
|
@ -35141,11 +35152,18 @@ public class THBaseService {
|
|||
public void write(org.apache.thrift.protocol.TProtocol prot, createTable_args struct) throws org.apache.thrift.TException {
|
||||
TTupleProtocol oprot = (TTupleProtocol) prot;
|
||||
struct.desc.write(oprot);
|
||||
{
|
||||
oprot.writeI32(struct.splitKeys.size());
|
||||
for (ByteBuffer _iter322 : struct.splitKeys)
|
||||
BitSet optionals = new BitSet();
|
||||
if (struct.isSetSplitKeys()) {
|
||||
optionals.set(0);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 1);
|
||||
if (struct.isSetSplitKeys()) {
|
||||
{
|
||||
oprot.writeBinary(_iter322);
|
||||
oprot.writeI32(struct.splitKeys.size());
|
||||
for (ByteBuffer _iter322 : struct.splitKeys)
|
||||
{
|
||||
oprot.writeBinary(_iter322);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -35156,17 +35174,20 @@ public class THBaseService {
|
|||
struct.desc = new TTableDescriptor();
|
||||
struct.desc.read(iprot);
|
||||
struct.setDescIsSet(true);
|
||||
{
|
||||
org.apache.thrift.protocol.TList _list323 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
|
||||
struct.splitKeys = new ArrayList<ByteBuffer>(_list323.size);
|
||||
ByteBuffer _elem324;
|
||||
for (int _i325 = 0; _i325 < _list323.size; ++_i325)
|
||||
BitSet incoming = iprot.readBitSet(1);
|
||||
if (incoming.get(0)) {
|
||||
{
|
||||
_elem324 = iprot.readBinary();
|
||||
struct.splitKeys.add(_elem324);
|
||||
org.apache.thrift.protocol.TList _list323 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
|
||||
struct.splitKeys = new ArrayList<ByteBuffer>(_list323.size);
|
||||
ByteBuffer _elem324;
|
||||
for (int _i325 = 0; _i325 < _list323.size; ++_i325)
|
||||
{
|
||||
_elem324 = iprot.readBinary();
|
||||
struct.splitKeys.add(_elem324);
|
||||
}
|
||||
}
|
||||
struct.setSplitKeysIsSet(true);
|
||||
}
|
||||
struct.setSplitKeysIsSet(true);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -41194,7 +41215,7 @@ public class THBaseService {
|
|||
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableName.class)));
|
||||
tmpMap.put(_Fields.SPLIT_KEYS, new org.apache.thrift.meta_data.FieldMetaData("splitKeys", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
tmpMap.put(_Fields.SPLIT_KEYS, new org.apache.thrift.meta_data.FieldMetaData("splitKeys", org.apache.thrift.TFieldRequirementType.DEFAULT,
|
||||
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))));
|
||||
metaDataMap = Collections.unmodifiableMap(tmpMap);
|
||||
|
@ -41482,9 +41503,6 @@ public class THBaseService {
|
|||
if (tableName == null) {
|
||||
throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' was not present! Struct: " + toString());
|
||||
}
|
||||
if (splitKeys == null) {
|
||||
throw new org.apache.thrift.protocol.TProtocolException("Required field 'splitKeys' was not present! Struct: " + toString());
|
||||
}
|
||||
// check for sub-struct validity
|
||||
if (tableName != null) {
|
||||
tableName.validate();
|
||||
|
@ -41602,11 +41620,18 @@ public class THBaseService {
|
|||
public void write(org.apache.thrift.protocol.TProtocol prot, isTableAvailableWithSplit_args struct) throws org.apache.thrift.TException {
|
||||
TTupleProtocol oprot = (TTupleProtocol) prot;
|
||||
struct.tableName.write(oprot);
|
||||
{
|
||||
oprot.writeI32(struct.splitKeys.size());
|
||||
for (ByteBuffer _iter330 : struct.splitKeys)
|
||||
BitSet optionals = new BitSet();
|
||||
if (struct.isSetSplitKeys()) {
|
||||
optionals.set(0);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 1);
|
||||
if (struct.isSetSplitKeys()) {
|
||||
{
|
||||
oprot.writeBinary(_iter330);
|
||||
oprot.writeI32(struct.splitKeys.size());
|
||||
for (ByteBuffer _iter330 : struct.splitKeys)
|
||||
{
|
||||
oprot.writeBinary(_iter330);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -41617,17 +41642,20 @@ public class THBaseService {
|
|||
struct.tableName = new TTableName();
|
||||
struct.tableName.read(iprot);
|
||||
struct.setTableNameIsSet(true);
|
||||
{
|
||||
org.apache.thrift.protocol.TList _list331 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
|
||||
struct.splitKeys = new ArrayList<ByteBuffer>(_list331.size);
|
||||
ByteBuffer _elem332;
|
||||
for (int _i333 = 0; _i333 < _list331.size; ++_i333)
|
||||
BitSet incoming = iprot.readBitSet(1);
|
||||
if (incoming.get(0)) {
|
||||
{
|
||||
_elem332 = iprot.readBinary();
|
||||
struct.splitKeys.add(_elem332);
|
||||
org.apache.thrift.protocol.TList _list331 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
|
||||
struct.splitKeys = new ArrayList<ByteBuffer>(_list331.size);
|
||||
ByteBuffer _elem332;
|
||||
for (int _i333 = 0; _i333 < _list331.size; ++_i333)
|
||||
{
|
||||
_elem332 = iprot.readBinary();
|
||||
struct.splitKeys.add(_elem332);
|
||||
}
|
||||
}
|
||||
struct.setSplitKeysIsSet(true);
|
||||
}
|
||||
struct.setSplitKeysIsSet(true);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class THRegionInfo implements org.apache.thrift.TBase<THRegionInfo, THRegionInfo._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionInfo> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionInfo");
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class THRegionLocation implements org.apache.thrift.TBase<THRegionLocation, THRegionLocation._Fields>, java.io.Serializable, Cloneable, Comparable<THRegionLocation> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THRegionLocation");
|
||||
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.slf4j.LoggerFactory;
|
|||
* to the HBase master or a HBase region server. Also used to return
|
||||
* more general HBase error conditions.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TIOError extends TException implements org.apache.thrift.TBase<TIOError, TIOError._Fields>, java.io.Serializable, Cloneable, Comparable<TIOError> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIOError");
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
|
|||
* A TIllegalArgument exception indicates an illegal or invalid
|
||||
* argument was passed into a procedure.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TIllegalArgument extends TException implements org.apache.thrift.TBase<TIllegalArgument, TIllegalArgument._Fields>, java.io.Serializable, Cloneable, Comparable<TIllegalArgument> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIllegalArgument");
|
||||
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
|
|||
* by changing the durability. If you don't provide durability, it defaults to
|
||||
* column family's default setting for durability.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TIncrement implements org.apache.thrift.TBase<TIncrement, TIncrement._Fields>, java.io.Serializable, Cloneable, Comparable<TIncrement> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TIncrement");
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.NamespaceDescriptor
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TNamespaceDescriptor implements org.apache.thrift.TBase<TNamespaceDescriptor, TNamespaceDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TNamespaceDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TNamespaceDescriptor");
|
||||
|
||||
|
|
|
@ -46,7 +46,7 @@ import org.slf4j.LoggerFactory;
|
|||
* by changing the durability. If you don't provide durability, it defaults to
|
||||
* column family's default setting for durability.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TPut implements org.apache.thrift.TBase<TPut, TPut._Fields>, java.io.Serializable, Cloneable, Comparable<TPut> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPut");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* if no Result is found, row and columnValues will not be set.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TResult implements org.apache.thrift.TBase<TResult, TResult._Fields>, java.io.Serializable, Cloneable, Comparable<TResult> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TResult");
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
|
|||
/**
|
||||
* A TRowMutations object is used to apply a number of Mutations to a single row.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TRowMutations implements org.apache.thrift.TBase<TRowMutations, TRowMutations._Fields>, java.io.Serializable, Cloneable, Comparable<TRowMutations> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowMutations");
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
|
|||
* Any timestamps in the columns are ignored but the colFamTimeRangeMap included, use timeRange to select by timestamp.
|
||||
* Max versions defaults to 1.
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable, Comparable<TScan> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
|
||||
|
||||
|
@ -59,6 +59,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
private static final org.apache.thrift.protocol.TField LIMIT_FIELD_DESC = new org.apache.thrift.protocol.TField("limit", org.apache.thrift.protocol.TType.I32, (short)15);
|
||||
private static final org.apache.thrift.protocol.TField CONSISTENCY_FIELD_DESC = new org.apache.thrift.protocol.TField("consistency", org.apache.thrift.protocol.TType.I32, (short)16);
|
||||
private static final org.apache.thrift.protocol.TField TARGET_REPLICA_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("targetReplicaId", org.apache.thrift.protocol.TType.I32, (short)17);
|
||||
private static final org.apache.thrift.protocol.TField FILTER_BYTES_FIELD_DESC = new org.apache.thrift.protocol.TField("filterBytes", org.apache.thrift.protocol.TType.STRING, (short)18);
|
||||
|
||||
private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
|
||||
static {
|
||||
|
@ -91,6 +92,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
*/
|
||||
public TConsistency consistency; // optional
|
||||
public int targetReplicaId; // optional
|
||||
public ByteBuffer filterBytes; // optional
|
||||
|
||||
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
|
||||
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
|
||||
|
@ -118,7 +120,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
* @see TConsistency
|
||||
*/
|
||||
CONSISTENCY((short)16, "consistency"),
|
||||
TARGET_REPLICA_ID((short)17, "targetReplicaId");
|
||||
TARGET_REPLICA_ID((short)17, "targetReplicaId"),
|
||||
FILTER_BYTES((short)18, "filterBytes");
|
||||
|
||||
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
|
||||
|
||||
|
@ -167,6 +170,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return CONSISTENCY;
|
||||
case 17: // TARGET_REPLICA_ID
|
||||
return TARGET_REPLICA_ID;
|
||||
case 18: // FILTER_BYTES
|
||||
return FILTER_BYTES;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
|
@ -215,7 +220,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
private static final int __LIMIT_ISSET_ID = 5;
|
||||
private static final int __TARGETREPLICAID_ISSET_ID = 6;
|
||||
private byte __isset_bitfield = 0;
|
||||
private static final _Fields optionals[] = {_Fields.START_ROW,_Fields.STOP_ROW,_Fields.COLUMNS,_Fields.CACHING,_Fields.MAX_VERSIONS,_Fields.TIME_RANGE,_Fields.FILTER_STRING,_Fields.BATCH_SIZE,_Fields.ATTRIBUTES,_Fields.AUTHORIZATIONS,_Fields.REVERSED,_Fields.CACHE_BLOCKS,_Fields.COL_FAM_TIME_RANGE_MAP,_Fields.READ_TYPE,_Fields.LIMIT,_Fields.CONSISTENCY,_Fields.TARGET_REPLICA_ID};
|
||||
private static final _Fields optionals[] = {_Fields.START_ROW,_Fields.STOP_ROW,_Fields.COLUMNS,_Fields.CACHING,_Fields.MAX_VERSIONS,_Fields.TIME_RANGE,_Fields.FILTER_STRING,_Fields.BATCH_SIZE,_Fields.ATTRIBUTES,_Fields.AUTHORIZATIONS,_Fields.REVERSED,_Fields.CACHE_BLOCKS,_Fields.COL_FAM_TIME_RANGE_MAP,_Fields.READ_TYPE,_Fields.LIMIT,_Fields.CONSISTENCY,_Fields.TARGET_REPLICA_ID,_Fields.FILTER_BYTES};
|
||||
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
|
||||
static {
|
||||
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
|
@ -258,6 +263,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TConsistency.class)));
|
||||
tmpMap.put(_Fields.TARGET_REPLICA_ID, new org.apache.thrift.meta_data.FieldMetaData("targetReplicaId", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
|
||||
tmpMap.put(_Fields.FILTER_BYTES, new org.apache.thrift.meta_data.FieldMetaData("filterBytes", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
|
||||
metaDataMap = Collections.unmodifiableMap(tmpMap);
|
||||
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TScan.class, metaDataMap);
|
||||
}
|
||||
|
@ -326,6 +333,9 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
this.consistency = other.consistency;
|
||||
}
|
||||
this.targetReplicaId = other.targetReplicaId;
|
||||
if (other.isSetFilterBytes()) {
|
||||
this.filterBytes = org.apache.thrift.TBaseHelper.copyBinary(other.filterBytes);
|
||||
}
|
||||
}
|
||||
|
||||
public TScan deepCopy() {
|
||||
|
@ -358,6 +368,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
this.consistency = null;
|
||||
setTargetReplicaIdIsSet(false);
|
||||
this.targetReplicaId = 0;
|
||||
this.filterBytes = null;
|
||||
}
|
||||
|
||||
public byte[] getStartRow() {
|
||||
|
@ -844,6 +855,40 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
__isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TARGETREPLICAID_ISSET_ID, value);
|
||||
}
|
||||
|
||||
public byte[] getFilterBytes() {
|
||||
setFilterBytes(org.apache.thrift.TBaseHelper.rightSize(filterBytes));
|
||||
return filterBytes == null ? null : filterBytes.array();
|
||||
}
|
||||
|
||||
public ByteBuffer bufferForFilterBytes() {
|
||||
return org.apache.thrift.TBaseHelper.copyBinary(filterBytes);
|
||||
}
|
||||
|
||||
public TScan setFilterBytes(byte[] filterBytes) {
|
||||
this.filterBytes = filterBytes == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(filterBytes, filterBytes.length));
|
||||
return this;
|
||||
}
|
||||
|
||||
public TScan setFilterBytes(ByteBuffer filterBytes) {
|
||||
this.filterBytes = org.apache.thrift.TBaseHelper.copyBinary(filterBytes);
|
||||
return this;
|
||||
}
|
||||
|
||||
public void unsetFilterBytes() {
|
||||
this.filterBytes = null;
|
||||
}
|
||||
|
||||
/** Returns true if field filterBytes is set (has been assigned a value) and false otherwise */
|
||||
public boolean isSetFilterBytes() {
|
||||
return this.filterBytes != null;
|
||||
}
|
||||
|
||||
public void setFilterBytesIsSet(boolean value) {
|
||||
if (!value) {
|
||||
this.filterBytes = null;
|
||||
}
|
||||
}
|
||||
|
||||
public void setFieldValue(_Fields field, Object value) {
|
||||
switch (field) {
|
||||
case START_ROW:
|
||||
|
@ -982,6 +1027,14 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
}
|
||||
break;
|
||||
|
||||
case FILTER_BYTES:
|
||||
if (value == null) {
|
||||
unsetFilterBytes();
|
||||
} else {
|
||||
setFilterBytes((ByteBuffer)value);
|
||||
}
|
||||
break;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1038,6 +1091,9 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
case TARGET_REPLICA_ID:
|
||||
return getTargetReplicaId();
|
||||
|
||||
case FILTER_BYTES:
|
||||
return getFilterBytes();
|
||||
|
||||
}
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
@ -1083,6 +1139,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return isSetConsistency();
|
||||
case TARGET_REPLICA_ID:
|
||||
return isSetTargetReplicaId();
|
||||
case FILTER_BYTES:
|
||||
return isSetFilterBytes();
|
||||
}
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
@ -1253,6 +1311,15 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return false;
|
||||
}
|
||||
|
||||
boolean this_present_filterBytes = true && this.isSetFilterBytes();
|
||||
boolean that_present_filterBytes = true && that.isSetFilterBytes();
|
||||
if (this_present_filterBytes || that_present_filterBytes) {
|
||||
if (!(this_present_filterBytes && that_present_filterBytes))
|
||||
return false;
|
||||
if (!this.filterBytes.equals(that.filterBytes))
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -1345,6 +1412,11 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
if (present_targetReplicaId)
|
||||
list.add(targetReplicaId);
|
||||
|
||||
boolean present_filterBytes = true && (isSetFilterBytes());
|
||||
list.add(present_filterBytes);
|
||||
if (present_filterBytes)
|
||||
list.add(filterBytes);
|
||||
|
||||
return list.hashCode();
|
||||
}
|
||||
|
||||
|
@ -1526,6 +1598,16 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return lastComparison;
|
||||
}
|
||||
}
|
||||
lastComparison = Boolean.valueOf(isSetFilterBytes()).compareTo(other.isSetFilterBytes());
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
if (isSetFilterBytes()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterBytes, other.filterBytes);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
@ -1687,6 +1769,16 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
sb.append(this.targetReplicaId);
|
||||
first = false;
|
||||
}
|
||||
if (isSetFilterBytes()) {
|
||||
if (!first) sb.append(", ");
|
||||
sb.append("filterBytes:");
|
||||
if (this.filterBytes == null) {
|
||||
sb.append("null");
|
||||
} else {
|
||||
org.apache.thrift.TBaseHelper.toString(this.filterBytes, sb);
|
||||
}
|
||||
first = false;
|
||||
}
|
||||
sb.append(")");
|
||||
return sb.toString();
|
||||
}
|
||||
|
@ -1912,6 +2004,14 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
break;
|
||||
case 18: // FILTER_BYTES
|
||||
if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
|
||||
struct.filterBytes = iprot.readBinary();
|
||||
struct.setFilterBytesIsSet(true);
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
|
||||
}
|
||||
|
@ -2055,6 +2155,13 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
oprot.writeI32(struct.targetReplicaId);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
if (struct.filterBytes != null) {
|
||||
if (struct.isSetFilterBytes()) {
|
||||
oprot.writeFieldBegin(FILTER_BYTES_FIELD_DESC);
|
||||
oprot.writeBinary(struct.filterBytes);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
}
|
||||
oprot.writeFieldStop();
|
||||
oprot.writeStructEnd();
|
||||
}
|
||||
|
@ -2124,7 +2231,10 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
if (struct.isSetTargetReplicaId()) {
|
||||
optionals.set(16);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 17);
|
||||
if (struct.isSetFilterBytes()) {
|
||||
optionals.set(17);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 18);
|
||||
if (struct.isSetStartRow()) {
|
||||
oprot.writeBinary(struct.startRow);
|
||||
}
|
||||
|
@ -2196,12 +2306,15 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
if (struct.isSetTargetReplicaId()) {
|
||||
oprot.writeI32(struct.targetReplicaId);
|
||||
}
|
||||
if (struct.isSetFilterBytes()) {
|
||||
oprot.writeBinary(struct.filterBytes);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void read(org.apache.thrift.protocol.TProtocol prot, TScan struct) throws org.apache.thrift.TException {
|
||||
TTupleProtocol iprot = (TTupleProtocol) prot;
|
||||
BitSet incoming = iprot.readBitSet(17);
|
||||
BitSet incoming = iprot.readBitSet(18);
|
||||
if (incoming.get(0)) {
|
||||
struct.startRow = iprot.readBinary();
|
||||
struct.setStartRowIsSet(true);
|
||||
|
@ -2305,6 +2418,10 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
struct.targetReplicaId = iprot.readI32();
|
||||
struct.setTargetReplicaIdIsSet(true);
|
||||
}
|
||||
if (incoming.get(17)) {
|
||||
struct.filterBytes = iprot.readBinary();
|
||||
struct.setFilterBytesIsSet(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TServerName implements org.apache.thrift.TBase<TServerName, TServerName._Fields>, java.io.Serializable, Cloneable, Comparable<TServerName> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TServerName");
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.client.TableDescriptor
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TTableDescriptor implements org.apache.thrift.TBase<TTableDescriptor, TTableDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<TTableDescriptor> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableDescriptor");
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
|
|||
* Thrift wrapper around
|
||||
* org.apache.hadoop.hbase.TableName
|
||||
*/
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TTableName implements org.apache.thrift.TBase<TTableName, TTableName._Fields>, java.io.Serializable, Cloneable, Comparable<TTableName> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableName");
|
||||
|
||||
|
@ -54,7 +54,7 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
|
|||
/**
|
||||
* namespace name
|
||||
*/
|
||||
public ByteBuffer ns; // required
|
||||
public ByteBuffer ns; // optional
|
||||
/**
|
||||
* tablename
|
||||
*/
|
||||
|
@ -128,10 +128,11 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
|
|||
}
|
||||
|
||||
// isset id assignments
|
||||
private static final _Fields optionals[] = {_Fields.NS};
|
||||
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
|
||||
static {
|
||||
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
tmpMap.put(_Fields.NS, new org.apache.thrift.meta_data.FieldMetaData("ns", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
|
||||
tmpMap.put(_Fields.QUALIFIER, new org.apache.thrift.meta_data.FieldMetaData("qualifier", org.apache.thrift.TFieldRequirementType.REQUIRED,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
|
||||
|
@ -143,11 +144,9 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
|
|||
}
|
||||
|
||||
public TTableName(
|
||||
ByteBuffer ns,
|
||||
ByteBuffer qualifier)
|
||||
{
|
||||
this();
|
||||
this.ns = org.apache.thrift.TBaseHelper.copyBinary(ns);
|
||||
this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(qualifier);
|
||||
}
|
||||
|
||||
|
@ -400,13 +399,15 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
|
|||
StringBuilder sb = new StringBuilder("TTableName(");
|
||||
boolean first = true;
|
||||
|
||||
sb.append("ns:");
|
||||
if (this.ns == null) {
|
||||
sb.append("null");
|
||||
} else {
|
||||
org.apache.thrift.TBaseHelper.toString(this.ns, sb);
|
||||
if (isSetNs()) {
|
||||
sb.append("ns:");
|
||||
if (this.ns == null) {
|
||||
sb.append("null");
|
||||
} else {
|
||||
org.apache.thrift.TBaseHelper.toString(this.ns, sb);
|
||||
}
|
||||
first = false;
|
||||
}
|
||||
first = false;
|
||||
if (!first) sb.append(", ");
|
||||
sb.append("qualifier:");
|
||||
if (this.qualifier == null) {
|
||||
|
@ -421,9 +422,6 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
|
|||
|
||||
public void validate() throws org.apache.thrift.TException {
|
||||
// check for required fields
|
||||
if (ns == null) {
|
||||
throw new org.apache.thrift.protocol.TProtocolException("Required field 'ns' was not present! Struct: " + toString());
|
||||
}
|
||||
if (qualifier == null) {
|
||||
throw new org.apache.thrift.protocol.TProtocolException("Required field 'qualifier' was not present! Struct: " + toString());
|
||||
}
|
||||
|
@ -496,9 +494,11 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
|
|||
|
||||
oprot.writeStructBegin(STRUCT_DESC);
|
||||
if (struct.ns != null) {
|
||||
oprot.writeFieldBegin(NS_FIELD_DESC);
|
||||
oprot.writeBinary(struct.ns);
|
||||
oprot.writeFieldEnd();
|
||||
if (struct.isSetNs()) {
|
||||
oprot.writeFieldBegin(NS_FIELD_DESC);
|
||||
oprot.writeBinary(struct.ns);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
}
|
||||
if (struct.qualifier != null) {
|
||||
oprot.writeFieldBegin(QUALIFIER_FIELD_DESC);
|
||||
|
@ -522,17 +522,27 @@ public class TTableName implements org.apache.thrift.TBase<TTableName, TTableNam
|
|||
@Override
|
||||
public void write(org.apache.thrift.protocol.TProtocol prot, TTableName struct) throws org.apache.thrift.TException {
|
||||
TTupleProtocol oprot = (TTupleProtocol) prot;
|
||||
oprot.writeBinary(struct.ns);
|
||||
oprot.writeBinary(struct.qualifier);
|
||||
BitSet optionals = new BitSet();
|
||||
if (struct.isSetNs()) {
|
||||
optionals.set(0);
|
||||
}
|
||||
oprot.writeBitSet(optionals, 1);
|
||||
if (struct.isSetNs()) {
|
||||
oprot.writeBinary(struct.ns);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void read(org.apache.thrift.protocol.TProtocol prot, TTableName struct) throws org.apache.thrift.TException {
|
||||
TTupleProtocol iprot = (TTupleProtocol) prot;
|
||||
struct.ns = iprot.readBinary();
|
||||
struct.setNsIsSet(true);
|
||||
struct.qualifier = iprot.readBinary();
|
||||
struct.setQualifierIsSet(true);
|
||||
BitSet incoming = iprot.readBitSet(1);
|
||||
if (incoming.get(0)) {
|
||||
struct.ns = iprot.readBinary();
|
||||
struct.setNsIsSet(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2018-12-28")
|
||||
@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)", date = "2019-01-03")
|
||||
public class TTimeRange implements org.apache.thrift.TBase<TTimeRange, TTimeRange._Fields>, java.io.Serializable, Cloneable, Comparable<TTimeRange> {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTimeRange");
|
||||
|
||||
|
|
|
@ -146,6 +146,7 @@ struct TGet {
|
|||
12: optional i32 storeLimit
|
||||
13: optional i32 storeOffset
|
||||
14: optional bool existence_only
|
||||
15: optional binary filterBytes
|
||||
|
||||
}
|
||||
|
||||
|
@ -262,6 +263,8 @@ struct TScan {
|
|||
15: optional i32 limit
|
||||
16: optional TConsistency consistency
|
||||
17: optional i32 targetReplicaId
|
||||
18: optional binary filterBytes
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -402,7 +405,7 @@ enum TKeepDeletedCells {
|
|||
*/
|
||||
struct TTableName {
|
||||
/** namespace name */
|
||||
1: required binary ns
|
||||
1: optional binary ns
|
||||
/** tablename */
|
||||
2: required binary qualifier
|
||||
}
|
||||
|
@ -828,7 +831,7 @@ service THBaseService {
|
|||
**/
|
||||
list<TTableDescriptor> getTableDescriptorsByPattern(
|
||||
/** The regular expression to match against */
|
||||
1: required string regex
|
||||
1: optional string regex
|
||||
/** set to false if match only against userspace tables */
|
||||
2: required bool includeSysTables
|
||||
) throws (1: TIOError io)
|
||||
|
@ -848,7 +851,7 @@ service THBaseService {
|
|||
**/
|
||||
list<TTableName> getTableNamesByPattern(
|
||||
/** The regular expression to match against */
|
||||
1: required string regex
|
||||
1: optional string regex
|
||||
/** set to false if match only against userspace tables */
|
||||
2: required bool includeSysTables
|
||||
) throws (1: TIOError io)
|
||||
|
@ -871,7 +874,7 @@ service THBaseService {
|
|||
/** table descriptor for table */
|
||||
1: required TTableDescriptor desc
|
||||
/** rray of split keys for the initial regions of the table */
|
||||
2: required list<binary> splitKeys
|
||||
2: optional list<binary> splitKeys
|
||||
) throws (1: TIOError io)
|
||||
|
||||
/**
|
||||
|
@ -946,7 +949,7 @@ service THBaseService {
|
|||
/** the tablename to check */
|
||||
1: required TTableName tableName
|
||||
/** keys to check if the table has been created with all split keys */
|
||||
2: required list<binary> splitKeys
|
||||
2: optional list<binary> splitKeys
|
||||
) throws (1: TIOError io)
|
||||
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,841 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.thrift2;
|
||||
|
||||
import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_INFO_SERVER_PORT;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompareOperator;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.filter.ColumnValueFilter;
|
||||
import org.apache.hadoop.hbase.filter.FilterList;
|
||||
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RestTests;
|
||||
import org.apache.hadoop.hbase.thrift.Constants;
|
||||
import org.apache.hadoop.hbase.thrift2.client.ThriftConnection;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@Category({ RestTests.class, MediumTests.class})
|
||||
|
||||
public class TestThriftConnection {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(TestThriftConnection.class);
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestThriftConnection.class);
|
||||
|
||||
private static final byte[] FAMILYA = Bytes.toBytes("fa");
|
||||
private static final byte[] FAMILYB = Bytes.toBytes("fb");
|
||||
private static final byte[] FAMILYC = Bytes.toBytes("fc");
|
||||
private static final byte[] FAMILYD = Bytes.toBytes("fd");
|
||||
|
||||
private static final byte[] ROW_1 = Bytes.toBytes("testrow1");
|
||||
private static final byte[] ROW_2 = Bytes.toBytes("testrow2");
|
||||
private static final byte[] ROW_3 = Bytes.toBytes("testrow3");
|
||||
private static final byte[] ROW_4 = Bytes.toBytes("testrow4");
|
||||
|
||||
private static final byte[] QUALIFIER_1 = Bytes.toBytes("1");
|
||||
private static final byte[] QUALIFIER_2 = Bytes.toBytes("2");
|
||||
private static final byte[] VALUE_1 = Bytes.toBytes("testvalue1");
|
||||
private static final byte[] VALUE_2 = Bytes.toBytes("testvalue2");
|
||||
|
||||
private static final long ONE_HOUR = 60 * 60 * 1000;
|
||||
private static final long TS_2 = System.currentTimeMillis();
|
||||
private static final long TS_1 = TS_2 - ONE_HOUR;
|
||||
|
||||
|
||||
protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
protected static ThriftServer thriftServer;
|
||||
|
||||
protected static ThriftServer thriftHttpServer;
|
||||
|
||||
protected static int thriftPort;
|
||||
protected static int httpPort;
|
||||
|
||||
protected static Connection thriftConnection;
|
||||
protected static Connection thriftHttpConnection;
|
||||
|
||||
private static Admin thriftAdmin;
|
||||
|
||||
private static ThriftServer startThriftServer(int port, boolean useHttp) {
|
||||
Configuration thriftServerConf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
|
||||
thriftServerConf.setInt(Constants.PORT_CONF_KEY, port);
|
||||
if (useHttp) {
|
||||
thriftServerConf.setBoolean(Constants.USE_HTTP_CONF_KEY, true);
|
||||
}
|
||||
ThriftServer server = new ThriftServer(thriftServerConf);
|
||||
Thread thriftServerThread = new Thread(() -> {
|
||||
try{
|
||||
server.run();
|
||||
} catch (Exception t) {
|
||||
LOG.error("Thrift Server failed", t);
|
||||
}
|
||||
});
|
||||
thriftServerThread.setDaemon(true);
|
||||
thriftServerThread.start();
|
||||
if (useHttp) {
|
||||
TEST_UTIL.waitFor(10000, () -> server.getHttpServer() != null);
|
||||
} else {
|
||||
TEST_UTIL.waitFor(10000, () -> server.getTserver() != null);
|
||||
}
|
||||
return server;
|
||||
}
|
||||
|
||||
private static Connection createConnection(int port, boolean useHttp) throws IOException {
|
||||
Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
|
||||
conf.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL,
|
||||
ThriftConnection.class.getName());
|
||||
if (useHttp) {
|
||||
conf.set(Constants.HBASE_THRIFT_CLIENT_BUIDLER_CLASS,
|
||||
ThriftConnection.HTTPThriftClientBuilder.class.getName());
|
||||
}
|
||||
String host = HConstants.LOCALHOST;
|
||||
if (useHttp) {
|
||||
host = "http://" + host;
|
||||
}
|
||||
conf.set(Constants.HBASE_THRIFT_SERVER_NAME, host);
|
||||
conf.setInt(Constants.HBASE_THRIFT_SERVER_PORT, port);
|
||||
return ConnectionFactory.createConnection(conf);
|
||||
}
|
||||
|
||||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
// Do not start info server
|
||||
TEST_UTIL.getConfiguration().setInt(THRIFT_INFO_SERVER_PORT , -1);
|
||||
TEST_UTIL.startMiniCluster();
|
||||
thriftPort = HBaseTestingUtility.randomFreePort();
|
||||
httpPort = HBaseTestingUtility.randomFreePort();
|
||||
// Start a thrift server
|
||||
thriftServer = startThriftServer(thriftPort, false);
|
||||
// Start a HTTP thrift server
|
||||
thriftHttpServer = startThriftServer(httpPort, true);
|
||||
thriftConnection = createConnection(thriftPort, false);
|
||||
thriftHttpConnection = createConnection(httpPort, true);
|
||||
thriftAdmin = thriftConnection.getAdmin();
|
||||
LOG.info("TS_1=" + TS_1);
|
||||
LOG.info("TS_2=" + TS_1);
|
||||
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void shutdown() throws Exception {
|
||||
if (thriftAdmin != null) {
|
||||
thriftAdmin.close();
|
||||
}
|
||||
if (thriftHttpConnection != null) {
|
||||
thriftHttpConnection.close();
|
||||
}
|
||||
if (thriftConnection != null) {
|
||||
thriftConnection.close();
|
||||
}
|
||||
if (thriftHttpServer != null) {
|
||||
thriftHttpServer.stop();
|
||||
}
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testThrfitAdmin() throws Exception {
|
||||
testThriftAdmin(thriftConnection, "testThrfitAdminNamesapce", "testThrfitAdminTable");
|
||||
testThriftAdmin(thriftHttpConnection, "testThrfitHttpAdminNamesapce",
|
||||
"testThrfitHttpAdminTable");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGet() throws Exception {
|
||||
testGet(thriftConnection, "testGetTable");
|
||||
testGet(thriftHttpConnection, "testGetHttpTable");
|
||||
|
||||
}
|
||||
|
||||
public void testGet(Connection connection, String tableName) throws IOException {
|
||||
createTable(thriftAdmin, tableName);
|
||||
try (Table table = connection.getTable(TableName.valueOf(tableName))){
|
||||
Get get = new Get(ROW_1);
|
||||
Result result = table.get(get);
|
||||
byte[] value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
byte[] value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_1, value1));
|
||||
assertNull(value2);
|
||||
|
||||
get = new Get(ROW_1);
|
||||
get.addFamily(FAMILYC);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNull(value1);
|
||||
assertNull(value2);
|
||||
|
||||
get = new Get(ROW_1);
|
||||
get.addColumn(FAMILYA, QUALIFIER_1);
|
||||
get.addColumn(FAMILYB, QUALIFIER_2);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_1, value1));
|
||||
assertNull(value2);
|
||||
|
||||
get = new Get(ROW_2);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_2, value1)); // @TS_2
|
||||
assertNotNull(value2);
|
||||
assertTrue(Bytes.equals(VALUE_2, value2));
|
||||
|
||||
get = new Get(ROW_2);
|
||||
get.addFamily(FAMILYA);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_2, value1)); // @TS_2
|
||||
assertNull(value2);
|
||||
|
||||
get = new Get(ROW_2);
|
||||
get.addColumn(FAMILYA, QUALIFIER_1);
|
||||
get.addColumn(FAMILYB, QUALIFIER_2);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_2, value1)); // @TS_2
|
||||
assertNotNull(value2);
|
||||
assertTrue(Bytes.equals(VALUE_2, value2));
|
||||
|
||||
// test timestamp
|
||||
|
||||
get = new Get(ROW_2);
|
||||
get.addFamily(FAMILYA);
|
||||
get.addFamily(FAMILYB);
|
||||
get.setTimestamp(TS_1);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_1, value1)); // @TS_1
|
||||
assertNull(value2);
|
||||
|
||||
// test timerange
|
||||
|
||||
get = new Get(ROW_2);
|
||||
get.addFamily(FAMILYA);
|
||||
get.addFamily(FAMILYB);
|
||||
get.setTimeRange(0, TS_1 + 1);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_1, value1)); // @TS_1
|
||||
assertNull(value2);
|
||||
|
||||
// test maxVersions
|
||||
|
||||
get = new Get(ROW_2);
|
||||
get.addFamily(FAMILYA);
|
||||
get.setMaxVersions(2);
|
||||
result = table.get(get);
|
||||
int count = 0;
|
||||
for (Cell kv: result.listCells()) {
|
||||
if (CellUtil.matchingFamily(kv, FAMILYA) && TS_1 == kv.getTimestamp()) {
|
||||
assertTrue(CellUtil.matchingValue(kv, VALUE_1)); // @TS_1
|
||||
count++;
|
||||
}
|
||||
if (CellUtil.matchingFamily(kv, FAMILYA) && TS_2 == kv.getTimestamp()) {
|
||||
assertTrue(CellUtil.matchingValue(kv, VALUE_2)); // @TS_2
|
||||
count++;
|
||||
}
|
||||
}
|
||||
assertEquals(2, count);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiGet() throws Exception {
|
||||
testMultiGet(thriftConnection, "testMultiGetTable");
|
||||
testMultiGet(thriftHttpConnection, "testMultiGetHttpTable");
|
||||
}
|
||||
|
||||
public void testMultiGet(Connection connection, String tableName) throws Exception {
|
||||
createTable(thriftAdmin, tableName);
|
||||
try (Table table = connection.getTable(TableName.valueOf(tableName))){
|
||||
ArrayList<Get> gets = new ArrayList<>(2);
|
||||
gets.add(new Get(ROW_1));
|
||||
gets.add(new Get(ROW_2));
|
||||
Result[] results = table.get(gets);
|
||||
assertNotNull(results);
|
||||
assertEquals(2, results.length);
|
||||
assertEquals(1, results[0].size());
|
||||
assertEquals(2, results[1].size());
|
||||
|
||||
//Test Versions
|
||||
gets = new ArrayList<>(2);
|
||||
Get g = new Get(ROW_1);
|
||||
g.setMaxVersions(3);
|
||||
gets.add(g);
|
||||
Get get2 = new Get(ROW_2);
|
||||
get2.setMaxVersions(3);
|
||||
gets.add(get2);
|
||||
results = table.get(gets);
|
||||
assertNotNull(results);
|
||||
assertEquals(2, results.length);
|
||||
assertEquals(1, results[0].size());
|
||||
assertEquals(3, results[1].size());
|
||||
|
||||
gets = new ArrayList<>(1);
|
||||
gets.add(new Get(Bytes.toBytes("RESALLYREALLYNOTTHERE")));
|
||||
results = table.get(gets);
|
||||
assertNotNull(results);
|
||||
assertTrue(results[0].isEmpty());
|
||||
|
||||
gets = new ArrayList<>(3);
|
||||
gets.add(new Get(Bytes.toBytes("RESALLYREALLYNOTTHERE")));
|
||||
gets.add(new Get(ROW_1));
|
||||
gets.add(new Get(ROW_2));
|
||||
results = table.get(gets);
|
||||
assertNotNull(results);
|
||||
assertEquals(3, results.length);
|
||||
assertTrue(results[0].isEmpty());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPut() throws Exception {
|
||||
testPut(thriftConnection, "testPutTable");
|
||||
testPut(thriftHttpConnection, "testPutHttpTable");
|
||||
}
|
||||
|
||||
public void testPut(Connection connection, String tableName) throws IOException {
|
||||
createTable(thriftAdmin, tableName);
|
||||
try (Table table = connection.getTable(TableName.valueOf(tableName))){
|
||||
Put put = new Put(ROW_3);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
table.put(put);
|
||||
|
||||
Get get = new Get(ROW_3);
|
||||
get.addFamily(FAMILYA);
|
||||
Result result = table.get(get);
|
||||
byte[] value = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
assertNotNull(value);
|
||||
assertTrue(Bytes.equals(VALUE_1, value));
|
||||
|
||||
// multiput
|
||||
|
||||
List<Put> puts = new ArrayList<>(3);
|
||||
put = new Put(ROW_3);
|
||||
put.addColumn(FAMILYB, QUALIFIER_2, VALUE_2);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_4);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_4);
|
||||
put.addColumn(FAMILYB, QUALIFIER_2, VALUE_2);
|
||||
puts.add(put);
|
||||
table.put(puts);
|
||||
|
||||
get = new Get(ROW_3);
|
||||
get.addFamily(FAMILYB);
|
||||
result = table.get(get);
|
||||
value = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value);
|
||||
assertTrue(Bytes.equals(VALUE_2, value));
|
||||
get = new Get(ROW_4);
|
||||
result = table.get(get);
|
||||
value = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
assertNotNull(value);
|
||||
assertTrue(Bytes.equals(VALUE_1, value));
|
||||
value = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value);
|
||||
assertTrue(Bytes.equals(VALUE_2, value));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDelete() throws Exception {
|
||||
testDelete(thriftConnection, "testDeleteTable");
|
||||
testDelete(thriftHttpConnection, "testDeleteHttpTable");
|
||||
}
|
||||
|
||||
public void testDelete(Connection connection, String tableName) throws IOException {
|
||||
createTable(thriftAdmin, tableName);
|
||||
try (Table table = connection.getTable(TableName.valueOf(tableName))){
|
||||
Put put = new Put(ROW_3);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
put.addColumn(FAMILYB, QUALIFIER_2, VALUE_2);
|
||||
put.addColumn(FAMILYC, QUALIFIER_1, VALUE_1);
|
||||
put.addColumn(FAMILYC, QUALIFIER_2, VALUE_2);
|
||||
table.put(put);
|
||||
|
||||
Get get = new Get(ROW_3);
|
||||
get.addFamily(FAMILYA);
|
||||
get.addFamily(FAMILYB);
|
||||
get.addFamily(FAMILYC);
|
||||
Result result = table.get(get);
|
||||
byte[] value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
byte[] value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
byte[] value3 = result.getValue(FAMILYC, QUALIFIER_1);
|
||||
byte[] value4 = result.getValue(FAMILYC, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_1, value1));
|
||||
assertNotNull(value2);
|
||||
assertTrue(Bytes.equals(VALUE_2, value2));
|
||||
assertNotNull(value3);
|
||||
assertTrue(Bytes.equals(VALUE_1, value3));
|
||||
assertNotNull(value4);
|
||||
assertTrue(Bytes.equals(VALUE_2, value4));
|
||||
|
||||
Delete delete = new Delete(ROW_3);
|
||||
delete.addColumn(FAMILYB, QUALIFIER_2);
|
||||
table.delete(delete);
|
||||
|
||||
get = new Get(ROW_3);
|
||||
get.addFamily(FAMILYA);
|
||||
get.addFamily(FAMILYB);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_1, value1));
|
||||
assertNull(value2);
|
||||
|
||||
delete = new Delete(ROW_3);
|
||||
delete.setTimestamp(1L);
|
||||
table.delete(delete);
|
||||
|
||||
get = new Get(ROW_3);
|
||||
get.addFamily(FAMILYA);
|
||||
get.addFamily(FAMILYB);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_1, value1));
|
||||
assertNull(value2);
|
||||
|
||||
// Delete column family from row
|
||||
delete = new Delete(ROW_3);
|
||||
delete.addFamily(FAMILYC);
|
||||
table.delete(delete);
|
||||
|
||||
get = new Get(ROW_3);
|
||||
get.addFamily(FAMILYC);
|
||||
result = table.get(get);
|
||||
value3 = result.getValue(FAMILYC, QUALIFIER_1);
|
||||
value4 = result.getValue(FAMILYC, QUALIFIER_2);
|
||||
assertNull(value3);
|
||||
assertNull(value4);
|
||||
|
||||
delete = new Delete(ROW_3);
|
||||
table.delete(delete);
|
||||
|
||||
get = new Get(ROW_3);
|
||||
get.addFamily(FAMILYA);
|
||||
get.addFamily(FAMILYB);
|
||||
result = table.get(get);
|
||||
value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNull(value1);
|
||||
assertNull(value2);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScanner() throws Exception {
|
||||
testScanner(thriftConnection, "testScannerTable");
|
||||
testScanner(thriftHttpConnection, "testScannerHttpTable");
|
||||
}
|
||||
|
||||
public void testScanner(Connection connection, String tableName) throws IOException {
|
||||
createTable(thriftAdmin, tableName);
|
||||
try (Table table = connection.getTable(TableName.valueOf(tableName))){
|
||||
List<Put> puts = new ArrayList<>(4);
|
||||
Put put = new Put(ROW_1);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_2);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_3);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_4);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
table.put(puts);
|
||||
|
||||
ResultScanner scanner = table.getScanner(new Scan());
|
||||
|
||||
Result[] results = scanner.next(1);
|
||||
assertNotNull(results);
|
||||
assertEquals(1, results.length);
|
||||
assertTrue(Bytes.equals(ROW_1, results[0].getRow()));
|
||||
|
||||
Result result = scanner.next();
|
||||
assertNotNull(result);
|
||||
assertTrue(Bytes.equals(ROW_2, result.getRow()));
|
||||
|
||||
results = scanner.next(2);
|
||||
assertNotNull(results);
|
||||
assertEquals(2, results.length);
|
||||
assertTrue(Bytes.equals(ROW_3, results[0].getRow()));
|
||||
assertTrue(Bytes.equals(ROW_4, results[1].getRow()));
|
||||
|
||||
results = scanner.next(1);
|
||||
assertTrue(results == null || results.length == 0);
|
||||
scanner.close();
|
||||
|
||||
scanner = table.getScanner(FAMILYA);
|
||||
results = scanner.next(4);
|
||||
assertNotNull(results);
|
||||
assertEquals(4, results.length);
|
||||
assertTrue(Bytes.equals(ROW_1, results[0].getRow()));
|
||||
assertTrue(Bytes.equals(ROW_2, results[1].getRow()));
|
||||
assertTrue(Bytes.equals(ROW_3, results[2].getRow()));
|
||||
assertTrue(Bytes.equals(ROW_4, results[3].getRow()));
|
||||
|
||||
scanner.close();
|
||||
|
||||
scanner = table.getScanner(FAMILYA,QUALIFIER_1);
|
||||
results = scanner.next(4);
|
||||
assertNotNull(results);
|
||||
assertEquals(4, results.length);
|
||||
assertTrue(Bytes.equals(ROW_1, results[0].getRow()));
|
||||
assertTrue(Bytes.equals(ROW_2, results[1].getRow()));
|
||||
assertTrue(Bytes.equals(ROW_3, results[2].getRow()));
|
||||
assertTrue(Bytes.equals(ROW_4, results[3].getRow()));
|
||||
scanner.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckAndDelete() throws Exception {
|
||||
testCheckAndDelete(thriftConnection, "testCheckAndDeleteTable");
|
||||
testCheckAndDelete(thriftHttpConnection, "testCheckAndDeleteHttpTable");
|
||||
}
|
||||
|
||||
|
||||
public void testCheckAndDelete(Connection connection, String tableName) throws IOException {
|
||||
createTable(thriftAdmin, tableName);
|
||||
try (Table table = connection.getTable(TableName.valueOf(tableName))){
|
||||
Get get = new Get(ROW_1);
|
||||
Result result = table.get(get);
|
||||
byte[] value1 = result.getValue(FAMILYA, QUALIFIER_1);
|
||||
byte[] value2 = result.getValue(FAMILYB, QUALIFIER_2);
|
||||
assertNotNull(value1);
|
||||
assertTrue(Bytes.equals(VALUE_1, value1));
|
||||
assertNull(value2);
|
||||
assertTrue(table.exists(get));
|
||||
assertEquals(1, table.existsAll(Collections.singletonList(get)).length);
|
||||
Delete delete = new Delete(ROW_1);
|
||||
|
||||
table.checkAndMutate(ROW_1, FAMILYA).qualifier(QUALIFIER_1)
|
||||
.ifEquals(VALUE_1).thenDelete(delete);
|
||||
assertFalse(table.exists(get));
|
||||
|
||||
Put put = new Put(ROW_1);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
table.put(put);
|
||||
|
||||
assertTrue(table.checkAndMutate(ROW_1, FAMILYA).qualifier(QUALIFIER_1)
|
||||
.ifEquals(VALUE_1).thenPut(put));
|
||||
assertFalse(table.checkAndMutate(ROW_1, FAMILYA).qualifier(QUALIFIER_1)
|
||||
.ifEquals(VALUE_2).thenPut(put));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIteratorScaner() throws Exception {
|
||||
testIteratorScanner(thriftConnection, "testIteratorScanerTable");
|
||||
testIteratorScanner(thriftHttpConnection, "testIteratorScanerHttpTable");
|
||||
}
|
||||
|
||||
public void testIteratorScanner(Connection connection, String tableName) throws IOException {
|
||||
createTable(thriftAdmin, tableName);
|
||||
try (Table table = connection.getTable(TableName.valueOf(tableName))){
|
||||
List<Put> puts = new ArrayList<>(4);
|
||||
Put put = new Put(ROW_1);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_2);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_3);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_4);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
table.put(puts);
|
||||
Scan scan = new Scan();
|
||||
scan.setCaching(1);
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
Iterator<Result> iterator = scanner.iterator();
|
||||
assertTrue(iterator.hasNext());
|
||||
int counter = 0;
|
||||
while (iterator.hasNext()) {
|
||||
iterator.next();
|
||||
counter++;
|
||||
}
|
||||
assertEquals(4, counter);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReverseScan() throws Exception {
|
||||
testReverseScan(thriftConnection, "testReverseScanTable");
|
||||
testReverseScan(thriftHttpConnection, "testReverseScanHttpTable");
|
||||
}
|
||||
|
||||
public void testReverseScan(Connection connection, String tableName) throws IOException {
|
||||
createTable(thriftAdmin, tableName);
|
||||
try (Table table = connection.getTable(TableName.valueOf(tableName))){
|
||||
List<Put> puts = new ArrayList<>(4);
|
||||
Put put = new Put(ROW_1);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_2);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_3);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
put = new Put(ROW_4);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, VALUE_1);
|
||||
puts.add(put);
|
||||
table.put(puts);
|
||||
Scan scan = new Scan();
|
||||
scan.setReversed(true);
|
||||
scan.setCaching(1);
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
Iterator<Result> iterator = scanner.iterator();
|
||||
assertTrue(iterator.hasNext());
|
||||
int counter = 0;
|
||||
Result lastResult = null;
|
||||
while (iterator.hasNext()) {
|
||||
Result current = iterator.next();
|
||||
if (lastResult != null) {
|
||||
assertTrue(Bytes.compareTo(lastResult.getRow(), current.getRow()) > 0);
|
||||
}
|
||||
lastResult = current;
|
||||
counter++;
|
||||
}
|
||||
assertEquals(4, counter);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testScanWithFilters() throws Exception {
|
||||
testIteratorScanner(thriftConnection, "testScanWithFiltersTable");
|
||||
testIteratorScanner(thriftHttpConnection, "testScanWithFiltersHttpTable");
|
||||
}
|
||||
|
||||
private void testScanWithFilters(Connection connection, String tableName) throws IOException {
|
||||
createTable(thriftAdmin, tableName);
|
||||
try (Table table = connection.getTable(TableName.valueOf(tableName))){
|
||||
FilterList filterList = new FilterList();
|
||||
PrefixFilter prefixFilter = new PrefixFilter(Bytes.toBytes("testrow"));
|
||||
ColumnValueFilter columnValueFilter = new ColumnValueFilter(FAMILYA, QUALIFIER_1,
|
||||
CompareOperator.EQUAL, VALUE_1);
|
||||
filterList.addFilter(prefixFilter);
|
||||
filterList.addFilter(columnValueFilter);
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(2);
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
Iterator<Result> iterator = scanner.iterator();
|
||||
assertTrue(iterator.hasNext());
|
||||
int counter = 0;
|
||||
while (iterator.hasNext()) {
|
||||
Result result = iterator.next();
|
||||
counter += result.size();
|
||||
}
|
||||
assertEquals(2, counter);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private TableDescriptor createTable(Admin admin, String tableName) throws IOException {
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(tableName));
|
||||
ColumnFamilyDescriptorBuilder familyABuilder = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(FAMILYA);
|
||||
familyABuilder.setMaxVersions(3);
|
||||
ColumnFamilyDescriptorBuilder familyBBuilder = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(FAMILYB);
|
||||
familyBBuilder.setMaxVersions(3);
|
||||
ColumnFamilyDescriptorBuilder familyCBuilder = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(FAMILYC);
|
||||
familyCBuilder.setMaxVersions(3);
|
||||
builder.setColumnFamily(familyABuilder.build());
|
||||
builder.setColumnFamily(familyBBuilder.build());
|
||||
builder.setColumnFamily(familyCBuilder.build());
|
||||
TableDescriptor tableDescriptor = builder.build();
|
||||
admin.createTable(tableDescriptor);
|
||||
try (Table table = TEST_UTIL.getConnection().getTable(TableName.valueOf(tableName))) {
|
||||
Put put = new Put(ROW_1);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, TS_2, VALUE_1);
|
||||
table.put(put);
|
||||
put = new Put(ROW_2);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, TS_1, VALUE_1);
|
||||
put.addColumn(FAMILYA, QUALIFIER_1, TS_2, VALUE_2);
|
||||
put.addColumn(FAMILYB, QUALIFIER_2, TS_2, VALUE_2);
|
||||
table.put(put);
|
||||
|
||||
}
|
||||
return tableDescriptor;
|
||||
|
||||
}
|
||||
|
||||
private void testThriftAdmin(Connection connection, String namespace, String table)
|
||||
throws Exception {
|
||||
try (Admin admin = connection.getAdmin()){
|
||||
//create name space
|
||||
NamespaceDescriptor namespaceDescriptor = NamespaceDescriptor.create(namespace).build();
|
||||
namespaceDescriptor.setConfiguration("key1", "value1");
|
||||
namespaceDescriptor.setConfiguration("key2", "value2");
|
||||
admin.createNamespace(namespaceDescriptor);
|
||||
//list namespace
|
||||
NamespaceDescriptor[] namespaceDescriptors = admin.listNamespaceDescriptors();
|
||||
boolean found = false;
|
||||
for (NamespaceDescriptor nd : namespaceDescriptors) {
|
||||
if (nd.getName().equals(namespace)) {
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(found);
|
||||
//modify namesapce
|
||||
namespaceDescriptor.setConfiguration("kye3", "value3");
|
||||
admin.modifyNamespace(namespaceDescriptor);
|
||||
//get namespace
|
||||
NamespaceDescriptor namespaceDescriptorReturned = admin.getNamespaceDescriptor(namespace);
|
||||
assertTrue(namespaceDescriptorReturned.getConfiguration().size() == 3);
|
||||
//create table
|
||||
TableDescriptor tableDescriptor = createTable(admin, table);
|
||||
//modify table
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableDescriptor);
|
||||
builder.setDurability(Durability.ASYNC_WAL);
|
||||
admin.modifyTable(builder.build());
|
||||
//modify column family
|
||||
ColumnFamilyDescriptor familyA = tableDescriptor.getColumnFamily(FAMILYA);
|
||||
ColumnFamilyDescriptorBuilder familyABuilder = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(familyA);
|
||||
familyABuilder.setInMemory(true);
|
||||
admin.modifyColumnFamily(tableDescriptor.getTableName(), familyABuilder.build());
|
||||
//add column family
|
||||
ColumnFamilyDescriptorBuilder familyDBuilder = ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(FAMILYD);
|
||||
familyDBuilder.setDataBlockEncoding(DataBlockEncoding.PREFIX);
|
||||
admin.addColumnFamily(tableDescriptor.getTableName(), familyDBuilder.build());
|
||||
//get table descriptor
|
||||
TableDescriptor tableDescriptorReturned = admin.getDescriptor(tableDescriptor.getTableName());
|
||||
assertTrue(tableDescriptorReturned.getColumnFamilies().length == 4);
|
||||
assertTrue(tableDescriptorReturned.getDurability() == Durability.ASYNC_WAL);
|
||||
ColumnFamilyDescriptor columnFamilyADescriptor1Returned = tableDescriptorReturned
|
||||
.getColumnFamily(FAMILYA);
|
||||
assertTrue(columnFamilyADescriptor1Returned.isInMemory() == true);
|
||||
//delete column family
|
||||
admin.deleteColumnFamily(tableDescriptor.getTableName(), FAMILYA);
|
||||
tableDescriptorReturned = admin.getDescriptor(tableDescriptor.getTableName());
|
||||
assertTrue(tableDescriptorReturned.getColumnFamilies().length == 3);
|
||||
//disable table
|
||||
admin.disableTable(tableDescriptor.getTableName());
|
||||
assertTrue(admin.isTableDisabled(tableDescriptor.getTableName()));
|
||||
//enable table
|
||||
admin.enableTable(tableDescriptor.getTableName());
|
||||
assertTrue(admin.isTableEnabled(tableDescriptor.getTableName()));
|
||||
assertTrue(admin.isTableAvailable(tableDescriptor.getTableName()));
|
||||
//truncate table
|
||||
admin.disableTable(tableDescriptor.getTableName());
|
||||
admin.truncateTable(tableDescriptor.getTableName(), true);
|
||||
assertTrue(admin.isTableAvailable(tableDescriptor.getTableName()));
|
||||
//delete table
|
||||
admin.disableTable(tableDescriptor.getTableName());
|
||||
admin.deleteTable(tableDescriptor.getTableName());
|
||||
assertFalse(admin.tableExists(tableDescriptor.getTableName()));
|
||||
//delete namespace
|
||||
admin.deleteNamespace(namespace);
|
||||
namespaceDescriptors = admin.listNamespaceDescriptors();
|
||||
// should have 2 namespace, default and hbase
|
||||
found = false;
|
||||
for (NamespaceDescriptor nd : namespaceDescriptors) {
|
||||
if (nd.getName().equals(namespace)) {
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertTrue(found == false);
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue