@InterfaceAudience.Private @InterfaceStability.Stable public class HTable extends Object implements Table
Table
. Used to communicate with a single HBase table.
Lightweight. Get as needed and just close when done.
Instances of this class SHOULD NOT be constructed directly.
Obtain an instance via Connection
. See ConnectionFactory
class comment for an example of how.
This class is thread safe since 2.0.0 if not invoking any of the setter methods.
All setters are moved into TableBuilder
and reserved here only for keeping
backward compatibility, and TODO will be removed soon.
HTable is no longer a client API. Use Table
instead. It is marked
InterfaceAudience.Private indicating that this is an HBase-internal class as defined in
Hadoop
Interface Classification
There are no guarantees for backwards source / binary compatibility and methods or class can
change or go away without deprecation.
Table
,
Admin
,
Connection
,
ConnectionFactory
Modifier and Type | Class and Description |
---|---|
private class |
HTable.CheckAndMutateBuilderImpl |
private class |
HTable.CheckAndMutateWithFilterBuilderImpl |
Table.CheckAndMutateBuilder, Table.CheckAndMutateWithFilterBuilder
Modifier and Type | Field and Description |
---|---|
private boolean |
cleanupPoolOnClose |
private boolean |
closed |
private org.apache.hadoop.conf.Configuration |
configuration |
private ConnectionConfiguration |
connConfiguration |
private ClusterConnection |
connection |
private static Consistency |
DEFAULT_CONSISTENCY |
private HRegionLocator |
locator |
private static org.slf4j.Logger |
LOG |
(package private) AsyncProcess |
multiAp
The Async process for batch
|
private int |
operationTimeoutMs |
private ExecutorService |
pool |
private int |
readRpcTimeoutMs |
private RpcRetryingCallerFactory |
rpcCallerFactory |
private RpcControllerFactory |
rpcControllerFactory |
private int |
rpcTimeoutMs |
private int |
scannerCaching |
private long |
scannerMaxResultSize |
private TableName |
tableName |
private int |
writeRpcTimeoutMs |
Modifier | Constructor and Description |
---|---|
protected |
HTable(ConnectionImplementation connection,
TableBuilderBase builder,
RpcRetryingCallerFactory rpcCallerFactory,
RpcControllerFactory rpcControllerFactory,
ExecutorService pool)
Creates an object to access a HBase table.
|
Modifier and Type | Method and Description |
---|---|
Result |
append(Append append)
Appends values to one or more columns within a single row.
|
void |
batch(List<? extends Row> actions,
Object[] results)
Method that does a batch call on Deletes, Gets, Puts, Increments, Appends, RowMutations.
|
void |
batch(List<? extends Row> actions,
Object[] results,
int rpcTimeout) |
<R> void |
batchCallback(List<? extends Row> actions,
Object[] results,
Batch.Callback<R> callback)
Same as
Table.batch(List, Object[]) , but with a callback. |
<R extends com.google.protobuf.Message> |
batchCoprocessorService(com.google.protobuf.Descriptors.MethodDescriptor methodDescriptor,
com.google.protobuf.Message request,
byte[] startKey,
byte[] endKey,
R responsePrototype)
Creates an instance of the given
Service subclass for each table
region spanning the range from the startKey row to endKey row (inclusive), all
the invocations to the same region server will be batched into one call. |
<R extends com.google.protobuf.Message> |
batchCoprocessorService(com.google.protobuf.Descriptors.MethodDescriptor methodDescriptor,
com.google.protobuf.Message request,
byte[] startKey,
byte[] endKey,
R responsePrototype,
Batch.Callback<R> callback)
Creates an instance of the given
Service subclass for each table
region spanning the range from the startKey row to endKey row (inclusive), all
the invocations to the same region server will be batched into one call. |
boolean |
checkAndDelete(byte[] row,
byte[] family,
byte[] qualifier,
byte[] value,
Delete delete)
Deprecated.
|
boolean |
checkAndDelete(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
byte[] value,
Delete delete)
Deprecated.
|
boolean |
checkAndDelete(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
byte[] value,
Delete delete)
Deprecated.
|
Table.CheckAndMutateBuilder |
checkAndMutate(byte[] row,
byte[] family)
Atomically checks if a row/family/qualifier value matches the expected value.
|
boolean |
checkAndMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
byte[] value,
RowMutations rm)
Deprecated.
|
boolean |
checkAndMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
byte[] value,
RowMutations rm)
Deprecated.
|
Table.CheckAndMutateWithFilterBuilder |
checkAndMutate(byte[] row,
Filter filter)
Atomically checks if a row matches the specified filter.
|
boolean |
checkAndPut(byte[] row,
byte[] family,
byte[] qualifier,
byte[] value,
Put put)
Deprecated.
|
boolean |
checkAndPut(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
byte[] value,
Put put)
Deprecated.
|
boolean |
checkAndPut(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
byte[] value,
Put put)
Deprecated.
|
void |
clearRegionCache()
Explicitly clears the region cache to fetch the latest value from META.
|
void |
close()
Releases any resources held or pending changes in internal buffers.
|
CoprocessorRpcChannel |
coprocessorService(byte[] row)
Creates and returns a
RpcChannel instance connected to the
table region containing the specified row. |
<T extends com.google.protobuf.Service,R> |
coprocessorService(Class<T> service,
byte[] startKey,
byte[] endKey,
Batch.Call<T,R> callable)
Creates an instance of the given
Service subclass for each table
region spanning the range from the startKey row to endKey row (inclusive), and
invokes the passed Batch.Call.call(T) method
with each Service instance. |
<T extends com.google.protobuf.Service,R> |
coprocessorService(Class<T> service,
byte[] startKey,
byte[] endKey,
Batch.Call<T,R> callable,
Batch.Callback<R> callback)
Creates an instance of the given
Service subclass for each table
region spanning the range from the startKey row to endKey row (inclusive), and
invokes the passed Batch.Call.call(T) method
with each Service instance. |
void |
delete(Delete delete)
Deletes the specified cells/row.
|
void |
delete(List<Delete> deletes)
Batch Deletes the specified cells/rows from the table.
|
static <R> void |
doBatchWithCallback(List<? extends Row> actions,
Object[] results,
Batch.Callback<R> callback,
ClusterConnection connection,
ExecutorService pool,
TableName tableName) |
private boolean |
doCheckAndDelete(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
byte[] value,
Filter filter,
TimeRange timeRange,
Delete delete) |
private boolean |
doCheckAndMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
byte[] value,
Filter filter,
TimeRange timeRange,
RowMutations rm) |
private boolean |
doCheckAndPut(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
byte[] value,
Filter filter,
TimeRange timeRange,
Put put) |
boolean |
exists(Get get)
Test for the existence of columns in the table, as specified by the Get.
|
boolean[] |
exists(List<Get> gets)
Test for the existence of columns in the table, as specified by the Gets.
|
Result |
get(Get get)
Extracts certain cells from a given row.
|
private Result |
get(Get get,
boolean checkExistenceOnly) |
Result[] |
get(List<Get> gets)
Extracts specified cells from the given rows, as a batch.
|
org.apache.hadoop.conf.Configuration |
getConfiguration()
Returns the
Configuration object used by this instance. |
protected Connection |
getConnection()
INTERNAL Used by unit tests and tools to do low-level
manipulations.
|
static ThreadPoolExecutor |
getDefaultExecutor(org.apache.hadoop.conf.Configuration conf) |
TableDescriptor |
getDescriptor()
Gets the
table descriptor for this table. |
private Pair<List<byte[]>,List<HRegionLocation>> |
getKeysAndRegionsInRange(byte[] startKey,
byte[] endKey,
boolean includeEndKey)
Get the corresponding start keys and regions for an arbitrary range of
keys.
|
private Pair<List<byte[]>,List<HRegionLocation>> |
getKeysAndRegionsInRange(byte[] startKey,
byte[] endKey,
boolean includeEndKey,
boolean reload)
Get the corresponding start keys and regions for an arbitrary range of
keys.
|
static int |
getMaxKeyValueSize(org.apache.hadoop.conf.Configuration conf) |
TableName |
getName()
Gets the fully qualified table name instance of this table.
|
int |
getOperationTimeout()
Deprecated.
|
long |
getOperationTimeout(TimeUnit unit)
Get timeout of each operation in Table instance.
|
(package private) ExecutorService |
getPool()
The pool is used for mutli requests for this HTable
|
int |
getReadRpcTimeout()
Deprecated.
|
long |
getReadRpcTimeout(TimeUnit unit)
Get timeout of each rpc read request in this Table instance.
|
RegionLocator |
getRegionLocator()
Gets the
RegionLocator for this table. |
int |
getRpcTimeout()
Deprecated.
|
long |
getRpcTimeout(TimeUnit unit)
Get timeout of each rpc request in this Table instance.
|
ResultScanner |
getScanner(byte[] family)
The underlying
HTable must not be closed. |
ResultScanner |
getScanner(byte[] family,
byte[] qualifier)
The underlying
HTable must not be closed. |
ResultScanner |
getScanner(Scan scan)
The underlying
HTable must not be closed. |
private List<byte[]> |
getStartKeysInRange(byte[] start,
byte[] end) |
HTableDescriptor |
getTableDescriptor()
Deprecated.
|
int |
getWriteRpcTimeout()
Deprecated.
|
long |
getWriteRpcTimeout(TimeUnit unit)
Get timeout of each rpc write request in this Table instance.
|
Result |
increment(Increment increment)
Increments one or more columns within a single row.
|
long |
incrementColumnValue(byte[] row,
byte[] family,
byte[] qualifier,
long amount)
|
long |
incrementColumnValue(byte[] row,
byte[] family,
byte[] qualifier,
long amount,
Durability durability)
Atomically increments a column value.
|
void |
mutateRow(RowMutations rm)
Performs multiple mutations atomically on a single row.
|
<R> void |
processBatchCallback(List<? extends Row> list,
Object[] results,
Batch.Callback<R> callback)
Process a mixed batch of Get, Put and Delete actions.
|
void |
put(List<Put> puts)
Batch puts the specified data into the table.
|
void |
put(Put put)
Puts some data in the table.
|
void |
setOperationTimeout(int operationTimeout)
Deprecated.
|
void |
setReadRpcTimeout(int readRpcTimeout)
Deprecated.
|
void |
setRpcTimeout(int rpcTimeout)
Deprecated.
|
void |
setWriteRpcTimeout(int writeRpcTimeout)
Deprecated.
|
private CompareOperator |
toCompareOperator(CompareFilter.CompareOp compareOp) |
String |
toString() |
private void |
validatePut(Put put) |
private static final org.slf4j.Logger LOG
private static final Consistency DEFAULT_CONSISTENCY
private final ClusterConnection connection
private final org.apache.hadoop.conf.Configuration configuration
private final ConnectionConfiguration connConfiguration
private boolean closed
private final int scannerCaching
private final long scannerMaxResultSize
private final ExecutorService pool
private int operationTimeoutMs
private final int rpcTimeoutMs
private int readRpcTimeoutMs
private int writeRpcTimeoutMs
private final boolean cleanupPoolOnClose
private final HRegionLocator locator
AsyncProcess multiAp
private final RpcRetryingCallerFactory rpcCallerFactory
private final RpcControllerFactory rpcControllerFactory
@InterfaceAudience.Private protected HTable(ConnectionImplementation connection, TableBuilderBase builder, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, ExecutorService pool)
ConnectionFactory
class comment for how to
get a Table
instance (use Table
instead of HTable
).connection
- Connection to be used.builder
- The table builderrpcCallerFactory
- The RPC caller factoryrpcControllerFactory
- The RPC controller factorypool
- ExecutorService to be used.@InterfaceAudience.Private public static ThreadPoolExecutor getDefaultExecutor(org.apache.hadoop.conf.Configuration conf)
public static int getMaxKeyValueSize(org.apache.hadoop.conf.Configuration conf)
public org.apache.hadoop.conf.Configuration getConfiguration()
Table
Configuration
object used by this instance.
The reference returned is not a copy, so any change made to it will affect this instance.
getConfiguration
in interface Table
public TableName getName()
Table
protected Connection getConnection()
@Deprecated public HTableDescriptor getTableDescriptor() throws IOException
Table
table descriptor
for this table.getTableDescriptor
in interface Table
IOException
- if a remote or network exception occurs.public TableDescriptor getDescriptor() throws IOException
Table
table descriptor
for this table.getDescriptor
in interface Table
IOException
- if a remote or network exception occurs.private Pair<List<byte[]>,List<HRegionLocation>> getKeysAndRegionsInRange(byte[] startKey, byte[] endKey, boolean includeEndKey) throws IOException
startKey
- Starting row in range, inclusiveendKey
- Ending row in rangeincludeEndKey
- true if endRow is inclusive, false if exclusiveIOException
- if a remote or network exception occursprivate Pair<List<byte[]>,List<HRegionLocation>> getKeysAndRegionsInRange(byte[] startKey, byte[] endKey, boolean includeEndKey, boolean reload) throws IOException
startKey
- Starting row in range, inclusiveendKey
- Ending row in rangeincludeEndKey
- true if endRow is inclusive, false if exclusivereload
- true to reload information or false to use cached informationIOException
- if a remote or network exception occurspublic ResultScanner getScanner(Scan scan) throws IOException
HTable
must not be closed.
Table.getScanner(Scan)
has other usage details.getScanner
in interface Table
scan
- A configured Scan
object.IOException
- if a remote or network exception occurs.public ResultScanner getScanner(byte[] family) throws IOException
HTable
must not be closed.
Table.getScanner(byte[])
has other usage details.getScanner
in interface Table
family
- The column family to scan.IOException
- if a remote or network exception occurs.public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException
HTable
must not be closed.
Table.getScanner(byte[], byte[])
has other usage details.getScanner
in interface Table
family
- The column family to scan.qualifier
- The column qualifier to scan.IOException
- if a remote or network exception occurs.public Result get(Get get) throws IOException
Table
get
in interface Table
get
- The object that specifies what data to fetch and from which row.Result
instance returned won't
contain any KeyValue
, as indicated by
Result.isEmpty()
.IOException
- if a remote or network exception occurs.private Result get(Get get, boolean checkExistenceOnly) throws IOException
IOException
public Result[] get(List<Get> gets) throws IOException
Table
get
in interface Table
gets
- The objects that specify what data to fetch and from which rows.Result
instance returned won't contain any
Cell
s, as indicated by Result.isEmpty()
. If there
are any failures even after retries, there will be a null
in the results' array
for those Gets, AND an exception will be thrown. The ordering of the Result array
corresponds to the order of the list of passed in Gets.IOException
- if a remote or network exception occurs.public void batch(List<? extends Row> actions, Object[] results) throws InterruptedException, IOException
Table
Table.batch(java.util.List<? extends org.apache.hadoop.hbase.client.Row>, java.lang.Object[])
call, you will not necessarily be
guaranteed that the Get returns what the Put had put.batch
in interface Table
actions
- list of Get, Put, Delete, Increment, Append, RowMutations.results
- Empty Object[], same size as actions. Provides access to partial
results, in case an exception is thrown. A null in the result array means that
the call for that action failed, even after retries. The order of the objects
in the results array corresponds to the order of actions in the request list.IOException
InterruptedException
public void batch(List<? extends Row> actions, Object[] results, int rpcTimeout) throws InterruptedException, IOException
InterruptedException
IOException
public <R> void batchCallback(List<? extends Row> actions, Object[] results, Batch.Callback<R> callback) throws IOException, InterruptedException
Table
Table.batch(List, Object[])
, but with a callback.batchCallback
in interface Table
IOException
InterruptedException
public static <R> void doBatchWithCallback(List<? extends Row> actions, Object[] results, Batch.Callback<R> callback, ClusterConnection connection, ExecutorService pool, TableName tableName) throws InterruptedIOException, RetriesExhaustedWithDetailsException
public void delete(Delete delete) throws IOException
Table
delete
in interface Table
delete
- The object that specifies what to delete.IOException
- if a remote or network exception occurs.public void delete(List<Delete> deletes) throws IOException
Table
If a specified row does not exist, Delete
will report as though sucessful
delete; no exception will be thrown. If there are any failures even after retries,
a RetriesExhaustedWithDetailsException
will be thrown.
RetriesExhaustedWithDetailsException contains lists of failed Delete
s and
corresponding remote exceptions.
delete
in interface Table
deletes
- List of things to delete. The input list gets modified by this
method. All successfully applied Delete
s in the list are removed (in particular it
gets re-ordered, so the order in which the elements are inserted in the list gives no
guarantee as to the order in which the Delete
s are executed).IOException
- if a remote or network exception occurs. In that case
the deletes
argument will contain the Delete
instances
that have not be successfully applied.public void put(Put put) throws IOException
Table
put
in interface Table
put
- The data to put.IOException
- if a remote or network exception occurs.public void put(List<Put> puts) throws IOException
Table
This can be used for group commit, or for submitting user defined batches. Before sending
a batch of mutations to the server, the client runs a few validations on the input list. If an
error is found, for example, a mutation was supplied but was missing it's column an
IllegalArgumentException
will be thrown and no mutations will be applied. If there
are any failures even after retries, a RetriesExhaustedWithDetailsException
will be
thrown. RetriesExhaustedWithDetailsException contains lists of failed mutations and
corresponding remote exceptions. The ordering of mutations and exceptions in the
encapsulating exception corresponds to the order of the input list of Put requests.
put
in interface Table
puts
- The list of mutations to apply.IOException
- if a remote or network exception occurs.public void mutateRow(RowMutations rm) throws IOException
Table
mutateRow
in interface Table
rm
- object that specifies the set of mutations to perform atomicallyIOException
public Result append(Append append) throws IOException
Table
This operation guaranteed atomicity to readers. Appends are done under a single row lock, so write operations to a row are synchronized, and readers are guaranteed to see this operation fully completed.
append
in interface Table
append
- object that specifies the columns and values to be appendedIOException
- epublic Result increment(Increment increment) throws IOException
Table
This operation ensures atomicity to readers. Increments are done under a single row lock, so write operations to a row are synchronized, and readers are guaranteed to see this operation fully completed.
increment
in interface Table
increment
- object that specifies the columns and amounts to be used
for the increment operationsIOException
- epublic long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException
Table
Table.incrementColumnValue(byte[], byte[], byte[], long, Durability)
The Durability
is defaulted to Durability.SYNC_WAL
.
incrementColumnValue
in interface Table
row
- The row that contains the cell to increment.family
- The column family of the cell to increment.qualifier
- The column qualifier of the cell to increment.amount
- The amount to increment the cell with (or decrement, if the
amount is negative).IOException
- if a remote or network exception occurs.public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, Durability durability) throws IOException
Table
amount
and
written to the specified column.
Setting durability to Durability.SKIP_WAL
means that in a fail
scenario you will lose any increments that have not been flushed.
incrementColumnValue
in interface Table
row
- The row that contains the cell to increment.family
- The column family of the cell to increment.qualifier
- The column qualifier of the cell to increment.amount
- The amount to increment the cell with (or decrement, if the
amount is negative).durability
- The persistence guarantee for this increment.IOException
- if a remote or network exception occurs.@Deprecated public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put) throws IOException
Table
checkAndPut
in interface Table
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkvalue
- the expected valueput
- data to put if check succeedsIOException
- e@Deprecated public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, byte[] value, Put put) throws IOException
Table
checkAndPut
in interface Table
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkcompareOp
- comparison operator to usevalue
- the expected valueput
- data to put if check succeedsIOException
- e@Deprecated public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Put put) throws IOException
Table
checkAndPut
in interface Table
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- comparison operator to usevalue
- the expected valueput
- data to put if check succeedsIOException
- eprivate boolean doCheckAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Filter filter, TimeRange timeRange, Put put) throws IOException
IOException
@Deprecated public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, byte[] value, Delete delete) throws IOException
Table
checkAndDelete
in interface Table
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkvalue
- the expected valuedelete
- data to delete if check succeedsIOException
- e@Deprecated public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, byte[] value, Delete delete) throws IOException
Table
checkAndDelete
in interface Table
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkcompareOp
- comparison operator to usevalue
- the expected valuedelete
- data to delete if check succeedsIOException
- e@Deprecated public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Delete delete) throws IOException
Table
checkAndDelete
in interface Table
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- comparison operator to usevalue
- the expected valuedelete
- data to delete if check succeedsIOException
- eprivate boolean doCheckAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Filter filter, TimeRange timeRange, Delete delete) throws IOException
IOException
public Table.CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family)
Table
Use the returned Table.CheckAndMutateBuilder
to construct your request and then execute it.
This is a fluent style API, the code is like:
table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put);
checkAndMutate
in interface Table
public Table.CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter)
Table
Use the returned Table.CheckAndMutateWithFilterBuilder
to construct your request and then
execute it. This is a fluent style API, the code is like:
table.checkAndMutate(row, filter).thenPut(put);
checkAndMutate
in interface Table
private boolean doCheckAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Filter filter, TimeRange timeRange, RowMutations rm) throws IOException
IOException
@Deprecated public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, byte[] value, RowMutations rm) throws IOException
Table
checkAndMutate
in interface Table
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkcompareOp
- the comparison operatorvalue
- the expected valuerm
- mutations to perform if check succeedsIOException
- e@Deprecated public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, RowMutations rm) throws IOException
Table
checkAndMutate
in interface Table
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- the comparison operatorvalue
- the expected valuerm
- mutations to perform if check succeedsIOException
- eprivate CompareOperator toCompareOperator(CompareFilter.CompareOp compareOp)
public boolean exists(Get get) throws IOException
Table
This will return true if the Get matches one or more keys, false if not.
This is a server-side call so it prevents any data from being transfered to the client.
exists
in interface Table
get
- the GetIOException
- epublic boolean[] exists(List<Get> gets) throws IOException
Table
This will return an array of booleans. Each value will be true if the related Get matches one or more keys, false if not.
This is a server-side call so it prevents any data from being transferred to the client.
exists
in interface Table
gets
- the GetsIOException
- epublic <R> void processBatchCallback(List<? extends Row> list, Object[] results, Batch.Callback<R> callback) throws IOException, InterruptedException
list
- The collection of actions.results
- An empty array, same size as list. If an exception is thrown,
you can test here for partial results, and to determine which actions
processed successfully.IOException
- if there are problems talking to META. Per-item
exceptions are stored in the results array.InterruptedException
public void close() throws IOException
Table
close
in interface Closeable
close
in interface AutoCloseable
close
in interface Table
IOException
- if a remote or network exception occurs.private void validatePut(Put put) throws IllegalArgumentException
IllegalArgumentException
ExecutorService getPool()
public void clearRegionCache()
public CoprocessorRpcChannel coprocessorService(byte[] row)
Table
RpcChannel
instance connected to the
table region containing the specified row. The row given does not actually have
to exist. Whichever region would contain the row based on start and end keys will
be used. Note that the row
parameter is also not passed to the
coprocessor handler registered for this protocol, unless the row
is separately passed as an argument in the service request. The parameter
here is only used to locate the region used to handle the call.
The obtained RpcChannel
instance can be used to access a published
coprocessor Service
using standard protobuf service invocations:
CoprocessorRpcChannel channel = myTable.coprocessorService(rowkey); MyService.BlockingInterface service = MyService.newBlockingStub(channel); MyCallRequest request = MyCallRequest.newBuilder() ... .build(); MyCallResponse response = service.myCall(null, request);
coprocessorService
in interface Table
row
- The row key used to identify the remote region locationpublic <T extends com.google.protobuf.Service,R> Map<byte[],R> coprocessorService(Class<T> service, byte[] startKey, byte[] endKey, Batch.Call<T,R> callable) throws com.google.protobuf.ServiceException, Throwable
Table
Service
subclass for each table
region spanning the range from the startKey
row to endKey
row (inclusive), and
invokes the passed Batch.Call.call(T)
method
with each Service
instance.coprocessorService
in interface Table
T
- the Service
subclass to connect toR
- Return type for the callable
parameter's Batch.Call.call(T)
methodservice
- the protocol buffer Service
implementation to callstartKey
- start region selection with region containing this row. If null
, the
selection will start with the first table region.endKey
- select regions up to and including the region containing this row. If
null
, selection will continue through the last table region.callable
- this instance's
Batch.Call.call(T)
method will be invoked once per table region, using the Service
instance connected to that region.com.google.protobuf.ServiceException
Throwable
public <T extends com.google.protobuf.Service,R> void coprocessorService(Class<T> service, byte[] startKey, byte[] endKey, Batch.Call<T,R> callable, Batch.Callback<R> callback) throws com.google.protobuf.ServiceException, Throwable
Table
Service
subclass for each table
region spanning the range from the startKey
row to endKey
row (inclusive), and
invokes the passed Batch.Call.call(T)
method
with each Service
instance.
The given
Batch.Callback.update(byte[],byte[],Object)
method will be called with the return value from each region's
Batch.Call.call(T)
invocation.
coprocessorService
in interface Table
T
- the Service
subclass to connect toR
- Return type for the callable
parameter's Batch.Call.call(T)
methodservice
- the protocol buffer Service
implementation to callstartKey
- start region selection with region containing this row. If null
, the
selection will start with the first table region.endKey
- select regions up to and including the region containing this row. If
null
, selection will continue through the last table region.callable
- this instance's
Batch.Call.call(T)
method will be invoked once per table region, using the Service
instance connected to
that region.com.google.protobuf.ServiceException
Throwable
private List<byte[]> getStartKeysInRange(byte[] start, byte[] end) throws IOException
IOException
public long getRpcTimeout(TimeUnit unit)
Table
getRpcTimeout
in interface Table
unit
- the unit of time the timeout to be represented inTable.getReadRpcTimeout(TimeUnit)
,
Table.getWriteRpcTimeout(TimeUnit)
@Deprecated public int getRpcTimeout()
Table
getRpcTimeout
in interface Table
@Deprecated public void setRpcTimeout(int rpcTimeout)
Table
NOTE: This will set both the read and write timeout settings to the provided value.
setRpcTimeout
in interface Table
rpcTimeout
- the timeout of each rpc request in millisecond.public long getReadRpcTimeout(TimeUnit unit)
Table
getReadRpcTimeout
in interface Table
unit
- the unit of time the timeout to be represented in@Deprecated public int getReadRpcTimeout()
Table
getReadRpcTimeout
in interface Table
@Deprecated public void setReadRpcTimeout(int readRpcTimeout)
Table
setReadRpcTimeout
in interface Table
readRpcTimeout
- the timeout for read rpc request in millisecondspublic long getWriteRpcTimeout(TimeUnit unit)
Table
getWriteRpcTimeout
in interface Table
unit
- the unit of time the timeout to be represented in@Deprecated public int getWriteRpcTimeout()
Table
getWriteRpcTimeout
in interface Table
@Deprecated public void setWriteRpcTimeout(int writeRpcTimeout)
Table
setWriteRpcTimeout
in interface Table
writeRpcTimeout
- the timeout for write rpc request in millisecondspublic long getOperationTimeout(TimeUnit unit)
Table
getOperationTimeout
in interface Table
unit
- the unit of time the timeout to be represented in@Deprecated public int getOperationTimeout()
Table
getOperationTimeout
in interface Table
@Deprecated public void setOperationTimeout(int operationTimeout)
Table
setOperationTimeout
in interface Table
operationTimeout
- the total timeout of each operation in millisecond.public <R extends com.google.protobuf.Message> Map<byte[],R> batchCoprocessorService(com.google.protobuf.Descriptors.MethodDescriptor methodDescriptor, com.google.protobuf.Message request, byte[] startKey, byte[] endKey, R responsePrototype) throws com.google.protobuf.ServiceException, Throwable
Table
Service
subclass for each table
region spanning the range from the startKey
row to endKey
row (inclusive), all
the invocations to the same region server will be batched into one call. The coprocessor
service is invoked according to the service instance, method name and parameters.batchCoprocessorService
in interface Table
R
- the response type for the coprocessor Service methodmethodDescriptor
- the descriptor for the protobuf service method to call.request
- the method call parametersstartKey
- start region selection with region containing this row. If null
, the
selection will start with the first table region.endKey
- select regions up to and including the region containing this row. If null
,
selection will continue through the last table region.responsePrototype
- the proto type of the response of the method in Service.com.google.protobuf.ServiceException
Throwable
public <R extends com.google.protobuf.Message> void batchCoprocessorService(com.google.protobuf.Descriptors.MethodDescriptor methodDescriptor, com.google.protobuf.Message request, byte[] startKey, byte[] endKey, R responsePrototype, Batch.Callback<R> callback) throws com.google.protobuf.ServiceException, Throwable
Table
Service
subclass for each table
region spanning the range from the startKey
row to endKey
row (inclusive), all
the invocations to the same region server will be batched into one call. The coprocessor
service is invoked according to the service instance, method name and parameters.
The given
Batch.Callback.update(byte[],byte[],Object)
method will be called with the return value from each region's invocation.
batchCoprocessorService
in interface Table
R
- the response type for the coprocessor Service methodmethodDescriptor
- the descriptor for the protobuf service method to call.request
- the method call parametersstartKey
- start region selection with region containing this row.
If null
, the selection will start with the first table region.endKey
- select regions up to and including the region containing this row.
If null
, selection will continue through the last table region.responsePrototype
- the proto type of the response of the method in Service.callback
- callback to invoke with the response for each regioncom.google.protobuf.ServiceException
Throwable
public RegionLocator getRegionLocator()
Table
RegionLocator
for this table.getRegionLocator
in interface Table
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.