@InterfaceAudience.Public public interface Table extends Closeable
Connection
and call close()
afterwards.
Table
can be used to get, put, delete or scan data from a table.
ConnectionFactory
,
Connection
,
Admin
,
RegionLocator
Modifier and Type | Interface and Description |
---|---|
static interface |
Table.CheckAndMutateBuilder
A helper class for sending checkAndMutate request.
|
static interface |
Table.CheckAndMutateWithFilterBuilder
A helper class for sending checkAndMutate request with a filter.
|
Modifier and Type | Method and Description |
---|---|
default Result |
append(Append append)
Appends values to one or more columns within a single row.
|
default void |
batch(List<? extends Row> actions,
Object[] results)
Method that does a batch call on Deletes, Gets, Puts, Increments, Appends, RowMutations.
|
default <R> void |
batchCallback(List<? extends Row> actions,
Object[] results,
Batch.Callback<R> callback)
Same as
batch(List, Object[]) , but with a callback. |
default <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. |
default <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. |
default boolean |
checkAndDelete(byte[] row,
byte[] family,
byte[] qualifier,
byte[] value,
Delete delete)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
checkAndMutate(byte[], byte[]) |
default boolean |
checkAndDelete(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
byte[] value,
Delete delete)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
checkAndMutate(byte[], byte[]) |
default boolean |
checkAndDelete(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
byte[] value,
Delete delete)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
checkAndMutate(byte[], byte[]) |
default Table.CheckAndMutateBuilder |
checkAndMutate(byte[] row,
byte[] family)
Atomically checks if a row/family/qualifier value matches the expected value.
|
default boolean |
checkAndMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
byte[] value,
RowMutations mutation)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
checkAndMutate(byte[], byte[]) |
default boolean |
checkAndMutate(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
byte[] value,
RowMutations mutation)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
checkAndMutate(byte[], byte[]) |
default Table.CheckAndMutateWithFilterBuilder |
checkAndMutate(byte[] row,
Filter filter)
Atomically checks if a row matches the specified filter.
|
default boolean |
checkAndPut(byte[] row,
byte[] family,
byte[] qualifier,
byte[] value,
Put put)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
checkAndMutate(byte[], byte[]) |
default boolean |
checkAndPut(byte[] row,
byte[] family,
byte[] qualifier,
CompareFilter.CompareOp compareOp,
byte[] value,
Put put)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
checkAndMutate(byte[], byte[]) |
default boolean |
checkAndPut(byte[] row,
byte[] family,
byte[] qualifier,
CompareOperator op,
byte[] value,
Put put)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
checkAndMutate(byte[], byte[]) |
default void |
close()
Releases any resources held or pending changes in internal buffers.
|
default CoprocessorRpcChannel |
coprocessorService(byte[] row)
Creates and returns a
RpcChannel instance connected to the
table region containing the specified row. |
default <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. |
default <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. |
default void |
delete(Delete delete)
Deletes the specified cells/row.
|
default void |
delete(List<Delete> deletes)
Batch Deletes the specified cells/rows from the table.
|
default boolean |
exists(Get get)
Test for the existence of columns in the table, as specified by the Get.
|
default boolean[] |
exists(List<Get> gets)
Test for the existence of columns in the table, as specified by the Gets.
|
default boolean[] |
existsAll(List<Get> gets)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
exists(List) |
default Result |
get(Get get)
Extracts certain cells from a given row.
|
default 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. |
TableDescriptor |
getDescriptor()
Gets the
table descriptor for this table. |
TableName |
getName()
Gets the fully qualified table name instance of this table.
|
default int |
getOperationTimeout()
Deprecated.
since 2.0 and will be removed in 3.0 version
use
getOperationTimeout(TimeUnit) instead |
default long |
getOperationTimeout(TimeUnit unit)
Get timeout of each operation in Table instance.
|
default int |
getReadRpcTimeout()
Deprecated.
since 2.0 and will be removed in 3.0 version
use
getReadRpcTimeout(TimeUnit) instead |
default long |
getReadRpcTimeout(TimeUnit unit)
Get timeout of each rpc read request in this Table instance.
|
RegionLocator |
getRegionLocator()
Gets the
RegionLocator for this table. |
default int |
getRpcTimeout()
Deprecated.
use
getReadRpcTimeout(TimeUnit) or
getWriteRpcTimeout(TimeUnit) instead |
default long |
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.
|
default ResultScanner |
getScanner(Scan scan)
Returns a scanner on the current table as specified by the
Scan
object. |
default HTableDescriptor |
getTableDescriptor()
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
getDescriptor() |
default int |
getWriteRpcTimeout()
Deprecated.
since 2.0 and will be removed in 3.0 version
use
getWriteRpcTimeout(TimeUnit) instead |
default long |
getWriteRpcTimeout(TimeUnit unit)
Get timeout of each rpc write request in this Table instance.
|
default Result |
increment(Increment increment)
Increments one or more columns within a single row.
|
default long |
incrementColumnValue(byte[] row,
byte[] family,
byte[] qualifier,
long amount)
|
default long |
incrementColumnValue(byte[] row,
byte[] family,
byte[] qualifier,
long amount,
Durability durability)
Atomically increments a column value.
|
default void |
mutateRow(RowMutations rm)
Performs multiple mutations atomically on a single row.
|
default void |
put(List<Put> puts)
Batch puts the specified data into the table.
|
default void |
put(Put put)
Puts some data in the table.
|
default void |
setOperationTimeout(int operationTimeout)
Deprecated.
since 2.0.0, use
TableBuilder.setOperationTimeout(int) instead |
default void |
setReadRpcTimeout(int readRpcTimeout)
Deprecated.
since 2.0.0, use
TableBuilder.setReadRpcTimeout(int) instead |
default void |
setRpcTimeout(int rpcTimeout)
Deprecated.
Use setReadRpcTimeout or setWriteRpcTimeout instead
|
default void |
setWriteRpcTimeout(int writeRpcTimeout)
Deprecated.
since 2.0.0, use
TableBuilder.setWriteRpcTimeout(int) instead |
org.apache.hadoop.conf.Configuration getConfiguration()
Configuration
object used by this instance.
The reference returned is not a copy, so any change made to it will affect this instance.
@Deprecated default HTableDescriptor getTableDescriptor() throws IOException
getDescriptor()
table descriptor
for this table.IOException
- if a remote or network exception occurs.TableDescriptor getDescriptor() throws IOException
table descriptor
for this table.IOException
- if a remote or network exception occurs.RegionLocator getRegionLocator() throws IOException
RegionLocator
for this table.IOException
default boolean exists(Get get) throws IOException
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.
get
- the GetIOException
- edefault boolean[] exists(List<Get> gets) throws IOException
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.
gets
- the GetsIOException
- e@Deprecated default boolean[] existsAll(List<Get> gets) throws IOException
exists(List)
gets
- the GetsIOException
- edefault void batch(List<? extends Row> actions, Object[] results) throws IOException, InterruptedException
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.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
default <R> void batchCallback(List<? extends Row> actions, Object[] results, Batch.Callback<R> callback) throws IOException, InterruptedException
batch(List, Object[])
, but with a callback.IOException
InterruptedException
default Result get(Get get) throws IOException
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.default Result[] get(List<Get> gets) throws IOException
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.put(List)
runs pre-flight validations on the input list on client.
Currently get(List)
doesn't run any validations on the client-side,
currently there is no need, but this may change in the future. An
IllegalArgumentException
will be thrown in this case.default ResultScanner getScanner(Scan scan) throws IOException
Scan
object.
Note that the passed Scan
's start row and caching properties
maybe changed.scan
- A configured Scan
object.IOException
- if a remote or network exception occurs.default ResultScanner getScanner(byte[] family) throws IOException
family
- The column family to scan.IOException
- if a remote or network exception occurs.default ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException
family
- The column family to scan.qualifier
- The column qualifier to scan.IOException
- if a remote or network exception occurs.default void put(Put put) throws IOException
put
- The data to put.IOException
- if a remote or network exception occurs.default void put(List<Put> puts) throws IOException
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.
puts
- The list of mutations to apply.IOException
- if a remote or network exception occurs.@Deprecated default boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, byte[] value, Put put) throws IOException
checkAndMutate(byte[], byte[])
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkvalue
- the expected valueput
- data to put if check succeedsIOException
- e@Deprecated default boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, byte[] value, Put put) throws IOException
checkAndMutate(byte[], byte[])
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 default boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Put put) throws IOException
checkAndMutate(byte[], byte[])
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- comparison operator to usevalue
- the expected valueput
- data to put if check succeedsIOException
- edefault void delete(Delete delete) throws IOException
delete
- The object that specifies what to delete.IOException
- if a remote or network exception occurs.default void delete(List<Delete> deletes) throws IOException
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.
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.deletes
will no longer be modified. Also,
put(List)
runs pre-flight validations on the input list on client. Currently
delete(List)
doesn't run validations on the client, there is no need
currently, but this may change in the future. An * IllegalArgumentException
will be thrown in this case.@Deprecated default boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, byte[] value, Delete delete) throws IOException
checkAndMutate(byte[], byte[])
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkvalue
- the expected valuedelete
- data to delete if check succeedsIOException
- e@Deprecated default boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, byte[] value, Delete delete) throws IOException
checkAndMutate(byte[], byte[])
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 default boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, Delete delete) throws IOException
checkAndMutate(byte[], byte[])
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- comparison operator to usevalue
- the expected valuedelete
- data to delete if check succeedsIOException
- edefault Table.CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family)
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);
default Table.CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter)
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);
default void mutateRow(RowMutations rm) throws IOException
rm
- object that specifies the set of mutations to perform atomicallyIOException
default Result append(Append append) throws IOException
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
- object that specifies the columns and values to be appendedIOException
- edefault Result increment(Increment increment) throws IOException
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
- object that specifies the columns and amounts to be used
for the increment operationsIOException
- edefault long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException
incrementColumnValue(byte[], byte[], byte[], long, Durability)
The Durability
is defaulted to Durability.SYNC_WAL
.
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.default long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, Durability durability) throws IOException
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.
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.default void close() throws IOException
close
in interface AutoCloseable
close
in interface Closeable
IOException
- if a remote or network exception occurs.default CoprocessorRpcChannel coprocessorService(byte[] row)
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);
row
- The row key used to identify the remote region locationdefault <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
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
- 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
default <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
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.
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
default <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
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.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
default <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
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.
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
@Deprecated default boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareFilter.CompareOp compareOp, byte[] value, RowMutations mutation) throws IOException
checkAndMutate(byte[], byte[])
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkcompareOp
- the comparison operatorvalue
- the expected valuemutation
- mutations to perform if check succeedsIOException
- e@Deprecated default boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, byte[] value, RowMutations mutation) throws IOException
checkAndMutate(byte[], byte[])
row
- to checkfamily
- column family to checkqualifier
- column qualifier to checkop
- the comparison operatorvalue
- the expected valuemutation
- mutations to perform if check succeedsIOException
- edefault long getRpcTimeout(TimeUnit unit)
unit
- the unit of time the timeout to be represented ingetReadRpcTimeout(TimeUnit)
,
getWriteRpcTimeout(TimeUnit)
@Deprecated default int getRpcTimeout()
getReadRpcTimeout(TimeUnit)
or
getWriteRpcTimeout(TimeUnit)
instead@Deprecated default void setRpcTimeout(int rpcTimeout)
NOTE: This will set both the read and write timeout settings to the provided value.
rpcTimeout
- the timeout of each rpc request in millisecond.default long getReadRpcTimeout(TimeUnit unit)
unit
- the unit of time the timeout to be represented in@Deprecated default int getReadRpcTimeout()
getReadRpcTimeout(TimeUnit)
instead@Deprecated default void setReadRpcTimeout(int readRpcTimeout)
TableBuilder.setReadRpcTimeout(int)
insteadreadRpcTimeout
- the timeout for read rpc request in millisecondsdefault long getWriteRpcTimeout(TimeUnit unit)
unit
- the unit of time the timeout to be represented in@Deprecated default int getWriteRpcTimeout()
getWriteRpcTimeout(TimeUnit)
instead@Deprecated default void setWriteRpcTimeout(int writeRpcTimeout)
TableBuilder.setWriteRpcTimeout(int)
insteadwriteRpcTimeout
- the timeout for write rpc request in millisecondsdefault long getOperationTimeout(TimeUnit unit)
unit
- the unit of time the timeout to be represented in@Deprecated default int getOperationTimeout()
getOperationTimeout(TimeUnit)
instead@Deprecated default void setOperationTimeout(int operationTimeout)
TableBuilder.setOperationTimeout(int)
insteadoperationTimeout
- the total timeout of each operation in millisecond.Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.