3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Version 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
11 * http://www.apache.org/licenses/LICENSE-2.0
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
19 package org
.apache
.hadoop
.hbase
.client
;
21 import java
.io
.Closeable
;
22 import java
.io
.IOException
;
23 import java
.util
.Collections
;
24 import java
.util
.List
;
26 import java
.util
.TreeMap
;
27 import java
.util
.concurrent
.TimeUnit
;
28 import org
.apache
.commons
.lang3
.NotImplementedException
;
29 import org
.apache
.hadoop
.conf
.Configuration
;
30 import org
.apache
.hadoop
.hbase
.Cell
;
31 import org
.apache
.hadoop
.hbase
.CompareOperator
;
32 import org
.apache
.hadoop
.hbase
.TableName
;
33 import org
.apache
.hadoop
.hbase
.client
.coprocessor
.Batch
;
34 import org
.apache
.hadoop
.hbase
.client
.coprocessor
.Batch
.Callback
;
35 import org
.apache
.hadoop
.hbase
.filter
.Filter
;
36 import org
.apache
.hadoop
.hbase
.io
.TimeRange
;
37 import org
.apache
.hadoop
.hbase
.ipc
.CoprocessorRpcChannel
;
38 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
39 import org
.apache
.yetus
.audience
.InterfaceAudience
;
41 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.Descriptors
;
42 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.Message
;
43 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.Service
;
44 import org
.apache
.hbase
.thirdparty
.com
.google
.protobuf
.ServiceException
;
47 * Used to communicate with a single HBase table.
48 * Obtain an instance from a {@link Connection} and call {@link #close()} afterwards.
50 * <p><code>Table</code> can be used to get, put, delete or scan data from a table.
51 * @see ConnectionFactory
57 @InterfaceAudience.Public
58 public interface Table
extends Closeable
{
60 * Gets the fully qualified table name instance of this table.
65 * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
67 * The reference returned is not a copy, so any change made to it will
68 * affect this instance.
70 Configuration
getConfiguration();
73 * Gets the {@link org.apache.hadoop.hbase.client.TableDescriptor table descriptor} for this table.
74 * @throws java.io.IOException if a remote or network exception occurs.
76 TableDescriptor
getDescriptor() throws IOException
;
79 * Gets the {@link RegionLocator} for this table.
81 RegionLocator
getRegionLocator() throws IOException
;
84 * Test for the existence of columns in the table, as specified by the Get.
87 * This will return true if the Get matches one or more keys, false if not.
90 * This is a server-side call so it prevents any data from being transfered to
94 * @return true if the specified Get matches one or more keys, false if not
95 * @throws IOException e
97 default boolean exists(Get get
) throws IOException
{
98 return exists(Collections
.singletonList(get
))[0];
102 * Test for the existence of columns in the table, as specified by the Gets.
105 * This will return an array of booleans. Each value will be true if the related Get matches
106 * one or more keys, false if not.
109 * This is a server-side call so it prevents any data from being transferred to
112 * @param gets the Gets
113 * @return Array of boolean. True if the specified Get matches one or more keys, false if not.
114 * @throws IOException e
116 default boolean[] exists(List
<Get
> gets
) throws IOException
{
117 throw new NotImplementedException("Add an implementation!");
121 * Method that does a batch call on Deletes, Gets, Puts, Increments, Appends, RowMutations.
122 * The ordering of execution of the actions is not defined. Meaning if you do a Put and a
123 * Get in the same {@link #batch} call, you will not necessarily be
124 * guaranteed that the Get returns what the Put had put.
126 * @param actions list of Get, Put, Delete, Increment, Append, RowMutations.
127 * @param results Empty Object[], same size as actions. Provides access to partial
128 * results, in case an exception is thrown. A null in the result array means that
129 * the call for that action failed, even after retries. The order of the objects
130 * in the results array corresponds to the order of actions in the request list.
131 * @throws IOException
134 default void batch(final List
<?
extends Row
> actions
, final Object
[] results
) throws IOException
,
135 InterruptedException
{
136 throw new NotImplementedException("Add an implementation!");
140 * Same as {@link #batch(List, Object[])}, but with a callback.
142 * @deprecated since 3.0.0, will removed in 4.0.0. Please use the batch related methods in
143 * {@link AsyncTable} directly if you want to use callback. We reuse the callback for
144 * coprocessor here, and the problem is that for batch operation, the
145 * {@link AsyncTable} does not tell us the region, so in this method we need an extra
146 * locating after we get the result, which is not good.
149 default <R
> void batchCallback(final List
<?
extends Row
> actions
, final Object
[] results
,
150 final Batch
.Callback
<R
> callback
) throws IOException
, InterruptedException
{
151 throw new NotImplementedException("Add an implementation!");
155 * Extracts certain cells from a given row.
156 * @param get The object that specifies what data to fetch and from which row.
157 * @return The data coming from the specified row, if it exists. If the row
158 * specified doesn't exist, the {@link Result} instance returned won't
159 * contain any {@link org.apache.hadoop.hbase.KeyValue}, as indicated by
160 * {@link Result#isEmpty()}.
161 * @throws IOException if a remote or network exception occurs.
164 default Result
get(Get get
) throws IOException
{
165 return get(Collections
.singletonList(get
))[0];
169 * Extracts specified cells from the given rows, as a batch.
171 * @param gets The objects that specify what data to fetch and from which rows.
172 * @return The data coming from the specified rows, if it exists. If the row specified doesn't
173 * exist, the {@link Result} instance returned won't contain any
174 * {@link org.apache.hadoop.hbase.Cell}s, as indicated by {@link Result#isEmpty()}. If there
175 * are any failures even after retries, there will be a <code>null</code> in the results' array
176 * for those Gets, AND an exception will be thrown. The ordering of the Result array
177 * corresponds to the order of the list of passed in Gets.
178 * @throws IOException if a remote or network exception occurs.
180 * @apiNote {@link #put(List)} runs pre-flight validations on the input list on client.
181 * Currently {@link #get(List)} doesn't run any validations on the client-side,
182 * currently there is no need, but this may change in the future. An
183 * {@link IllegalArgumentException} will be thrown in this case.
185 default Result
[] get(List
<Get
> gets
) throws IOException
{
186 throw new NotImplementedException("Add an implementation!");
190 * Returns a scanner on the current table as specified by the {@link Scan}
192 * Note that the passed {@link Scan}'s start row and caching properties
195 * @param scan A configured {@link Scan} object.
197 * @throws IOException if a remote or network exception occurs.
200 default ResultScanner
getScanner(Scan scan
) throws IOException
{
201 throw new NotImplementedException("Add an implementation!");
205 * Gets a scanner on the current table for the given family.
207 * @param family The column family to scan.
209 * @throws IOException if a remote or network exception occurs.
212 default ResultScanner
getScanner(byte[] family
) throws IOException
{
213 throw new NotImplementedException("Add an implementation!");
217 * Gets a scanner on the current table for the given family and qualifier.
219 * @param family The column family to scan.
220 * @param qualifier The column qualifier to scan.
222 * @throws IOException if a remote or network exception occurs.
225 default ResultScanner
getScanner(byte[] family
, byte[] qualifier
) throws IOException
{
226 throw new NotImplementedException("Add an implementation!");
230 * Puts some data in the table.
232 * @param put The data to put.
233 * @throws IOException if a remote or network exception occurs.
236 default void put(Put put
) throws IOException
{
237 put(Collections
.singletonList(put
));
241 * Batch puts the specified data into the table.
243 * This can be used for group commit, or for submitting user defined batches. Before sending
244 * a batch of mutations to the server, the client runs a few validations on the input list. If an
245 * error is found, for example, a mutation was supplied but was missing it's column an
246 * {@link IllegalArgumentException} will be thrown and no mutations will be applied. If there
247 * are any failures even after retries, a {@link RetriesExhaustedWithDetailsException} will be
248 * thrown. RetriesExhaustedWithDetailsException contains lists of failed mutations and
249 * corresponding remote exceptions. The ordering of mutations and exceptions in the
250 * encapsulating exception corresponds to the order of the input list of Put requests.
252 * @param puts The list of mutations to apply.
253 * @throws IOException if a remote or network exception occurs.
256 default void put(List
<Put
> puts
) throws IOException
{
257 throw new NotImplementedException("Add an implementation!");
261 * Deletes the specified cells/row.
263 * @param delete The object that specifies what to delete.
264 * @throws IOException if a remote or network exception occurs.
267 default void delete(Delete delete
) throws IOException
{
268 throw new NotImplementedException("Add an implementation!");
272 * Batch Deletes the specified cells/rows from the table.
274 * If a specified row does not exist, {@link Delete} will report as though sucessful
275 * delete; no exception will be thrown. If there are any failures even after retries,
276 * a {@link RetriesExhaustedWithDetailsException} will be thrown.
277 * RetriesExhaustedWithDetailsException contains lists of failed {@link Delete}s and
278 * corresponding remote exceptions.
280 * @param deletes List of things to delete. The input list gets modified by this
281 * method. All successfully applied {@link Delete}s in the list are removed (in particular it
282 * gets re-ordered, so the order in which the elements are inserted in the list gives no
283 * guarantee as to the order in which the {@link Delete}s are executed).
284 * @throws IOException if a remote or network exception occurs. In that case
285 * the {@code deletes} argument will contain the {@link Delete} instances
286 * that have not be successfully applied.
288 * @apiNote In 3.0.0 version, the input list {@code deletes} will no longer be modified. Also,
289 * {@link #put(List)} runs pre-flight validations on the input list on client. Currently
290 * {@link #delete(List)} doesn't run validations on the client, there is no need
291 * currently, but this may change in the future. An * {@link IllegalArgumentException}
292 * will be thrown in this case.
294 default void delete(List
<Delete
> deletes
) throws IOException
{
295 throw new NotImplementedException("Add an implementation!");
299 * Atomically checks if a row/family/qualifier value matches the expected value. If it does, it
300 * adds the Put/Delete/RowMutations.
302 * Use the returned {@link CheckAndMutateBuilder} to construct your request and then execute it.
303 * This is a fluent style API, the code is like:
307 * table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put);
311 default CheckAndMutateBuilder
checkAndMutate(byte[] row
, byte[] family
) {
312 throw new NotImplementedException("Add an implementation!");
316 * A helper class for sending checkAndMutate request.
318 interface CheckAndMutateBuilder
{
321 * @param qualifier column qualifier to check.
323 CheckAndMutateBuilder
qualifier(byte[] qualifier
);
326 * @param timeRange timeRange to check
328 CheckAndMutateBuilder
timeRange(TimeRange timeRange
);
331 * Check for lack of column.
333 CheckAndMutateBuilder
ifNotExists();
336 * Check for equality.
337 * @param value the expected value
339 default CheckAndMutateBuilder
ifEquals(byte[] value
) {
340 return ifMatches(CompareOperator
.EQUAL
, value
);
344 * @param compareOp comparison operator to use
345 * @param value the expected value
347 CheckAndMutateBuilder
ifMatches(CompareOperator compareOp
, byte[] value
);
350 * @param put data to put if check succeeds
351 * @return {@code true} if the new put was executed, {@code false} otherwise.
353 boolean thenPut(Put put
) throws IOException
;
356 * @param delete data to delete if check succeeds
357 * @return {@code true} if the new delete was executed, {@code false} otherwise.
359 boolean thenDelete(Delete delete
) throws IOException
;
362 * @param mutation mutations to perform if check succeeds
363 * @return true if the new mutation was executed, false otherwise.
365 boolean thenMutate(RowMutations mutation
) throws IOException
;
369 * Atomically checks if a row matches the specified filter. If it does, it adds the
370 * Put/Delete/RowMutations.
372 * Use the returned {@link CheckAndMutateWithFilterBuilder} to construct your request and then
373 * execute it. This is a fluent style API, the code is like:
377 * table.checkAndMutate(row, filter).thenPut(put);
381 default CheckAndMutateWithFilterBuilder
checkAndMutate(byte[] row
, Filter filter
) {
382 throw new NotImplementedException("Add an implementation!");
386 * A helper class for sending checkAndMutate request with a filter.
388 interface CheckAndMutateWithFilterBuilder
{
391 * @param timeRange timeRange to check
393 CheckAndMutateWithFilterBuilder
timeRange(TimeRange timeRange
);
396 * @param put data to put if check succeeds
397 * @return {@code true} if the new put was executed, {@code false} otherwise.
399 boolean thenPut(Put put
) throws IOException
;
402 * @param delete data to delete if check succeeds
403 * @return {@code true} if the new delete was executed, {@code false} otherwise.
405 boolean thenDelete(Delete delete
) throws IOException
;
408 * @param mutation mutations to perform if check succeeds
409 * @return true if the new mutation was executed, false otherwise.
411 boolean thenMutate(RowMutations mutation
) throws IOException
;
415 * Performs multiple mutations atomically on a single row. Currently
416 * {@link Put} and {@link Delete} are supported.
418 * @param rm object that specifies the set of mutations to perform atomically
419 * @throws IOException
421 default void mutateRow(final RowMutations rm
) throws IOException
{
422 throw new NotImplementedException("Add an implementation!");
426 * Appends values to one or more columns within a single row.
428 * This operation guaranteed atomicity to readers. Appends are done
429 * under a single row lock, so write operations to a row are synchronized, and
430 * readers are guaranteed to see this operation fully completed.
432 * @param append object that specifies the columns and values to be appended
433 * @throws IOException e
434 * @return values of columns after the append operation (maybe null)
436 default Result
append(final Append append
) throws IOException
{
437 throw new NotImplementedException("Add an implementation!");
441 * Increments one or more columns within a single row.
443 * This operation ensures atomicity to readers. Increments are done
444 * under a single row lock, so write operations to a row are synchronized, and
445 * readers are guaranteed to see this operation fully completed.
447 * @param increment object that specifies the columns and amounts to be used
448 * for the increment operations
449 * @throws IOException e
450 * @return values of columns after the increment
452 default Result
increment(final Increment increment
) throws IOException
{
453 throw new NotImplementedException("Add an implementation!");
457 * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
459 * The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.
460 * @param row The row that contains the cell to increment.
461 * @param family The column family of the cell to increment.
462 * @param qualifier The column qualifier of the cell to increment.
463 * @param amount The amount to increment the cell with (or decrement, if the
464 * amount is negative).
465 * @return The new value, post increment.
466 * @throws IOException if a remote or network exception occurs.
468 default long incrementColumnValue(byte[] row
, byte[] family
, byte[] qualifier
, long amount
)
470 Increment increment
= new Increment(row
).addColumn(family
, qualifier
, amount
);
471 Cell cell
= increment(increment
).getColumnLatestCell(family
, qualifier
);
472 return Bytes
.toLong(cell
.getValueArray(), cell
.getValueOffset(), cell
.getValueLength());
476 * Atomically increments a column value. If the column value already exists
477 * and is not a big-endian long, this could throw an exception. If the column
478 * value does not yet exist it is initialized to <code>amount</code> and
479 * written to the specified column.
481 * <p>Setting durability to {@link Durability#SKIP_WAL} means that in a fail
482 * scenario you will lose any increments that have not been flushed.
483 * @param row The row that contains the cell to increment.
484 * @param family The column family of the cell to increment.
485 * @param qualifier The column qualifier of the cell to increment.
486 * @param amount The amount to increment the cell with (or decrement, if the
487 * amount is negative).
488 * @param durability The persistence guarantee for this increment.
489 * @return The new value, post increment.
490 * @throws IOException if a remote or network exception occurs.
492 default long incrementColumnValue(byte[] row
, byte[] family
, byte[] qualifier
,
493 long amount
, Durability durability
) throws IOException
{
494 Increment increment
= new Increment(row
)
495 .addColumn(family
, qualifier
, amount
)
496 .setDurability(durability
);
497 Cell cell
= increment(increment
).getColumnLatestCell(family
, qualifier
);
498 return Bytes
.toLong(cell
.getValueArray(), cell
.getValueOffset(), cell
.getValueLength());
502 * Releases any resources held or pending changes in internal buffers.
504 * @throws IOException if a remote or network exception occurs.
507 default void close() throws IOException
{
508 throw new NotImplementedException("Add an implementation!");
512 * Creates and returns a {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel}
513 * instance connected to the table region containing the specified row. The row given does not
514 * actually have to exist. Whichever region would contain the row based on start and end keys will
515 * be used. Note that the {@code row} parameter is also not passed to the coprocessor handler
516 * registered for this protocol, unless the {@code row} is separately passed as an argument in the
517 * service request. The parameter here is only used to locate the region used to handle the call.
519 * The obtained {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel} instance can be
520 * used to access a published coprocessor {@link Service} using standard protobuf service
523 * <div style="background-color: #cccccc; padding: 2px"> <blockquote>
525 * CoprocessorRpcChannel channel = myTable.coprocessorService(rowkey);
526 * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
527 * MyCallRequest request = MyCallRequest.newBuilder()
530 * MyCallResponse response = service.myCall(null, request);
534 * @param row The row key used to identify the remote region location
535 * @return A CoprocessorRpcChannel instance
536 * @deprecated since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any
537 * more. Use the coprocessorService methods in {@link AsyncTable} instead.
538 * @see Connection#toAsyncConnection()
541 default CoprocessorRpcChannel
coprocessorService(byte[] row
) {
542 throw new NotImplementedException("Add an implementation!");
546 * Creates an instance of the given {@link Service} subclass for each table region spanning the
547 * range from the {@code startKey} row to {@code endKey} row (inclusive), and invokes the passed
548 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method with each
549 * {@link Service} instance.
550 * @param service the protocol buffer {@code Service} implementation to call
551 * @param startKey start region selection with region containing this row. If {@code null}, the
552 * selection will start with the first table region.
553 * @param endKey select regions up to and including the region containing this row. If
554 * {@code null}, selection will continue through the last table region.
555 * @param callable this instance's
556 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method will be
557 * invoked once per table region, using the {@link Service} instance connected to that
559 * @param <T> the {@link Service} subclass to connect to
560 * @param <R> Return type for the {@code callable} parameter's
561 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
562 * @return a map of result values keyed by region name
563 * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking
564 * interface for of a protobuf stub, so it is not possible to do it in an asynchronous
565 * way, even if now we are building the {@link Table} implementation based on the
566 * {@link AsyncTable}, which is not good. Use the coprocessorService methods in
567 * {@link AsyncTable} directly instead.
568 * @see Connection#toAsyncConnection()
571 default <T
extends Service
, R
> Map
<byte[], R
> coprocessorService(final Class
<T
> service
,
572 byte[] startKey
, byte[] endKey
, final Batch
.Call
<T
, R
> callable
)
573 throws ServiceException
, Throwable
{
574 Map
<byte[], R
> results
=
575 Collections
.synchronizedMap(new TreeMap
<byte[], R
>(Bytes
.BYTES_COMPARATOR
));
576 coprocessorService(service
, startKey
, endKey
, callable
, new Batch
.Callback
<R
>() {
578 public void update(byte[] region
, byte[] row
, R value
) {
579 if (region
!= null) {
580 results
.put(region
, value
);
588 * Creates an instance of the given {@link Service} subclass for each table region spanning the
589 * range from the {@code startKey} row to {@code endKey} row (inclusive), and invokes the passed
590 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method with each
591 * {@link Service} instance.
594 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}
595 * method will be called with the return value from each region's
596 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} invocation.
597 * @param service the protocol buffer {@code Service} implementation to call
598 * @param startKey start region selection with region containing this row. If {@code null}, the
599 * selection will start with the first table region.
600 * @param endKey select regions up to and including the region containing this row. If
601 * {@code null}, selection will continue through the last table region.
602 * @param callable this instance's
603 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method will be
604 * invoked once per table region, using the {@link Service} instance connected to that
606 * @param <T> the {@link Service} subclass to connect to
607 * @param <R> Return type for the {@code callable} parameter's
608 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
609 * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking
610 * interface for of a protobuf stub, so it is not possible to do it in an asynchronous
611 * way, even if now we are building the {@link Table} implementation based on the
612 * {@link AsyncTable}, which is not good. Use the coprocessorService methods in
613 * {@link AsyncTable} directly instead.
614 * @see Connection#toAsyncConnection()
617 default <T
extends Service
, R
> void coprocessorService(final Class
<T
> service
, byte[] startKey
,
618 byte[] endKey
, final Batch
.Call
<T
, R
> callable
, final Batch
.Callback
<R
> callback
)
619 throws ServiceException
, Throwable
{
620 throw new NotImplementedException("Add an implementation!");
624 * Creates an instance of the given {@link Service} subclass for each table region spanning the
625 * range from the {@code startKey} row to {@code endKey} row (inclusive), all the invocations to
626 * the same region server will be batched into one call. The coprocessor service is invoked
627 * according to the service instance, method name and parameters.
628 * @param methodDescriptor the descriptor for the protobuf service method to call.
629 * @param request the method call parameters
630 * @param startKey start region selection with region containing this row. If {@code null}, the
631 * selection will start with the first table region.
632 * @param endKey select regions up to and including the region containing this row. If
633 * {@code null}, selection will continue through the last table region.
634 * @param responsePrototype the proto type of the response of the method in Service.
635 * @param <R> the response type for the coprocessor Service method
636 * @return a map of result values keyed by region name
637 * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking
638 * interface for of a protobuf stub, so it is not possible to do it in an asynchronous
639 * way, even if now we are building the {@link Table} implementation based on the
640 * {@link AsyncTable}, which is not good. Use the coprocessorService methods in
641 * {@link AsyncTable} directly instead.
642 * @see Connection#toAsyncConnection()
645 default <R
extends Message
> Map
<byte[], R
> batchCoprocessorService(
646 Descriptors
.MethodDescriptor methodDescriptor
, Message request
, byte[] startKey
,
647 byte[] endKey
, R responsePrototype
) throws ServiceException
, Throwable
{
648 final Map
<byte[], R
> results
=
649 Collections
.synchronizedMap(new TreeMap
<byte[], R
>(Bytes
.BYTES_COMPARATOR
));
650 batchCoprocessorService(methodDescriptor
, request
, startKey
, endKey
, responsePrototype
,
653 public void update(byte[] region
, byte[] row
, R result
) {
654 if (region
!= null) {
655 results
.put(region
, result
);
663 * Creates an instance of the given {@link Service} subclass for each table region spanning the
664 * range from the {@code startKey} row to {@code endKey} row (inclusive), all the invocations to
665 * the same region server will be batched into one call. The coprocessor service is invoked
666 * according to the service instance, method name and parameters.
669 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}
670 * method will be called with the return value from each region's invocation.
671 * @param methodDescriptor the descriptor for the protobuf service method to call.
672 * @param request the method call parameters
673 * @param startKey start region selection with region containing this row. If {@code null}, the
674 * selection will start with the first table region.
675 * @param endKey select regions up to and including the region containing this row. If
676 * {@code null}, selection will continue through the last table region.
677 * @param responsePrototype the proto type of the response of the method in Service.
678 * @param callback callback to invoke with the response for each region
679 * @param <R> the response type for the coprocessor Service method
680 * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking
681 * interface for of a protobuf stub, so it is not possible to do it in an asynchronous
682 * way, even if now we are building the {@link Table} implementation based on the
683 * {@link AsyncTable}, which is not good. Use the coprocessorService methods in
684 * {@link AsyncTable} directly instead.
685 * @see Connection#toAsyncConnection()
688 default <R
extends Message
> void batchCoprocessorService(
689 Descriptors
.MethodDescriptor methodDescriptor
, Message request
, byte[] startKey
,
690 byte[] endKey
, R responsePrototype
, Batch
.Callback
<R
> callback
)
691 throws ServiceException
, Throwable
{
692 throw new NotImplementedException("Add an implementation!");
696 * Get timeout of each rpc request in this Table instance. It will be overridden by a more
697 * specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
698 * @see #getReadRpcTimeout(TimeUnit)
699 * @see #getWriteRpcTimeout(TimeUnit)
700 * @param unit the unit of time the timeout to be represented in
701 * @return rpc timeout in the specified time unit
703 default long getRpcTimeout(TimeUnit unit
) {
704 throw new NotImplementedException("Add an implementation!");
708 * Get timeout of each rpc read request in this Table instance.
709 * @param unit the unit of time the timeout to be represented in
710 * @return read rpc timeout in the specified time unit
712 default long getReadRpcTimeout(TimeUnit unit
) {
713 throw new NotImplementedException("Add an implementation!");
717 * Get timeout of each rpc write request in this Table instance.
718 * @param unit the unit of time the timeout to be represented in
719 * @return write rpc timeout in the specified time unit
721 default long getWriteRpcTimeout(TimeUnit unit
) {
722 throw new NotImplementedException("Add an implementation!");
726 * Get timeout of each operation in Table instance.
727 * @param unit the unit of time the timeout to be represented in
728 * @return operation rpc timeout in the specified time unit
730 default long getOperationTimeout(TimeUnit unit
) {
731 throw new NotImplementedException("Add an implementation!");