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.checkHasFamilies;
022import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
023import static org.apache.hadoop.hbase.client.ConnectionUtils.timelineConsistentRead;
024import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePut;
025import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
026
027import com.google.protobuf.RpcChannel;
028import java.io.IOException;
029import java.util.ArrayList;
030import java.util.Arrays;
031import java.util.List;
032import java.util.concurrent.CompletableFuture;
033import java.util.concurrent.TimeUnit;
034import java.util.concurrent.atomic.AtomicBoolean;
035import java.util.concurrent.atomic.AtomicInteger;
036import java.util.function.Function;
037import org.apache.hadoop.conf.Configuration;
038import org.apache.hadoop.hbase.CompareOperator;
039import org.apache.hadoop.hbase.HConstants;
040import org.apache.hadoop.hbase.HRegionLocation;
041import org.apache.hadoop.hbase.TableName;
042import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.SingleRequestCallerBuilder;
043import org.apache.hadoop.hbase.client.ConnectionUtils.Converter;
044import org.apache.hadoop.hbase.filter.BinaryComparator;
045import org.apache.hadoop.hbase.io.TimeRange;
046import org.apache.hadoop.hbase.ipc.HBaseRpcController;
047import org.apache.hadoop.hbase.util.Bytes;
048import org.apache.hadoop.hbase.util.ReflectionUtils;
049import org.apache.yetus.audience.InterfaceAudience;
050import org.slf4j.Logger;
051import org.slf4j.LoggerFactory;
052
053import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
054import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
055import org.apache.hbase.thirdparty.io.netty.util.Timer;
056
057import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
058import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
059import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
060import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
061import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
062import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;
063import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
064import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;
065import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
066import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
067import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
068import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType;
069
070/**
071 * The implementation of RawAsyncTable.
072 * <p/>
073 * The word 'Raw' means that this is a low level class. The returned {@link CompletableFuture} will
074 * be finished inside the rpc framework thread, which means that the callbacks registered to the
075 * {@link CompletableFuture} will also be executed inside the rpc framework thread. So users who use
076 * this class should not try to do time consuming tasks in the callbacks.
077 * @since 2.0.0
078 * @see AsyncTableImpl
079 */
080@InterfaceAudience.Private
081class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
082
083  private static final Logger LOG = LoggerFactory.getLogger(RawAsyncTableImpl.class);
084
085  private final AsyncConnectionImpl conn;
086
087  private final Timer retryTimer;
088
089  private final TableName tableName;
090
091  private final int defaultScannerCaching;
092
093  private final long defaultScannerMaxResultSize;
094
095  private final long rpcTimeoutNs;
096
097  private final long readRpcTimeoutNs;
098
099  private final long writeRpcTimeoutNs;
100
101  private final long operationTimeoutNs;
102
103  private final long scanTimeoutNs;
104
105  private final long pauseNs;
106
107  private final long pauseForCQTBENs;
108
109  private final int maxAttempts;
110
111  private final int startLogErrorsCnt;
112
113  RawAsyncTableImpl(AsyncConnectionImpl conn, Timer retryTimer, AsyncTableBuilderBase<?> builder) {
114    this.conn = conn;
115    this.retryTimer = retryTimer;
116    this.tableName = builder.tableName;
117    this.rpcTimeoutNs = builder.rpcTimeoutNs;
118    this.readRpcTimeoutNs = builder.readRpcTimeoutNs;
119    this.writeRpcTimeoutNs = builder.writeRpcTimeoutNs;
120    this.operationTimeoutNs = builder.operationTimeoutNs;
121    this.scanTimeoutNs = builder.scanTimeoutNs;
122    this.pauseNs = builder.pauseNs;
123    if (builder.pauseForCQTBENs < builder.pauseNs) {
124      LOG.warn(
125        "Configured value of pauseForCQTBENs is {} ms, which is less than" +
126          " the normal pause value {} ms, use the greater one instead",
127        TimeUnit.NANOSECONDS.toMillis(builder.pauseForCQTBENs),
128        TimeUnit.NANOSECONDS.toMillis(builder.pauseNs));
129      this.pauseForCQTBENs = builder.pauseNs;
130    } else {
131      this.pauseForCQTBENs = builder.pauseForCQTBENs;
132    }
133    this.maxAttempts = builder.maxAttempts;
134    this.startLogErrorsCnt = builder.startLogErrorsCnt;
135    this.defaultScannerCaching = tableName.isSystemTable() ? conn.connConf.getMetaScannerCaching()
136      : conn.connConf.getScannerCaching();
137    this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize();
138  }
139
140  @Override
141  public TableName getName() {
142    return tableName;
143  }
144
145  @Override
146  public Configuration getConfiguration() {
147    return conn.getConfiguration();
148  }
149
150  @Override
151  public CompletableFuture<TableDescriptor> getDescriptor() {
152    return conn.getAdmin().getDescriptor(tableName);
153  }
154
155  @Override
156  public AsyncTableRegionLocator getRegionLocator() {
157    return conn.getRegionLocator(tableName);
158  }
159
160  private static <REQ, RESP> CompletableFuture<RESP> mutate(HBaseRpcController controller,
161      HRegionLocation loc, ClientService.Interface stub, REQ req,
162      Converter<MutateRequest, byte[], REQ> reqConvert,
163      Converter<RESP, HBaseRpcController, MutateResponse> respConverter) {
164    return ConnectionUtils.call(controller, loc, stub, req, reqConvert,
165      (s, c, r, done) -> s.mutate(c, r, done), respConverter);
166  }
167
168  private static <REQ> CompletableFuture<Void> voidMutate(HBaseRpcController controller,
169      HRegionLocation loc, ClientService.Interface stub, REQ req,
170      Converter<MutateRequest, byte[], REQ> reqConvert) {
171    return mutate(controller, loc, stub, req, reqConvert, (c, resp) -> {
172      return null;
173    });
174  }
175
176  private static Result toResult(HBaseRpcController controller, MutateResponse resp)
177      throws IOException {
178    if (!resp.hasResult()) {
179      return null;
180    }
181    return ProtobufUtil.toResult(resp.getResult(), controller.cellScanner());
182  }
183
184  @FunctionalInterface
185  private interface NoncedConverter<D, I, S> {
186    D convert(I info, S src, long nonceGroup, long nonce) throws IOException;
187  }
188
189  private <REQ, RESP> CompletableFuture<RESP> noncedMutate(long nonceGroup, long nonce,
190      HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req,
191      NoncedConverter<MutateRequest, byte[], REQ> reqConvert,
192      Converter<RESP, HBaseRpcController, MutateResponse> respConverter) {
193    return mutate(controller, loc, stub, req,
194      (info, src) -> reqConvert.convert(info, src, nonceGroup, nonce), respConverter);
195  }
196
197  private <T> SingleRequestCallerBuilder<T> newCaller(byte[] row, int priority, long rpcTimeoutNs) {
198    return conn.callerFactory.<T> single().table(tableName).row(row).priority(priority)
199      .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS)
200      .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
201      .pause(pauseNs, TimeUnit.NANOSECONDS).pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS)
202      .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt);
203  }
204
205  private <T, R extends OperationWithAttributes & Row> SingleRequestCallerBuilder<T> newCaller(
206      R row, long rpcTimeoutNs) {
207    return newCaller(row.getRow(), row.getPriority(), rpcTimeoutNs);
208  }
209
210  private CompletableFuture<Result> get(Get get, int replicaId) {
211    return this.<Result, Get> newCaller(get, readRpcTimeoutNs)
212      .action((controller, loc, stub) -> ConnectionUtils
213        .<Get, GetRequest, GetResponse, Result> call(controller, loc, stub, get,
214          RequestConverter::buildGetRequest, (s, c, req, done) -> s.get(c, req, done),
215          (c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner())))
216      .replicaId(replicaId).call();
217  }
218
219  @Override
220  public CompletableFuture<Result> get(Get get) {
221    return timelineConsistentRead(conn.getLocator(), tableName, get, get.getRow(),
222      RegionLocateType.CURRENT, replicaId -> get(get, replicaId), readRpcTimeoutNs,
223      conn.connConf.getPrimaryCallTimeoutNs(), retryTimer, conn.getConnectionMetrics());
224  }
225
226  @Override
227  public CompletableFuture<Void> put(Put put) {
228    validatePut(put, conn.connConf.getMaxKeyValueSize());
229    return this.<Void, Put> newCaller(put, writeRpcTimeoutNs)
230      .action((controller, loc, stub) -> RawAsyncTableImpl.<Put> voidMutate(controller, loc, stub,
231        put, RequestConverter::buildMutateRequest))
232      .call();
233  }
234
235  @Override
236  public CompletableFuture<Void> delete(Delete delete) {
237    return this.<Void, Delete> newCaller(delete, writeRpcTimeoutNs)
238      .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete> voidMutate(controller, loc,
239        stub, delete, RequestConverter::buildMutateRequest))
240      .call();
241  }
242
243  @Override
244  public CompletableFuture<Result> append(Append append) {
245    checkHasFamilies(append);
246    long nonceGroup = conn.getNonceGenerator().getNonceGroup();
247    long nonce = conn.getNonceGenerator().newNonce();
248    return this.<Result, Append> newCaller(append, rpcTimeoutNs)
249      .action(
250        (controller, loc, stub) -> this.<Append, Result> noncedMutate(nonceGroup, nonce, controller,
251          loc, stub, append, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
252      .call();
253  }
254
255  @Override
256  public CompletableFuture<Result> increment(Increment increment) {
257    checkHasFamilies(increment);
258    long nonceGroup = conn.getNonceGenerator().getNonceGroup();
259    long nonce = conn.getNonceGenerator().newNonce();
260    return this.<Result, Increment> newCaller(increment, rpcTimeoutNs)
261      .action((controller, loc, stub) -> this.<Increment, Result> noncedMutate(nonceGroup, nonce,
262        controller, loc, stub, increment, RequestConverter::buildMutateRequest,
263        RawAsyncTableImpl::toResult))
264      .call();
265  }
266
267  private final class CheckAndMutateBuilderImpl implements CheckAndMutateBuilder {
268
269    private final byte[] row;
270
271    private final byte[] family;
272
273    private byte[] qualifier;
274
275    private TimeRange timeRange;
276
277    private CompareOperator op;
278
279    private byte[] value;
280
281    public CheckAndMutateBuilderImpl(byte[] row, byte[] family) {
282      this.row = Preconditions.checkNotNull(row, "row is null");
283      this.family = Preconditions.checkNotNull(family, "family is null");
284    }
285
286    @Override
287    public CheckAndMutateBuilder qualifier(byte[] qualifier) {
288      this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using" +
289        " an empty byte array, or just do not call this method if you want a null qualifier");
290      return this;
291    }
292
293    @Override
294    public CheckAndMutateBuilder timeRange(TimeRange timeRange) {
295      this.timeRange = timeRange;
296      return this;
297    }
298
299    @Override
300    public CheckAndMutateBuilder ifNotExists() {
301      this.op = CompareOperator.EQUAL;
302      this.value = null;
303      return this;
304    }
305
306    @Override
307    public CheckAndMutateBuilder ifMatches(CompareOperator compareOp, byte[] value) {
308      this.op = Preconditions.checkNotNull(compareOp, "compareOp is null");
309      this.value = Preconditions.checkNotNull(value, "value is null");
310      return this;
311    }
312
313    private void preCheck() {
314      Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by" +
315        " calling ifNotExists/ifEquals/ifMatches before executing the request");
316    }
317
318    @Override
319    public CompletableFuture<Boolean> thenPut(Put put) {
320      validatePut(put, conn.connConf.getMaxKeyValueSize());
321      preCheck();
322      return RawAsyncTableImpl.this.<Boolean> newCaller(row, put.getPriority(), rpcTimeoutNs)
323        .action((controller, loc, stub) -> RawAsyncTableImpl.<Put, Boolean> mutate(controller, loc,
324          stub, put,
325          (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
326            new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, p),
327          (c, r) -> r.getProcessed()))
328        .call();
329    }
330
331    @Override
332    public CompletableFuture<Boolean> thenDelete(Delete delete) {
333      preCheck();
334      return RawAsyncTableImpl.this.<Boolean> newCaller(row, delete.getPriority(), rpcTimeoutNs)
335        .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete, Boolean> mutate(controller,
336          loc, stub, delete,
337          (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
338            new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, d),
339          (c, r) -> r.getProcessed()))
340        .call();
341    }
342
343    @Override
344    public CompletableFuture<Boolean> thenMutate(RowMutations mutation) {
345      preCheck();
346      return RawAsyncTableImpl.this
347        .<Boolean> newCaller(row, mutation.getMaxPriority(), rpcTimeoutNs)
348        .action((controller, loc, stub) -> RawAsyncTableImpl.this.<Boolean> mutateRow(controller,
349          loc, stub, mutation,
350          (rn, rm) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier,
351            new BinaryComparator(value), CompareType.valueOf(op.name()), timeRange, rm),
352          resp -> resp.getExists()))
353        .call();
354    }
355  }
356
357  @Override
358  public CheckAndMutateBuilder checkAndMutate(byte[] row, byte[] family) {
359    return new CheckAndMutateBuilderImpl(row, family);
360  }
361
362  // We need the MultiRequest when constructing the org.apache.hadoop.hbase.client.MultiResponse,
363  // so here I write a new method as I do not want to change the abstraction of call method.
364  private <RESP> CompletableFuture<RESP> mutateRow(HBaseRpcController controller,
365      HRegionLocation loc, ClientService.Interface stub, RowMutations mutation,
366      Converter<MultiRequest, byte[], RowMutations> reqConvert,
367      Function<Result, RESP> respConverter) {
368    CompletableFuture<RESP> future = new CompletableFuture<>();
369    try {
370      byte[] regionName = loc.getRegion().getRegionName();
371      MultiRequest req = reqConvert.convert(regionName, mutation);
372      stub.multi(controller, req, new RpcCallback<MultiResponse>() {
373
374        @Override
375        public void run(MultiResponse resp) {
376          if (controller.failed()) {
377            future.completeExceptionally(controller.getFailed());
378          } else {
379            try {
380              org.apache.hadoop.hbase.client.MultiResponse multiResp =
381                ResponseConverter.getResults(req, resp, controller.cellScanner());
382              ConnectionUtils.updateStats(conn.getStatisticsTracker(), conn.getConnectionMetrics(),
383                loc.getServerName(), multiResp);
384              Throwable ex = multiResp.getException(regionName);
385              if (ex != null) {
386                future.completeExceptionally(ex instanceof IOException ? ex
387                  : new IOException(
388                    "Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex));
389              } else {
390                future.complete(respConverter
391                  .apply((Result) multiResp.getResults().get(regionName).result.get(0)));
392              }
393            } catch (IOException e) {
394              future.completeExceptionally(e);
395            }
396          }
397        }
398      });
399    } catch (IOException e) {
400      future.completeExceptionally(e);
401    }
402    return future;
403  }
404
405  @Override
406  public CompletableFuture<Void> mutateRow(RowMutations mutation) {
407    return this.<Void> newCaller(mutation.getRow(), mutation.getMaxPriority(), writeRpcTimeoutNs)
408      .action((controller, loc, stub) -> this.<Void> mutateRow(controller, loc, stub, mutation,
409        (rn, rm) -> {
410          RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(rn, rm);
411          regionMutationBuilder.setAtomic(true);
412          return MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
413        }, resp -> null))
414      .call();
415  }
416
417  private Scan setDefaultScanConfig(Scan scan) {
418    // always create a new scan object as we may reset the start row later.
419    Scan newScan = ReflectionUtils.newInstance(scan.getClass(), scan);
420    if (newScan.getCaching() <= 0) {
421      newScan.setCaching(defaultScannerCaching);
422    }
423    if (newScan.getMaxResultSize() <= 0) {
424      newScan.setMaxResultSize(defaultScannerMaxResultSize);
425    }
426    return newScan;
427  }
428
429  @Override
430  public void scan(Scan scan, AdvancedScanResultConsumer consumer) {
431    new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, retryTimer,
432      pauseNs, pauseForCQTBENs, maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt)
433        .start();
434  }
435
436  private long resultSize2CacheSize(long maxResultSize) {
437    // * 2 if possible
438    return maxResultSize > Long.MAX_VALUE / 2 ? maxResultSize : maxResultSize * 2;
439  }
440
441  @Override
442  public ResultScanner getScanner(Scan scan) {
443    return new AsyncTableResultScanner(this, ReflectionUtils.newInstance(scan.getClass(), scan),
444      resultSize2CacheSize(
445        scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize));
446  }
447
448  @Override
449  public CompletableFuture<List<Result>> scanAll(Scan scan) {
450    CompletableFuture<List<Result>> future = new CompletableFuture<>();
451    List<Result> scanResults = new ArrayList<>();
452    scan(scan, new AdvancedScanResultConsumer() {
453
454      @Override
455      public void onNext(Result[] results, ScanController controller) {
456        scanResults.addAll(Arrays.asList(results));
457      }
458
459      @Override
460      public void onError(Throwable error) {
461        future.completeExceptionally(error);
462      }
463
464      @Override
465      public void onComplete() {
466        future.complete(scanResults);
467      }
468    });
469    return future;
470  }
471
472  @Override
473  public List<CompletableFuture<Result>> get(List<Get> gets) {
474    return batch(gets, readRpcTimeoutNs);
475  }
476
477  @Override
478  public List<CompletableFuture<Void>> put(List<Put> puts) {
479    return voidMutate(puts);
480  }
481
482  @Override
483  public List<CompletableFuture<Void>> delete(List<Delete> deletes) {
484    return voidMutate(deletes);
485  }
486
487  @Override
488  public <T> List<CompletableFuture<T>> batch(List<? extends Row> actions) {
489    return batch(actions, rpcTimeoutNs);
490  }
491
492  private List<CompletableFuture<Void>> voidMutate(List<? extends Row> actions) {
493    return this.<Object> batch(actions, writeRpcTimeoutNs).stream()
494      .map(f -> f.<Void> thenApply(r -> null)).collect(toList());
495  }
496
497  private <T> List<CompletableFuture<T>> batch(List<? extends Row> actions, long rpcTimeoutNs) {
498    actions.stream().filter(action -> action instanceof Put).map(action -> (Put) action)
499      .forEach(put -> validatePut(put, conn.connConf.getMaxKeyValueSize()));
500    return conn.callerFactory.batch().table(tableName).actions(actions)
501      .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS)
502      .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS)
503      .pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts)
504      .startLogErrorsCnt(startLogErrorsCnt).call();
505  }
506
507  @Override
508  public long getRpcTimeout(TimeUnit unit) {
509    return unit.convert(rpcTimeoutNs, TimeUnit.NANOSECONDS);
510  }
511
512  @Override
513  public long getReadRpcTimeout(TimeUnit unit) {
514    return unit.convert(readRpcTimeoutNs, TimeUnit.NANOSECONDS);
515  }
516
517  @Override
518  public long getWriteRpcTimeout(TimeUnit unit) {
519    return unit.convert(writeRpcTimeoutNs, TimeUnit.NANOSECONDS);
520  }
521
522  @Override
523  public long getOperationTimeout(TimeUnit unit) {
524    return unit.convert(operationTimeoutNs, TimeUnit.NANOSECONDS);
525  }
526
527  @Override
528  public long getScanTimeout(TimeUnit unit) {
529    return unit.convert(scanTimeoutNs, TimeUnit.NANOSECONDS);
530  }
531
532  private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
533      ServiceCaller<S, R> callable, RegionInfo region, byte[] row) {
534    RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
535      region, row, rpcTimeoutNs, operationTimeoutNs);
536    S stub = stubMaker.apply(channel);
537    CompletableFuture<R> future = new CompletableFuture<>();
538    ClientCoprocessorRpcController controller = new ClientCoprocessorRpcController();
539    callable.call(stub, controller, resp -> {
540      if (controller.failed()) {
541        future.completeExceptionally(controller.getFailed());
542      } else {
543        future.complete(resp);
544      }
545    });
546    return future;
547  }
548
549  @Override
550  public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
551      ServiceCaller<S, R> callable, byte[] row) {
552    return coprocessorService(stubMaker, callable, null, row);
553  }
554
555  private boolean locateFinished(RegionInfo region, byte[] endKey, boolean endKeyInclusive) {
556    if (isEmptyStopRow(endKey)) {
557      if (isEmptyStopRow(region.getEndKey())) {
558        return true;
559      }
560      return false;
561    } else {
562      if (isEmptyStopRow(region.getEndKey())) {
563        return true;
564      }
565      int c = Bytes.compareTo(endKey, region.getEndKey());
566      // 1. if the region contains endKey
567      // 2. endKey is equal to the region's endKey and we do not want to include endKey.
568      return c < 0 || c == 0 && !endKeyInclusive;
569    }
570  }
571
572  private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
573      ServiceCaller<S, R> callable, CoprocessorCallback<R> callback, List<HRegionLocation> locs,
574      byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,
575      AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {
576    if (error != null) {
577      callback.onError(error);
578      return;
579    }
580    unfinishedRequest.incrementAndGet();
581    RegionInfo region = loc.getRegion();
582    if (locateFinished(region, endKey, endKeyInclusive)) {
583      locateFinished.set(true);
584    } else {
585      addListener(
586        conn.getLocator().getRegionLocation(tableName, region.getEndKey(), RegionLocateType.CURRENT,
587          operationTimeoutNs),
588        (l, e) -> onLocateComplete(stubMaker, callable, callback, locs, endKey, endKeyInclusive,
589          locateFinished, unfinishedRequest, l, e));
590    }
591    addListener(coprocessorService(stubMaker, callable, region, region.getStartKey()), (r, e) -> {
592      if (e != null) {
593        callback.onRegionError(region, e);
594      } else {
595        callback.onRegionComplete(region, r);
596      }
597      if (unfinishedRequest.decrementAndGet() == 0 && locateFinished.get()) {
598        callback.onComplete();
599      }
600    });
601  }
602
603  private final class CoprocessorServiceBuilderImpl<S, R>
604      implements CoprocessorServiceBuilder<S, R> {
605
606    private final Function<RpcChannel, S> stubMaker;
607
608    private final ServiceCaller<S, R> callable;
609
610    private final CoprocessorCallback<R> callback;
611
612    private byte[] startKey = HConstants.EMPTY_START_ROW;
613
614    private boolean startKeyInclusive;
615
616    private byte[] endKey = HConstants.EMPTY_END_ROW;
617
618    private boolean endKeyInclusive;
619
620    public CoprocessorServiceBuilderImpl(Function<RpcChannel, S> stubMaker,
621        ServiceCaller<S, R> callable, CoprocessorCallback<R> callback) {
622      this.stubMaker = Preconditions.checkNotNull(stubMaker, "stubMaker is null");
623      this.callable = Preconditions.checkNotNull(callable, "callable is null");
624      this.callback = Preconditions.checkNotNull(callback, "callback is null");
625    }
626
627    @Override
628    public CoprocessorServiceBuilderImpl<S, R> fromRow(byte[] startKey, boolean inclusive) {
629      this.startKey = Preconditions.checkNotNull(startKey,
630        "startKey is null. Consider using" +
631          " an empty byte array, or just do not call this method if you want to start selection" +
632          " from the first region");
633      this.startKeyInclusive = inclusive;
634      return this;
635    }
636
637    @Override
638    public CoprocessorServiceBuilderImpl<S, R> toRow(byte[] endKey, boolean inclusive) {
639      this.endKey = Preconditions.checkNotNull(endKey,
640        "endKey is null. Consider using" +
641          " an empty byte array, or just do not call this method if you want to continue" +
642          " selection to the last region");
643      this.endKeyInclusive = inclusive;
644      return this;
645    }
646
647    @Override
648    public void execute() {
649      addListener(conn.getLocator().getRegionLocation(tableName, startKey,
650        startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER, operationTimeoutNs),
651        (loc, error) -> onLocateComplete(stubMaker, callable, callback, new ArrayList<>(), endKey,
652          endKeyInclusive, new AtomicBoolean(false), new AtomicInteger(0), loc, error));
653    }
654  }
655
656  @Override
657  public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
658      Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
659      CoprocessorCallback<R> callback) {
660    return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback);
661  }
662}