001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.client;
019
020import static java.util.stream.Collectors.toList;
021import static org.apache.hadoop.hbase.client.ConnectionUtils.allOf;
022import static org.apache.hadoop.hbase.client.ConnectionUtils.toCheckExistenceOnly;
023
024import java.util.List;
025import java.util.concurrent.CompletableFuture;
026import java.util.concurrent.TimeUnit;
027import java.util.function.Function;
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.hbase.CompareOperator;
030import org.apache.hadoop.hbase.TableName;
031import org.apache.hadoop.hbase.filter.Filter;
032import org.apache.hadoop.hbase.io.TimeRange;
033import org.apache.hadoop.hbase.util.Bytes;
034import org.apache.yetus.audience.InterfaceAudience;
035
036import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
037import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
038
039/**
040 * The interface for asynchronous version of Table. Obtain an instance from a
041 * {@link AsyncConnection}.
042 * <p>
043 * The implementation is required to be thread safe.
044 * <p>
045 * Usually the implementation will not throw any exception directly. You need to get the exception
046 * from the returned {@link CompletableFuture}.
047 * @since 2.0.0
048 */
049@InterfaceAudience.Public
050public interface AsyncTable<C extends ScanResultConsumerBase> {
051
052  /**
053   * Gets the fully qualified table name instance of this table.
054   */
055  TableName getName();
056
057  /**
058   * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
059   * <p>
060   * The reference returned is not a copy, so any change made to it will affect this instance.
061   */
062  Configuration getConfiguration();
063
064  /**
065   * Gets the {@link TableDescriptor} for this table.
066   */
067  CompletableFuture<TableDescriptor> getDescriptor();
068
069  /**
070   * Gets the {@link AsyncTableRegionLocator} for this table.
071   */
072  AsyncTableRegionLocator getRegionLocator();
073
074  /**
075   * Get timeout of each rpc request in this Table instance. It will be overridden by a more
076   * specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
077   * @see #getReadRpcTimeout(TimeUnit)
078   * @see #getWriteRpcTimeout(TimeUnit)
079   * @param unit the unit of time the timeout to be represented in
080   * @return rpc timeout in the specified time unit
081   */
082  long getRpcTimeout(TimeUnit unit);
083
084  /**
085   * Get timeout of each rpc read request in this Table instance.
086   * @param unit the unit of time the timeout to be represented in
087   * @return read rpc timeout in the specified time unit
088   */
089  long getReadRpcTimeout(TimeUnit unit);
090
091  /**
092   * Get timeout of each rpc write request in this Table instance.
093   * @param unit the unit of time the timeout to be represented in
094   * @return write rpc timeout in the specified time unit
095   */
096  long getWriteRpcTimeout(TimeUnit unit);
097
098  /**
099   * 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
102   */
103  long getOperationTimeout(TimeUnit unit);
104
105  /**
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
110   */
111  long getScanTimeout(TimeUnit unit);
112
113  /**
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}.
121   */
122  default CompletableFuture<Boolean> exists(Get get) {
123    return get(toCheckExistenceOnly(get)).thenApply(r -> r.getExists());
124  }
125
126  /**
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}.
133   */
134  CompletableFuture<Result> get(Get get);
135
136  /**
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.
140   */
141  CompletableFuture<Void> put(Put put);
142
143  /**
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.
147   */
148  CompletableFuture<Void> delete(Delete delete);
149
150  /**
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}.
160   */
161  CompletableFuture<Result> append(Append append);
162
163  /**
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}.
173   */
174  CompletableFuture<Result> increment(Increment increment);
175
176  /**
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}.
186   */
187  default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
188    long amount) {
189    return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
190  }
191
192  /**
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}.
206   */
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)));
214  }
215
216  /**
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:
222   *
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");
231   *       }
232   *     });
233   * </code>
234   * </pre>
235   *
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.
238   */
239  @Deprecated
240  CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family);
241
242  /**
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.
246   */
247  @Deprecated
248  interface CheckAndMutateBuilder {
249
250    /**
251     * @param qualifier column qualifier to check.
252     */
253    CheckAndMutateBuilder qualifier(byte[] qualifier);
254
255    /**
256     * @param timeRange time range to check.
257     */
258    CheckAndMutateBuilder timeRange(TimeRange timeRange);
259
260    /**
261     * Check for lack of column.
262     */
263    CheckAndMutateBuilder ifNotExists();
264
265    /**
266     * Check for equality.
267     * @param value the expected value
268     */
269    default CheckAndMutateBuilder ifEquals(byte[] value) {
270      return ifMatches(CompareOperator.EQUAL, value);
271    }
272
273    /**
274     * @param compareOp comparison operator to use
275     * @param value the expected value
276     */
277    CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);
278
279    /**
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}.
283     */
284    CompletableFuture<Boolean> thenPut(Put put);
285
286    /**
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}.
290     */
291    CompletableFuture<Boolean> thenDelete(Delete delete);
292
293    /**
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}.
297     */
298    CompletableFuture<Boolean> thenMutate(RowMutations mutation);
299  }
300
301  /**
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:
307   *
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");
316   *       }
317   *     });
318   * </code>
319   * </pre>
320   *
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.
323   */
324  @Deprecated
325  CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter);
326
327  /**
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.
331   */
332  @Deprecated
333  interface CheckAndMutateWithFilterBuilder {
334
335    /**
336     * @param timeRange time range to check.
337     */
338    CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange);
339
340    /**
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}.
344     */
345    CompletableFuture<Boolean> thenPut(Put put);
346
347    /**
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}.
351     */
352    CompletableFuture<Boolean> thenDelete(Delete delete);
353
354    /**
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}.
358     */
359    CompletableFuture<Boolean> thenMutate(RowMutations mutation);
360  }
361
362  /**
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.
367   */
368  CompletableFuture<CheckAndMutateResult> checkAndMutate(CheckAndMutate checkAndMutate);
369
370  /**
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.
376   */
377  List<CompletableFuture<CheckAndMutateResult>>
378    checkAndMutate(List<CheckAndMutate> checkAndMutates);
379
380  /**
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.
384   */
385  default CompletableFuture<List<CheckAndMutateResult>>
386    checkAndMutateAll(List<CheckAndMutate> checkAndMutates) {
387    return allOf(checkAndMutate(checkAndMutates));
388  }
389
390  /**
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
395   */
396  CompletableFuture<Result> mutateRow(RowMutations mutation);
397
398  /**
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
404   */
405  void scan(Scan scan, C consumer);
406
407  /**
408   * Gets a scanner on the current table for the given family.
409   * @param family The column family to scan.
410   * @return A scanner.
411   */
412  default ResultScanner getScanner(byte[] family) {
413    return getScanner(new Scan().addFamily(family));
414  }
415
416  /**
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.
421   */
422  default ResultScanner getScanner(byte[] family, byte[] qualifier) {
423    return getScanner(new Scan().addColumn(family, qualifier));
424  }
425
426  /**
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.
430   */
431  ResultScanner getScanner(Scan scan);
432
433  /**
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>
439   *
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()));
448   *   }
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}.
467   */
468  CompletableFuture<List<Result>> scanAll(Scan scan);
469
470  /**
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.
479   */
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());
483  }
484
485  /**
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.
490   */
491  default CompletableFuture<List<Boolean>> existsAll(List<Get> gets) {
492    return allOf(exists(gets));
493  }
494
495  /**
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.
503   */
504  List<CompletableFuture<Result>> get(List<Get> gets);
505
506  /**
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.
511   */
512  default CompletableFuture<List<Result>> getAll(List<Get> gets) {
513    return allOf(get(gets));
514  }
515
516  /**
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.
520   */
521  List<CompletableFuture<Void>> put(List<Put> puts);
522
523  /**
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.
527   */
528  default CompletableFuture<Void> putAll(List<Put> puts) {
529    return allOf(put(puts)).thenApply(r -> null);
530  }
531
532  /**
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.
536   */
537  List<CompletableFuture<Void>> delete(List<Delete> deletes);
538
539  /**
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.
543   */
544  default CompletableFuture<Void> deleteAll(List<Delete> deletes) {
545    return allOf(delete(deletes)).thenApply(r -> null);
546  }
547
548  /**
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.
555   */
556  <T> List<CompletableFuture<T>> batch(List<? extends Row> actions);
557
558  /**
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}.
563   */
564  default <T> CompletableFuture<List<T>> batchAll(List<? extends Row> actions) {
565    return allOf(batch(actions));
566  }
567
568  /**
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:
573   *
574   * <pre>
575   * <code>
576   * channel -> xxxService.newStub(channel)
577   * </code>
578   * </pre>
579   *
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
588   */
589  <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
590    ServiceCaller<S, R> callable, byte[] row);
591
592  /**
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}.
607   *
608   * <pre>
609   * locateThenCall(byte[] row) {
610   *   locate(row).whenComplete((location, locateError) -> {
611   *     if (locateError != null) {
612   *       callback.onError(locateError);
613   *       return;
614   *     }
615   *     incPendingCall();
616   *     region = location.getRegion();
617   *     if (region.getEndKey() > endKey) {
618   *       locateEnd = true;
619   *     } else {
620   *       locateThenCall(region.getEndKey());
621   *     }
622   *     sendCall().whenComplete((resp, error) -> {
623   *       if (error != null) {
624   *         callback.onRegionError(region, error);
625   *       } else {
626   *         callback.onRegionComplete(region, resp);
627   *       }
628   *       if (locateEnd && decPendingCallAndGet() == 0) {
629   *         callback.onComplete();
630   *       }
631   *     });
632   *   });
633   * }
634   * </pre>
635   */
636  @InterfaceAudience.Public
637  interface CoprocessorCallback<R> {
638
639    /**
640     * @param region the region that the response belongs to
641     * @param resp the response of the coprocessor call
642     */
643    void onRegionComplete(RegionInfo region, R resp);
644
645    /**
646     * @param region the region that the error belongs to
647     * @param error the response error of the coprocessor call
648     */
649    void onRegionError(RegionInfo region, Throwable error);
650
651    /**
652     * Indicate that all responses of the regions have been notified by calling
653     * {@link #onRegionComplete(RegionInfo, Object)} or
654     * {@link #onRegionError(RegionInfo, Throwable)}.
655     */
656    void onComplete();
657
658    /**
659     * Indicate that we got an error which does not belong to any regions. Usually a locating error.
660     */
661    void onError(Throwable error);
662  }
663
664  /**
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.
672   */
673  interface CoprocessorServiceBuilder<S, R> {
674
675    /**
676     * @param startKey start region selection with region containing this row, inclusive.
677     */
678    default CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey) {
679      return fromRow(startKey, true);
680    }
681
682    /**
683     * @param startKey start region selection with region containing this row
684     * @param inclusive whether to include the startKey
685     */
686    CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey, boolean inclusive);
687
688    /**
689     * @param endKey select regions up to and including the region containing this row, exclusive.
690     */
691    default CoprocessorServiceBuilder<S, R> toRow(byte[] endKey) {
692      return toRow(endKey, false);
693    }
694
695    /**
696     * @param endKey select regions up to and including the region containing this row
697     * @param inclusive whether to include the endKey
698     */
699    CoprocessorServiceBuilder<S, R> toRow(byte[] endKey, boolean inclusive);
700
701    /**
702     * Execute the coprocessorService request. You can get the response through the
703     * {@link CoprocessorCallback}.
704     */
705    void execute();
706  }
707
708  /**
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:
715   *
716   * <pre>
717   * <code>
718   * channel -> xxxService.newStub(channel)
719   * </code>
720   * </pre>
721   *
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.
727   */
728  <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(Function<RpcChannel, S> stubMaker,
729    ServiceCaller<S, R> callable, CoprocessorCallback<R> callback);
730}