Interface AsyncTable<C extends ScanResultConsumerBase>
- All Known Implementing Classes:
AsyncTableImpl
,RawAsyncTableImpl
AsyncConnection
.
The implementation is required to be thread safe.
Usually the implementation will not throw any exception directly. You need to get the exception
from the returned CompletableFuture
.
- Since:
- 2.0.0
-
Nested Class Summary
Modifier and TypeInterfaceDescriptionstatic interface
Deprecated.Since 3.0.0, will be removed in 4.0.0.static interface
Deprecated.Since 3.0.0, will be removed in 4.0.0.static interface
The callback when we want to execute a coprocessor call on a range of regions.static interface
Helper class for sending coprocessorService request that executes a coprocessor call on regions which are covered by a range.static interface
Some coprocessors may support the idea of "partial results." If for some reason a coprocessor cannot return all results for a given region in a single response, the client side can be designed to recognize this and continuing requesting more results until they are completely accumulated in the client. -
Method Summary
Modifier and TypeMethodDescriptionAppends values to one or more columns within a single row.<T> List<CompletableFuture<T>>
Method that does a batch call on Deletes, Gets, Puts, Increments, Appends and RowMutations.default <T> CompletableFuture<List<T>>
A simple version of batch.checkAndMutate
(byte[] row, byte[] family) Deprecated.Since 3.0.0, will be removed in 4.0.0.checkAndMutate
(byte[] row, Filter filter) Deprecated.Since 3.0.0, will be removed in 4.0.0.checkAndMutate
(List<CheckAndMutate> checkAndMutates) Batch version of checkAndMutate.checkAndMutate
(CheckAndMutate checkAndMutate) checkAndMutate that atomically checks if a row matches the specified condition.default CompletableFuture<List<CheckAndMutateResult>>
checkAndMutateAll
(List<CheckAndMutate> checkAndMutates) A simple version of batch checkAndMutate.<S,
R> CompletableFuture<R> coprocessorService
(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, byte[] row) Execute the given coprocessor call on the region which contains the givenrow
.<S,
R> AsyncTable.CoprocessorServiceBuilder<S, R> coprocessorService
(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, AsyncTable.CoprocessorCallback<R> callback) Execute a coprocessor call on the regions which are covered by a range.<S,
R> AsyncTable.CoprocessorServiceBuilder<S, R> coprocessorService
(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, AsyncTable.PartialResultCoprocessorCallback<S, R> callback) Similar to above.Deletes the specified cells/rows in bulk.Deletes the specified cells/row.default CompletableFuture<Void>
A simple version of batch delete.default List<CompletableFuture<Boolean>>
Test for the existence of columns in the table, as specified by the Gets.default CompletableFuture<Boolean>
Test for the existence of columns in the table, as specified by the Get.default CompletableFuture<List<Boolean>>
A simple version for batch exists.Extracts certain cells from the given rows, in batch.Extracts certain cells from a given row.default CompletableFuture<List<Result>>
A simple version for batch get.org.apache.hadoop.conf.Configuration
Returns theConfiguration
object used by this instance.Gets theTableDescriptor
for this table.getName()
Gets the fully qualified table name instance of this table.long
getOperationTimeout
(TimeUnit unit) Get timeout of each operation in Table instance.long
getReadRpcTimeout
(TimeUnit unit) Get timeout of each rpc read request in this Table instance.Gets theAsyncTableRegionLocator
for this table.Get the map of request attributeslong
getRpcTimeout
(TimeUnit unit) Get timeout of each rpc request in this Table instance.default ResultScanner
getScanner
(byte[] family) Gets a scanner on the current table for the given family.default ResultScanner
getScanner
(byte[] family, byte[] qualifier) Gets a scanner on the current table for the given family and qualifier.getScanner
(Scan scan) Returns a scanner on the current table as specified by theScan
object.long
getScanTimeout
(TimeUnit unit) Get the timeout of a single operation in a scan.long
getWriteRpcTimeout
(TimeUnit unit) Get timeout of each rpc write request in this Table instance.Increments one or more columns within a single row.default CompletableFuture<Long>
incrementColumnValue
(byte[] row, byte[] family, byte[] qualifier, long amount) default CompletableFuture<Long>
incrementColumnValue
(byte[] row, byte[] family, byte[] qualifier, long amount, Durability durability) Atomically increments a column value.mutateRow
(RowMutations mutation) Performs multiple mutations atomically on a single row.Puts some data in the table, in batch.Puts some data to the table.default CompletableFuture<Void>
A simple version of batch put.void
The scan API uses the observer pattern.Return all the results that match the given scan object.
-
Method Details
-
getName
Gets the fully qualified table name instance of this table. -
getConfiguration
org.apache.hadoop.conf.Configuration getConfiguration()Returns theConfiguration
object used by this instance.The reference returned is not a copy, so any change made to it will affect this instance.
-
getDescriptor
Gets theTableDescriptor
for this table. -
getRegionLocator
Gets theAsyncTableRegionLocator
for this table. -
getRpcTimeout
Get timeout of each rpc request in this Table instance. It will be overridden by a more specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.- Parameters:
unit
- the unit of time the timeout to be represented in- Returns:
- rpc timeout in the specified time unit
- See Also:
-
getReadRpcTimeout
Get timeout of each rpc read request in this Table instance.- Parameters:
unit
- the unit of time the timeout to be represented in- Returns:
- read rpc timeout in the specified time unit
-
getWriteRpcTimeout
Get timeout of each rpc write request in this Table instance.- Parameters:
unit
- the unit of time the timeout to be represented in- Returns:
- write rpc timeout in the specified time unit
-
getOperationTimeout
Get timeout of each operation in Table instance.- Parameters:
unit
- the unit of time the timeout to be represented in- Returns:
- operation rpc timeout in the specified time unit
-
getScanTimeout
Get the timeout of a single operation in a scan. It works like operation timeout for other operations.- Parameters:
unit
- the unit of time the timeout to be represented in- Returns:
- scan rpc timeout in the specified time unit
-
getRequestAttributes
Get the map of request attributes- Returns:
- a map of request attributes supplied by the client
-
exists
Test for the existence of columns in the table, as specified by the Get.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.
- Returns:
- true if the specified Get matches one or more keys, false if not. The return value will
be wrapped by a
CompletableFuture
.
-
get
Extracts certain cells from a given row.- Parameters:
get
- The object that specifies what data to fetch and from which row.- Returns:
- The data coming from the specified row, if it exists. If the row specified doesn't
exist, the
Result
instance returned won't contain anyKeyValue
, as indicated byResult.isEmpty()
. The return value will be wrapped by aCompletableFuture
.
-
put
Puts some data to the table.- Parameters:
put
- The data to put.- Returns:
- A
CompletableFuture
that always returns null when complete normally.
-
delete
Deletes the specified cells/row.- Parameters:
delete
- The object that specifies what to delete.- Returns:
- A
CompletableFuture
that always returns null when complete normally.
-
append
Appends values to one or more columns within a single row.This operation does not appear atomic to readers. Appends are done under a single row lock, so write operations to a row are synchronized, but readers do not take row locks so get and scan operations can see this operation partially completed.
- Parameters:
append
- object that specifies the columns and amounts to be used for the increment operations- Returns:
- values of columns after the append operation (maybe null). The return value will be
wrapped by a
CompletableFuture
.
-
increment
Increments one or more columns within a single row.This operation does not appear atomic to readers. Increments are done under a single row lock, so write operations to a row are synchronized, but readers do not take row locks so get and scan operations can see this operation partially completed.
- Parameters:
increment
- object that specifies the columns and amounts to be used for the increment operations- Returns:
- values of columns after the increment. The return value will be wrapped by a
CompletableFuture
.
-
incrementColumnValue
default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) SeeincrementColumnValue(byte[], byte[], byte[], long, Durability)
The
Durability
is defaulted toDurability.SYNC_WAL
.- Parameters:
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).- Returns:
- The new value, post increment. The return value will be wrapped by a
CompletableFuture
.
-
incrementColumnValue
default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, Durability durability) Atomically increments a column value. If the column value already exists and is not a big-endian long, this could throw an exception. If the column value does not yet exist it is initialized toamount
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.- Parameters:
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.- Returns:
- The new value, post increment. The return value will be wrapped by a
CompletableFuture
.
-
checkAndMutate
Deprecated.Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it any more.Atomically checks if a row/family/qualifier value matches the expected value. If it does, it adds the Put/Delete/RowMutations.Use the returned
AsyncTable.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) .thenAccept(succ -> { if (succ) { System.out.println("Check and put succeeded"); } else { System.out.println("Check and put failed"); } });
-
checkAndMutate
Deprecated.Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it any more.Atomically checks if a row matches the specified filter. If it does, it adds the Put/Delete/RowMutations.Use the returned
AsyncTable.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).thenAccept(succ -> { if (succ) { System.out.println("Check and put succeeded"); } else { System.out.println("Check and put failed"); } });
-
checkAndMutate
checkAndMutate that atomically checks if a row matches the specified condition. If it does, it performs the specified action.- Parameters:
checkAndMutate
- The CheckAndMutate object.- Returns:
- A
CompletableFuture
s that represent the result for the CheckAndMutate.
-
checkAndMutate
Batch version of checkAndMutate. The specified CheckAndMutates are batched only in the sense that they are sent to a RS in one RPC, but each CheckAndMutate operation is still executed atomically (and thus, each may fail independently of others).- Parameters:
checkAndMutates
- The list of CheckAndMutate.- Returns:
- A list of
CompletableFuture
s that represent the result for each CheckAndMutate.
-
checkAndMutateAll
default CompletableFuture<List<CheckAndMutateResult>> checkAndMutateAll(List<CheckAndMutate> checkAndMutates) A simple version of batch checkAndMutate. It will fail if there are any failures.- Parameters:
checkAndMutates
- The list of rows to apply.- Returns:
- A
CompletableFuture
that wrapper the result list.
-
mutateRow
- Parameters:
mutation
- object that specifies the set of mutations to perform atomically- Returns:
- A
CompletableFuture
that returns results of Increment/Append operations
-
scan
The scan API uses the observer pattern.- Parameters:
scan
- A configuredScan
object.consumer
- the consumer used to receive results.- See Also:
-
getScanner
Gets a scanner on the current table for the given family.- Parameters:
family
- The column family to scan.- Returns:
- A scanner.
-
getScanner
Gets a scanner on the current table for the given family and qualifier.- Parameters:
family
- The column family to scan.qualifier
- The column qualifier to scan.- Returns:
- A scanner.
-
getScanner
Returns a scanner on the current table as specified by theScan
object.- Parameters:
scan
- A configuredScan
object.- Returns:
- A scanner.
-
scanAll
Return all the results that match the given scan object.Notice that usually you should use this method with a
Scan
object that has limit set. For example, if you want to get the closest row after a given row, you could do this:table.scanAll(new Scan().withStartRow(row, false).setLimit(1)).thenAccept(results -> { if (results.isEmpty()) { System.out.println("No row after " + Bytes.toStringBinary(row)); } else { System.out.println("The closest row after " + Bytes.toStringBinary(row) + " is " + Bytes.toStringBinary(results.stream().findFirst().get().getRow())); } });
If your result set is very large, you should use other scan method to get a scanner or use callback to process the results. They will do chunking to prevent OOM. The scanAll method will fetch all the results and store them in a List and then return the list to you.
The scan metrics will be collected background if you enable it but you have no way to get it. Usually you can get scan metrics from
ResultScanner
, or throughScanResultConsumer.onScanMetricsCreated
but this method only returns a list of results. So if you really care about scan metrics then you'd better use other scan methods which return aResultScanner
or let you pass in aScanResultConsumer
. There is no performance difference between these scan methods so do not worry.- Parameters:
scan
- A configuredScan
object. So if you use this method to fetch a really large result set, it is likely to cause OOM.- Returns:
- The results of this small scan operation. The return value will be wrapped by a
CompletableFuture
.
-
exists
Test for the existence of columns in the table, as specified by the Gets.This will return a list 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.
- Parameters:
gets
- the Gets- Returns:
- A list of
CompletableFuture
s that represent the existence for each get.
-
existsAll
A simple version for batch exists. It will fail if there are any failures and you will get the whole result boolean list at once if the operation is succeeded.- Parameters:
gets
- the Gets- Returns:
- A
CompletableFuture
that wrapper the result boolean list.
-
get
Extracts certain cells from the given rows, in batch.Notice that you may not get all the results with this function, which means some of the returned
CompletableFuture
s may succeed while some of the other returnedCompletableFuture
s may fail.- Parameters:
gets
- The objects that specify what data to fetch and from which rows.- Returns:
- A list of
CompletableFuture
s that represent the result for each get.
-
getAll
A simple version for batch get. It will fail if there are any failures and you will get the whole result list at once if the operation is succeeded.- Parameters:
gets
- The objects that specify what data to fetch and from which rows.- Returns:
- A
CompletableFuture
that wrapper the result list.
-
put
Puts some data in the table, in batch.- Parameters:
puts
- The list of mutations to apply.- Returns:
- A list of
CompletableFuture
s that represent the result for each put.
-
putAll
A simple version of batch put. It will fail if there are any failures.- Parameters:
puts
- The list of mutations to apply.- Returns:
- A
CompletableFuture
that always returns null when complete normally.
-
delete
Deletes the specified cells/rows in bulk.- Parameters:
deletes
- list of things to delete.- Returns:
- A list of
CompletableFuture
s that represent the result for each delete.
-
deleteAll
A simple version of batch delete. It will fail if there are any failures.- Parameters:
deletes
- list of things to delete.- Returns:
- A
CompletableFuture
that always returns null when complete normally.
-
batch
Method that does a batch call on Deletes, Gets, Puts, Increments, Appends and RowMutations. The ordering of execution of the actions is not defined. Meaning if you do a Put and a Get in the samebatch(java.util.List<? extends org.apache.hadoop.hbase.client.Row>)
call, you will not necessarily be guaranteed that the Get returns what the Put had put.- Parameters:
actions
- list of Get, Put, Delete, Increment, Append, and RowMutations objects- Returns:
- A list of
CompletableFuture
s that represent the result for each action.
-
batchAll
A simple version of batch. It will fail if there are any failures and you will get the whole result list at once if the operation is succeeded.- Parameters:
actions
- list of Get, Put, Delete, Increment, Append and RowMutations objects- Returns:
- A list of the result for the actions. Wrapped by a
CompletableFuture
.
-
coprocessorService
<S,R> CompletableFuture<R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, byte[] row) Execute the given coprocessor call on the region which contains the givenrow
.The
stubMaker
is just a delegation to thenewStub
call. Usually it is only a one line lambda expression, like:channel -> xxxService.newStub(channel)
- Type Parameters:
S
- the type of the asynchronous stubR
- the type of the return value- Parameters:
stubMaker
- a delegation to the actualnewStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment ofServiceCaller
for more details.row
- The row key used to identify the remote region location- Returns:
- the return value of the protobuf rpc call, wrapped by a
CompletableFuture
. - See Also:
-
coprocessorService
<S,R> AsyncTable.CoprocessorServiceBuilder<S,R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, AsyncTable.CoprocessorCallback<R> callback) Execute a coprocessor call on the regions which are covered by a range.Use the returned
AsyncTable.CoprocessorServiceBuilder
construct your request and then execute it.The
stubMaker
is just a delegation to thexxxService.newStub
call. Usually it is only a one line lambda expression, like:channel -> xxxService.newStub(channel)
- Parameters:
stubMaker
- a delegation to the actualnewStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment ofServiceCaller
for more details.callback
- callback to get the response. See the comment ofAsyncTable.CoprocessorCallback
for more details.
-
coprocessorService
<S,R> AsyncTable.CoprocessorServiceBuilder<S,R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, AsyncTable.PartialResultCoprocessorCallback<S, R> callback) Similar to above. Use when your coprocessor client+endpoint supports partial results. If the server does not offer partial results, it is still safe to use this, assuming you implement yourAsyncTable.PartialResultCoprocessorCallback.getNextCallable(Object, RegionInfo)
correctly.
-