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