HBASE-26688 Threads shared EMPTY_RESULT may lead to unexpected client job down. ...
[hbase.git] / hbase-client / src / main / java / org / apache / hadoop / hbase / client / AsyncTable.java
blobdf25351e1017acfdf7c45dc9452b5fed40dae476
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.client;
20 import static java.util.stream.Collectors.toList;
21 import static org.apache.hadoop.hbase.client.ConnectionUtils.allOf;
22 import static org.apache.hadoop.hbase.client.ConnectionUtils.toCheckExistenceOnly;
24 import java.util.List;
25 import java.util.concurrent.CompletableFuture;
26 import java.util.concurrent.TimeUnit;
27 import java.util.function.Function;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.CompareOperator;
30 import org.apache.hadoop.hbase.TableName;
31 import org.apache.hadoop.hbase.filter.Filter;
32 import org.apache.hadoop.hbase.io.TimeRange;
33 import org.apache.hadoop.hbase.util.Bytes;
34 import org.apache.yetus.audience.InterfaceAudience;
36 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
37 import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
39 /**
40 * The interface for asynchronous version of Table. Obtain an instance from a
41 * {@link AsyncConnection}.
42 * <p>
43 * The implementation is required to be thread safe.
44 * <p>
45 * Usually the implementation will not throw any exception directly. You need to get the exception
46 * from the returned {@link CompletableFuture}.
47 * @since 2.0.0
49 @InterfaceAudience.Public
50 public interface AsyncTable<C extends ScanResultConsumerBase> {
52 /**
53 * Gets the fully qualified table name instance of this table.
55 TableName getName();
57 /**
58 * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
59 * <p>
60 * The reference returned is not a copy, so any change made to it will affect this instance.
62 Configuration getConfiguration();
64 /**
65 * Gets the {@link TableDescriptor} for this table.
67 CompletableFuture<TableDescriptor> getDescriptor();
69 /**
70 * Gets the {@link AsyncTableRegionLocator} for this table.
72 AsyncTableRegionLocator getRegionLocator();
74 /**
75 * Get timeout of each rpc request in this Table instance. It will be overridden by a more
76 * specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
77 * @see #getReadRpcTimeout(TimeUnit)
78 * @see #getWriteRpcTimeout(TimeUnit)
79 * @param unit the unit of time the timeout to be represented in
80 * @return rpc timeout in the specified time unit
82 long getRpcTimeout(TimeUnit unit);
84 /**
85 * Get timeout of each rpc read request in this Table instance.
86 * @param unit the unit of time the timeout to be represented in
87 * @return read rpc timeout in the specified time unit
89 long getReadRpcTimeout(TimeUnit unit);
91 /**
92 * Get timeout of each rpc write request in this Table instance.
93 * @param unit the unit of time the timeout to be represented in
94 * @return write rpc timeout in the specified time unit
96 long getWriteRpcTimeout(TimeUnit unit);
98 /**
99 * Get timeout of each operation in Table instance.
100 * @param unit the unit of time the timeout to be represented in
101 * @return operation rpc timeout in the specified time unit
103 long getOperationTimeout(TimeUnit unit);
106 * Get the timeout of a single operation in a scan. It works like operation timeout for other
107 * operations.
108 * @param unit the unit of time the timeout to be represented in
109 * @return scan rpc timeout in the specified time unit
111 long getScanTimeout(TimeUnit unit);
114 * Test for the existence of columns in the table, as specified by the Get.
115 * <p>
116 * This will return true if the Get matches one or more keys, false if not.
117 * <p>
118 * This is a server-side call so it prevents any data from being transfered to the client.
119 * @return true if the specified Get matches one or more keys, false if not. The return value will
120 * be wrapped by a {@link CompletableFuture}.
122 default CompletableFuture<Boolean> exists(Get get) {
123 return get(toCheckExistenceOnly(get)).thenApply(r -> r.getExists());
127 * Extracts certain cells from a given row.
128 * @param get The object that specifies what data to fetch and from which row.
129 * @return The data coming from the specified row, if it exists. If the row specified doesn't
130 * exist, the {@link Result} instance returned won't contain any
131 * {@link org.apache.hadoop.hbase.KeyValue}, as indicated by {@link Result#isEmpty()}. The
132 * return value will be wrapped by a {@link CompletableFuture}.
134 CompletableFuture<Result> get(Get get);
137 * Puts some data to the table.
138 * @param put The data to put.
139 * @return A {@link CompletableFuture} that always returns null when complete normally.
141 CompletableFuture<Void> put(Put put);
144 * Deletes the specified cells/row.
145 * @param delete The object that specifies what to delete.
146 * @return A {@link CompletableFuture} that always returns null when complete normally.
148 CompletableFuture<Void> delete(Delete delete);
151 * Appends values to one or more columns within a single row.
152 * <p>
153 * This operation does not appear atomic to readers. Appends are done under a single row lock, so
154 * write operations to a row are synchronized, but readers do not take row locks so get and scan
155 * operations can see this operation partially completed.
156 * @param append object that specifies the columns and amounts to be used for the increment
157 * operations
158 * @return values of columns after the append operation (maybe null). The return value will be
159 * wrapped by a {@link CompletableFuture}.
161 CompletableFuture<Result> append(Append append);
164 * Increments one or more columns within a single row.
165 * <p>
166 * This operation does not appear atomic to readers. Increments are done under a single row lock,
167 * so write operations to a row are synchronized, but readers do not take row locks so get and
168 * scan operations can see this operation partially completed.
169 * @param increment object that specifies the columns and amounts to be used for the increment
170 * operations
171 * @return values of columns after the increment. The return value will be wrapped by a
172 * {@link CompletableFuture}.
174 CompletableFuture<Result> increment(Increment increment);
177 * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
178 * <p>
179 * The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.
180 * @param row The row that contains the cell to increment.
181 * @param family The column family of the cell to increment.
182 * @param qualifier The column qualifier of the cell to increment.
183 * @param amount The amount to increment the cell with (or decrement, if the amount is negative).
184 * @return The new value, post increment. The return value will be wrapped by a
185 * {@link CompletableFuture}.
187 default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
188 long amount) {
189 return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
193 * Atomically increments a column value. If the column value already exists and is not a
194 * big-endian long, this could throw an exception. If the column value does not yet exist it is
195 * initialized to <code>amount</code> and written to the specified column.
196 * <p>
197 * Setting durability to {@link Durability#SKIP_WAL} means that in a fail scenario you will lose
198 * any increments that have not been flushed.
199 * @param row The row that contains the cell to increment.
200 * @param family The column family of the cell to increment.
201 * @param qualifier The column qualifier of the cell to increment.
202 * @param amount The amount to increment the cell with (or decrement, if the amount is negative).
203 * @param durability The persistence guarantee for this increment.
204 * @return The new value, post increment. The return value will be wrapped by a
205 * {@link CompletableFuture}.
207 default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
208 long amount, Durability durability) {
209 Preconditions.checkNotNull(row, "row is null");
210 Preconditions.checkNotNull(family, "family is null");
211 return increment(
212 new Increment(row).addColumn(family, qualifier, amount).setDurability(durability))
213 .thenApply(r -> Bytes.toLong(r.getValue(family, qualifier)));
217 * Atomically checks if a row/family/qualifier value matches the expected value. If it does, it
218 * adds the Put/Delete/RowMutations.
219 * <p>
220 * Use the returned {@link CheckAndMutateBuilder} to construct your request and then execute it.
221 * This is a fluent style API, the code is like:
223 * <pre>
224 * <code>
225 * table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put)
226 * .thenAccept(succ -> {
227 * if (succ) {
228 * System.out.println("Check and put succeeded");
229 * } else {
230 * System.out.println("Check and put failed");
232 * });
233 * </code>
234 * </pre>
236 * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it
237 * any more.
239 @Deprecated
240 CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family);
243 * A helper class for sending checkAndMutate request.
244 * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it
245 * any more.
247 @Deprecated
248 interface CheckAndMutateBuilder {
251 * @param qualifier column qualifier to check.
253 CheckAndMutateBuilder qualifier(byte[] qualifier);
256 * @param timeRange time range to check.
258 CheckAndMutateBuilder timeRange(TimeRange timeRange);
261 * Check for lack of column.
263 CheckAndMutateBuilder ifNotExists();
266 * Check for equality.
267 * @param value the expected value
269 default CheckAndMutateBuilder ifEquals(byte[] value) {
270 return ifMatches(CompareOperator.EQUAL, value);
274 * @param compareOp comparison operator to use
275 * @param value the expected value
277 CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);
280 * @param put data to put if check succeeds
281 * @return {@code true} if the new put was executed, {@code false} otherwise. The return value
282 * will be wrapped by a {@link CompletableFuture}.
284 CompletableFuture<Boolean> thenPut(Put put);
287 * @param delete data to delete if check succeeds
288 * @return {@code true} if the new delete was executed, {@code false} otherwise. The return
289 * value will be wrapped by a {@link CompletableFuture}.
291 CompletableFuture<Boolean> thenDelete(Delete delete);
294 * @param mutation mutations to perform if check succeeds
295 * @return true if the new mutation was executed, false otherwise. The return value will be
296 * wrapped by a {@link CompletableFuture}.
298 CompletableFuture<Boolean> thenMutate(RowMutations mutation);
302 * Atomically checks if a row matches the specified filter. If it does, it adds the
303 * Put/Delete/RowMutations.
304 * <p>
305 * Use the returned {@link CheckAndMutateWithFilterBuilder} to construct your request and then
306 * execute it. This is a fluent style API, the code is like:
308 * <pre>
309 * <code>
310 * table.checkAndMutate(row, filter).thenPut(put)
311 * .thenAccept(succ -> {
312 * if (succ) {
313 * System.out.println("Check and put succeeded");
314 * } else {
315 * System.out.println("Check and put failed");
317 * });
318 * </code>
319 * </pre>
321 * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it
322 * any more.
324 @Deprecated
325 CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter);
328 * A helper class for sending checkAndMutate request with a filter.
329 * @deprecated Since 3.0.0, will be removed in 4.0.0. For internal test use only, do not use it
330 * any more.
332 @Deprecated
333 interface CheckAndMutateWithFilterBuilder {
336 * @param timeRange time range to check.
338 CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange);
341 * @param put data to put if check succeeds
342 * @return {@code true} if the new put was executed, {@code false} otherwise. The return value
343 * will be wrapped by a {@link CompletableFuture}.
345 CompletableFuture<Boolean> thenPut(Put put);
348 * @param delete data to delete if check succeeds
349 * @return {@code true} if the new delete was executed, {@code false} otherwise. The return
350 * value will be wrapped by a {@link CompletableFuture}.
352 CompletableFuture<Boolean> thenDelete(Delete delete);
355 * @param mutation mutations to perform if check succeeds
356 * @return true if the new mutation was executed, false otherwise. The return value will be
357 * wrapped by a {@link CompletableFuture}.
359 CompletableFuture<Boolean> thenMutate(RowMutations mutation);
363 * checkAndMutate that atomically checks if a row matches the specified condition. If it does, it
364 * performs the specified action.
365 * @param checkAndMutate The CheckAndMutate object.
366 * @return A {@link CompletableFuture}s that represent the result for the CheckAndMutate.
368 CompletableFuture<CheckAndMutateResult> checkAndMutate(CheckAndMutate checkAndMutate);
371 * Batch version of checkAndMutate. The specified CheckAndMutates are batched only in the sense
372 * that they are sent to a RS in one RPC, but each CheckAndMutate operation is still executed
373 * atomically (and thus, each may fail independently of others).
374 * @param checkAndMutates The list of CheckAndMutate.
375 * @return A list of {@link CompletableFuture}s that represent the result for each CheckAndMutate.
377 List<CompletableFuture<CheckAndMutateResult>>
378 checkAndMutate(List<CheckAndMutate> checkAndMutates);
381 * A simple version of batch checkAndMutate. It will fail if there are any failures.
382 * @param checkAndMutates The list of rows to apply.
383 * @return A {@link CompletableFuture} that wrapper the result list.
385 default CompletableFuture<List<CheckAndMutateResult>>
386 checkAndMutateAll(List<CheckAndMutate> checkAndMutates) {
387 return allOf(checkAndMutate(checkAndMutates));
391 * Performs multiple mutations atomically on a single row. Currently {@link Put} and
392 * {@link Delete} are supported.
393 * @param mutation object that specifies the set of mutations to perform atomically
394 * @return A {@link CompletableFuture} that returns results of Increment/Append operations
396 CompletableFuture<Result> mutateRow(RowMutations mutation);
399 * The scan API uses the observer pattern.
400 * @param scan A configured {@link Scan} object.
401 * @param consumer the consumer used to receive results.
402 * @see ScanResultConsumer
403 * @see AdvancedScanResultConsumer
405 void scan(Scan scan, C consumer);
408 * Gets a scanner on the current table for the given family.
409 * @param family The column family to scan.
410 * @return A scanner.
412 default ResultScanner getScanner(byte[] family) {
413 return getScanner(new Scan().addFamily(family));
417 * Gets a scanner on the current table for the given family and qualifier.
418 * @param family The column family to scan.
419 * @param qualifier The column qualifier to scan.
420 * @return A scanner.
422 default ResultScanner getScanner(byte[] family, byte[] qualifier) {
423 return getScanner(new Scan().addColumn(family, qualifier));
427 * Returns a scanner on the current table as specified by the {@link Scan} object.
428 * @param scan A configured {@link Scan} object.
429 * @return A scanner.
431 ResultScanner getScanner(Scan scan);
434 * Return all the results that match the given scan object.
435 * <p>
436 * Notice that usually you should use this method with a {@link Scan} object that has limit set.
437 * For example, if you want to get the closest row after a given row, you could do this:
438 * <p>
440 * <pre>
441 * <code>
442 * table.scanAll(new Scan().withStartRow(row, false).setLimit(1)).thenAccept(results -> {
443 * if (results.isEmpty()) {
444 * System.out.println("No row after " + Bytes.toStringBinary(row));
445 * } else {
446 * System.out.println("The closest row after " + Bytes.toStringBinary(row) + " is "
447 * + Bytes.toStringBinary(results.stream().findFirst().get().getRow()));
449 * });
450 * </code>
451 * </pre>
452 * <p>
453 * If your result set is very large, you should use other scan method to get a scanner or use
454 * callback to process the results. They will do chunking to prevent OOM. The scanAll method will
455 * fetch all the results and store them in a List and then return the list to you.
456 * <p>
457 * The scan metrics will be collected background if you enable it but you have no way to get it.
458 * Usually you can get scan metrics from {@code ResultScanner}, or through
459 * {@code ScanResultConsumer.onScanMetricsCreated} but this method only returns a list of results.
460 * So if you really care about scan metrics then you'd better use other scan methods which return
461 * a {@code ResultScanner} or let you pass in a {@code ScanResultConsumer}. There is no
462 * performance difference between these scan methods so do not worry.
463 * @param scan A configured {@link Scan} object. So if you use this method to fetch a really large
464 * result set, it is likely to cause OOM.
465 * @return The results of this small scan operation. The return value will be wrapped by a
466 * {@link CompletableFuture}.
468 CompletableFuture<List<Result>> scanAll(Scan scan);
471 * Test for the existence of columns in the table, as specified by the Gets.
472 * <p>
473 * This will return a list of booleans. Each value will be true if the related Get matches one or
474 * more keys, false if not.
475 * <p>
476 * This is a server-side call so it prevents any data from being transferred to the client.
477 * @param gets the Gets
478 * @return A list of {@link CompletableFuture}s that represent the existence for each get.
480 default List<CompletableFuture<Boolean>> exists(List<Get> gets) {
481 return get(toCheckExistenceOnly(gets)).stream()
482 .<CompletableFuture<Boolean>> map(f -> f.thenApply(r -> r.getExists())).collect(toList());
486 * A simple version for batch exists. It will fail if there are any failures and you will get the
487 * whole result boolean list at once if the operation is succeeded.
488 * @param gets the Gets
489 * @return A {@link CompletableFuture} that wrapper the result boolean list.
491 default CompletableFuture<List<Boolean>> existsAll(List<Get> gets) {
492 return allOf(exists(gets));
496 * Extracts certain cells from the given rows, in batch.
497 * <p>
498 * Notice that you may not get all the results with this function, which means some of the
499 * returned {@link CompletableFuture}s may succeed while some of the other returned
500 * {@link CompletableFuture}s may fail.
501 * @param gets The objects that specify what data to fetch and from which rows.
502 * @return A list of {@link CompletableFuture}s that represent the result for each get.
504 List<CompletableFuture<Result>> get(List<Get> gets);
507 * A simple version for batch get. It will fail if there are any failures and you will get the
508 * whole result list at once if the operation is succeeded.
509 * @param gets The objects that specify what data to fetch and from which rows.
510 * @return A {@link CompletableFuture} that wrapper the result list.
512 default CompletableFuture<List<Result>> getAll(List<Get> gets) {
513 return allOf(get(gets));
517 * Puts some data in the table, in batch.
518 * @param puts The list of mutations to apply.
519 * @return A list of {@link CompletableFuture}s that represent the result for each put.
521 List<CompletableFuture<Void>> put(List<Put> puts);
524 * A simple version of batch put. It will fail if there are any failures.
525 * @param puts The list of mutations to apply.
526 * @return A {@link CompletableFuture} that always returns null when complete normally.
528 default CompletableFuture<Void> putAll(List<Put> puts) {
529 return allOf(put(puts)).thenApply(r -> null);
533 * Deletes the specified cells/rows in bulk.
534 * @param deletes list of things to delete.
535 * @return A list of {@link CompletableFuture}s that represent the result for each delete.
537 List<CompletableFuture<Void>> delete(List<Delete> deletes);
540 * A simple version of batch delete. It will fail if there are any failures.
541 * @param deletes list of things to delete.
542 * @return A {@link CompletableFuture} that always returns null when complete normally.
544 default CompletableFuture<Void> deleteAll(List<Delete> deletes) {
545 return allOf(delete(deletes)).thenApply(r -> null);
549 * Method that does a batch call on Deletes, Gets, Puts, Increments, Appends and RowMutations. The
550 * ordering of execution of the actions is not defined. Meaning if you do a Put and a Get in the
551 * same {@link #batch} call, you will not necessarily be guaranteed that the Get returns what the
552 * Put had put.
553 * @param actions list of Get, Put, Delete, Increment, Append, and RowMutations objects
554 * @return A list of {@link CompletableFuture}s that represent the result for each action.
556 <T> List<CompletableFuture<T>> batch(List<? extends Row> actions);
559 * A simple version of batch. It will fail if there are any failures and you will get the whole
560 * result list at once if the operation is succeeded.
561 * @param actions list of Get, Put, Delete, Increment, Append and RowMutations objects
562 * @return A list of the result for the actions. Wrapped by a {@link CompletableFuture}.
564 default <T> CompletableFuture<List<T>> batchAll(List<? extends Row> actions) {
565 return allOf(batch(actions));
569 * Execute the given coprocessor call on the region which contains the given {@code row}.
570 * <p>
571 * The {@code stubMaker} is just a delegation to the {@code newStub} call. Usually it is only a
572 * one line lambda expression, like:
574 * <pre>
575 * <code>
576 * channel -> xxxService.newStub(channel)
577 * </code>
578 * </pre>
580 * @param stubMaker a delegation to the actual {@code newStub} call.
581 * @param callable a delegation to the actual protobuf rpc call. See the comment of
582 * {@link ServiceCaller} for more details.
583 * @param row The row key used to identify the remote region location
584 * @param <S> the type of the asynchronous stub
585 * @param <R> the type of the return value
586 * @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
587 * @see ServiceCaller
589 <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
590 ServiceCaller<S, R> callable, byte[] row);
593 * The callback when we want to execute a coprocessor call on a range of regions.
594 * <p>
595 * As the locating itself also takes some time, the implementation may want to send rpc calls on
596 * the fly, which means we do not know how many regions we have when we get the return value of
597 * the rpc calls, so we need an {@link #onComplete()} which is used to tell you that we have
598 * passed all the return values to you(through the {@link #onRegionComplete(RegionInfo, Object)}
599 * or {@link #onRegionError(RegionInfo, Throwable)} calls), i.e, there will be no
600 * {@link #onRegionComplete(RegionInfo, Object)} or {@link #onRegionError(RegionInfo, Throwable)}
601 * calls in the future.
602 * <p>
603 * Here is a pseudo code to describe a typical implementation of a range coprocessor service
604 * method to help you better understand how the {@link CoprocessorCallback} will be called. The
605 * {@code callback} in the pseudo code is our {@link CoprocessorCallback}. And notice that the
606 * {@code whenComplete} is {@code CompletableFuture.whenComplete}.
608 * <pre>
609 * locateThenCall(byte[] row) {
610 * locate(row).whenComplete((location, locateError) -> {
611 * if (locateError != null) {
612 * callback.onError(locateError);
613 * return;
615 * incPendingCall();
616 * region = location.getRegion();
617 * if (region.getEndKey() > endKey) {
618 * locateEnd = true;
619 * } else {
620 * locateThenCall(region.getEndKey());
622 * sendCall().whenComplete((resp, error) -> {
623 * if (error != null) {
624 * callback.onRegionError(region, error);
625 * } else {
626 * callback.onRegionComplete(region, resp);
628 * if (locateEnd && decPendingCallAndGet() == 0) {
629 * callback.onComplete();
631 * });
632 * });
634 * </pre>
636 @InterfaceAudience.Public
637 interface CoprocessorCallback<R> {
640 * @param region the region that the response belongs to
641 * @param resp the response of the coprocessor call
643 void onRegionComplete(RegionInfo region, R resp);
646 * @param region the region that the error belongs to
647 * @param error the response error of the coprocessor call
649 void onRegionError(RegionInfo region, Throwable error);
652 * Indicate that all responses of the regions have been notified by calling
653 * {@link #onRegionComplete(RegionInfo, Object)} or
654 * {@link #onRegionError(RegionInfo, Throwable)}.
656 void onComplete();
659 * Indicate that we got an error which does not belong to any regions. Usually a locating error.
661 void onError(Throwable error);
665 * Helper class for sending coprocessorService request that executes a coprocessor call on regions
666 * which are covered by a range.
667 * <p>
668 * If {@code fromRow} is not specified the selection will start with the first table region. If
669 * {@code toRow} is not specified the selection will continue through the last table region.
670 * @param <S> the type of the protobuf Service you want to call.
671 * @param <R> the type of the return value.
673 interface CoprocessorServiceBuilder<S, R> {
676 * @param startKey start region selection with region containing this row, inclusive.
678 default CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey) {
679 return fromRow(startKey, true);
683 * @param startKey start region selection with region containing this row
684 * @param inclusive whether to include the startKey
686 CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey, boolean inclusive);
689 * @param endKey select regions up to and including the region containing this row, exclusive.
691 default CoprocessorServiceBuilder<S, R> toRow(byte[] endKey) {
692 return toRow(endKey, false);
696 * @param endKey select regions up to and including the region containing this row
697 * @param inclusive whether to include the endKey
699 CoprocessorServiceBuilder<S, R> toRow(byte[] endKey, boolean inclusive);
702 * Execute the coprocessorService request. You can get the response through the
703 * {@link CoprocessorCallback}.
705 void execute();
709 * Execute a coprocessor call on the regions which are covered by a range.
710 * <p>
711 * Use the returned {@link CoprocessorServiceBuilder} construct your request and then execute it.
712 * <p>
713 * The {@code stubMaker} is just a delegation to the {@code xxxService.newStub} call. Usually it
714 * is only a one line lambda expression, like:
716 * <pre>
717 * <code>
718 * channel -> xxxService.newStub(channel)
719 * </code>
720 * </pre>
722 * @param stubMaker a delegation to the actual {@code newStub} call.
723 * @param callable a delegation to the actual protobuf rpc call. See the comment of
724 * {@link ServiceCaller} for more details.
725 * @param callback callback to get the response. See the comment of {@link CoprocessorCallback}
726 * for more details.
728 <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(Function<RpcChannel, S> stubMaker,
729 ServiceCaller<S, R> callable, CoprocessorCallback<R> callback);