HBASE-26474 Implement connection-level attributes (addendum)
[hbase.git] / hbase-client / src / main / java / org / apache / hadoop / hbase / client / Table.java
blob53da0cfb91204bfcae4e599748fbc1c7c6906e46
1 /**
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;
25 import java.util.Map;
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;
46 /**
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
52 * @see Connection
53 * @see Admin
54 * @see RegionLocator
55 * @since 0.99.0
57 @InterfaceAudience.Public
58 public interface Table extends Closeable {
59 /**
60 * Gets the fully qualified table name instance of this table.
62 TableName getName();
64 /**
65 * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
66 * <p>
67 * The reference returned is not a copy, so any change made to it will
68 * affect this instance.
70 Configuration getConfiguration();
72 /**
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;
78 /**
79 * Gets the {@link RegionLocator} for this table.
81 RegionLocator getRegionLocator() throws IOException;
83 /**
84 * Test for the existence of columns in the table, as specified by the Get.
85 * <p>
87 * This will return true if the Get matches one or more keys, false if not.
88 * <p>
90 * This is a server-side call so it prevents any data from being transfered to
91 * the client.
93 * @param get the Get
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.
103 * <p>
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.
107 * <p>
109 * This is a server-side call so it prevents any data from being transferred to
110 * the client.
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
132 * @since 0.90.0
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.
141 * @since 0.96.0
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.
148 @Deprecated
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.
162 * @since 0.20.0
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.
179 * @since 0.90.0
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}
191 * object.
192 * Note that the passed {@link Scan}'s start row and caching properties
193 * maybe changed.
195 * @param scan A configured {@link Scan} object.
196 * @return A scanner.
197 * @throws IOException if a remote or network exception occurs.
198 * @since 0.20.0
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.
208 * @return A scanner.
209 * @throws IOException if a remote or network exception occurs.
210 * @since 0.20.0
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.
221 * @return A scanner.
222 * @throws IOException if a remote or network exception occurs.
223 * @since 0.20.0
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.
234 * @since 0.20.0
236 default void put(Put put) throws IOException {
237 put(Collections.singletonList(put));
241 * Batch puts the specified data into the table.
242 * <p>
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.
254 * @since 0.20.0
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.
265 * @since 0.20.0
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.
273 * <p>
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.
287 * @since 0.20.1
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.
301 * <p>
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:
305 * <pre>
306 * <code>
307 * table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put);
308 * </code>
309 * </pre>
311 * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it
312 * any more.
314 @Deprecated
315 default CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
316 throw new NotImplementedException("Add an implementation!");
320 * A helper class for sending checkAndMutate request.
322 * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it
323 * any more.
325 @Deprecated
326 interface CheckAndMutateBuilder {
329 * @param qualifier column qualifier to check.
331 CheckAndMutateBuilder qualifier(byte[] qualifier);
334 * @param timeRange timeRange to check
336 CheckAndMutateBuilder timeRange(TimeRange timeRange);
339 * Check for lack of column.
341 CheckAndMutateBuilder ifNotExists();
344 * Check for equality.
345 * @param value the expected value
347 default CheckAndMutateBuilder ifEquals(byte[] value) {
348 return ifMatches(CompareOperator.EQUAL, value);
352 * @param compareOp comparison operator to use
353 * @param value the expected value
355 CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);
358 * @param put data to put if check succeeds
359 * @return {@code true} if the new put was executed, {@code false} otherwise.
361 boolean thenPut(Put put) throws IOException;
364 * @param delete data to delete if check succeeds
365 * @return {@code true} if the new delete was executed, {@code false} otherwise.
367 boolean thenDelete(Delete delete) throws IOException;
370 * @param mutation mutations to perform if check succeeds
371 * @return true if the new mutation was executed, false otherwise.
373 boolean thenMutate(RowMutations mutation) throws IOException;
377 * Atomically checks if a row matches the specified filter. If it does, it adds the
378 * Put/Delete/RowMutations.
379 * <p>
380 * Use the returned {@link CheckAndMutateWithFilterBuilder} to construct your request and then
381 * execute it. This is a fluent style API, the code is like:
383 * <pre>
384 * <code>
385 * table.checkAndMutate(row, filter).thenPut(put);
386 * </code>
387 * </pre>
389 * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it
390 * any more.
392 @Deprecated
393 default CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter) {
394 throw new NotImplementedException("Add an implementation!");
398 * A helper class for sending checkAndMutate request with a filter.
400 * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it
401 * any more.
403 @Deprecated
404 interface CheckAndMutateWithFilterBuilder {
407 * @param timeRange timeRange to check
409 CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange);
412 * @param put data to put if check succeeds
413 * @return {@code true} if the new put was executed, {@code false} otherwise.
415 boolean thenPut(Put put) throws IOException;
418 * @param delete data to delete if check succeeds
419 * @return {@code true} if the new delete was executed, {@code false} otherwise.
421 boolean thenDelete(Delete delete) throws IOException;
424 * @param mutation mutations to perform if check succeeds
425 * @return true if the new mutation was executed, false otherwise.
427 boolean thenMutate(RowMutations mutation) throws IOException;
431 * checkAndMutate that atomically checks if a row matches the specified condition. If it does,
432 * it performs the specified action.
434 * @param checkAndMutate The CheckAndMutate object.
435 * @return A CheckAndMutateResult object that represents the result for the CheckAndMutate.
436 * @throws IOException if a remote or network exception occurs.
438 default CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException {
439 return checkAndMutate(Collections.singletonList(checkAndMutate)).get(0);
443 * Batch version of checkAndMutate. The specified CheckAndMutates are batched only in the sense
444 * that they are sent to a RS in one RPC, but each CheckAndMutate operation is still executed
445 * atomically (and thus, each may fail independently of others).
447 * @param checkAndMutates The list of CheckAndMutate.
448 * @return A list of CheckAndMutateResult objects that represents the result for each
449 * CheckAndMutate.
450 * @throws IOException if a remote or network exception occurs.
452 default List<CheckAndMutateResult> checkAndMutate(List<CheckAndMutate> checkAndMutates)
453 throws IOException {
454 throw new NotImplementedException("Add an implementation!");
458 * Performs multiple mutations atomically on a single row. Currently
459 * {@link Put} and {@link Delete} are supported.
461 * @param rm object that specifies the set of mutations to perform atomically
462 * @return results of Increment/Append operations
463 * @throws IOException if a remote or network exception occurs.
465 default Result mutateRow(final RowMutations rm) throws IOException {
466 throw new NotImplementedException("Add an implementation!");
470 * Appends values to one or more columns within a single row.
471 * <p>
472 * This operation guaranteed atomicity to readers. Appends are done
473 * under a single row lock, so write operations to a row are synchronized, and
474 * readers are guaranteed to see this operation fully completed.
476 * @param append object that specifies the columns and values to be appended
477 * @throws IOException e
478 * @return values of columns after the append operation (maybe null)
480 default Result append(final Append append) throws IOException {
481 throw new NotImplementedException("Add an implementation!");
485 * Increments one or more columns within a single row.
486 * <p>
487 * This operation ensures atomicity to readers. Increments are done
488 * under a single row lock, so write operations to a row are synchronized, and
489 * readers are guaranteed to see this operation fully completed.
491 * @param increment object that specifies the columns and amounts to be used
492 * for the increment operations
493 * @throws IOException e
494 * @return values of columns after the increment
496 default Result increment(final Increment increment) throws IOException {
497 throw new NotImplementedException("Add an implementation!");
501 * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
502 * <p>
503 * The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.
504 * @param row The row that contains the cell to increment.
505 * @param family The column family of the cell to increment.
506 * @param qualifier The column qualifier of the cell to increment.
507 * @param amount The amount to increment the cell with (or decrement, if the
508 * amount is negative).
509 * @return The new value, post increment.
510 * @throws IOException if a remote or network exception occurs.
512 default long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount)
513 throws IOException {
514 Increment increment = new Increment(row).addColumn(family, qualifier, amount);
515 Cell cell = increment(increment).getColumnLatestCell(family, qualifier);
516 return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
520 * Atomically increments a column value. If the column value already exists
521 * and is not a big-endian long, this could throw an exception. If the column
522 * value does not yet exist it is initialized to <code>amount</code> and
523 * written to the specified column.
525 * <p>Setting durability to {@link Durability#SKIP_WAL} means that in a fail
526 * scenario you will lose any increments that have not been flushed.
527 * @param row The row that contains the cell to increment.
528 * @param family The column family of the cell to increment.
529 * @param qualifier The column qualifier of the cell to increment.
530 * @param amount The amount to increment the cell with (or decrement, if the
531 * amount is negative).
532 * @param durability The persistence guarantee for this increment.
533 * @return The new value, post increment.
534 * @throws IOException if a remote or network exception occurs.
536 default long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
537 long amount, Durability durability) throws IOException {
538 Increment increment = new Increment(row)
539 .addColumn(family, qualifier, amount)
540 .setDurability(durability);
541 Cell cell = increment(increment).getColumnLatestCell(family, qualifier);
542 return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
546 * Releases any resources held or pending changes in internal buffers.
548 * @throws IOException if a remote or network exception occurs.
550 @Override
551 default void close() throws IOException {
552 throw new NotImplementedException("Add an implementation!");
556 * Creates and returns a {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel}
557 * instance connected to the table region containing the specified row. The row given does not
558 * actually have to exist. Whichever region would contain the row based on start and end keys will
559 * be used. Note that the {@code row} parameter is also not passed to the coprocessor handler
560 * registered for this protocol, unless the {@code row} is separately passed as an argument in the
561 * service request. The parameter here is only used to locate the region used to handle the call.
562 * <p/>
563 * The obtained {@link org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel} instance can be
564 * used to access a published coprocessor {@link Service} using standard protobuf service
565 * invocations:
566 * <p/>
567 * <div style="background-color: #cccccc; padding: 2px"> <blockquote>
568 * <pre>
569 * CoprocessorRpcChannel channel = myTable.coprocessorService(rowkey);
570 * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
571 * MyCallRequest request = MyCallRequest.newBuilder()
572 * ...
573 * .build();
574 * MyCallResponse response = service.myCall(null, request);
575 * </pre>
576 * </blockquote>
577 * </div>
578 * @param row The row key used to identify the remote region location
579 * @return A CoprocessorRpcChannel instance
580 * @deprecated since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any
581 * more. Use the coprocessorService methods in {@link AsyncTable} instead.
582 * @see Connection#toAsyncConnection()
584 @Deprecated
585 default CoprocessorRpcChannel coprocessorService(byte[] row) {
586 throw new NotImplementedException("Add an implementation!");
590 * Creates an instance of the given {@link Service} subclass for each table region spanning the
591 * range from the {@code startKey} row to {@code endKey} row (inclusive), and invokes the passed
592 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method with each
593 * {@link Service} instance.
594 * @param service the protocol buffer {@code Service} implementation to call
595 * @param startKey start region selection with region containing this row. If {@code null}, the
596 * selection will start with the first table region.
597 * @param endKey select regions up to and including the region containing this row. If
598 * {@code null}, selection will continue through the last table region.
599 * @param callable this instance's
600 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method will be
601 * invoked once per table region, using the {@link Service} instance connected to that
602 * region.
603 * @param <T> the {@link Service} subclass to connect to
604 * @param <R> Return type for the {@code callable} parameter's
605 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
606 * @return a map of result values keyed by region name
607 * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking
608 * interface for of a protobuf stub, so it is not possible to do it in an asynchronous
609 * way, even if now we are building the {@link Table} implementation based on the
610 * {@link AsyncTable}, which is not good. Use the coprocessorService methods in
611 * {@link AsyncTable} directly instead.
612 * @see Connection#toAsyncConnection()
614 @Deprecated
615 default <T extends Service, R> Map<byte[], R> coprocessorService(final Class<T> service,
616 byte[] startKey, byte[] endKey, final Batch.Call<T, R> callable)
617 throws ServiceException, Throwable {
618 Map<byte[], R> results =
619 Collections.synchronizedMap(new TreeMap<byte[], R>(Bytes.BYTES_COMPARATOR));
620 coprocessorService(service, startKey, endKey, callable, new Batch.Callback<R>() {
621 @Override
622 public void update(byte[] region, byte[] row, R value) {
623 if (region != null) {
624 results.put(region, value);
628 return results;
632 * Creates an instance of the given {@link Service} subclass for each table region spanning the
633 * range from the {@code startKey} row to {@code endKey} row (inclusive), and invokes the passed
634 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method with each
635 * {@link Service} instance.
636 * <p/>
637 * The given
638 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}
639 * method will be called with the return value from each region's
640 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} invocation.
641 * @param service the protocol buffer {@code Service} implementation to call
642 * @param startKey start region selection with region containing this row. If {@code null}, the
643 * selection will start with the first table region.
644 * @param endKey select regions up to and including the region containing this row. If
645 * {@code null}, selection will continue through the last table region.
646 * @param callable this instance's
647 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method will be
648 * invoked once per table region, using the {@link Service} instance connected to that
649 * region.
650 * @param <T> the {@link Service} subclass to connect to
651 * @param <R> Return type for the {@code callable} parameter's
652 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call} method
653 * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking
654 * interface for of a protobuf stub, so it is not possible to do it in an asynchronous
655 * way, even if now we are building the {@link Table} implementation based on the
656 * {@link AsyncTable}, which is not good. Use the coprocessorService methods in
657 * {@link AsyncTable} directly instead.
658 * @see Connection#toAsyncConnection()
660 @Deprecated
661 default <T extends Service, R> void coprocessorService(final Class<T> service, byte[] startKey,
662 byte[] endKey, final Batch.Call<T, R> callable, final Batch.Callback<R> callback)
663 throws ServiceException, Throwable {
664 throw new NotImplementedException("Add an implementation!");
668 * Creates an instance of the given {@link Service} subclass for each table region spanning the
669 * range from the {@code startKey} row to {@code endKey} row (inclusive), all the invocations to
670 * the same region server will be batched into one call. The coprocessor service is invoked
671 * according to the service instance, method name and parameters.
672 * @param methodDescriptor the descriptor for the protobuf service method to call.
673 * @param request the method call parameters
674 * @param startKey start region selection with region containing this row. If {@code null}, the
675 * selection will start with the first table region.
676 * @param endKey select regions up to and including the region containing this row. If
677 * {@code null}, selection will continue through the last table region.
678 * @param responsePrototype the proto type of the response of the method in Service.
679 * @param <R> the response type for the coprocessor Service method
680 * @return a map of result values keyed by region name
681 * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking
682 * interface for of a protobuf stub, so it is not possible to do it in an asynchronous
683 * way, even if now we are building the {@link Table} implementation based on the
684 * {@link AsyncTable}, which is not good. Use the coprocessorService methods in
685 * {@link AsyncTable} directly instead.
686 * @see Connection#toAsyncConnection()
688 @Deprecated
689 default <R extends Message> Map<byte[], R> batchCoprocessorService(
690 Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey,
691 byte[] endKey, R responsePrototype) throws ServiceException, Throwable {
692 final Map<byte[], R> results =
693 Collections.synchronizedMap(new TreeMap<byte[], R>(Bytes.BYTES_COMPARATOR));
694 batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,
695 new Callback<R>() {
696 @Override
697 public void update(byte[] region, byte[] row, R result) {
698 if (region != null) {
699 results.put(region, result);
703 return results;
707 * Creates an instance of the given {@link Service} subclass for each table region spanning the
708 * range from the {@code startKey} row to {@code endKey} row (inclusive), all the invocations to
709 * the same region server will be batched into one call. The coprocessor service is invoked
710 * according to the service instance, method name and parameters.
711 * <p/>
712 * The given
713 * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[],byte[],Object)}
714 * method will be called with the return value from each region's invocation.
715 * @param methodDescriptor the descriptor for the protobuf service method to call.
716 * @param request the method call parameters
717 * @param startKey start region selection with region containing this row. If {@code null}, the
718 * selection will start with the first table region.
719 * @param endKey select regions up to and including the region containing this row. If
720 * {@code null}, selection will continue through the last table region.
721 * @param responsePrototype the proto type of the response of the method in Service.
722 * @param callback callback to invoke with the response for each region
723 * @param <R> the response type for the coprocessor Service method
724 * @deprecated since 3.0.0, will removed in 4.0.0. The batch call here references the blocking
725 * interface for of a protobuf stub, so it is not possible to do it in an asynchronous
726 * way, even if now we are building the {@link Table} implementation based on the
727 * {@link AsyncTable}, which is not good. Use the coprocessorService methods in
728 * {@link AsyncTable} directly instead.
729 * @see Connection#toAsyncConnection()
731 @Deprecated
732 default <R extends Message> void batchCoprocessorService(
733 Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey,
734 byte[] endKey, R responsePrototype, Batch.Callback<R> callback)
735 throws ServiceException, Throwable {
736 throw new NotImplementedException("Add an implementation!");
740 * Get timeout of each rpc request in this Table instance. It will be overridden by a more
741 * specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
742 * @see #getReadRpcTimeout(TimeUnit)
743 * @see #getWriteRpcTimeout(TimeUnit)
744 * @param unit the unit of time the timeout to be represented in
745 * @return rpc timeout in the specified time unit
747 default long getRpcTimeout(TimeUnit unit) {
748 throw new NotImplementedException("Add an implementation!");
752 * Get timeout of each rpc read request in this Table instance.
753 * @param unit the unit of time the timeout to be represented in
754 * @return read rpc timeout in the specified time unit
756 default long getReadRpcTimeout(TimeUnit unit) {
757 throw new NotImplementedException("Add an implementation!");
761 * Get timeout of each rpc write request in this Table instance.
762 * @param unit the unit of time the timeout to be represented in
763 * @return write rpc timeout in the specified time unit
765 default long getWriteRpcTimeout(TimeUnit unit) {
766 throw new NotImplementedException("Add an implementation!");
770 * Get timeout of each operation in Table instance.
771 * @param unit the unit of time the timeout to be represented in
772 * @return operation rpc timeout in the specified time unit
774 default long getOperationTimeout(TimeUnit unit) {
775 throw new NotImplementedException("Add an implementation!");