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.toCheckExistenceOnly;
022import static org.apache.hadoop.hbase.util.FutureUtils.allOf;
023
024import com.google.protobuf.RpcChannel;
025import java.time.Duration;
026import java.util.Collections;
027import java.util.List;
028import java.util.Map;
029import java.util.concurrent.CompletableFuture;
030import java.util.concurrent.TimeUnit;
031import java.util.function.Function;
032import org.apache.hadoop.conf.Configuration;
033import org.apache.hadoop.hbase.CompareOperator;
034import org.apache.hadoop.hbase.TableName;
035import org.apache.hadoop.hbase.filter.Filter;
036import org.apache.hadoop.hbase.io.TimeRange;
037import org.apache.hadoop.hbase.util.Bytes;
038import org.apache.yetus.audience.InterfaceAudience;
039import org.slf4j.LoggerFactory;
040
041import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
042
043/**
044 * The interface for asynchronous version of Table. Obtain an instance from a
045 * {@link AsyncConnection}.
046 * <p>
047 * The implementation is required to be thread safe.
048 * <p>
049 * Usually the implementation will not throw any exception directly. You need to get the exception
050 * from the returned {@link CompletableFuture}.
051 * @since 2.0.0
052 */
053@InterfaceAudience.Public
054public interface AsyncTable<C extends ScanResultConsumerBase> {
055
056  /**
057   * Gets the fully qualified table name instance of this table.
058   */
059  TableName getName();
060
061  /**
062   * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
063   * <p>
064   * The reference returned is not a copy, so any change made to it will affect this instance.
065   */
066  Configuration getConfiguration();
067
068  /**
069   * Gets the {@link TableDescriptor} for this table.
070   */
071  CompletableFuture<TableDescriptor> getDescriptor();
072
073  /**
074   * Gets the {@link AsyncTableRegionLocator} for this table.
075   */
076  AsyncTableRegionLocator getRegionLocator();
077
078  /**
079   * Get timeout of each rpc request in this Table instance. It will be overridden by a more
080   * specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
081   * @see #getReadRpcTimeout(TimeUnit)
082   * @see #getWriteRpcTimeout(TimeUnit)
083   * @param unit the unit of time the timeout to be represented in
084   * @return rpc timeout in the specified time unit
085   */
086  long getRpcTimeout(TimeUnit unit);
087
088  /**
089   * Get timeout of each rpc read request in this Table instance.
090   * @param unit the unit of time the timeout to be represented in
091   * @return read rpc timeout in the specified time unit
092   */
093  long getReadRpcTimeout(TimeUnit unit);
094
095  /**
096   * Get timeout of each rpc write request in this Table instance.
097   * @param unit the unit of time the timeout to be represented in
098   * @return write rpc timeout in the specified time unit
099   */
100  long getWriteRpcTimeout(TimeUnit unit);
101
102  /**
103   * Get timeout of each operation in Table instance.
104   * @param unit the unit of time the timeout to be represented in
105   * @return operation rpc timeout in the specified time unit
106   */
107  long getOperationTimeout(TimeUnit unit);
108
109  /**
110   * Get the timeout of a single operation in a scan. It works like operation timeout for other
111   * operations.
112   * @param unit the unit of time the timeout to be represented in
113   * @return scan rpc timeout in the specified time unit
114   */
115  long getScanTimeout(TimeUnit unit);
116
117  /**
118   * Get the map of request attributes
119   * @return a map of request attributes supplied by the client
120   */
121  default Map<String, byte[]> getRequestAttributes() {
122    return Collections.emptyMap();
123  }
124
125  /**
126   * Test for the existence of columns in the table, as specified by the Get.
127   * <p>
128   * This will return true if the Get matches one or more keys, false if not.
129   * <p>
130   * This is a server-side call so it prevents any data from being transfered to the client.
131   * @return true if the specified Get matches one or more keys, false if not. The return value will
132   *         be wrapped by a {@link CompletableFuture}.
133   */
134  default CompletableFuture<Boolean> exists(Get get) {
135    return get(toCheckExistenceOnly(get)).thenApply(r -> r.getExists());
136  }
137
138  /**
139   * Extracts certain cells from a given row.
140   * @param get The object that specifies what data to fetch and from which row.
141   * @return The data coming from the specified row, if it exists. If the row specified doesn't
142   *         exist, the {@link Result} instance returned won't contain any
143   *         {@link org.apache.hadoop.hbase.KeyValue}, as indicated by {@link Result#isEmpty()}. The
144   *         return value will be wrapped by a {@link CompletableFuture}.
145   */
146  CompletableFuture<Result> get(Get get);
147
148  /**
149   * Puts some data to the table.
150   * @param put The data to put.
151   * @return A {@link CompletableFuture} that always returns null when complete normally.
152   */
153  CompletableFuture<Void> put(Put put);
154
155  /**
156   * Deletes the specified cells/row.
157   * @param delete The object that specifies what to delete.
158   * @return A {@link CompletableFuture} that always returns null when complete normally.
159   */
160  CompletableFuture<Void> delete(Delete delete);
161
162  /**
163   * Appends values to one or more columns within a single row.
164   * <p>
165   * This operation does not appear atomic to readers. Appends are done under a single row lock, so
166   * write operations to a row are synchronized, but readers do not take row locks so get and scan
167   * operations can see this operation partially completed.
168   * @param append object that specifies the columns and amounts to be used for the increment
169   *               operations
170   * @return values of columns after the append operation (maybe null). The return value will be
171   *         wrapped by a {@link CompletableFuture}.
172   */
173  CompletableFuture<Result> append(Append append);
174
175  /**
176   * Increments one or more columns within a single row.
177   * <p>
178   * This operation does not appear atomic to readers. Increments are done under a single row lock,
179   * so write operations to a row are synchronized, but readers do not take row locks so get and
180   * scan operations can see this operation partially completed.
181   * @param increment object that specifies the columns and amounts to be used for the increment
182   *                  operations
183   * @return values of columns after the increment. The return value will be wrapped by a
184   *         {@link CompletableFuture}.
185   */
186  CompletableFuture<Result> increment(Increment increment);
187
188  /**
189   * See {@link #incrementColumnValue(byte[], byte[], byte[], long, Durability)}
190   * <p>
191   * The {@link Durability} is defaulted to {@link Durability#SYNC_WAL}.
192   * @param row       The row that contains the cell to increment.
193   * @param family    The column family of the cell to increment.
194   * @param qualifier The column qualifier of the cell to increment.
195   * @param amount    The amount to increment the cell with (or decrement, if the amount is
196   *                  negative).
197   * @return The new value, post increment. The return value will be wrapped by a
198   *         {@link CompletableFuture}.
199   */
200  default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
201    long amount) {
202    return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
203  }
204
205  /**
206   * Atomically increments a column value. If the column value already exists and is not a
207   * big-endian long, this could throw an exception. If the column value does not yet exist it is
208   * initialized to <code>amount</code> and written to the specified column.
209   * <p>
210   * Setting durability to {@link Durability#SKIP_WAL} means that in a fail scenario you will lose
211   * any increments that have not been flushed.
212   * @param row        The row that contains the cell to increment.
213   * @param family     The column family of the cell to increment.
214   * @param qualifier  The column qualifier of the cell to increment.
215   * @param amount     The amount to increment the cell with (or decrement, if the amount is
216   *                   negative).
217   * @param durability The persistence guarantee for this increment.
218   * @return The new value, post increment. The return value will be wrapped by a
219   *         {@link CompletableFuture}.
220   */
221  default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
222    long amount, Durability durability) {
223    Preconditions.checkNotNull(row, "row is null");
224    Preconditions.checkNotNull(family, "family is null");
225    return increment(
226      new Increment(row).addColumn(family, qualifier, amount).setDurability(durability))
227      .thenApply(r -> Bytes.toLong(r.getValue(family, qualifier)));
228  }
229
230  /**
231   * Atomically checks if a row/family/qualifier value matches the expected value. If it does, it
232   * adds the Put/Delete/RowMutations.
233   * <p>
234   * Use the returned {@link CheckAndMutateBuilder} to construct your request and then execute it.
235   * This is a fluent style API, the code is like:
236   *
237   * <pre>
238   * table.checkAndMutate(row, family).qualifier(qualifier).ifNotExists().thenPut(put)
239   *   .thenAccept(succ -&gt; {
240   *     if (succ) {
241   *       System.out.println("Check and put succeeded");
242   *     } else {
243   *       System.out.println("Check and put failed");
244   *     }
245   *   });
246   * </pre>
247   *
248   * @deprecated Since 2.4.0, will be removed in 4.0.0. For internal test use only, do not use it
249   *             any more.
250   */
251  @Deprecated
252  CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family);
253
254  /**
255   * A helper class for sending checkAndMutate request.
256   * @deprecated Since 2.4.0, will be removed in 4.0.0. For internal test use only, do not use it
257   *             any more.
258   */
259  @Deprecated
260  interface CheckAndMutateBuilder {
261
262    /**
263     * Match a qualifier.
264     * @param qualifier column qualifier to check.
265     */
266    CheckAndMutateBuilder qualifier(byte[] qualifier);
267
268    /**
269     * Match a timerange.
270     * @param timeRange time range to check.
271     */
272    CheckAndMutateBuilder timeRange(TimeRange timeRange);
273
274    /**
275     * Check for lack of column.
276     */
277    CheckAndMutateBuilder ifNotExists();
278
279    /**
280     * Check for equality.
281     * @param value the expected value
282     */
283    default CheckAndMutateBuilder ifEquals(byte[] value) {
284      return ifMatches(CompareOperator.EQUAL, value);
285    }
286
287    /**
288     * Compare a value
289     * @param compareOp comparison operator to use
290     * @param value     the expected value
291     */
292    CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value);
293
294    /**
295     * Specify a Put to commit if the check succeeds.
296     * @param put data to put if check succeeds
297     * @return {@code true} if the new put was executed, {@code false} otherwise. The return value
298     *         will be wrapped by a {@link CompletableFuture}.
299     */
300    CompletableFuture<Boolean> thenPut(Put put);
301
302    /**
303     * Specify a Delete to commit if the check succeeds.
304     * @param delete data to delete if check succeeds
305     * @return {@code true} if the new delete was executed, {@code false} otherwise. The return
306     *         value will be wrapped by a {@link CompletableFuture}.
307     */
308    CompletableFuture<Boolean> thenDelete(Delete delete);
309
310    /**
311     * Specify a RowMutations to commit if the check succeeds.
312     * @param mutation mutations to perform if check succeeds
313     * @return true if the new mutation was executed, false otherwise. The return value will be
314     *         wrapped by a {@link CompletableFuture}.
315     */
316    CompletableFuture<Boolean> thenMutate(RowMutations mutation);
317  }
318
319  /**
320   * Atomically checks if a row matches the specified filter. If it does, it adds the
321   * Put/Delete/RowMutations.
322   * <p>
323   * Use the returned {@link CheckAndMutateWithFilterBuilder} to construct your request and then
324   * execute it. This is a fluent style API, the code is like:
325   *
326   * <pre>
327   * table.checkAndMutate(row, filter).thenPut(put).thenAccept(succ -&gt; {
328   *   if (succ) {
329   *     System.out.println("Check and put succeeded");
330   *   } else {
331   *     System.out.println("Check and put failed");
332   *   }
333   * });
334   * </pre>
335   *
336   * @deprecated Since 2.4.0, will be removed in 4.0.0. For internal test use only, do not use it
337   *             any more.
338   */
339  @Deprecated
340  CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter);
341
342  /**
343   * A helper class for sending checkAndMutate request with a filter.
344   * @deprecated Since 2.4.0, will be removed in 4.0.0. For internal test use only, do not use it
345   *             any more.
346   */
347  @Deprecated
348  interface CheckAndMutateWithFilterBuilder {
349
350    /**
351     * Match a timerange.
352     * @param timeRange time range to check.
353     */
354    CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange);
355
356    /**
357     * Specify a Put to commit if the check succeeds.
358     * @param put data to put if check succeeds
359     * @return {@code true} if the new put was executed, {@code false} otherwise. The return value
360     *         will be wrapped by a {@link CompletableFuture}.
361     */
362    CompletableFuture<Boolean> thenPut(Put put);
363
364    /**
365     * Specify a Delete to commit if the check succeeds.
366     * @param delete data to delete if check succeeds
367     * @return {@code true} if the new delete was executed, {@code false} otherwise. The return
368     *         value will be wrapped by a {@link CompletableFuture}.
369     */
370    CompletableFuture<Boolean> thenDelete(Delete delete);
371
372    /**
373     * Specify a RowMutations to commit if the check succeeds.
374     * @param mutation mutations to perform if check succeeds
375     * @return true if the new mutation was executed, false otherwise. The return value will be
376     *         wrapped by a {@link CompletableFuture}.
377     */
378    CompletableFuture<Boolean> thenMutate(RowMutations mutation);
379  }
380
381  /**
382   * checkAndMutate that atomically checks if a row matches the specified condition. If it does, it
383   * performs the specified action.
384   * @param checkAndMutate The CheckAndMutate object.
385   * @return A {@link CompletableFuture}s that represent the result for the CheckAndMutate.
386   */
387  CompletableFuture<CheckAndMutateResult> checkAndMutate(CheckAndMutate checkAndMutate);
388
389  /**
390   * Batch version of checkAndMutate. The specified CheckAndMutates are batched only in the sense
391   * that they are sent to a RS in one RPC, but each CheckAndMutate operation is still executed
392   * atomically (and thus, each may fail independently of others).
393   * @param checkAndMutates The list of CheckAndMutate.
394   * @return A list of {@link CompletableFuture}s that represent the result for each CheckAndMutate.
395   */
396  List<CompletableFuture<CheckAndMutateResult>>
397    checkAndMutate(List<CheckAndMutate> checkAndMutates);
398
399  /**
400   * A simple version of batch checkAndMutate. It will fail if there are any failures.
401   * @param checkAndMutates The list of rows to apply.
402   * @return A {@link CompletableFuture} that wrapper the result list.
403   */
404  default CompletableFuture<List<CheckAndMutateResult>>
405    checkAndMutateAll(List<CheckAndMutate> checkAndMutates) {
406    return allOf(checkAndMutate(checkAndMutates));
407  }
408
409  /**
410   * Performs multiple mutations atomically on a single row. Currently {@link Put} and
411   * {@link Delete} are supported.
412   * @param mutation object that specifies the set of mutations to perform atomically
413   * @return A {@link CompletableFuture} that returns results of Increment/Append operations
414   */
415  CompletableFuture<Result> mutateRow(RowMutations mutation);
416
417  /**
418   * The scan API uses the observer pattern.
419   * @param scan     A configured {@link Scan} object.
420   * @param consumer the consumer used to receive results.
421   * @see ScanResultConsumer
422   * @see AdvancedScanResultConsumer
423   */
424  void scan(Scan scan, C consumer);
425
426  /**
427   * Gets a scanner on the current table for the given family.
428   * @param family The column family to scan.
429   * @return A scanner.
430   */
431  default ResultScanner getScanner(byte[] family) {
432    return getScanner(new Scan().addFamily(family));
433  }
434
435  /**
436   * Gets a scanner on the current table for the given family and qualifier.
437   * @param family    The column family to scan.
438   * @param qualifier The column qualifier to scan.
439   * @return A scanner.
440   */
441  default ResultScanner getScanner(byte[] family, byte[] qualifier) {
442    return getScanner(new Scan().addColumn(family, qualifier));
443  }
444
445  /**
446   * Returns a scanner on the current table as specified by the {@link Scan} object.
447   * @param scan A configured {@link Scan} object.
448   * @return A scanner.
449   */
450  ResultScanner getScanner(Scan scan);
451
452  /**
453   * Return all the results that match the given scan object.
454   * <p>
455   * Notice that usually you should use this method with a {@link Scan} object that has limit set.
456   * For example, if you want to get the closest row after a given row, you could do this:
457   * <p>
458   *
459   * <pre>
460   * table.scanAll(new Scan().withStartRow(row, false).setLimit(1)).thenAccept(results -&gt; {
461   *   if (results.isEmpty()) {
462   *     System.out.println("No row after " + Bytes.toStringBinary(row));
463   *   } else {
464   *     System.out.println("The closest row after " + Bytes.toStringBinary(row) + " is "
465   *       + Bytes.toStringBinary(results.stream().findFirst().get().getRow()));
466   *   }
467   * });
468   * </pre>
469   * <p>
470   * If your result set is very large, you should use other scan method to get a scanner or use
471   * callback to process the results. They will do chunking to prevent OOM. The scanAll method will
472   * fetch all the results and store them in a List and then return the list to you.
473   * <p>
474   * The scan metrics will be collected background if you enable it but you have no way to get it.
475   * Usually you can get scan metrics from {@code ResultScanner}, or through
476   * {@code ScanResultConsumer.onScanMetricsCreated} but this method only returns a list of results.
477   * So if you really care about scan metrics then you'd better use other scan methods which return
478   * a {@code ResultScanner} or let you pass in a {@code ScanResultConsumer}. There is no
479   * performance difference between these scan methods so do not worry.
480   * @param scan A configured {@link Scan} object. So if you use this method to fetch a really large
481   *             result set, it is likely to cause OOM.
482   * @return The results of this small scan operation. The return value will be wrapped by a
483   *         {@link CompletableFuture}.
484   */
485  CompletableFuture<List<Result>> scanAll(Scan scan);
486
487  /**
488   * Test for the existence of columns in the table, as specified by the Gets.
489   * <p>
490   * This will return a list of booleans. Each value will be true if the related Get matches one or
491   * more keys, false if not.
492   * <p>
493   * This is a server-side call so it prevents any data from being transferred to the client.
494   * @param gets the Gets
495   * @return A list of {@link CompletableFuture}s that represent the existence for each get.
496   */
497  default List<CompletableFuture<Boolean>> exists(List<Get> gets) {
498    return get(toCheckExistenceOnly(gets)).stream()
499      .<CompletableFuture<Boolean>> map(f -> f.thenApply(r -> r.getExists())).collect(toList());
500  }
501
502  /**
503   * A simple version for batch exists. It will fail if there are any failures and you will get the
504   * whole result boolean list at once if the operation is succeeded.
505   * @param gets the Gets
506   * @return A {@link CompletableFuture} that wrapper the result boolean list.
507   */
508  default CompletableFuture<List<Boolean>> existsAll(List<Get> gets) {
509    return allOf(exists(gets));
510  }
511
512  /**
513   * Extracts certain cells from the given rows, in batch.
514   * <p>
515   * Notice that you may not get all the results with this function, which means some of the
516   * returned {@link CompletableFuture}s may succeed while some of the other returned
517   * {@link CompletableFuture}s may fail.
518   * @param gets The objects that specify what data to fetch and from which rows.
519   * @return A list of {@link CompletableFuture}s that represent the result for each get.
520   */
521  List<CompletableFuture<Result>> get(List<Get> gets);
522
523  /**
524   * A simple version for batch get. It will fail if there are any failures and you will get the
525   * whole result list at once if the operation is succeeded.
526   * @param gets The objects that specify what data to fetch and from which rows.
527   * @return A {@link CompletableFuture} that wrapper the result list.
528   */
529  default CompletableFuture<List<Result>> getAll(List<Get> gets) {
530    return allOf(get(gets));
531  }
532
533  /**
534   * Puts some data in the table, in batch.
535   * @param puts The list of mutations to apply.
536   * @return A list of {@link CompletableFuture}s that represent the result for each put.
537   */
538  List<CompletableFuture<Void>> put(List<Put> puts);
539
540  /**
541   * A simple version of batch put. It will fail if there are any failures.
542   * @param puts The list of mutations to apply.
543   * @return A {@link CompletableFuture} that always returns null when complete normally.
544   */
545  default CompletableFuture<Void> putAll(List<Put> puts) {
546    return allOf(put(puts)).thenApply(r -> null);
547  }
548
549  /**
550   * Deletes the specified cells/rows in bulk.
551   * @param deletes list of things to delete.
552   * @return A list of {@link CompletableFuture}s that represent the result for each delete.
553   */
554  List<CompletableFuture<Void>> delete(List<Delete> deletes);
555
556  /**
557   * A simple version of batch delete. It will fail if there are any failures.
558   * @param deletes list of things to delete.
559   * @return A {@link CompletableFuture} that always returns null when complete normally.
560   */
561  default CompletableFuture<Void> deleteAll(List<Delete> deletes) {
562    return allOf(delete(deletes)).thenApply(r -> null);
563  }
564
565  /**
566   * Method that does a batch call on Deletes, Gets, Puts, Increments, Appends and RowMutations. The
567   * ordering of execution of the actions is not defined. Meaning if you do a Put and a Get in the
568   * same {@link #batch} call, you will not necessarily be guaranteed that the Get returns what the
569   * Put had put.
570   * @param actions list of Get, Put, Delete, Increment, Append, and RowMutations objects
571   * @return A list of {@link CompletableFuture}s that represent the result for each action.
572   */
573  <T> List<CompletableFuture<T>> batch(List<? extends Row> actions);
574
575  /**
576   * A simple version of batch. It will fail if there are any failures and you will get the whole
577   * result list at once if the operation is succeeded.
578   * @param actions list of Get, Put, Delete, Increment, Append and RowMutations objects
579   * @return A list of the result for the actions. Wrapped by a {@link CompletableFuture}.
580   */
581  default <T> CompletableFuture<List<T>> batchAll(List<? extends Row> actions) {
582    return allOf(batch(actions));
583  }
584
585  /**
586   * Execute the given coprocessor call on the region which contains the given {@code row}.
587   * <p>
588   * The {@code stubMaker} is just a delegation to the {@code newStub} call. Usually it is only a
589   * one line lambda expression, like:
590   *
591   * <pre>
592   * channel -&gt; xxxService.newStub(channel)
593   * </pre>
594   *
595   * @param stubMaker a delegation to the actual {@code newStub} call.
596   * @param callable  a delegation to the actual protobuf rpc call. See the comment of
597   *                  {@link ServiceCaller} for more details.
598   * @param row       The row key used to identify the remote region location
599   * @param <S>       the type of the asynchronous stub
600   * @param <R>       the type of the return value
601   * @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
602   * @see ServiceCaller
603   */
604  <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
605    ServiceCaller<S, R> callable, byte[] row);
606
607  /**
608   * The callback when we want to execute a coprocessor call on a range of regions.
609   * <p>
610   * As the locating itself also takes some time, the implementation may want to send rpc calls on
611   * the fly, which means we do not know how many regions we have when we get the return value of
612   * the rpc calls, so we need an {@link #onComplete()} which is used to tell you that we have
613   * passed all the return values to you(through the {@link #onRegionComplete(RegionInfo, Object)}
614   * or {@link #onRegionError(RegionInfo, Throwable)} calls), i.e, there will be no
615   * {@link #onRegionComplete(RegionInfo, Object)} or {@link #onRegionError(RegionInfo, Throwable)}
616   * calls in the future.
617   * <p>
618   * Here is a pseudo code to describe a typical implementation of a range coprocessor service
619   * method to help you better understand how the {@link CoprocessorCallback} will be called. The
620   * {@code callback} in the pseudo code is our {@link CoprocessorCallback}. And notice that the
621   * {@code whenComplete} is {@code CompletableFuture.whenComplete}.
622   *
623   * <pre>
624   * locateThenCall(byte[] row) {
625   *   locate(row).whenComplete((location, locateError) -&gt; {
626   *     if (locateError != null) {
627   *       callback.onError(locateError);
628   *       return;
629   *     }
630   *     incPendingCall();
631   *     region = location.getRegion();
632   *     if (region.getEndKey() > endKey) {
633   *       locateEnd = true;
634   *     } else {
635   *       locateThenCall(region.getEndKey());
636   *     }
637   *     sendCall().whenComplete((resp, error) -&gt; {
638   *       if (error != null) {
639   *         callback.onRegionError(region, error);
640   *       } else {
641   *         callback.onRegionComplete(region, resp);
642   *       }
643   *       if (locateEnd && decPendingCallAndGet() == 0) {
644   *         callback.onComplete();
645   *       }
646   *     });
647   *   });
648   * }
649   * </pre>
650   */
651  @InterfaceAudience.Public
652  interface CoprocessorCallback<R> {
653
654    /**
655     * Indicate that the respose of a region is available
656     * @param region the region that the response belongs to
657     * @param resp   the response of the coprocessor call
658     */
659    void onRegionComplete(RegionInfo region, R resp);
660
661    /**
662     * Indicate that the error for a region is available
663     * @param region the region that the error belongs to
664     * @param error  the response error of the coprocessor call
665     */
666    void onRegionError(RegionInfo region, Throwable error);
667
668    /**
669     * Indicate that all responses of the regions have been notified by calling
670     * {@link #onRegionComplete(RegionInfo, Object)} or
671     * {@link #onRegionError(RegionInfo, Throwable)}.
672     */
673    void onComplete();
674
675    /**
676     * Indicate that we got an error which does not belong to any regions. Usually a locating error.
677     */
678    void onError(Throwable error);
679  }
680
681  /**
682   * Some coprocessors may support the idea of "partial results." If for some reason a coprocessor
683   * cannot return all results for a given region in a single response, the client side can be
684   * designed to recognize this and continuing requesting more results until they are completely
685   * accumulated in the client.
686   * <p>
687   * It is up to a particular coprocessor implementation and its corresponding clients to agree on
688   * what it means for results to be incomplete, how this state is communicated, and how multiple
689   * incomplete results are accumulated together.
690   * <p>
691   * Use this callback when you want to execute a coprocessor call on a range of regions, and that
692   * coprocessor may return incomplete results for a given region. See also the docs for
693   * {@link CoprocessorCallback}, which all apply here to its child interface too.
694   */
695  @InterfaceAudience.Public
696  interface PartialResultCoprocessorCallback<S, R> extends CoprocessorCallback<R> {
697    /**
698     * Subclasses should implement this to tell AsyncTable whether the given response is "final" or
699     * whether the AsyncTable should send another request to the coprocessor to fetch more results
700     * from the given region. This method of fetching more results can be used many times until
701     * there are no more results to fetch from the region.
702     * @param response The response received from the coprocessor
703     * @param region   The region the response came from
704     * @return A ServiceCaller object if the response was not final and therefore another request is
705     *         required to continuing fetching results. null if no more requests need to be sent to
706     *         the region.
707     */
708    ServiceCaller<S, R> getNextCallable(R response, RegionInfo region);
709
710    /**
711     * Subclasses should implement this such that, when the above method returns non-null, this
712     * method returns the duration that AsyncTable should wait before sending the next request to
713     * the given region. You can use this to create a back-off behavior to reduce load on the
714     * RegionServer. If that's not desired, you can always return {@link Duration.ZERO}.
715     * @param response The response received from the coprocessor
716     * @param region   The region the response came from
717     * @return The duration to wait.
718     */
719    Duration getWaitInterval(R response, RegionInfo region);
720  }
721
722  /**
723   * Helper class for sending coprocessorService request that executes a coprocessor call on regions
724   * which are covered by a range.
725   * <p>
726   * If {@code fromRow} is not specified the selection will start with the first table region. If
727   * {@code toRow} is not specified the selection will continue through the last table region.
728   * @param <S> the type of the protobuf Service you want to call.
729   * @param <R> the type of the return value.
730   */
731  interface CoprocessorServiceBuilder<S, R> {
732
733    /**
734     * Specify a start row
735     * @param startKey start region selection with region containing this row, inclusive.
736     */
737    default CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey) {
738      return fromRow(startKey, true);
739    }
740
741    /**
742     * Specify a start row
743     * @param startKey  start region selection with region containing this row
744     * @param inclusive whether to include the startKey
745     */
746    CoprocessorServiceBuilder<S, R> fromRow(byte[] startKey, boolean inclusive);
747
748    /**
749     * Specify a stop row
750     * @param endKey select regions up to and including the region containing this row, exclusive.
751     */
752    default CoprocessorServiceBuilder<S, R> toRow(byte[] endKey) {
753      return toRow(endKey, false);
754    }
755
756    /**
757     * Specify a stop row
758     * @param endKey    select regions up to and including the region containing this row
759     * @param inclusive whether to include the endKey
760     */
761    CoprocessorServiceBuilder<S, R> toRow(byte[] endKey, boolean inclusive);
762
763    /**
764     * Execute the coprocessorService request. You can get the response through the
765     * {@link CoprocessorCallback}.
766     */
767    void execute();
768  }
769
770  /**
771   * Execute a coprocessor call on the regions which are covered by a range.
772   * <p>
773   * Use the returned {@link CoprocessorServiceBuilder} construct your request and then execute it.
774   * <p>
775   * The {@code stubMaker} is just a delegation to the {@code xxxService.newStub} call. Usually it
776   * is only a one line lambda expression, like:
777   *
778   * <pre>
779   * channel -&gt; xxxService.newStub(channel)
780   * </pre>
781   *
782   * @param stubMaker a delegation to the actual {@code newStub} call.
783   * @param callable  a delegation to the actual protobuf rpc call. See the comment of
784   *                  {@link ServiceCaller} for more details.
785   * @param callback  callback to get the response. See the comment of {@link CoprocessorCallback}
786   *                  for more details.
787   */
788  <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(Function<RpcChannel, S> stubMaker,
789    ServiceCaller<S, R> callable, CoprocessorCallback<R> callback);
790
791  /**
792   * Similar to above. Use when your coprocessor client+endpoint supports partial results. If the
793   * server does not offer partial results, it is still safe to use this, assuming you implement
794   * your {@link PartialResultCoprocessorCallback#getNextCallable(Object, RegionInfo)} correctly.
795   */
796  default <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
797    Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
798    PartialResultCoprocessorCallback<S, R> callback) {
799    LoggerFactory.getLogger(AsyncTable.class).warn(
800      "Calling coprocessorService with default implementation that does not fully implement the"
801        + " PartialResultCoprocessorCallback interface.");
802    return coprocessorService(stubMaker, callable, (CoprocessorCallback<R>) callback);
803  }
804}