From 235308d8bf88d3ca13b88bc093745c5fa3addc1a Mon Sep 17 00:00:00 2001 From: Nick Dimiduk Date: Thu, 31 Mar 2022 17:29:51 +0200 Subject: [PATCH] HBASE-26545 Implement tracing of scan * on `AsyncTable`, both `scan` and `scanAll` methods should result in `SCAN` table operations. * the span of the `SCAN` table operation should have children representing all the RPC calls involved in servicing the scan. * when a user provides custom implementation of `AdvancedScanResultConsumer`, any spans emitted from the callback methods should also be tied to the span that represents the `SCAN` table operation. This is easily done because these callbacks are executed on the RPC thread. * when a user provides a custom implementation of `ScanResultConsumer`, any spans emitted from the callback methods should be also be tied to the span that represents the `SCAN` table operation. This accomplished by carefully passing the span instance around after it is created. Signed-off-by: Andrew Purtell Signed-off-by: Duo Zhang --- .../hbase/client/AsyncClientScanner.java | 147 ++- ...syncScanSingleRegionRpcRetryingCaller.java | 43 +- .../hadoop/hbase/client/AsyncTableImpl.java | 38 +- .../hbase/client/AsyncTableResultScanner.java | 22 +- .../client/ClientAsyncPrefetchScanner.java | 31 +- .../hadoop/hbase/client/ClientScanner.java | 108 ++- .../client/ConnectionImplementation.java | 506 +++++----- .../apache/hadoop/hbase/client/HTable.java | 887 ++++++++---------- .../hbase/client/RawAsyncTableImpl.java | 368 ++++---- .../hadoop/hbase/client/ResultScanner.java | 10 +- .../hbase/client/TestAsyncTableTracing.java | 282 +++--- .../trace/hamcrest/SpanDataMatchers.java | 84 +- .../hadoop/hbase/StartMiniClusterOption.java | 45 +- .../client/AbstractTestAsyncTableScan.java | 194 ++-- .../client/LimitedScanResultConsumer.java | 35 + .../client/SimpleScanResultConsumer.java | 58 +- .../client/SimpleScanResultConsumerImpl.java | 76 ++ .../hbase/client/TestAsyncTableScan.java | 131 ++- .../hbase/client/TestAsyncTableScanAll.java | 65 ++ .../client/TestAsyncTableScanMetrics.java | 4 +- .../hbase/client/TestAsyncTableScanner.java | 69 +- .../hbase/client/TestRawAsyncTableScan.java | 101 +- .../client/TestResultScannerTracing.java | 292 ++++++ .../TracedAdvancedScanResultConsumer.java | 61 ++ .../client/TracedScanResultConsumer.java | 65 ++ pom.xml | 3 +- 26 files changed, 2295 insertions(+), 1430 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/LimitedScanResultConsumer.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/SimpleScanResultConsumerImpl.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerTracing.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TracedAdvancedScanResultConsumer.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TracedScanResultConsumer.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java index 48f004c0a29..0e2c9c616df 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncClientScanner.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,6 +28,9 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.isRemote; import static org.apache.hadoop.hbase.client.ConnectionUtils.timelineConsistentRead; import static org.apache.hadoop.hbase.util.FutureUtils.addListener; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.StatusCode; +import io.opentelemetry.context.Scope; import java.io.IOException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -35,7 +38,9 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; +import org.apache.hadoop.hbase.client.trace.TableOperationSpanBuilder; import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.io.netty.util.Timer; @@ -85,6 +90,8 @@ class AsyncClientScanner { private final ScanResultCache resultCache; + private final Span span; + public AsyncClientScanner(Scan scan, AdvancedScanResultConsumer consumer, TableName tableName, AsyncConnectionImpl conn, Timer retryTimer, long pauseNs, long pauseForCQTBENs, int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt) { @@ -112,6 +119,18 @@ class AsyncClientScanner { } else { this.scanMetrics = null; } + + /* + * Assumes that the `start()` method is called immediately after construction. If this is no + * longer the case, for tracing correctness, we should move the start of the span into the + * `start()` method. The cost of doing so would be making access to the `span` safe for + * concurrent threads. + */ + span = new TableOperationSpanBuilder(conn).setTableName(tableName).setOperation(scan).build(); + if (consumer instanceof AsyncTableResultScanner) { + AsyncTableResultScanner scanner = (AsyncTableResultScanner) consumer; + scanner.setSpan(span); + } } private static final class OpenScannerResponse { @@ -140,64 +159,87 @@ class AsyncClientScanner { private CompletableFuture callOpenScanner(HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub) { - boolean isRegionServerRemote = isRemote(loc.getHostname()); - incRPCCallsMetrics(scanMetrics, isRegionServerRemote); - if (openScannerTries.getAndIncrement() > 1) { - incRPCRetriesMetrics(scanMetrics, isRegionServerRemote); + try (Scope ignored = span.makeCurrent()) { + boolean isRegionServerRemote = isRemote(loc.getHostname()); + incRPCCallsMetrics(scanMetrics, isRegionServerRemote); + if (openScannerTries.getAndIncrement() > 1) { + incRPCRetriesMetrics(scanMetrics, isRegionServerRemote); + } + CompletableFuture future = new CompletableFuture<>(); + try { + ScanRequest request = RequestConverter.buildScanRequest(loc.getRegion().getRegionName(), + scan, scan.getCaching(), false); + stub.scan(controller, request, resp -> { + try (Scope ignored1 = span.makeCurrent()) { + if (controller.failed()) { + final IOException e = controller.getFailed(); + future.completeExceptionally(e); + TraceUtil.setError(span, e); + span.end(); + return; + } + future.complete( + new OpenScannerResponse(loc, isRegionServerRemote, stub, controller, resp)); + } + }); + } catch (IOException e) { + // span is closed by listener attached to the Future in `openScanner()` + future.completeExceptionally(e); + } + return future; } - CompletableFuture future = new CompletableFuture<>(); - try { - ScanRequest request = RequestConverter.buildScanRequest(loc.getRegion().getRegionName(), scan, - scan.getCaching(), false); - stub.scan(controller, request, resp -> { - if (controller.failed()) { - future.completeExceptionally(controller.getFailed()); - return; - } - future.complete(new OpenScannerResponse(loc, isRegionServerRemote, stub, controller, resp)); - }); - } catch (IOException e) { - future.completeExceptionally(e); - } - return future; } private void startScan(OpenScannerResponse resp) { addListener( conn.callerFactory.scanSingleRegion().id(resp.resp.getScannerId()).location(resp.loc) - .remote(resp.isRegionServerRemote) - .scannerLeaseTimeoutPeriod(resp.resp.getTtl(), TimeUnit.MILLISECONDS).stub(resp.stub) - .setScan(scan).metrics(scanMetrics).consumer(consumer).resultCache(resultCache) - .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS) - .scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) - .pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts) - .startLogErrorsCnt(startLogErrorsCnt).start(resp.controller, resp.resp), + .remote(resp.isRegionServerRemote) + .scannerLeaseTimeoutPeriod(resp.resp.getTtl(), TimeUnit.MILLISECONDS).stub(resp.stub) + .setScan(scan).metrics(scanMetrics).consumer(consumer).resultCache(resultCache) + .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS) + .scanTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) + .pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts) + .startLogErrorsCnt(startLogErrorsCnt).start(resp.controller, resp.resp), (hasMore, error) -> { - if (error != null) { - consumer.onError(error); - return; - } - if (hasMore) { - openScanner(); - } else { - consumer.onComplete(); + try (Scope ignored = span.makeCurrent()) { + if (error != null) { + try { + consumer.onError(error); + return; + } finally { + TraceUtil.setError(span, error); + span.end(); + } + } + if (hasMore) { + openScanner(); + } else { + try { + consumer.onComplete(); + } finally { + span.setStatus(StatusCode.OK); + span.end(); + } + } } }); } private CompletableFuture openScanner(int replicaId) { - return conn.callerFactory. single().table(tableName) - .row(scan.getStartRow()).replicaId(replicaId).locateType(getLocateType(scan)) - .priority(scan.getPriority()) - .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS) - .operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) - .pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts) - .startLogErrorsCnt(startLogErrorsCnt).action(this::callOpenScanner).call(); + try (Scope ignored = span.makeCurrent()) { + return conn.callerFactory. single().table(tableName) + .row(scan.getStartRow()).replicaId(replicaId).locateType(getLocateType(scan)) + .priority(scan.getPriority()).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS) + .operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS) + .pause(pauseNs, TimeUnit.NANOSECONDS).pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS) + .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt) + .action(this::callOpenScanner).call(); + } } private long getPrimaryTimeoutNs() { return TableName.isMetaTableName(tableName) ? conn.connConf.getPrimaryMetaScanTimeoutNs() - : conn.connConf.getPrimaryScanTimeoutNs(); + : conn.connConf.getPrimaryScanTimeoutNs(); } private void openScanner() { @@ -206,15 +248,24 @@ class AsyncClientScanner { addListener(timelineConsistentRead(conn.getLocator(), tableName, scan, scan.getStartRow(), getLocateType(scan), this::openScanner, rpcTimeoutNs, getPrimaryTimeoutNs(), retryTimer, conn.getConnectionMetrics()), (resp, error) -> { - if (error != null) { - consumer.onError(error); - return; + try (Scope ignored = span.makeCurrent()) { + if (error != null) { + try { + consumer.onError(error); + return; + } finally { + TraceUtil.setError(span, error); + span.end(); + } + } + startScan(resp); } - startScan(resp); }); } public void start() { - openScanner(); + try (Scope ignored = span.makeCurrent()) { + openScanner(); + } } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java index 7f19180a0ab..e9aa962edb3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,6 +28,8 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException; import static org.apache.hadoop.hbase.client.ConnectionUtils.updateResultsMetrics; import static org.apache.hadoop.hbase.client.ConnectionUtils.updateServerSideMetrics; +import io.opentelemetry.context.Context; +import io.opentelemetry.context.Scope; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -170,8 +172,8 @@ class AsyncScanSingleRegionRpcRetryingCaller { private void preCheck() { Preconditions.checkState(Thread.currentThread() == callerThread, - "The current thread is %s, expected thread is %s, " + - "you should not call this method outside onNext or onHeartbeat", + "The current thread is %s, expected thread is %s, " + + "you should not call this method outside onNext or onHeartbeat", Thread.currentThread(), callerThread); Preconditions.checkState(state.equals(ScanControllerState.INITIALIZED), "Invalid Stopper state %s", state); @@ -201,7 +203,7 @@ class AsyncScanSingleRegionRpcRetryingCaller { @Override public Optional cursor() { - return cursor; + return cursor; } } @@ -352,9 +354,9 @@ class AsyncScanSingleRegionRpcRetryingCaller { ScanRequest req = RequestConverter.buildScanRequest(this.scannerId, 0, true, false); stub.scan(controller, req, resp -> { if (controller.failed()) { - LOG.warn("Call to " + loc.getServerName() + " for closing scanner id = " + scannerId + - " for " + loc.getRegion().getEncodedName() + " of " + - loc.getRegion().getTable() + " failed, ignore, probably already closed", + LOG.warn("Call to " + loc.getServerName() + " for closing scanner id = " + scannerId + + " for " + loc.getRegion().getEncodedName() + " of " + loc.getRegion().getTable() + + " failed, ignore, probably already closed", controller.getFailed()); } }); @@ -392,19 +394,19 @@ class AsyncScanSingleRegionRpcRetryingCaller { private void onError(Throwable error) { error = translateException(error); if (tries > startLogErrorsCnt) { - LOG.warn("Call to " + loc.getServerName() + " for scanner id = " + scannerId + " for " + - loc.getRegion().getEncodedName() + " of " + loc.getRegion().getTable() + - " failed, , tries = " + tries + ", maxAttempts = " + maxAttempts + ", timeout = " + - TimeUnit.NANOSECONDS.toMillis(scanTimeoutNs) + " ms, time elapsed = " + elapsedMs() + - " ms", + LOG.warn("Call to " + loc.getServerName() + " for scanner id = " + scannerId + " for " + + loc.getRegion().getEncodedName() + " of " + loc.getRegion().getTable() + + " failed, , tries = " + tries + ", maxAttempts = " + maxAttempts + ", timeout = " + + TimeUnit.NANOSECONDS.toMillis(scanTimeoutNs) + " ms, time elapsed = " + elapsedMs() + + " ms", error); } - boolean scannerClosed = - error instanceof UnknownScannerException || error instanceof NotServingRegionException || - error instanceof RegionServerStoppedException || error instanceof ScannerResetException; + boolean scannerClosed = error instanceof UnknownScannerException + || error instanceof NotServingRegionException + || error instanceof RegionServerStoppedException || error instanceof ScannerResetException; RetriesExhaustedException.ThrowableWithExtraContext qt = - new RetriesExhaustedException.ThrowableWithExtraContext(error, - EnvironmentEdgeManager.currentTime(), ""); + new RetriesExhaustedException.ThrowableWithExtraContext(error, + EnvironmentEdgeManager.currentTime(), ""); exceptions.add(qt); if (tries >= maxAttempts) { completeExceptionally(!scannerClosed); @@ -573,7 +575,12 @@ class AsyncScanSingleRegionRpcRetryingCaller { resetController(controller, callTimeoutNs, priority); ScanRequest req = RequestConverter.buildScanRequest(scannerId, scan.getCaching(), false, nextCallSeq, scan.isScanMetricsEnabled(), false, scan.getLimit()); - stub.scan(controller, req, resp -> onComplete(controller, resp)); + final Context context = Context.current(); + stub.scan(controller, req, resp -> { + try (Scope ignored = context.makeCurrent()) { + onComplete(controller, resp); + } + }); } private void next() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java index 0ef6edf080c..9f780fbf9e9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableImpl.java @@ -18,8 +18,11 @@ package org.apache.hadoop.hbase.client; import static java.util.stream.Collectors.toList; + import com.google.protobuf.RpcChannel; +import io.opentelemetry.api.trace.Span; import io.opentelemetry.context.Context; +import io.opentelemetry.context.Scope; import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -177,8 +180,7 @@ class AsyncTableImpl implements AsyncTable { public CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter) { return new CheckAndMutateWithFilterBuilder() { - private final CheckAndMutateWithFilterBuilder builder = - rawTable.checkAndMutate(row, filter); + private final CheckAndMutateWithFilterBuilder builder = rawTable.checkAndMutate(row, filter); @Override public CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange) { @@ -209,10 +211,9 @@ class AsyncTableImpl implements AsyncTable { } @Override - public List> checkAndMutate( - List checkAndMutates) { - return rawTable.checkAndMutate(checkAndMutates).stream() - .map(this::wrap).collect(toList()); + public List> + checkAndMutate(List checkAndMutates) { + return rawTable.checkAndMutate(checkAndMutates).stream().map(this::wrap).collect(toList()); } @Override @@ -231,22 +232,29 @@ class AsyncTableImpl implements AsyncTable { } private void scan0(Scan scan, ScanResultConsumer consumer) { - try (ResultScanner scanner = getScanner(scan)) { - consumer.onScanMetricsCreated(scanner.getScanMetrics()); - for (Result result; (result = scanner.next()) != null;) { - if (!consumer.onNext(result)) { - break; + Span span = null; + try (AsyncTableResultScanner scanner = rawTable.getScanner(scan)) { + span = scanner.getSpan(); + try (Scope ignored = span.makeCurrent()) { + consumer.onScanMetricsCreated(scanner.getScanMetrics()); + for (Result result; (result = scanner.next()) != null;) { + if (!consumer.onNext(result)) { + break; + } } + consumer.onComplete(); } - consumer.onComplete(); } catch (IOException e) { - consumer.onError(e); + try (Scope ignored = span.makeCurrent()) { + consumer.onError(e); + } } } @Override public void scan(Scan scan, ScanResultConsumer consumer) { - pool.execute(() -> scan0(scan, consumer)); + final Context context = Context.current(); + pool.execute(context.wrap(() -> scan0(scan, consumer))); } @Override @@ -303,7 +311,7 @@ class AsyncTableImpl implements AsyncTable { } }; CoprocessorServiceBuilder builder = - rawTable.coprocessorService(stubMaker, callable, wrappedCallback); + rawTable.coprocessorService(stubMaker, callable, wrappedCallback); return new CoprocessorServiceBuilder() { @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java index 616cf6b9e60..39427fbcd4c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableResultScanner.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize; +import io.opentelemetry.api.trace.Span; import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayDeque; @@ -58,6 +59,9 @@ class AsyncTableResultScanner implements ResultScanner, AdvancedScanResultConsum private ScanResumer resumer; + // Used to pass the span instance to the `AsyncTableImpl` from its underlying `rawAsyncTable`. + private Span span = null; + public AsyncTableResultScanner(TableName tableName, Scan scan, long maxCacheSize) { this.tableName = tableName; this.maxCacheSize = maxCacheSize; @@ -71,14 +75,22 @@ class AsyncTableResultScanner implements ResultScanner, AdvancedScanResultConsum private void stopPrefetch(ScanController controller) { if (LOG.isDebugEnabled()) { - LOG.debug("{} stop prefetching when scanning {} as the cache size {}" + - " is greater than the maxCacheSize {}", - String.format("0x%x", System.identityHashCode(this)), tableName, cacheSize, - maxCacheSize); + LOG.debug( + "{} stop prefetching when scanning {} as the cache size {}" + + " is greater than the maxCacheSize {}", + String.format("0x%x", System.identityHashCode(this)), tableName, cacheSize, maxCacheSize); } resumer = controller.suspend(); } + Span getSpan() { + return span; + } + + void setSpan(final Span span) { + this.span = span; + } + @Override public synchronized void onNext(Result[] results, ScanController controller) { assert results.length > 0; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java index 0c832acdb37..e8d453f83c9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientAsyncPrefetchScanner.java @@ -18,6 +18,9 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize; + +import io.opentelemetry.context.Context; +import io.opentelemetry.context.Scope; import java.io.IOException; import java.io.InterruptedIOException; import java.util.Queue; @@ -36,14 +39,13 @@ import org.apache.hadoop.hbase.util.Threads; import org.apache.yetus.audience.InterfaceAudience; /** - * ClientAsyncPrefetchScanner implements async scanner behaviour. - * Specifically, the cache used by this scanner is a concurrent queue which allows both - * the producer (hbase client) and consumer (application) to access the queue in parallel. - * The number of rows returned in a prefetch is defined by the caching factor and the result size - * factor. - * This class allocates a buffer cache, whose size is a function of both factors. - * The prefetch is invoked when the cache is half­filled, instead of waiting for it to be empty. - * This is defined in the method {@link ClientAsyncPrefetchScanner#prefetchCondition()}. + * ClientAsyncPrefetchScanner implements async scanner behaviour. Specifically, the cache used by + * this scanner is a concurrent queue which allows both the producer (hbase client) and consumer + * (application) to access the queue in parallel. The number of rows returned in a prefetch is + * defined by the caching factor and the result size factor. This class allocates a buffer cache, + * whose size is a function of both factors. The prefetch is invoked when the cache is half-filled, + * instead of waiting for it to be empty. This is defined in the method + * {@link ClientAsyncPrefetchScanner#prefetchCondition()}. */ @InterfaceAudience.Private public class ClientAsyncPrefetchScanner extends ClientSimpleScanner { @@ -66,7 +68,9 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner { super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool, replicaCallTimeoutMicroSecondScan); exceptionsQueue = new ConcurrentLinkedQueue<>(); - Threads.setDaemonThreadRunning(new Thread(new PrefetchRunnable()), name + ".asyncPrefetcher"); + final Context context = Context.current(); + final Runnable runnable = context.wrap(new PrefetchRunnable()); + Threads.setDaemonThreadRunning(new Thread(runnable), name + ".asyncPrefetcher"); } void setPrefetchListener(Consumer prefetchListener) { @@ -88,7 +92,7 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner { @Override public Result next() throws IOException { - try { + try (Scope ignored = span.makeCurrent()) { lock.lock(); while (cache.isEmpty()) { handleException(); @@ -98,6 +102,7 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner { try { notEmpty.await(); } catch (InterruptedException e) { + span.recordException(e); throw new InterruptedIOException("Interrupted when wait to load cache"); } } @@ -132,8 +137,8 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner { } private void handleException() throws IOException { - //The prefetch task running in the background puts any exception it - //catches into this exception queue. + // The prefetch task running in the background puts any exception it + // catches into this exception queue. // Rethrow the exception so the application can handle it. while (!exceptionsQueue.isEmpty()) { Exception first = exceptionsQueue.peek(); @@ -171,6 +176,7 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner { succeed = true; } catch (Exception e) { exceptionsQueue.add(e); + span.recordException(e); } finally { notEmpty.signalAll(); lock.unlock(); @@ -180,7 +186,6 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner { } } } - } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java index ac5a7110a06..904df3b78ad 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientScanner.java @@ -1,4 +1,4 @@ -/** +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -21,6 +21,9 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.calcEstimatedSize; import static org.apache.hadoop.hbase.client.ConnectionUtils.createScanResultCache; import static org.apache.hadoop.hbase.client.ConnectionUtils.incRegionCountMetrics; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.StatusCode; +import io.opentelemetry.context.Scope; import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayDeque; @@ -40,13 +43,14 @@ import org.apache.hadoop.hbase.exceptions.ScannerResetException; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.regionserver.LeaseException; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; + /** * Implements the scanner interface for the HBase client. If there are multiple regions in a table, * this scanner will iterate through them all. @@ -76,6 +80,7 @@ public abstract class ClientScanner extends AbstractClientScanner { protected RpcRetryingCaller caller; protected RpcControllerFactory rpcControllerFactory; protected Configuration conf; + protected final Span span; // The timeout on the primary. Applicable if there are multiple replicas for a region // In that case, we will only wait for this much timeout on the primary before going // to the replicas and trying the same scan. Note that the retries will still happen @@ -92,7 +97,6 @@ public abstract class ClientScanner extends AbstractClientScanner { * @param scan {@link Scan} to use in this scanner * @param tableName The table that we wish to scan * @param connection Connection identifying the cluster - * @throws IOException */ public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, @@ -117,7 +121,7 @@ public abstract class ClientScanner extends AbstractClientScanner { HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE); } this.scannerTimeout = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, - HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD); + HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD); // check if application wants to collect scan metrics initScanMetrics(scan); @@ -134,14 +138,14 @@ public abstract class ClientScanner extends AbstractClientScanner { this.rpcControllerFactory = controllerFactory; this.conf = conf; + this.span = Span.current(); this.scanResultCache = createScanResultCache(scan); initCache(); } protected final int getScanReplicaId() { - return scan.getReplicaId() >= RegionReplicaUtil.DEFAULT_REPLICA_ID ? scan.getReplicaId() : - RegionReplicaUtil.DEFAULT_REPLICA_ID; + return Math.max(scan.getReplicaId(), RegionReplicaUtil.DEFAULT_REPLICA_ID); } protected ClusterConnection getConnection() { @@ -238,8 +242,8 @@ public abstract class ClientScanner extends AbstractClientScanner { if (LOG.isDebugEnabled() && this.currentRegion != null) { // Only worth logging if NOT first region in scan. LOG.debug( - "Advancing internal scanner to startKey at '" + Bytes.toStringBinary(scan.getStartRow()) + - "', " + (scan.includeStartRow() ? "inclusive" : "exclusive")); + "Advancing internal scanner to startKey at '" + Bytes.toStringBinary(scan.getStartRow()) + + "', " + (scan.includeStartRow() ? "inclusive" : "exclusive")); } // clear the current region, we will set a new value to it after the first call of the new // callable. @@ -331,8 +335,8 @@ public abstract class ClientScanner extends AbstractClientScanner { // old time we always return empty result for a open scanner operation so we add a check here to // keep compatible with the old logic. Should remove the isOpenScanner in the future. // 2. Server tells us that it has no more results for this region. - return (values.length == 0 && !callable.isHeartbeatMessage()) || - callable.moreResultsInRegion() == MoreResults.NO; + return (values.length == 0 && !callable.isHeartbeatMessage()) + || callable.moreResultsInRegion() == MoreResults.NO; } private void closeScannerIfExhausted(boolean exhausted) throws IOException { @@ -362,10 +366,10 @@ public abstract class ClientScanner extends AbstractClientScanner { // If exception is any but the list below throw it back to the client; else setup // the scanner and retry. Throwable cause = e.getCause(); - if ((cause != null && cause instanceof NotServingRegionException) || - (cause != null && cause instanceof RegionServerStoppedException) || - e instanceof OutOfOrderScannerNextException || e instanceof UnknownScannerException || - e instanceof ScannerResetException || e instanceof LeaseException) { + if ((cause != null && cause instanceof NotServingRegionException) + || (cause != null && cause instanceof RegionServerStoppedException) + || e instanceof OutOfOrderScannerNextException || e instanceof UnknownScannerException + || e instanceof ScannerResetException || e instanceof LeaseException) { // Pass. It is easier writing the if loop test as list of what is allowed rather than // as a list of what is not allowed... so if in here, it means we do not throw. if (retriesLeft <= 0) { @@ -489,8 +493,8 @@ public abstract class ClientScanner extends AbstractClientScanner { // processing of the scan is taking a long time server side. Rather than continue to // loop until a limit (e.g. size or caching) is reached, break out early to avoid causing // unnecesary delays to the caller - LOG.trace("Heartbeat message received and cache contains Results. " + - "Breaking out of scan loop"); + LOG.trace("Heartbeat message received and cache contains Results. " + + "Breaking out of scan loop"); // we know that the region has not been exhausted yet so just break without calling // closeScannerIfExhausted break; @@ -546,40 +550,52 @@ public abstract class ClientScanner extends AbstractClientScanner { @Override public void close() { - if (!scanMetricsPublished) writeScanMetrics(); - if (callable != null) { - callable.setClose(); - try { - call(callable, caller, scannerTimeout, false); - } catch (UnknownScannerException e) { - // We used to catch this error, interpret, and rethrow. However, we - // have since decided that it's not nice for a scanner's close to - // throw exceptions. Chances are it was just due to lease time out. - LOG.debug("scanner failed to close", e); - } catch (IOException e) { - /* An exception other than UnknownScanner is unexpected. */ - LOG.warn("scanner failed to close.", e); + try (Scope ignored = span.makeCurrent()) { + if (!scanMetricsPublished) { + writeScanMetrics(); } - callable = null; + if (callable != null) { + callable.setClose(); + try { + call(callable, caller, scannerTimeout, false); + } catch (UnknownScannerException e) { + // We used to catch this error, interpret, and rethrow. However, we + // have since decided that it's not nice for a scanner's close to + // throw exceptions. Chances are it was just due to lease time out. + LOG.debug("scanner failed to close", e); + } catch (IOException e) { + /* An exception other than UnknownScanner is unexpected. */ + LOG.warn("scanner failed to close.", e); + span.recordException(e); + span.setStatus(StatusCode.ERROR); + } + callable = null; + } + closed = true; + span.setStatus(StatusCode.OK); + } finally { + span.end(); } - closed = true; } @Override public boolean renewLease() { - if (callable == null) { - return false; - } - // do not return any rows, do not advance the scanner - callable.setRenew(true); - try { - this.caller.callWithoutRetries(callable, this.scannerTimeout); - return true; - } catch (Exception e) { - LOG.debug("scanner failed to renew lease", e); - return false; - } finally { - callable.setRenew(false); + try (Scope ignored = span.makeCurrent()) { + if (callable == null) { + return false; + } + // do not return any rows, do not advance the scanner + callable.setRenew(true); + try { + this.caller.callWithoutRetries(callable, this.scannerTimeout); + return true; + } catch (Exception e) { + LOG.debug("scanner failed to renew lease", e); + span.recordException(e); + return false; + } finally { + callable.setRenew(false); + } } } @@ -589,6 +605,8 @@ public abstract class ClientScanner extends AbstractClientScanner { @Override public Result next() throws IOException { - return nextWithSyncCache(); + try (Scope ignored = span.makeCurrent()) { + return nextWithSyncCache(); + } } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 13715d9471a..2fc3d54d220 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -1,5 +1,4 @@ -/** - * +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -30,6 +29,8 @@ import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent; import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsentEx; import edu.umd.cs.findbugs.annotations.Nullable; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.context.Scope; import java.io.Closeable; import java.io.IOException; import java.io.InterruptedIOException; @@ -69,6 +70,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Scan.ReadType; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory; +import org.apache.hadoop.hbase.client.trace.TableOperationSpanBuilder; import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; import org.apache.hadoop.hbase.exceptions.ConnectionClosedException; import org.apache.hadoop.hbase.exceptions.RegionMovedException; @@ -163,12 +165,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Updat import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; /** - * Main implementation of {@link Connection} and {@link ClusterConnection} interfaces. - * Encapsulates connection to zookeeper and regionservers. + * Main implementation of {@link Connection} and {@link ClusterConnection} interfaces. Encapsulates + * connection to zookeeper and regionservers. */ @edu.umd.cs.findbugs.annotations.SuppressWarnings( - value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION", - justification="Access to the conncurrent hash map is under a lock so should be fine.") + value = "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION", + justification = "Access to the conncurrent hash map is under a lock so should be fine.") @InterfaceAudience.Private public class ConnectionImplementation implements ClusterConnection, Closeable { public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server"; @@ -186,8 +188,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { final int rpcTimeout; /** - * Global nonceGenerator shared per client.Currently there's no reason to limit its scope. - * Once it's set under nonceGeneratorCreateLock, it is never unset or changed. + * Global nonceGenerator shared per client.Currently there's no reason to limit its scope. Once + * it's set under nonceGeneratorCreateLock, it is never unset or changed. */ private static volatile NonceGenerator nonceGenerator = null; /** The nonce generator lock. Only taken when creating Connection, which gets a private copy. */ @@ -243,8 +245,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { private final ClientBackoffPolicy backoffPolicy; /** - * Allow setting an alternate BufferedMutator implementation via - * config. If null, use default. + * Allow setting an alternate BufferedMutator implementation via config. If null, use default. */ private final String alternateBufferedMutatorClassName; @@ -265,7 +266,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { * Constructor, for creating cluster connection with provided ConnectionRegistry. */ ConnectionImplementation(Configuration conf, ExecutorService pool, User user, - ConnectionRegistry registry) throws IOException { + ConnectionRegistry registry) throws IOException { this.conf = conf; this.user = user; if (user != null && user.isLoginFromKeytab()) { @@ -274,8 +275,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { this.batchPool = (ThreadPoolExecutor) pool; this.connectionConfig = new ConnectionConfiguration(conf); this.closed = false; - this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, - HConstants.DEFAULT_HBASE_CLIENT_PAUSE); + this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, HConstants.DEFAULT_HBASE_CLIENT_PAUSE); long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause); if (configuredPauseForCQTBE < pause) { LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: " @@ -290,9 +290,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { // how many times to try, one more than max *retry* time this.numTries = retries2Attempts(connectionConfig.getRetriesNumber()); - this.rpcTimeout = conf.getInt( - HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT); + this.rpcTimeout = + conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT); if (conf.getBoolean(NonceGenerator.CLIENT_NONCES_ENABLED_KEY, true)) { synchronized (nonceGeneratorCreateLock) { if (nonceGenerator == null) { @@ -311,22 +310,20 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory); if (conf.getBoolean(CLIENT_SIDE_METRICS_ENABLED_KEY, false)) { this.metrics = - new MetricsConnection(this.toString(), this::getBatchPool, this::getMetaLookupPool); + new MetricsConnection(this.toString(), this::getBatchPool, this::getMetaLookupPool); } else { this.metrics = null; } this.metaCache = new MetaCache(this.metrics); - boolean shouldListen = conf.getBoolean(HConstants.STATUS_PUBLISHED, - HConstants.STATUS_PUBLISHED_DEFAULT); - Class listenerClass = - conf.getClass(ClusterStatusListener.STATUS_LISTENER_CLASS, - ClusterStatusListener.DEFAULT_STATUS_LISTENER_CLASS, - ClusterStatusListener.Listener.class); + boolean shouldListen = + conf.getBoolean(HConstants.STATUS_PUBLISHED, HConstants.STATUS_PUBLISHED_DEFAULT); + Class listenerClass = conf.getClass( + ClusterStatusListener.STATUS_LISTENER_CLASS, + ClusterStatusListener.DEFAULT_STATUS_LISTENER_CLASS, ClusterStatusListener.Listener.class); // Is there an alternate BufferedMutator to use? - this.alternateBufferedMutatorClassName = - this.conf.get(BufferedMutator.CLASSNAME_KEY); + this.alternateBufferedMutatorClassName = this.conf.get(BufferedMutator.CLASSNAME_KEY); try { if (registry == null) { @@ -341,11 +338,11 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { // Do we publish the status? if (shouldListen) { if (listenerClass == null) { - LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " + - ClusterStatusListener.STATUS_LISTENER_CLASS + " is not set - not listening status"); + LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " + + ClusterStatusListener.STATUS_LISTENER_CLASS + " is not set - not listening status"); } else { - clusterStatusListener = new ClusterStatusListener( - new ClusterStatusListener.DeadServerHandler() { + clusterStatusListener = + new ClusterStatusListener(new ClusterStatusListener.DeadServerHandler() { @Override public void newDead(ServerName sn) { clearCaches(sn); @@ -362,33 +359,32 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } // Get the region locator's meta replica mode. - this.metaReplicaMode = CatalogReplicaMode.fromString(conf.get(LOCATOR_META_REPLICAS_MODE, - CatalogReplicaMode.NONE.toString())); + this.metaReplicaMode = CatalogReplicaMode + .fromString(conf.get(LOCATOR_META_REPLICAS_MODE, CatalogReplicaMode.NONE.toString())); switch (this.metaReplicaMode) { case LOAD_BALANCE: - String replicaSelectorClass = conf.get( - RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR, - CatalogReplicaLoadBalanceSimpleSelector.class.getName()); + String replicaSelectorClass = + conf.get(RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR, + CatalogReplicaLoadBalanceSimpleSelector.class.getName()); - this.metaReplicaSelector = CatalogReplicaLoadBalanceSelectorFactory.createSelector( - replicaSelectorClass, META_TABLE_NAME, getChoreService(), () -> { - int numOfReplicas = 1; - try { - RegionLocations metaLocations = this.registry.getMetaRegionLocations().get( - connectionConfig.getReadRpcTimeout(), TimeUnit.MILLISECONDS); - numOfReplicas = metaLocations.size(); - } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", META_TABLE_NAME, e); - } - return numOfReplicas; - }); + this.metaReplicaSelector = CatalogReplicaLoadBalanceSelectorFactory + .createSelector(replicaSelectorClass, META_TABLE_NAME, getChoreService(), () -> { + int numOfReplicas = 1; + try { + RegionLocations metaLocations = this.registry.getMetaRegionLocations() + .get(connectionConfig.getReadRpcTimeout(), TimeUnit.MILLISECONDS); + numOfReplicas = metaLocations.size(); + } catch (Exception e) { + LOG.error("Failed to get table {}'s region replication, ", META_TABLE_NAME, e); + } + return numOfReplicas; + }); break; case NONE: // If user does not configure LOCATOR_META_REPLICAS_MODE, let's check the legacy config. - boolean useMetaReplicas = conf.getBoolean(USE_META_REPLICAS, - DEFAULT_USE_META_REPLICAS); + boolean useMetaReplicas = conf.getBoolean(USE_META_REPLICAS, DEFAULT_USE_META_REPLICAS); if (useMetaReplicas) { this.metaReplicaMode = CatalogReplicaMode.HEDGED_READ; } @@ -408,12 +404,10 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { * @param cnm Replaces the nonce generator used, for testing. * @return old nonce generator. */ - static NonceGenerator injectNonceGeneratorForTesting( - ClusterConnection conn, NonceGenerator cnm) { - ConnectionImplementation connImpl = (ConnectionImplementation)conn; + static NonceGenerator injectNonceGeneratorForTesting(ClusterConnection conn, NonceGenerator cnm) { + ConnectionImplementation connImpl = (ConnectionImplementation) conn; NonceGenerator ng = connImpl.getNonceGenerator(); - LOG.warn("Nonce generator is being replaced by test code for " - + cnm.getClass().getName()); + LOG.warn("Nonce generator is being replaced by test code for " + cnm.getClass().getName()); nonceGenerator = cnm; return ng; } @@ -445,11 +439,11 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } if (params.getWriteBufferPeriodicFlushTimeoutMs() == BufferedMutatorParams.UNSET) { params.setWriteBufferPeriodicFlushTimeoutMs( - connectionConfig.getWriteBufferPeriodicFlushTimeoutMs()); + connectionConfig.getWriteBufferPeriodicFlushTimeoutMs()); } if (params.getWriteBufferPeriodicFlushTimerTickMs() == BufferedMutatorParams.UNSET) { params.setWriteBufferPeriodicFlushTimerTickMs( - connectionConfig.getWriteBufferPeriodicFlushTimerTickMs()); + connectionConfig.getWriteBufferPeriodicFlushTimerTickMs()); } if (params.getMaxKeyValueSize() == BufferedMutatorParams.UNSET) { params.maxKeyValueSize(connectionConfig.getMaxKeyValueSize()); @@ -464,8 +458,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { return new BufferedMutatorImpl(this, rpcCallerFactory, rpcControllerFactory, params); } try { - return (BufferedMutator)ReflectionUtils.newInstance(Class.forName(implementationClassName), - this, rpcCallerFactory, rpcControllerFactory, params); + return (BufferedMutator) ReflectionUtils.newInstance(Class.forName(implementationClassName), + this, rpcCallerFactory, rpcControllerFactory, params); } catch (ClassNotFoundException e) { throw new RuntimeException(e); } @@ -500,16 +494,16 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { throw new RegionServerStoppedException(masterServer + " is dead."); } String key = - getStubKey(MasterProtos.HbckService.BlockingInterface.class.getName(), masterServer); + getStubKey(MasterProtos.HbckService.BlockingInterface.class.getName(), masterServer); return new HBaseHbck( - (MasterProtos.HbckService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> { - BlockingRpcChannel channel = - this.rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout); - return MasterProtos.HbckService.newBlockingStub(channel); - }), rpcControllerFactory); + (MasterProtos.HbckService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> { + BlockingRpcChannel channel = + this.rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout); + return MasterProtos.HbckService.newBlockingStub(channel); + }), rpcControllerFactory); }, () -> TraceUtil.createSpan(this.getClass().getSimpleName() + ".getHbck") - .setAttribute(HBaseSemanticAttributes.SERVER_NAME_KEY, masterServer.getServerName())); + .setAttribute(HBaseSemanticAttributes.SERVER_NAME_KEY, masterServer.getServerName())); } @Override @@ -552,16 +546,14 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { long keepAliveTime = conf.getLong("hbase.hconnection.threads.keepalivetime", 60); BlockingQueue workQueue = passedWorkQueue; if (workQueue == null) { - workQueue = - new LinkedBlockingQueue<>(maxThreads * - conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, - HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)); + workQueue = new LinkedBlockingQueue<>(maxThreads * conf.getInt( + HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS, HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)); coreThreads = maxThreads; } ThreadPoolExecutor tpe = - new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue, - new ThreadFactoryBuilder().setNameFormat(toString() + nameHint + "-pool-%d") - .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build()); + new ThreadPoolExecutor(coreThreads, maxThreads, keepAliveTime, TimeUnit.SECONDS, workQueue, + new ThreadFactoryBuilder().setNameFormat(toString() + nameHint + "-pool-%d") + .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build()); tpe.allowCoreThreadTimeOut(true); return tpe; } @@ -570,15 +562,13 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { if (this.metaLookupPool == null) { synchronized (this) { if (this.metaLookupPool == null) { - //Some of the threads would be used for meta replicas - //To start with, threads.max.core threads can hit the meta (including replicas). - //After that, requests will get queued up in the passed queue, and only after - //the queue is full, a new thread will be started + // Some of the threads would be used for meta replicas + // To start with, threads.max.core threads can hit the meta (including replicas). + // After that, requests will get queued up in the passed queue, and only after + // the queue is full, a new thread will be started int threads = conf.getInt("hbase.hconnection.meta.lookup.threads.max", 128); - this.metaLookupPool = getThreadPool( - threads, - threads, - "-metaLookup-shared-", new LinkedBlockingQueue<>()); + this.metaLookupPool = + getThreadPool(threads, threads, "-metaLookup-shared-", new LinkedBlockingQueue<>()); } } } @@ -624,7 +614,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { * An identifier that will remain the same for a given connection. */ @Override - public String toString(){ + public String toString() { return "hconnection-0x" + Integer.toHexString(hashCode()); } @@ -668,9 +658,9 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } /** - * Like {@link ConnectionClosedException} but thrown from the checkClosed call which looks - * at the local this.closed flag. We use this rather than {@link ConnectionClosedException} - * because the latter does not inherit from DoNotRetryIOE (it should. TODO). + * Like {@link ConnectionClosedException} but thrown from the checkClosed call which looks at the + * local this.closed flag. We use this rather than {@link ConnectionClosedException} because the + * latter does not inherit from DoNotRetryIOE (it should. TODO). */ private static class LocalConnectionClosedException extends DoNotRetryIOException { LocalConnectionClosedException(String message) { @@ -705,7 +695,6 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { return reload ? relocateRegion(tableName, row) : locateRegion(tableName, row); } - @Override public boolean isTableEnabled(TableName tableName) throws IOException { return getTableState(tableName).inStates(TableState.State.ENABLED); @@ -730,7 +719,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { return true; } List> locations = - MetaTableAccessor.getTableRegionsAndLocations(this, tableName, true); + MetaTableAccessor.getTableRegionsAndLocations(this, tableName, true); int notDeployed = 0; int regionCount = 0; @@ -738,21 +727,21 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { RegionInfo info = pair.getFirst(); if (pair.getSecond() == null) { LOG.debug("Table {} has not deployed region {}", tableName, - pair.getFirst().getEncodedName()); + pair.getFirst().getEncodedName()); notDeployed++; - } else if (splitKeys != null - && !Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { - for (byte[] splitKey : splitKeys) { - // Just check if the splitkey is available - if (Bytes.equals(info.getStartKey(), splitKey)) { - regionCount++; - break; + } else + if (splitKeys != null && !Bytes.equals(info.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) { + for (byte[] splitKey : splitKeys) { + // Just check if the splitkey is available + if (Bytes.equals(info.getStartKey(), splitKey)) { + regionCount++; + break; + } } + } else { + // Always empty start row should be counted + regionCount++; } - } else { - // Always empty start row should be counted - regionCount++; - } } if (notDeployed > 0) { if (LOG.isDebugEnabled()) { @@ -762,7 +751,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } else if (splitKeys != null && regionCount != splitKeys.length + 1) { if (LOG.isDebugEnabled()) { LOG.debug("Table {} expected to have {} regions, but only {} available", tableName, - splitKeys.length + 1, regionCount); + splitKeys.length + 1, regionCount); } return false; } else { @@ -832,16 +821,16 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { public HRegionLocation relocateRegion(final TableName tableName, final byte[] row) throws IOException { RegionLocations locations = - relocateRegion(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID); + relocateRegion(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID); return locations == null ? null - : locations.getRegionLocation(RegionReplicaUtil.DEFAULT_REPLICA_ID); + : locations.getRegionLocation(RegionReplicaUtil.DEFAULT_REPLICA_ID); } @Override - public RegionLocations relocateRegion(final TableName tableName, - final byte [] row, int replicaId) throws IOException{ + public RegionLocations relocateRegion(final TableName tableName, final byte[] row, int replicaId) + throws IOException { // Since this is an explicit request not to use any caching, finding - // disabled tables should not be desirable. This will ensure that an exception is thrown when + // disabled tables should not be desirable. This will ensure that an exception is thrown when // the first time a disabled table is interacted with. if (!tableName.equals(TableName.META_TABLE_NAME) && isTableDisabled(tableName)) { throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled."); @@ -871,8 +860,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } } - private RegionLocations locateMeta(final TableName tableName, - boolean useCache, int replicaId) throws IOException { + private RegionLocations locateMeta(final TableName tableName, boolean useCache, int replicaId) + throws IOException { // HBASE-10785: We cache the location of the META itself, so that we are not overloading // zookeeper with one request for every region lookup. We cache the META with empty row // key in MetaCache. @@ -924,15 +913,15 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { // without knowing the precise region names. byte[] metaStartKey = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false); byte[] metaStopKey = - RegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW, "", false); + RegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW, "", false); Scan s = new Scan().withStartRow(metaStartKey).withStopRow(metaStopKey, true) - .addFamily(HConstants.CATALOG_FAMILY).setReversed(true).setCaching(1) - .setReadType(ReadType.PREAD); + .addFamily(HConstants.CATALOG_FAMILY).setReversed(true).setCaching(1) + .setReadType(ReadType.PREAD); switch (this.metaReplicaMode) { case LOAD_BALANCE: - int metaReplicaId = this.metaReplicaSelector.select(tableName, row, - RegionLocateType.CURRENT); + int metaReplicaId = + this.metaReplicaSelector.select(tableName, row, RegionLocateType.CURRENT); if (metaReplicaId != RegionInfo.DEFAULT_REPLICA_ID) { // If the selector gives a non-primary meta replica region, then go with it. // Otherwise, just go to primary in non-hedgedRead mode. @@ -948,7 +937,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } int maxAttempts = (retry ? numTries : 1); boolean relocateMeta = false; - for (int tries = 0; ; tries++) { + for (int tries = 0;; tries++) { if (tries >= maxAttempts) { throw new NoServerForRegionException("Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName + " after " + tries + " tries."); @@ -981,9 +970,12 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { RegionInfo.DEFAULT_REPLICA_ID); } s.resetMvccReadPoint(); - try (ReversedClientScanner rcs = - new ReversedClientScanner(conf, s, TableName.META_TABLE_NAME, this, rpcCallerFactory, - rpcControllerFactory, getMetaLookupPool(), metaReplicaCallTimeoutScanInMicroSecond)) { + final Span span = new TableOperationSpanBuilder(this) + .setTableName(TableName.META_TABLE_NAME).setOperation(s).build(); + try (Scope ignored = span.makeCurrent(); + ReversedClientScanner rcs = new ReversedClientScanner(conf, s, + TableName.META_TABLE_NAME, this, rpcCallerFactory, rpcControllerFactory, + getMetaLookupPool(), metaReplicaCallTimeoutScanInMicroSecond)) { boolean tableNotFound = true; for (;;) { Result regionInfoRow = rcs.next(); @@ -992,7 +984,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { throw new TableNotFoundException(tableName); } else { throw new IOException( - "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName); + "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName); } } tableNotFound = false; @@ -1003,8 +995,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } RegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegion(); if (regionInfo == null) { - throw new IOException("RegionInfo null or empty in " + TableName.META_TABLE_NAME + - ", row=" + regionInfoRow); + throw new IOException("RegionInfo null or empty in " + TableName.META_TABLE_NAME + + ", row=" + regionInfoRow); } // See HBASE-20182. It is possible that we locate to a split parent even after the // children are online, so here we need to skip this region and go to the next one. @@ -1012,26 +1004,26 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { continue; } if (regionInfo.isOffline()) { - throw new RegionOfflineException("Region offline; disable table call? " + - regionInfo.getRegionNameAsString()); + throw new RegionOfflineException( + "Region offline; disable table call? " + regionInfo.getRegionNameAsString()); } // It is possible that the split children have not been online yet and we have skipped // the parent in the above condition, so we may have already reached a region which does // not contains us. if (!regionInfo.containsRow(row)) { throw new IOException( - "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName); + "Unable to find region for " + Bytes.toStringBinary(row) + " in " + tableName); } ServerName serverName = locations.getRegionLocation(replicaId).getServerName(); if (serverName == null) { - throw new NoServerForRegionException("No server address listed in " + - TableName.META_TABLE_NAME + " for region " + regionInfo.getRegionNameAsString() + - " containing row " + Bytes.toStringBinary(row)); + throw new NoServerForRegionException("No server address listed in " + + TableName.META_TABLE_NAME + " for region " + regionInfo.getRegionNameAsString() + + " containing row " + Bytes.toStringBinary(row)); } if (isDeadServer(serverName)) { throw new RegionServerStoppedException( - "hbase:meta says the region " + regionInfo.getRegionNameAsString() + - " is managed by the server " + serverName + ", but it is dead."); + "hbase:meta says the region " + regionInfo.getRegionNameAsString() + + " is managed by the server " + serverName + ", but it is dead."); } // Instantiate the location cacheLocation(tableName, locations); @@ -1050,29 +1042,31 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } catch (IOException e) { ExceptionUtil.rethrowIfInterrupt(e); if (e instanceof RemoteException) { - e = ((RemoteException)e).unwrapRemoteException(); + e = ((RemoteException) e).unwrapRemoteException(); } if (e instanceof CallQueueTooBigException) { // Give a special check on CallQueueTooBigException, see #HBASE-17114 pauseBase = this.pauseForCQTBE; } if (tries < maxAttempts - 1) { - LOG.debug("locateRegionInMeta parentTable='{}', attempt={} of {} failed; retrying " + - "after sleep of {}", TableName.META_TABLE_NAME, tries, maxAttempts, maxAttempts, e); + LOG.debug( + "locateRegionInMeta parentTable='{}', attempt={} of {} failed; retrying " + + "after sleep of {}", + TableName.META_TABLE_NAME, tries, maxAttempts, maxAttempts, e); } else { throw e; } // Only relocate the parent region if necessary relocateMeta = - !(e instanceof RegionOfflineException || e instanceof NoServerForRegionException); + !(e instanceof RegionOfflineException || e instanceof NoServerForRegionException); } finally { userRegionLock.unlock(); } - try{ + try { Thread.sleep(ConnectionUtils.getPauseTime(pauseBase, tries)); } catch (InterruptedException e) { - throw new InterruptedIOException("Giving up trying to location region in " + - "meta: thread is interrupted."); + throw new InterruptedIOException( + "Giving up trying to location region in " + "meta: thread is interrupted."); } } } @@ -1081,8 +1075,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { try { long waitTime = connectionConfig.getMetaOperationTimeout(); if (!userRegionLock.tryLock(waitTime, TimeUnit.MILLISECONDS)) { - throw new LockTimeoutException("Failed to get user region lock in" - + waitTime + " ms. " + " for accessing meta region server."); + throw new LockTimeoutException("Failed to get user region lock in" + waitTime + " ms. " + + " for accessing meta region server."); } } catch (InterruptedException ie) { LOG.error("Interrupted while waiting for a lock", ie); @@ -1101,12 +1095,11 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } /** - * Search the cache for a location that fits our table and row key. - * Return null if no suitable region is located. + * Search the cache for a location that fits our table and row key. Return null if no suitable + * region is located. * @return Null or region location found in cache. */ - RegionLocations getCachedLocation(final TableName tableName, - final byte [] row) { + RegionLocations getCachedLocation(final TableName tableName, final byte[] row) { return metaCache.getCachedLocation(tableName, row); } @@ -1180,7 +1173,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } catch (Exception e) { throw ProtobufUtil.handleRemoteException(e); } - return response != null? response.getIsMasterRunning(): false; + return response != null ? response.getIsMasterRunning() : false; } } @@ -1189,7 +1182,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { */ static class ServerErrorTracker { // We need a concurrent map here, as we could have multiple threads updating it in parallel. - private final ConcurrentMap errorsByServer = new ConcurrentHashMap<>(); + private final ConcurrentMap errorsByServer = + new ConcurrentHashMap<>(); private final long canRetryUntil; private final int maxTries;// max number to try private final long startTrackingTime; @@ -1211,14 +1205,13 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { */ boolean canTryMore(int numAttempt) { // If there is a single try we must not take into account the time. - return numAttempt < maxTries || (maxTries > 1 && - EnvironmentEdgeManager.currentTime() < this.canRetryUntil); + return numAttempt < maxTries + || (maxTries > 1 && EnvironmentEdgeManager.currentTime() < this.canRetryUntil); } /** * Calculates the back-off time for a retrying request to a particular server. - * - * @param server The server in question. + * @param server The server in question. * @param basePause The default hci pause. * @return The time to wait before sending next request. */ @@ -1292,8 +1285,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { throw new MasterNotRunningException(sn + " is dead."); } // Use the security info interface name as our stub key - String key = - getStubKey(MasterProtos.MasterService.getDescriptor().getName(), sn); + String key = getStubKey(MasterProtos.MasterService.getDescriptor().getName(), sn); MasterProtos.MasterService.BlockingInterface stub = (MasterProtos.MasterService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> { BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout); @@ -1355,8 +1347,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { if (isDeadServer(serverName)) { throw new RegionServerStoppedException(serverName + " is dead."); } - String key = getStubKey(ClientProtos.ClientService.BlockingInterface.class.getName(), - serverName); + String key = + getStubKey(ClientProtos.ClientService.BlockingInterface.class.getName(), serverName); return (ClientProtos.ClientService.BlockingInterface) computeIfAbsentEx(stubs, key, () -> { BlockingRpcChannel channel = this.rpcClient.createBlockingRpcChannel(serverName, user, rpcTimeout); @@ -1389,44 +1381,38 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { MasterServiceState mss = masterServiceState; @Override - public MasterProtos.AbortProcedureResponse abortProcedure( - RpcController controller, + public MasterProtos.AbortProcedureResponse abortProcedure(RpcController controller, MasterProtos.AbortProcedureRequest request) throws ServiceException { return stub.abortProcedure(controller, request); } @Override - public MasterProtos.GetProceduresResponse getProcedures( - RpcController controller, + public MasterProtos.GetProceduresResponse getProcedures(RpcController controller, MasterProtos.GetProceduresRequest request) throws ServiceException { return stub.getProcedures(controller, request); } @Override - public MasterProtos.GetLocksResponse getLocks( - RpcController controller, + public MasterProtos.GetLocksResponse getLocks(RpcController controller, MasterProtos.GetLocksRequest request) throws ServiceException { return stub.getLocks(controller, request); } @Override - public MasterProtos.AddColumnResponse addColumn( - RpcController controller, + public MasterProtos.AddColumnResponse addColumn(RpcController controller, MasterProtos.AddColumnRequest request) throws ServiceException { return stub.addColumn(controller, request); } @Override public MasterProtos.DeleteColumnResponse deleteColumn(RpcController controller, - MasterProtos.DeleteColumnRequest request) - throws ServiceException { + MasterProtos.DeleteColumnRequest request) throws ServiceException { return stub.deleteColumn(controller, request); } @Override public MasterProtos.ModifyColumnResponse modifyColumn(RpcController controller, - MasterProtos.ModifyColumnRequest request) - throws ServiceException { + MasterProtos.ModifyColumnRequest request) throws ServiceException { return stub.modifyColumn(controller, request); } @@ -1437,9 +1423,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public MasterProtos.MergeTableRegionsResponse mergeTableRegions( - RpcController controller, MasterProtos.MergeTableRegionsRequest request) - throws ServiceException { + public MasterProtos.MergeTableRegionsResponse mergeTableRegions(RpcController controller, + MasterProtos.MergeTableRegionsRequest request) throws ServiceException { return stub.mergeTableRegions(controller, request); } @@ -1517,8 +1502,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { @Override public MasterProtos.IsInMaintenanceModeResponse isMasterInMaintenanceMode( - final RpcController controller, - final MasterProtos.IsInMaintenanceModeRequest request) throws ServiceException { + final RpcController controller, final MasterProtos.IsInMaintenanceModeRequest request) + throws ServiceException { return stub.isMasterInMaintenanceMode(controller, request); } @@ -1529,22 +1514,20 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public MasterProtos.SetBalancerRunningResponse setBalancerRunning( - RpcController controller, MasterProtos.SetBalancerRunningRequest request) - throws ServiceException { + public MasterProtos.SetBalancerRunningResponse setBalancerRunning(RpcController controller, + MasterProtos.SetBalancerRunningRequest request) throws ServiceException { return stub.setBalancerRunning(controller, request); } @Override - public NormalizeResponse normalize(RpcController controller, - NormalizeRequest request) throws ServiceException { + public NormalizeResponse normalize(RpcController controller, NormalizeRequest request) + throws ServiceException { return stub.normalize(controller, request); } @Override - public SetNormalizerRunningResponse setNormalizerRunning( - RpcController controller, SetNormalizerRunningRequest request) - throws ServiceException { + public SetNormalizerRunningResponse setNormalizerRunning(RpcController controller, + SetNormalizerRunningRequest request) throws ServiceException { return stub.setNormalizerRunning(controller, request); } @@ -1570,8 +1553,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { @Override public MasterProtos.RunCleanerChoreResponse runCleanerChore(RpcController controller, - MasterProtos.RunCleanerChoreRequest request) - throws ServiceException { + MasterProtos.RunCleanerChoreRequest request) throws ServiceException { return stub.runCleanerChore(controller, request); } @@ -1590,9 +1572,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public ClientProtos.CoprocessorServiceResponse execMasterService( - RpcController controller, ClientProtos.CoprocessorServiceRequest request) - throws ServiceException { + public ClientProtos.CoprocessorServiceResponse execMasterService(RpcController controller, + ClientProtos.CoprocessorServiceRequest request) throws ServiceException { return stub.execMasterService(controller, request); } @@ -1622,16 +1603,14 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public MasterProtos.RestoreSnapshotResponse restoreSnapshot( - RpcController controller, MasterProtos.RestoreSnapshotRequest request) - throws ServiceException { + public MasterProtos.RestoreSnapshotResponse restoreSnapshot(RpcController controller, + MasterProtos.RestoreSnapshotRequest request) throws ServiceException { return stub.restoreSnapshot(controller, request); } @Override - public MasterProtos.SetSnapshotCleanupResponse switchSnapshotCleanup( - RpcController controller, MasterProtos.SetSnapshotCleanupRequest request) - throws ServiceException { + public MasterProtos.SetSnapshotCleanupResponse switchSnapshotCleanup(RpcController controller, + MasterProtos.SetSnapshotCleanupRequest request) throws ServiceException { return stub.switchSnapshotCleanup(controller, request); } @@ -1643,16 +1622,14 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public MasterProtos.ExecProcedureResponse execProcedure( - RpcController controller, MasterProtos.ExecProcedureRequest request) - throws ServiceException { + public MasterProtos.ExecProcedureResponse execProcedure(RpcController controller, + MasterProtos.ExecProcedureRequest request) throws ServiceException { return stub.execProcedure(controller, request); } @Override - public MasterProtos.ExecProcedureResponse execProcedureWithRet( - RpcController controller, MasterProtos.ExecProcedureRequest request) - throws ServiceException { + public MasterProtos.ExecProcedureResponse execProcedureWithRet(RpcController controller, + MasterProtos.ExecProcedureRequest request) throws ServiceException { return stub.execProcedureWithRet(controller, request); } @@ -1669,72 +1646,66 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public MasterProtos.IsMasterRunningResponse isMasterRunning( - RpcController controller, MasterProtos.IsMasterRunningRequest request) - throws ServiceException { + public MasterProtos.IsMasterRunningResponse isMasterRunning(RpcController controller, + MasterProtos.IsMasterRunningRequest request) throws ServiceException { return stub.isMasterRunning(controller, request); } @Override public MasterProtos.ModifyNamespaceResponse modifyNamespace(RpcController controller, - MasterProtos.ModifyNamespaceRequest request) - throws ServiceException { + MasterProtos.ModifyNamespaceRequest request) throws ServiceException { return stub.modifyNamespace(controller, request); } @Override - public MasterProtos.CreateNamespaceResponse createNamespace( - RpcController controller, + public MasterProtos.CreateNamespaceResponse createNamespace(RpcController controller, MasterProtos.CreateNamespaceRequest request) throws ServiceException { return stub.createNamespace(controller, request); } @Override - public MasterProtos.DeleteNamespaceResponse deleteNamespace( - RpcController controller, + public MasterProtos.DeleteNamespaceResponse deleteNamespace(RpcController controller, MasterProtos.DeleteNamespaceRequest request) throws ServiceException { return stub.deleteNamespace(controller, request); } @Override - public MasterProtos.ListNamespacesResponse listNamespaces( - RpcController controller, + public MasterProtos.ListNamespacesResponse listNamespaces(RpcController controller, MasterProtos.ListNamespacesRequest request) throws ServiceException { return stub.listNamespaces(controller, request); } @Override public MasterProtos.GetNamespaceDescriptorResponse getNamespaceDescriptor( - RpcController controller, - MasterProtos.GetNamespaceDescriptorRequest request) throws ServiceException { + RpcController controller, MasterProtos.GetNamespaceDescriptorRequest request) + throws ServiceException { return stub.getNamespaceDescriptor(controller, request); } @Override public MasterProtos.ListNamespaceDescriptorsResponse listNamespaceDescriptors( - RpcController controller, - MasterProtos.ListNamespaceDescriptorsRequest request) throws ServiceException { + RpcController controller, MasterProtos.ListNamespaceDescriptorsRequest request) + throws ServiceException { return stub.listNamespaceDescriptors(controller, request); } @Override public MasterProtos.ListTableDescriptorsByNamespaceResponse listTableDescriptorsByNamespace( RpcController controller, MasterProtos.ListTableDescriptorsByNamespaceRequest request) - throws ServiceException { + throws ServiceException { return stub.listTableDescriptorsByNamespace(controller, request); } @Override public MasterProtos.ListTableNamesByNamespaceResponse listTableNamesByNamespace( RpcController controller, MasterProtos.ListTableNamesByNamespaceRequest request) - throws ServiceException { + throws ServiceException { return stub.listTableNamesByNamespace(controller, request); } @Override - public MasterProtos.GetTableStateResponse getTableState( - RpcController controller, MasterProtos.GetTableStateRequest request) - throws ServiceException { + public MasterProtos.GetTableStateResponse getTableState(RpcController controller, + MasterProtos.GetTableStateRequest request) throws ServiceException { return stub.getTableState(controller, request); } @@ -1751,30 +1722,26 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public MasterProtos.GetTableDescriptorsResponse getTableDescriptors( - RpcController controller, MasterProtos.GetTableDescriptorsRequest request) - throws ServiceException { + public MasterProtos.GetTableDescriptorsResponse getTableDescriptors(RpcController controller, + MasterProtos.GetTableDescriptorsRequest request) throws ServiceException { return stub.getTableDescriptors(controller, request); } @Override - public MasterProtos.GetTableNamesResponse getTableNames( - RpcController controller, MasterProtos.GetTableNamesRequest request) - throws ServiceException { + public MasterProtos.GetTableNamesResponse getTableNames(RpcController controller, + MasterProtos.GetTableNamesRequest request) throws ServiceException { return stub.getTableNames(controller, request); } @Override - public MasterProtos.GetClusterStatusResponse getClusterStatus( - RpcController controller, MasterProtos.GetClusterStatusRequest request) - throws ServiceException { + public MasterProtos.GetClusterStatusResponse getClusterStatus(RpcController controller, + MasterProtos.GetClusterStatusRequest request) throws ServiceException { return stub.getClusterStatus(controller, request); } @Override - public MasterProtos.SetQuotaResponse setQuota( - RpcController controller, MasterProtos.SetQuotaRequest request) - throws ServiceException { + public MasterProtos.SetQuotaResponse setQuota(RpcController controller, + MasterProtos.SetQuotaRequest request) throws ServiceException { return stub.setQuota(controller, request); } @@ -1800,15 +1767,15 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { @Override public MasterProtos.SetSplitOrMergeEnabledResponse setSplitOrMergeEnabled( - RpcController controller, MasterProtos.SetSplitOrMergeEnabledRequest request) - throws ServiceException { + RpcController controller, MasterProtos.SetSplitOrMergeEnabledRequest request) + throws ServiceException { return stub.setSplitOrMergeEnabled(controller, request); } @Override public MasterProtos.IsSplitOrMergeEnabledResponse isSplitOrMergeEnabled( - RpcController controller, MasterProtos.IsSplitOrMergeEnabledRequest request) - throws ServiceException { + RpcController controller, MasterProtos.IsSplitOrMergeEnabledRequest request) + throws ServiceException { return stub.isSplitOrMergeEnabled(controller, request); } @@ -1849,8 +1816,9 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public ListDecommissionedRegionServersResponse listDecommissionedRegionServers(RpcController controller, - ListDecommissionedRegionServersRequest request) throws ServiceException { + public ListDecommissionedRegionServersResponse listDecommissionedRegionServers( + RpcController controller, ListDecommissionedRegionServersRequest request) + throws ServiceException { return stub.listDecommissionedRegionServers(controller, request); } @@ -1861,9 +1829,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public RecommissionRegionServerResponse recommissionRegionServer( - RpcController controller, RecommissionRegionServerRequest request) - throws ServiceException { + public RecommissionRegionServerResponse recommissionRegionServer(RpcController controller, + RecommissionRegionServerRequest request) throws ServiceException { return stub.recommissionRegionServer(controller, request); } @@ -1887,15 +1854,14 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes( - RpcController controller, GetSpaceQuotaRegionSizesRequest request) - throws ServiceException { + public GetSpaceQuotaRegionSizesResponse getSpaceQuotaRegionSizes(RpcController controller, + GetSpaceQuotaRegionSizesRequest request) throws ServiceException { return stub.getSpaceQuotaRegionSizes(controller, request); } @Override - public GetQuotaStatesResponse getQuotaStates( - RpcController controller, GetQuotaStatesRequest request) throws ServiceException { + public GetQuotaStatesResponse getQuotaStates(RpcController controller, + GetQuotaStatesRequest request) throws ServiceException { return stub.getQuotaStates(controller, request); } @@ -1955,15 +1921,15 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { @Override public ModifyTableStoreFileTrackerResponse modifyTableStoreFileTracker( - RpcController controller, ModifyTableStoreFileTrackerRequest request) - throws ServiceException { + RpcController controller, ModifyTableStoreFileTrackerRequest request) + throws ServiceException { return stub.modifyTableStoreFileTracker(controller, request); } @Override public ModifyColumnStoreFileTrackerResponse modifyColumnStoreFileTracker( - RpcController controller, ModifyColumnStoreFileTrackerRequest request) - throws ServiceException { + RpcController controller, ModifyColumnStoreFileTrackerRequest request) + throws ServiceException { return stub.modifyColumnStoreFileTracker(controller, request); } }; @@ -1971,19 +1937,19 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { private static void release(MasterServiceState mss) { if (mss != null && mss.connection != null) { - ((ConnectionImplementation)mss.connection).releaseMaster(mss); + ((ConnectionImplementation) mss.connection).releaseMaster(mss); } } private boolean isKeepAliveMasterConnectedAndRunning(MasterServiceState mss) { - if (mss.getStub() == null){ + if (mss.getStub() == null) { return false; } try { return mss.isMasterRunning(); } catch (UndeclaredThrowableException e) { // It's somehow messy, but we can receive exceptions such as - // java.net.ConnectException but they're not declared. So we catch it... + // java.net.ConnectException but they're not declared. So we catch it... LOG.info("Master connection is not running anymore", e.getUndeclaredThrowable()); return false; } catch (IOException se) { @@ -2030,19 +1996,19 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } /** - * Update the location with the new value (if the exception is a RegionMovedException) - * or delete it from the cache. Does nothing if we can be sure from the exception that - * the location is still accurate, or if the cache has already been updated. - * @param exception an object (to simplify user code) on which we will try to find a nested - * or wrapped or both RegionMovedException + * Update the location with the new value (if the exception is a RegionMovedException) or delete + * it from the cache. Does nothing if we can be sure from the exception that the location is still + * accurate, or if the cache has already been updated. + * @param exception an object (to simplify user code) on which we will try to find a nested or + * wrapped or both RegionMovedException * @param source server that is the source of the location update. */ @Override public void updateCachedLocations(final TableName tableName, byte[] regionName, byte[] rowkey, - final Object exception, final ServerName source) { + final Object exception, final ServerName source) { if (rowkey == null || tableName == null) { - LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) + - ", tableName=" + (tableName == null ? "null" : tableName)); + LOG.warn("Coding error, see method javadoc. row=" + (rowkey == null ? "null" : rowkey) + + ", tableName=" + (tableName == null ? "null" : tableName)); return; } @@ -2068,7 +2034,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } if (oldLocation == null || !source.equals(oldLocation.getServerName())) { // There is no such location in the cache (it's been removed already) or - // the cache has already been refreshed with a different location. => nothing to do + // the cache has already been refreshed with a different location. => nothing to do return; } @@ -2083,14 +2049,12 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { if (cause instanceof RegionMovedException) { RegionMovedException rme = (RegionMovedException) cause; if (LOG.isTraceEnabled()) { - LOG.trace("Region " + regionInfo.getRegionNameAsString() + " moved to " + - rme.getHostname() + ":" + rme.getPort() + - " according to " + source.getAddress()); + LOG.trace("Region " + regionInfo.getRegionNameAsString() + " moved to " + + rme.getHostname() + ":" + rme.getPort() + " according to " + source.getAddress()); } // We know that the region is not anymore on this region server, but we know - // the new location. - updateCachedLocation( - regionInfo, source, rme.getServerName(), rme.getLocationSeqNum()); + // the new location. + updateCachedLocation(regionInfo, source, rme.getServerName(), rme.getLocationSeqNum()); return; } } @@ -2127,8 +2091,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } /* - * Return the number of cached region for a table. It will only be called - * from a unit test. + * Return the number of cached region for a table. It will only be called from a unit test. */ int getNumberOfCachedRegionLocations(final TableName tableName) { return metaCache.getNumberOfCachedRegionLocations(tableName); @@ -2152,7 +2115,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } @Override - public boolean isAborted(){ + public boolean isAborted() { return this.aborted; } @@ -2187,12 +2150,11 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } /** - * Close the connection for good. On the off chance that someone is unable to close - * the connection, perhaps because it bailed out prematurely, the method - * below will ensure that this instance is cleaned up. - * Caveat: The JVM may take an unknown amount of time to call finalize on an - * unreachable object, so our hope is that every consumer cleans up after - * itself, like any good citizen. + * Close the connection for good. On the off chance that someone is unable to close the + * connection, perhaps because it bailed out prematurely, the method below will ensure that this + * instance is cleaned up. Caveat: The JVM may take an unknown amount of time to call finalize on + * an unreachable object, so our hope is that every consumer cleans up after itself, like any good + * citizen. */ @Override protected void finalize() throws Throwable { @@ -2217,8 +2179,8 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { @Override public RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf) { - return RpcRetryingCallerFactory - .instantiate(conf, this.interceptor, this.getStatisticsTracker()); + return RpcRetryingCallerFactory.instantiate(conf, this.interceptor, + this.getStatisticsTracker()); } @Override @@ -2256,7 +2218,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { @Override public List getLiveRegionServers(Supplier masterAddrTracker, int count) - throws IOException { + throws IOException { RegionServerStatusService.BlockingInterface stub = RegionServerStatusService.newBlockingStub( rpcClient.createBlockingRpcChannel(masterAddrTracker.get(), user, rpcTimeout)); GetLiveRegionServersResponse resp; @@ -2269,13 +2231,13 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { throw new IOException(t); } return resp.getServerList().stream().map(ProtobufUtil::toServerName) - .collect(Collectors.toList()); + .collect(Collectors.toList()); } @Override public List getAllBootstrapNodes(ServerName regionServer) throws IOException { BootstrapNodeService.BlockingInterface stub = BootstrapNodeService - .newBlockingStub(rpcClient.createBlockingRpcChannel(regionServer, user, rpcTimeout)); + .newBlockingStub(rpcClient.createBlockingRpcChannel(regionServer, user, rpcTimeout)); GetAllBootstrapNodesResponse resp; try { resp = stub.getAllBootstrapNodes(null, GetAllBootstrapNodesRequest.getDefaultInstance()); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index f6221fd22f1..fe2ae0c82ce 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -19,14 +19,14 @@ package org.apache.hadoop.hbase.client; // DO NOT MAKE USE OF THESE IMPORTS! THEY ARE HERE FOR COPROCESSOR ENDPOINTS ONLY. // Internally, we use shaded protobuf. This below are part of our public API. -//SEE ABOVE NOTE! +// SEE ABOVE NOTE! import static org.apache.hadoop.hbase.client.ConnectionUtils.checkHasFamilies; + import com.google.protobuf.Descriptors; import com.google.protobuf.Message; import com.google.protobuf.Service; import com.google.protobuf.ServiceException; - import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.SpanKind; import io.opentelemetry.api.trace.StatusCode; @@ -75,6 +75,7 @@ import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; @@ -84,23 +85,20 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse; /** - * An implementation of {@link Table}. Used to communicate with a single HBase table. - * Lightweight. Get as needed and just close when done. - * Instances of this class SHOULD NOT be constructed directly. - * Obtain an instance via {@link Connection}. See {@link ConnectionFactory} - * class comment for an example of how. - * - *

This class is thread safe since 2.0.0 if not invoking any of the setter methods. - * All setters are moved into {@link TableBuilder} and reserved here only for keeping - * backward compatibility, and TODO will be removed soon. - * - *

HTable is no longer a client API. Use {@link Table} instead. It is marked - * InterfaceAudience.Private indicating that this is an HBase-internal class as defined in - * Hadoop - * Interface Classification - * There are no guarantees for backwards source / binary compatibility and methods or class can - * change or go away without deprecation. - * + * An implementation of {@link Table}. Used to communicate with a single HBase table. Lightweight. + * Get as needed and just close when done. Instances of this class SHOULD NOT be constructed + * directly. Obtain an instance via {@link Connection}. See {@link ConnectionFactory} class comment + * for an example of how. + *

+ * This class is thread safe since 2.0.0 if not invoking any of the setter methods. All setters are + * moved into {@link TableBuilder} and reserved here only for keeping backward compatibility, and + * TODO will be removed soon. + *

+ * HTable is no longer a client API. Use {@link Table} instead. It is marked + * InterfaceAudience.Private indicating that this is an HBase-internal class as defined in Hadoop + * Interface Classification There are no guarantees for backwards source / binary compatibility + * and methods or class can change or go away without deprecation. * @see Table * @see Admin * @see Connection @@ -118,7 +116,7 @@ public class HTable implements Table { private boolean closed = false; private final int scannerCaching; private final long scannerMaxResultSize; - private final ExecutorService pool; // For Multi & Scan + private final ExecutorService pool; // For Multi & Scan private int operationTimeoutMs; // global timeout for each blocking method with retrying rpc private final int rpcTimeoutMs; // FIXME we should use this for rpc like batch and checkAndXXX private int readRpcTimeoutMs; // timeout for each read rpc request @@ -146,17 +144,17 @@ public class HTable implements Table { // we only create as many Runnables as there are region servers. It means // it also scales when new region servers are added. ThreadPoolExecutor pool = - new ThreadPoolExecutor(corePoolSize, maxThreads, keepAliveTime, TimeUnit.SECONDS, - new SynchronousQueue<>(), new ThreadFactoryBuilder().setNameFormat("htable-pool-%d") - .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build()); + new ThreadPoolExecutor(corePoolSize, maxThreads, keepAliveTime, TimeUnit.SECONDS, + new SynchronousQueue<>(), new ThreadFactoryBuilder().setNameFormat("htable-pool-%d") + .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build()); pool.allowCoreThreadTimeOut(true); return pool; } /** - * Creates an object to access a HBase table. - * Used by HBase internally. DO NOT USE. See {@link ConnectionFactory} class comment for how to - * get a {@link Table} instance (use {@link Table} instead of {@link HTable}). + * Creates an object to access a HBase table. Used by HBase internally. DO NOT USE. See + * {@link ConnectionFactory} class comment for how to get a {@link Table} instance (use + * {@link Table} instead of {@link HTable}). * @param connection Connection to be used. * @param builder The table builder * @param rpcCallerFactory The RPC caller factory @@ -164,11 +162,9 @@ public class HTable implements Table { * @param pool ExecutorService to be used. */ @InterfaceAudience.Private - protected HTable(final ConnectionImplementation connection, - final TableBuilderBase builder, + protected HTable(final ConnectionImplementation connection, final TableBuilderBase builder, final RpcRetryingCallerFactory rpcCallerFactory, - final RpcControllerFactory rpcControllerFactory, - final ExecutorService pool) { + final RpcControllerFactory rpcControllerFactory, final ExecutorService pool) { this.connection = Preconditions.checkNotNull(connection, "connection is null"); this.configuration = connection.getConfiguration(); this.connConfiguration = connection.getConnectionConfiguration(); @@ -222,8 +218,7 @@ public class HTable implements Table { } /** - * INTERNAL Used by unit tests and tools to do low-level - * manipulations. + * INTERNAL Used by unit tests and tools to do low-level manipulations. * @return A Connection instance. */ protected Connection getConnection() { @@ -248,42 +243,37 @@ public class HTable implements Table { } /** - * Get the corresponding start keys and regions for an arbitrary range of - * keys. + * Get the corresponding start keys and regions for an arbitrary range of keys. *

* @param startKey Starting row in range, inclusive * @param endKey Ending row in range * @param includeEndKey true if endRow is inclusive, false if exclusive - * @return A pair of list of start keys and list of HRegionLocations that - * contain the specified range + * @return A pair of list of start keys and list of HRegionLocations that contain the specified + * range * @throws IOException if a remote or network exception occurs */ - private Pair, List> getKeysAndRegionsInRange( - final byte[] startKey, final byte[] endKey, final boolean includeEndKey) - throws IOException { + private Pair, List> getKeysAndRegionsInRange(final byte[] startKey, + final byte[] endKey, final boolean includeEndKey) throws IOException { return getKeysAndRegionsInRange(startKey, endKey, includeEndKey, false); } /** - * Get the corresponding start keys and regions for an arbitrary range of - * keys. + * Get the corresponding start keys and regions for an arbitrary range of keys. *

* @param startKey Starting row in range, inclusive * @param endKey Ending row in range * @param includeEndKey true if endRow is inclusive, false if exclusive * @param reload true to reload information or false to use cached information - * @return A pair of list of start keys and list of HRegionLocations that - * contain the specified range + * @return A pair of list of start keys and list of HRegionLocations that contain the specified + * range * @throws IOException if a remote or network exception occurs */ - private Pair, List> getKeysAndRegionsInRange( - final byte[] startKey, final byte[] endKey, final boolean includeEndKey, - final boolean reload) throws IOException { - final boolean endKeyIsEndOfTable = Bytes.equals(endKey,HConstants.EMPTY_END_ROW); + private Pair, List> getKeysAndRegionsInRange(final byte[] startKey, + final byte[] endKey, final boolean includeEndKey, final boolean reload) throws IOException { + final boolean endKeyIsEndOfTable = Bytes.equals(endKey, HConstants.EMPTY_END_ROW); if ((Bytes.compareTo(startKey, endKey) > 0) && !endKeyIsEndOfTable) { - throw new IllegalArgumentException( - "Invalid range: " + Bytes.toStringBinary(startKey) + - " > " + Bytes.toStringBinary(endKey)); + throw new IllegalArgumentException("Invalid range: " + Bytes.toStringBinary(startKey) + " > " + + Bytes.toStringBinary(endKey)); } List keysInRange = new ArrayList<>(); List regionsInRange = new ArrayList<>(); @@ -300,61 +290,60 @@ public class HTable implements Table { } /** - * The underlying {@link HTable} must not be closed. - * {@link Table#getScanner(Scan)} has other usage details. + * The underlying {@link HTable} must not be closed. {@link Table#getScanner(Scan)} has other + * usage details. */ @Override public ResultScanner getScanner(Scan scan) throws IOException { - if (scan.getCaching() <= 0) { - scan.setCaching(scannerCaching); - } - if (scan.getMaxResultSize() <= 0) { - scan.setMaxResultSize(scannerMaxResultSize); - } - if (scan.getMvccReadPoint() > 0) { - // it is not supposed to be set by user, clear - scan.resetMvccReadPoint(); - } - Boolean async = scan.isAsyncPrefetch(); - if (async == null) { - async = connConfiguration.isClientScannerAsyncPrefetch(); - } + final Span span = new TableOperationSpanBuilder(connection).setTableName(tableName) + .setOperation(scan).build(); + try (Scope ignored = span.makeCurrent()) { + if (scan.getCaching() <= 0) { + scan.setCaching(scannerCaching); + } + if (scan.getMaxResultSize() <= 0) { + scan.setMaxResultSize(scannerMaxResultSize); + } + if (scan.getMvccReadPoint() > 0) { + // it is not supposed to be set by user, clear + scan.resetMvccReadPoint(); + } + final boolean async = scan.isAsyncPrefetch() != null ? scan.isAsyncPrefetch() + : connConfiguration.isClientScannerAsyncPrefetch(); + final int timeout = connConfiguration.getReplicaCallTimeoutMicroSecondScan(); - if (scan.isReversed()) { - return new ReversedClientScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); - } else { - if (async) { - return new ClientAsyncPrefetchScanner(getConfiguration(), scan, getName(), this.connection, - this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); + if (scan.isReversed()) { + return new ReversedClientScanner(getConfiguration(), scan, getName(), connection, + rpcCallerFactory, rpcControllerFactory, pool, timeout); } else { - return new ClientSimpleScanner(getConfiguration(), scan, getName(), this.connection, - this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); + if (async) { + return new ClientAsyncPrefetchScanner(getConfiguration(), scan, getName(), connection, + rpcCallerFactory, rpcControllerFactory, pool, timeout); + } else { + return new ClientSimpleScanner(getConfiguration(), scan, getName(), connection, + rpcCallerFactory, rpcControllerFactory, pool, timeout); + } } } } /** - * The underlying {@link HTable} must not be closed. - * {@link Table#getScanner(byte[])} has other usage details. + * The underlying {@link HTable} must not be closed. {@link Table#getScanner(byte[])} has other + * usage details. */ @Override - public ResultScanner getScanner(byte [] family) throws IOException { + public ResultScanner getScanner(byte[] family) throws IOException { Scan scan = new Scan(); scan.addFamily(family); return getScanner(scan); } /** - * The underlying {@link HTable} must not be closed. - * {@link Table#getScanner(byte[], byte[])} has other usage details. + * The underlying {@link HTable} must not be closed. {@link Table#getScanner(byte[], byte[])} has + * other usage details. */ @Override - public ResultScanner getScanner(byte [] family, byte [] qualifier) - throws IOException { + public ResultScanner getScanner(byte[] family, byte[] qualifier) throws IOException { Scan scan = new Scan(); scan.addColumn(family, qualifier); return getScanner(scan); @@ -362,9 +351,8 @@ public class HTable implements Table { @Override public Result get(final Get get) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(get); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName).setOperation(get); return TraceUtil.trace(() -> get(get, get.isCheckExistenceOnly()), supplier); } @@ -373,42 +361,41 @@ public class HTable implements Table { if (get.isCheckExistenceOnly() != checkExistenceOnly || get.getConsistency() == null) { get = ReflectionUtils.newInstance(get.getClass(), get); get.setCheckExistenceOnly(checkExistenceOnly); - if (get.getConsistency() == null){ + if (get.getConsistency() == null) { get.setConsistency(DEFAULT_CONSISTENCY); } } if (get.getConsistency() == Consistency.STRONG) { final Get configuredGet = get; - ClientServiceCallable callable = new ClientServiceCallable(this.connection, getName(), - get.getRow(), this.rpcControllerFactory.newController(), get.getPriority()) { + ClientServiceCallable callable = new ClientServiceCallable(this.connection, + getName(), get.getRow(), this.rpcControllerFactory.newController(), get.getPriority()) { @Override protected Result rpcCall() throws Exception { - ClientProtos.GetRequest request = RequestConverter.buildGetRequest( - getLocation().getRegionInfo().getRegionName(), configuredGet); + ClientProtos.GetRequest request = RequestConverter + .buildGetRequest(getLocation().getRegionInfo().getRegionName(), configuredGet); ClientProtos.GetResponse response = doGet(request); - return response == null? null: - ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); + return response == null ? null + : ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); } }; - return rpcCallerFactory.newCaller(readRpcTimeoutMs).callWithRetries(callable, - this.operationTimeoutMs); + return rpcCallerFactory. newCaller(readRpcTimeoutMs).callWithRetries(callable, + this.operationTimeoutMs); } // Call that takes into account the replica - RpcRetryingCallerWithReadReplicas callable = new RpcRetryingCallerWithReadReplicas( - rpcControllerFactory, tableName, this.connection, get, pool, - connConfiguration.getRetriesNumber(), operationTimeoutMs, readRpcTimeoutMs, - connConfiguration.getPrimaryCallTimeoutMicroSecond()); + RpcRetryingCallerWithReadReplicas callable = + new RpcRetryingCallerWithReadReplicas(rpcControllerFactory, tableName, this.connection, get, + pool, connConfiguration.getRetriesNumber(), operationTimeoutMs, readRpcTimeoutMs, + connConfiguration.getPrimaryCallTimeoutMicroSecond()); return callable.call(operationTimeoutMs); } @Override public Result[] get(List gets) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.BATCH) - .setContainerOperations(gets); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName) + .setOperation(HBaseSemanticAttributes.Operation.BATCH).setContainerOperations(gets); return TraceUtil.trace(() -> { if (gets.size() == 1) { return new Result[] { get(gets.get(0)) }; @@ -458,20 +445,13 @@ public class HTable implements Table { public void batch(final List actions, final Object[] results, int rpcTimeout) throws InterruptedException, IOException { - AsyncProcessTask task = AsyncProcessTask.newBuilder() - .setPool(pool) - .setTableName(tableName) - .setRowAccess(actions) - .setResults(results) - .setRpcTimeout(rpcTimeout) - .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .build(); - final Span span = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.BATCH) - .setContainerOperations(actions) - .build(); + AsyncProcessTask task = + AsyncProcessTask.newBuilder().setPool(pool).setTableName(tableName).setRowAccess(actions) + .setResults(results).setRpcTimeout(rpcTimeout).setOperationTimeout(operationTimeoutMs) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).build(); + final Span span = new TableOperationSpanBuilder(connection).setTableName(tableName) + .setOperation(HBaseSemanticAttributes.Operation.BATCH).setContainerOperations(actions) + .build(); try (Scope ignored = span.makeCurrent()) { AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); @@ -486,33 +466,25 @@ public class HTable implements Table { } @Override - public void batchCallback( - final List actions, final Object[] results, final Batch.Callback callback) - throws IOException, InterruptedException { + public void batchCallback(final List actions, final Object[] results, + final Batch.Callback callback) throws IOException, InterruptedException { doBatchWithCallback(actions, results, callback, connection, pool, tableName); } public static void doBatchWithCallback(List actions, Object[] results, - Callback callback, ClusterConnection connection, ExecutorService pool, TableName tableName) - throws InterruptedIOException, RetriesExhaustedWithDetailsException { + Callback callback, ClusterConnection connection, ExecutorService pool, TableName tableName) + throws InterruptedIOException, RetriesExhaustedWithDetailsException { int operationTimeout = connection.getConnectionConfiguration().getOperationTimeout(); int writeTimeout = connection.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY, - connection.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, - HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); - AsyncProcessTask task = AsyncProcessTask.newBuilder(callback) - .setPool(pool) - .setTableName(tableName) - .setRowAccess(actions) - .setResults(results) - .setOperationTimeout(operationTimeout) - .setRpcTimeout(writeTimeout) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .build(); - final Span span = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.BATCH) - .setContainerOperations(actions) - .build(); + connection.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, + HConstants.DEFAULT_HBASE_RPC_TIMEOUT)); + AsyncProcessTask task = AsyncProcessTask.newBuilder(callback).setPool(pool) + .setTableName(tableName).setRowAccess(actions).setResults(results) + .setOperationTimeout(operationTimeout).setRpcTimeout(writeTimeout) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).build(); + final Span span = new TableOperationSpanBuilder(connection).setTableName(tableName) + .setOperation(HBaseSemanticAttributes.Operation.BATCH).setContainerOperations(actions) + .build(); try (Scope ignored = span.makeCurrent()) { AsyncRequestFuture ars = connection.getAsyncProcess().submit(task); ars.waitUntilDone(); @@ -527,23 +499,22 @@ public class HTable implements Table { @Override public void delete(final Delete delete) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(delete); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName).setOperation(delete); TraceUtil.trace(() -> { ClientServiceCallable callable = - new ClientServiceCallable(this.connection, getName(), delete.getRow(), - this.rpcControllerFactory.newController(), delete.getPriority()) { - @Override - protected Void rpcCall() throws Exception { - MutateRequest request = RequestConverter - .buildMutateRequest(getLocation().getRegionInfo().getRegionName(), delete); - doMutate(request); - return null; - } - }; - rpcCallerFactory.newCaller(this.writeRpcTimeoutMs) - .callWithRetries(callable, this.operationTimeoutMs); + new ClientServiceCallable(this.connection, getName(), delete.getRow(), + this.rpcControllerFactory.newController(), delete.getPriority()) { + @Override + protected Void rpcCall() throws Exception { + MutateRequest request = RequestConverter + .buildMutateRequest(getLocation().getRegionInfo().getRegionName(), delete); + doMutate(request); + return null; + } + }; + rpcCallerFactory. newCaller(this.writeRpcTimeoutMs).callWithRetries(callable, + this.operationTimeoutMs); }, supplier); } @@ -570,24 +541,22 @@ public class HTable implements Table { @Override public void put(final Put put) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(put); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName).setOperation(put); TraceUtil.trace(() -> { validatePut(put); - ClientServiceCallable callable = - new ClientServiceCallable(this.connection, getName(), put.getRow(), - this.rpcControllerFactory.newController(), put.getPriority()) { - @Override - protected Void rpcCall() throws Exception { - MutateRequest request = RequestConverter + ClientServiceCallable callable = new ClientServiceCallable(this.connection, + getName(), put.getRow(), this.rpcControllerFactory.newController(), put.getPriority()) { + @Override + protected Void rpcCall() throws Exception { + MutateRequest request = RequestConverter .buildMutateRequest(getLocation().getRegionInfo().getRegionName(), put); - doMutate(request); - return null; - } - }; - rpcCallerFactory.newCaller(this.writeRpcTimeoutMs) - .callWithRetries(callable, this.operationTimeoutMs); + doMutate(request); + return null; + } + }; + rpcCallerFactory. newCaller(this.writeRpcTimeoutMs).callWithRetries(callable, + this.operationTimeoutMs); }, supplier); } @@ -606,44 +575,38 @@ public class HTable implements Table { @Override public Result mutateRow(final RowMutations rm) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.BATCH) - .setContainerOperations(rm); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName) + .setOperation(HBaseSemanticAttributes.Operation.BATCH).setContainerOperations(rm); return TraceUtil.trace(() -> { long nonceGroup = getNonceGroup(); long nonce = getNonce(); CancellableRegionServerCallable callable = - new CancellableRegionServerCallable(this.connection, getName(), rm.getRow(), - rpcControllerFactory.newController(), writeRpcTimeoutMs, - new RetryingTimeTracker().start(), rm.getMaxPriority()) { - @Override - protected MultiResponse rpcCall() throws Exception { - MultiRequest request = RequestConverter.buildMultiRequest( - getLocation().getRegionInfo().getRegionName(), rm, nonceGroup, nonce); - ClientProtos.MultiResponse response = doMulti(request); - ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0); - if (res.hasException()) { - Throwable ex = ProtobufUtil.toException(res.getException()); - if (ex instanceof IOException) { - throw (IOException) ex; + new CancellableRegionServerCallable(this.connection, getName(), + rm.getRow(), rpcControllerFactory.newController(), writeRpcTimeoutMs, + new RetryingTimeTracker().start(), rm.getMaxPriority()) { + @Override + protected MultiResponse rpcCall() throws Exception { + MultiRequest request = RequestConverter.buildMultiRequest( + getLocation().getRegionInfo().getRegionName(), rm, nonceGroup, nonce); + ClientProtos.MultiResponse response = doMulti(request); + ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0); + if (res.hasException()) { + Throwable ex = ProtobufUtil.toException(res.getException()); + if (ex instanceof IOException) { + throw (IOException) ex; + } + throw new IOException("Failed to mutate row: " + Bytes.toStringBinary(rm.getRow()), + ex); + } + return ResponseConverter.getResults(request, response, getRpcControllerCellScanner()); } - throw new IOException("Failed to mutate row: " + Bytes.toStringBinary(rm.getRow()), ex); - } - return ResponseConverter.getResults(request, response, getRpcControllerCellScanner()); - } - }; + }; Object[] results = new Object[rm.getMutations().size()]; - AsyncProcessTask task = AsyncProcessTask.newBuilder() - .setPool(pool) - .setTableName(tableName) - .setRowAccess(rm.getMutations()) - .setCallable(callable) - .setRpcTimeout(writeRpcTimeoutMs) - .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .setResults(results) - .build(); + AsyncProcessTask task = AsyncProcessTask.newBuilder().setPool(pool).setTableName(tableName) + .setRowAccess(rm.getMutations()).setCallable(callable).setRpcTimeout(writeRpcTimeoutMs) + .setOperationTimeout(operationTimeoutMs) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).setResults(results).build(); AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); if (ars.hasError()) { @@ -663,70 +626,65 @@ public class HTable implements Table { @Override public Result append(final Append append) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(append); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName).setOperation(append); return TraceUtil.trace(() -> { checkHasFamilies(append); NoncedRegionServerCallable callable = new NoncedRegionServerCallable(this.connection, getName(), append.getRow(), this.rpcControllerFactory.newController(), append.getPriority()) { - @Override - protected Result rpcCall() throws Exception { - MutateRequest request = RequestConverter.buildMutateRequest( - getLocation().getRegionInfo().getRegionName(), append, super.getNonceGroup(), - super.getNonce()); - MutateResponse response = doMutate(request); - if (!response.hasResult()) { - return null; - } - return ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); - } - }; - return rpcCallerFactory. newCaller(this.writeRpcTimeoutMs). - callWithRetries(callable, this.operationTimeoutMs); + @Override + protected Result rpcCall() throws Exception { + MutateRequest request = + RequestConverter.buildMutateRequest(getLocation().getRegionInfo().getRegionName(), + append, super.getNonceGroup(), super.getNonce()); + MutateResponse response = doMutate(request); + if (!response.hasResult()) { + return null; + } + return ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); + } + }; + return rpcCallerFactory. newCaller(this.writeRpcTimeoutMs).callWithRetries(callable, + this.operationTimeoutMs); }, supplier); } @Override public Result increment(final Increment increment) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(increment); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName).setOperation(increment); return TraceUtil.trace(() -> { checkHasFamilies(increment); NoncedRegionServerCallable callable = new NoncedRegionServerCallable(this.connection, getName(), increment.getRow(), this.rpcControllerFactory.newController(), increment.getPriority()) { - @Override - protected Result rpcCall() throws Exception { - MutateRequest request = RequestConverter.buildMutateRequest( - getLocation().getRegionInfo().getRegionName(), increment, super.getNonceGroup(), - super.getNonce()); - MutateResponse response = doMutate(request); - // Should this check for null like append does? - return ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); - } - }; + @Override + protected Result rpcCall() throws Exception { + MutateRequest request = + RequestConverter.buildMutateRequest(getLocation().getRegionInfo().getRegionName(), + increment, super.getNonceGroup(), super.getNonce()); + MutateResponse response = doMutate(request); + // Should this check for null like append does? + return ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); + } + }; return rpcCallerFactory. newCaller(writeRpcTimeoutMs).callWithRetries(callable, - this.operationTimeoutMs); + this.operationTimeoutMs); }, supplier); } @Override - public long incrementColumnValue(final byte [] row, final byte [] family, - final byte [] qualifier, final long amount) - throws IOException { + public long incrementColumnValue(final byte[] row, final byte[] family, final byte[] qualifier, + final long amount) throws IOException { return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL); } @Override - public long incrementColumnValue(final byte [] row, final byte [] family, - final byte [] qualifier, final long amount, final Durability durability) - throws IOException { + public long incrementColumnValue(final byte[] row, final byte[] family, final byte[] qualifier, + final long amount, final Durability durability) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.INCREMENT); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.INCREMENT); return TraceUtil.trace(() -> { NullPointerException npe = null; if (row == null) { @@ -735,65 +693,62 @@ public class HTable implements Table { npe = new NullPointerException("family is null"); } if (npe != null) { - throw new IOException( - "Invalid arguments to incrementColumnValue", npe); + throw new IOException("Invalid arguments to incrementColumnValue", npe); } NoncedRegionServerCallable callable = new NoncedRegionServerCallable(this.connection, getName(), row, this.rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) { - @Override - protected Long rpcCall() throws Exception { - MutateRequest request = RequestConverter.buildIncrementRequest( - getLocation().getRegionInfo().getRegionName(), row, family, - qualifier, amount, durability, super.getNonceGroup(), super.getNonce()); - MutateResponse response = doMutate(request); - Result result = - ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); - return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier))); - } - }; - return rpcCallerFactory. newCaller(this.writeRpcTimeoutMs). - callWithRetries(callable, this.operationTimeoutMs); + @Override + protected Long rpcCall() throws Exception { + MutateRequest request = RequestConverter.buildIncrementRequest( + getLocation().getRegionInfo().getRegionName(), row, family, qualifier, amount, + durability, super.getNonceGroup(), super.getNonce()); + MutateResponse response = doMutate(request); + Result result = + ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); + return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier))); + } + }; + return rpcCallerFactory. newCaller(this.writeRpcTimeoutMs).callWithRetries(callable, + this.operationTimeoutMs); }, supplier); } @Override @Deprecated - public boolean checkAndPut(final byte [] row, final byte [] family, final byte [] qualifier, - final byte [] value, final Put put) throws IOException { + public boolean checkAndPut(final byte[] row, final byte[] family, final byte[] qualifier, + final byte[] value, final Put put) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, HBaseSemanticAttributes.Operation.PUT); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, + HBaseSemanticAttributes.Operation.PUT); return TraceUtil.trace( () -> doCheckAndMutate(row, family, qualifier, CompareOperator.EQUAL, value, null, null, put) - .isSuccess(), + .isSuccess(), supplier); } @Override @Deprecated - public boolean checkAndPut(final byte [] row, final byte [] family, final byte [] qualifier, - final CompareOp compareOp, final byte [] value, final Put put) throws IOException { + public boolean checkAndPut(final byte[] row, final byte[] family, final byte[] qualifier, + final CompareOp compareOp, final byte[] value, final Put put) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, HBaseSemanticAttributes.Operation.PUT); - return TraceUtil.trace( - () -> doCheckAndMutate(row, family, qualifier, toCompareOperator(compareOp), value, null, - null, put).isSuccess(), - supplier); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, + HBaseSemanticAttributes.Operation.PUT); + return TraceUtil.trace(() -> doCheckAndMutate(row, family, qualifier, + toCompareOperator(compareOp), value, null, null, put).isSuccess(), supplier); } @Override @Deprecated - public boolean checkAndPut(final byte [] row, final byte [] family, final byte [] qualifier, - final CompareOperator op, final byte [] value, final Put put) throws IOException { + public boolean checkAndPut(final byte[] row, final byte[] family, final byte[] qualifier, + final CompareOperator op, final byte[] value, final Put put) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, HBaseSemanticAttributes.Operation.PUT); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, + HBaseSemanticAttributes.Operation.PUT); return TraceUtil.trace( () -> doCheckAndMutate(row, family, qualifier, op, value, null, null, put).isSuccess(), supplier); @@ -802,39 +757,35 @@ public class HTable implements Table { @Override @Deprecated public boolean checkAndDelete(final byte[] row, final byte[] family, final byte[] qualifier, - final byte[] value, final Delete delete) throws IOException { + final byte[] value, final Delete delete) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, HBaseSemanticAttributes.Operation.DELETE); - return TraceUtil.trace( - () -> doCheckAndMutate(row, family, qualifier, CompareOperator.EQUAL, value, null, null, - delete).isSuccess(), - supplier); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, + HBaseSemanticAttributes.Operation.DELETE); + return TraceUtil.trace(() -> doCheckAndMutate(row, family, qualifier, CompareOperator.EQUAL, + value, null, null, delete).isSuccess(), supplier); } @Override @Deprecated public boolean checkAndDelete(final byte[] row, final byte[] family, final byte[] qualifier, - final CompareOp compareOp, final byte[] value, final Delete delete) throws IOException { + final CompareOp compareOp, final byte[] value, final Delete delete) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, HBaseSemanticAttributes.Operation.DELETE); - return TraceUtil.trace( - () -> doCheckAndMutate(row, family, qualifier, toCompareOperator(compareOp), value, null, - null, delete).isSuccess(), - supplier); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, + HBaseSemanticAttributes.Operation.DELETE); + return TraceUtil.trace(() -> doCheckAndMutate(row, family, qualifier, + toCompareOperator(compareOp), value, null, null, delete).isSuccess(), supplier); } @Override @Deprecated public boolean checkAndDelete(final byte[] row, final byte[] family, final byte[] qualifier, - final CompareOperator op, final byte[] value, final Delete delete) throws IOException { + final CompareOperator op, final byte[] value, final Delete delete) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, HBaseSemanticAttributes.Operation.DELETE); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE, + HBaseSemanticAttributes.Operation.DELETE); return TraceUtil.trace( () -> doCheckAndMutate(row, family, qualifier, op, value, null, null, delete).isSuccess(), supplier); @@ -853,49 +804,44 @@ public class HTable implements Table { } private CheckAndMutateResult doCheckAndMutate(final byte[] row, final byte[] family, - final byte[] qualifier, final CompareOperator op, final byte[] value, final Filter filter, - final TimeRange timeRange, final RowMutations rm) throws IOException { + final byte[] qualifier, final CompareOperator op, final byte[] value, final Filter filter, + final TimeRange timeRange, final RowMutations rm) throws IOException { long nonceGroup = getNonceGroup(); long nonce = getNonce(); CancellableRegionServerCallable callable = - new CancellableRegionServerCallable(connection, getName(), rm.getRow(), - rpcControllerFactory.newController(), writeRpcTimeoutMs, new RetryingTimeTracker().start(), - rm.getMaxPriority()) { - @Override - protected MultiResponse rpcCall() throws Exception { - MultiRequest request = RequestConverter - .buildMultiRequest(getLocation().getRegionInfo().getRegionName(), row, family, - qualifier, op, value, filter, timeRange, rm, nonceGroup, nonce); - ClientProtos.MultiResponse response = doMulti(request); - ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0); - if (res.hasException()) { - Throwable ex = ProtobufUtil.toException(res.getException()); - if (ex instanceof IOException) { - throw (IOException) ex; + new CancellableRegionServerCallable(connection, getName(), rm.getRow(), + rpcControllerFactory.newController(), writeRpcTimeoutMs, + new RetryingTimeTracker().start(), rm.getMaxPriority()) { + @Override + protected MultiResponse rpcCall() throws Exception { + MultiRequest request = + RequestConverter.buildMultiRequest(getLocation().getRegionInfo().getRegionName(), + row, family, qualifier, op, value, filter, timeRange, rm, nonceGroup, nonce); + ClientProtos.MultiResponse response = doMulti(request); + ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0); + if (res.hasException()) { + Throwable ex = ProtobufUtil.toException(res.getException()); + if (ex instanceof IOException) { + throw (IOException) ex; + } + throw new IOException( + "Failed to checkAndMutate row: " + Bytes.toStringBinary(rm.getRow()), ex); + } + return ResponseConverter.getResults(request, response, getRpcControllerCellScanner()); } - throw new IOException( - "Failed to checkAndMutate row: " + Bytes.toStringBinary(rm.getRow()), ex); - } - return ResponseConverter.getResults(request, response, getRpcControllerCellScanner()); - } - }; + }; /** - * Currently, we use one array to store 'processed' flag which is returned by server. - * It is excessive to send such a large array, but that is required by the framework right now - * */ + * Currently, we use one array to store 'processed' flag which is returned by server. It is + * excessive to send such a large array, but that is required by the framework right now + */ Object[] results = new Object[rm.getMutations().size()]; - AsyncProcessTask task = AsyncProcessTask.newBuilder() - .setPool(pool) - .setTableName(tableName) - .setRowAccess(rm.getMutations()) - .setResults(results) - .setCallable(callable) - // TODO any better timeout? - .setRpcTimeout(Math.max(readRpcTimeoutMs, writeRpcTimeoutMs)) - .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .build(); + AsyncProcessTask task = AsyncProcessTask.newBuilder().setPool(pool).setTableName(tableName) + .setRowAccess(rm.getMutations()).setResults(results).setCallable(callable) + // TODO any better timeout? + .setRpcTimeout(Math.max(readRpcTimeoutMs, writeRpcTimeoutMs)) + .setOperationTimeout(operationTimeoutMs) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).build(); AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); if (ars.hasError()) { @@ -907,26 +853,22 @@ public class HTable implements Table { @Override @Deprecated - public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier, - final CompareOp compareOp, final byte [] value, final RowMutations rm) throws IOException { + public boolean checkAndMutate(final byte[] row, final byte[] family, final byte[] qualifier, + final CompareOp compareOp, final byte[] value, final RowMutations rm) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(rm); - return TraceUtil.trace( - () -> doCheckAndMutate(row, family, qualifier, toCompareOperator(compareOp), value, null, - null, rm).isSuccess(), - supplier); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(rm); + return TraceUtil.trace(() -> doCheckAndMutate(row, family, qualifier, + toCompareOperator(compareOp), value, null, null, rm).isSuccess(), supplier); } @Override @Deprecated - public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier, - final CompareOperator op, final byte [] value, final RowMutations rm) throws IOException { + public boolean checkAndMutate(final byte[] row, final byte[] family, final byte[] qualifier, + final CompareOperator op, final byte[] value, final RowMutations rm) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(rm); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(rm); return TraceUtil.trace( () -> doCheckAndMutate(row, family, qualifier, op, value, null, null, rm).isSuccess(), supplier); @@ -934,14 +876,13 @@ public class HTable implements Table { @Override public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(checkAndMutate) - .setContainerOperations(checkAndMutate); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName) + .setOperation(checkAndMutate).setContainerOperations(checkAndMutate); return TraceUtil.trace(() -> { Row action = checkAndMutate.getAction(); - if (action instanceof Put || action instanceof Delete || action instanceof Increment || - action instanceof Append) { + if (action instanceof Put || action instanceof Delete || action instanceof Increment + || action instanceof Append) { if (action instanceof Put) { validatePut((Put) action); } @@ -957,37 +898,36 @@ public class HTable implements Table { } private CheckAndMutateResult doCheckAndMutate(final byte[] row, final byte[] family, - final byte[] qualifier, final CompareOperator op, final byte[] value, final Filter filter, - final TimeRange timeRange, final Mutation mutation) throws IOException { + final byte[] qualifier, final CompareOperator op, final byte[] value, final Filter filter, + final TimeRange timeRange, final Mutation mutation) throws IOException { long nonceGroup = getNonceGroup(); long nonce = getNonce(); ClientServiceCallable callable = - new ClientServiceCallable(this.connection, getName(), row, - this.rpcControllerFactory.newController(), mutation.getPriority()) { - @Override - protected CheckAndMutateResult rpcCall() throws Exception { - MutateRequest request = RequestConverter.buildMutateRequest( - getLocation().getRegionInfo().getRegionName(), row, family, qualifier, op, value, - filter, timeRange, mutation, nonceGroup, nonce); - MutateResponse response = doMutate(request); - if (response.hasResult()) { - return new CheckAndMutateResult(response.getProcessed(), - ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner())); + new ClientServiceCallable(this.connection, getName(), row, + this.rpcControllerFactory.newController(), mutation.getPriority()) { + @Override + protected CheckAndMutateResult rpcCall() throws Exception { + MutateRequest request = RequestConverter.buildMutateRequest( + getLocation().getRegionInfo().getRegionName(), row, family, qualifier, op, value, + filter, timeRange, mutation, nonceGroup, nonce); + MutateResponse response = doMutate(request); + if (response.hasResult()) { + return new CheckAndMutateResult(response.getProcessed(), + ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner())); + } + return new CheckAndMutateResult(response.getProcessed(), null); } - return new CheckAndMutateResult(response.getProcessed(), null); - } - }; + }; return rpcCallerFactory. newCaller(this.writeRpcTimeoutMs) - .callWithRetries(callable, this.operationTimeoutMs); + .callWithRetries(callable, this.operationTimeoutMs); } @Override public List checkAndMutate(List checkAndMutates) - throws IOException { + throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.BATCH) - .setContainerOperations(checkAndMutates); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.BATCH) + .setContainerOperations(checkAndMutates); return TraceUtil.trace(() -> { if (checkAndMutates.isEmpty()) { return Collections.emptyList(); @@ -1043,9 +983,8 @@ public class HTable implements Table { @Override public boolean exists(final Get get) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(get); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName).setOperation(get); return TraceUtil.trace(() -> { Result r = get(get, true); assert r.getExists() != null; @@ -1055,10 +994,9 @@ public class HTable implements Table { @Override public boolean[] exists(List gets) throws IOException { - final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.BATCH) - .setContainerOperations(gets); + final Supplier supplier = + new TableOperationSpanBuilder(connection).setTableName(tableName) + .setOperation(HBaseSemanticAttributes.Operation.BATCH).setContainerOperations(gets); return TraceUtil.trace(() -> { if (gets.isEmpty()) { return new boolean[] {}; @@ -1094,28 +1032,23 @@ public class HTable implements Table { } /** - * Process a mixed batch of Get, Put and Delete actions. All actions for a - * RegionServer are forwarded in one RPC call. Queries are executed in parallel. - * + * Process a mixed batch of Get, Put and Delete actions. All actions for a RegionServer are + * forwarded in one RPC call. Queries are executed in parallel. * @param list The collection of actions. - * @param results An empty array, same size as list. If an exception is thrown, - * you can test here for partial results, and to determine which actions - * processed successfully. - * @throws IOException if there are problems talking to META. Per-item - * exceptions are stored in the results array. + * @param results An empty array, same size as list. If an exception is thrown, you can test here + * for partial results, and to determine which actions processed successfully. + * @throws IOException if there are problems talking to META. Per-item exceptions are stored in + * the results array. */ - public void processBatchCallback( - final List list, final Object[] results, final Batch.Callback callback) - throws IOException, InterruptedException { + public void processBatchCallback(final List list, final Object[] results, + final Batch.Callback callback) throws IOException, InterruptedException { this.batchCallback(list, results, callback); } @Override public void close() throws IOException { - final Supplier supplier = new TableSpanBuilder(connection) - .setName("HTable.close") - .setTableName(tableName) - .setSpanKind(SpanKind.INTERNAL); + final Supplier supplier = new TableSpanBuilder(connection).setName("HTable.close") + .setTableName(tableName).setSpanKind(SpanKind.INTERNAL); TraceUtil.trace(() -> { if (this.closed) { return; @@ -1151,8 +1084,8 @@ public class HTable implements Table { } /** - * Explicitly clears the region cache to fetch the latest value from META. - * This is a power user function: avoid unless you know the ramifications. + * Explicitly clears the region cache to fetch the latest value from META. This is a power user + * function: avoid unless you know the ramifications. */ public void clearRegionCache() { this.connection.clearRegionLocationCache(); @@ -1164,11 +1097,11 @@ public class HTable implements Table { } @Override - public Map coprocessorService(final Class service, - byte[] startKey, byte[] endKey, final Batch.Call callable) + public Map coprocessorService(final Class service, + byte[] startKey, byte[] endKey, final Batch.Call callable) throws ServiceException, Throwable { - final Map results = Collections.synchronizedMap( - new TreeMap<>(Bytes.BYTES_COMPARATOR)); + final Map results = + Collections.synchronizedMap(new TreeMap<>(Bytes.BYTES_COMPARATOR)); coprocessorService(service, startKey, endKey, callable, (region, row, value) -> { if (region != null) { results.put(region, value); @@ -1178,24 +1111,23 @@ public class HTable implements Table { } @Override - public void coprocessorService(final Class service, - byte[] startKey, byte[] endKey, final Batch.Call callable, - final Batch.Callback callback) throws ServiceException, Throwable { + public void coprocessorService(final Class service, byte[] startKey, + byte[] endKey, final Batch.Call callable, final Batch.Callback callback) + throws ServiceException, Throwable { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC); TraceUtil.trace(() -> { final Context context = Context.current(); final ExecutorService wrappedPool = context.wrap(pool); // get regions covered by the row range List keys = getStartKeysInRange(startKey, endKey); - Map> futures = new TreeMap<>(Bytes.BYTES_COMPARATOR); + Map> futures = new TreeMap<>(Bytes.BYTES_COMPARATOR); for (final byte[] r : keys) { final RegionCoprocessorRpcChannel channel = - new RegionCoprocessorRpcChannel(connection, tableName, r); + new RegionCoprocessorRpcChannel(connection, tableName, r); Future future = wrappedPool.submit(() -> { T instance = - org.apache.hadoop.hbase.protobuf.ProtobufUtil.newServiceStub(service, channel); + org.apache.hadoop.hbase.protobuf.ProtobufUtil.newServiceStub(service, channel); R result = callable.call(instance); byte[] region = channel.getLastRegion(); if (callback != null) { @@ -1205,7 +1137,7 @@ public class HTable implements Table { }); futures.put(r, future); } - for (Map.Entry> e : futures.entrySet()) { + for (Map.Entry> e : futures.entrySet()) { try { e.getValue().get(); } catch (ExecutionException ee) { @@ -1214,14 +1146,13 @@ public class HTable implements Table { throw ee.getCause(); } catch (InterruptedException ie) { throw new InterruptedIOException("Interrupted calling coprocessor service " - + service.getName() + " for row " + Bytes.toStringBinary(e.getKey())).initCause(ie); + + service.getName() + " for row " + Bytes.toStringBinary(e.getKey())).initCause(ie); } } }, supplier); } - private List getStartKeysInRange(byte[] start, byte[] end) - throws IOException { + private List getStartKeysInRange(byte[] start, byte[] end) throws IOException { if (start == null) { start = HConstants.EMPTY_START_ROW; } @@ -1307,10 +1238,10 @@ public class HTable implements Table { @Override public Map batchCoprocessorService( - Descriptors.MethodDescriptor methodDescriptor, Message request, - byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable { - final Map results = Collections.synchronizedMap(new TreeMap<>( - Bytes.BYTES_COMPARATOR)); + Descriptors.MethodDescriptor methodDescriptor, Message request, byte[] startKey, + byte[] endKey, R responsePrototype) throws ServiceException, Throwable { + final Map results = + Collections.synchronizedMap(new TreeMap<>(Bytes.BYTES_COMPARATOR)); batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype, (region, row, result) -> { if (region != null) { @@ -1322,22 +1253,20 @@ public class HTable implements Table { @Override public void batchCoprocessorService( - final Descriptors.MethodDescriptor methodDescriptor, final Message request, - byte[] startKey, byte[] endKey, final R responsePrototype, final Callback callback) + final Descriptors.MethodDescriptor methodDescriptor, final Message request, byte[] startKey, + byte[] endKey, final R responsePrototype, final Callback callback) throws ServiceException, Throwable { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC); TraceUtil.trace(() -> { final Context context = Context.current(); - final byte[] sanitizedStartKey = Optional.ofNullable(startKey) - .orElse(HConstants.EMPTY_START_ROW); - final byte[] sanitizedEndKey = Optional.ofNullable(endKey) - .orElse(HConstants.EMPTY_END_ROW); + final byte[] sanitizedStartKey = + Optional.ofNullable(startKey).orElse(HConstants.EMPTY_START_ROW); + final byte[] sanitizedEndKey = Optional.ofNullable(endKey).orElse(HConstants.EMPTY_END_ROW); // get regions covered by the row range Pair, List> keysAndRegions = - getKeysAndRegionsInRange(sanitizedStartKey, sanitizedEndKey, true); + getKeysAndRegionsInRange(sanitizedStartKey, sanitizedEndKey, true); List keys = keysAndRegions.getFirst(); List regions = keysAndRegions.getSecond(); @@ -1350,12 +1279,12 @@ public class HTable implements Table { List execs = new ArrayList<>(keys.size()); final Map execsByRow = - new TreeMap<>(Bytes.BYTES_COMPARATOR); + new TreeMap<>(Bytes.BYTES_COMPARATOR); for (int i = 0; i < keys.size(); i++) { final byte[] rowKey = keys.get(i); final byte[] region = regions.get(i).getRegionInfo().getRegionName(); RegionCoprocessorServiceExec exec = - new RegionCoprocessorServiceExec(region, rowKey, methodDescriptor, request); + new RegionCoprocessorServiceExec(region, rowKey, methodDescriptor, request); execs.add(exec); execsByRow.put(rowKey, exec); } @@ -1368,47 +1297,42 @@ public class HTable implements Table { Object[] results = new Object[execs.size()]; AsyncProcess asyncProcess = new AsyncProcess(connection, configuration, - RpcRetryingCallerFactory.instantiate(configuration, connection.getStatisticsTracker()), - RpcControllerFactory.instantiate(configuration)); + RpcRetryingCallerFactory.instantiate(configuration, connection.getStatisticsTracker()), + RpcControllerFactory.instantiate(configuration)); Callback resultsCallback = - (byte[] region, byte[] row, ClientProtos.CoprocessorServiceResult serviceResult) -> { - if (LOG.isTraceEnabled()) { - LOG.trace("Received result for endpoint {}: region={}, row={}, value={}", - methodDescriptor.getFullName(), Bytes.toStringBinary(region), - Bytes.toStringBinary(row), serviceResult.getValue().getValue()); - } - try { - Message.Builder builder = responsePrototype.newBuilderForType(); - org.apache.hadoop.hbase.protobuf.ProtobufUtil.mergeFrom(builder, - serviceResult.getValue().getValue().toByteArray()); - callback.update(region, row, (R) builder.build()); - } catch (IOException e) { - LOG.error("Unexpected response type from endpoint {}", methodDescriptor.getFullName(), - e); - callbackErrorExceptions.add(e); - callbackErrorActions.add(execsByRow.get(row)); - callbackErrorServers.add("null"); - } - }; + (byte[] region, byte[] row, ClientProtos.CoprocessorServiceResult serviceResult) -> { + if (LOG.isTraceEnabled()) { + LOG.trace("Received result for endpoint {}: region={}, row={}, value={}", + methodDescriptor.getFullName(), Bytes.toStringBinary(region), + Bytes.toStringBinary(row), serviceResult.getValue().getValue()); + } + try { + Message.Builder builder = responsePrototype.newBuilderForType(); + org.apache.hadoop.hbase.protobuf.ProtobufUtil.mergeFrom(builder, + serviceResult.getValue().getValue().toByteArray()); + callback.update(region, row, (R) builder.build()); + } catch (IOException e) { + LOG.error("Unexpected response type from endpoint {}", methodDescriptor.getFullName(), + e); + callbackErrorExceptions.add(e); + callbackErrorActions.add(execsByRow.get(row)); + callbackErrorServers.add("null"); + } + }; AsyncProcessTask task = - AsyncProcessTask.newBuilder(resultsCallback) - .setPool(context.wrap(pool)) - .setTableName(tableName) - .setRowAccess(execs) - .setResults(results) - .setRpcTimeout(readRpcTimeoutMs) - .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .build(); + AsyncProcessTask.newBuilder(resultsCallback).setPool(context.wrap(pool)) + .setTableName(tableName).setRowAccess(execs).setResults(results) + .setRpcTimeout(readRpcTimeoutMs).setOperationTimeout(operationTimeoutMs) + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).build(); AsyncRequestFuture future = asyncProcess.submit(task); future.waitUntilDone(); if (future.hasError()) { throw future.getErrors(); } else if (!callbackErrorExceptions.isEmpty()) { - throw new RetriesExhaustedWithDetailsException( - callbackErrorExceptions, callbackErrorActions, callbackErrorServers); + throw new RetriesExhaustedWithDetailsException(callbackErrorExceptions, + callbackErrorActions, callbackErrorServers); } }, supplier); } @@ -1434,8 +1358,8 @@ public class HTable implements Table { @Override public CheckAndMutateBuilder qualifier(byte[] qualifier) { - this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using" + - " an empty byte array, or just do not call this method if you want a null qualifier"); + this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using" + + " an empty byte array, or just do not call this method if you want a null qualifier"); return this; } @@ -1460,44 +1384,41 @@ public class HTable implements Table { } private void preCheck() { - Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by" + - " calling ifNotExists/ifEquals/ifMatches before executing the request"); + Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by" + + " calling ifNotExists/ifEquals/ifMatches before executing the request"); } @Override public boolean thenPut(Put put) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); return TraceUtil.trace(() -> { validatePut(put); preCheck(); return doCheckAndMutate(row, family, qualifier, op, value, null, timeRange, put) - .isSuccess(); + .isSuccess(); }, supplier); } @Override public boolean thenDelete(Delete delete) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); return TraceUtil.trace(() -> { preCheck(); return doCheckAndMutate(row, family, qualifier, op, value, null, timeRange, delete) - .isSuccess(); + .isSuccess(); }, supplier); } @Override public boolean thenMutate(RowMutations mutation) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); return TraceUtil.trace(() -> { preCheck(); return doCheckAndMutate(row, family, qualifier, op, value, null, timeRange, mutation) - .isSuccess(); + .isSuccess(); }, supplier); } } @@ -1522,20 +1443,17 @@ public class HTable implements Table { @Override public boolean thenPut(Put put) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); return TraceUtil.trace(() -> { validatePut(put); - return doCheckAndMutate(row, null, null, null, null, filter, timeRange, put) - .isSuccess(); + return doCheckAndMutate(row, null, null, null, null, filter, timeRange, put).isSuccess(); }, supplier); } @Override public boolean thenDelete(Delete delete) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); return TraceUtil.trace( () -> doCheckAndMutate(row, null, null, null, null, filter, timeRange, delete).isSuccess(), supplier); @@ -1544,12 +1462,11 @@ public class HTable implements Table { @Override public boolean thenMutate(RowMutations mutation) throws IOException { final Supplier supplier = new TableOperationSpanBuilder(connection) - .setTableName(tableName) - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); - return TraceUtil.trace( - () -> doCheckAndMutate(row, null, null, null, null, filter, timeRange, mutation) - .isSuccess(), - supplier); + .setTableName(tableName).setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE); + return TraceUtil + .trace(() -> doCheckAndMutate(row, null, null, null, null, filter, timeRange, mutation) + .isSuccess(), + supplier); } } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java index 12ddbd86cd9..c20610465b7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java @@ -26,6 +26,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePutsInRowMu import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture; import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFutures; import static org.apache.hadoop.hbase.util.FutureUtils.addListener; + import com.google.protobuf.RpcChannel; import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.StatusCode; @@ -58,9 +59,11 @@ import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; import org.apache.hbase.thirdparty.io.netty.util.Timer; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; @@ -127,8 +130,8 @@ class RawAsyncTableImpl implements AsyncTable { this.pauseNs = builder.pauseNs; if (builder.pauseForCQTBENs < builder.pauseNs) { LOG.warn( - "Configured value of pauseForCQTBENs is {} ms, which is less than" + - " the normal pause value {} ms, use the greater one instead", + "Configured value of pauseForCQTBENs is {} ms, which is less than" + + " the normal pause value {} ms, use the greater one instead", TimeUnit.NANOSECONDS.toMillis(builder.pauseForCQTBENs), TimeUnit.NANOSECONDS.toMillis(builder.pauseNs)); this.pauseForCQTBENs = builder.pauseNs; @@ -137,8 +140,8 @@ class RawAsyncTableImpl implements AsyncTable { } this.maxAttempts = builder.maxAttempts; this.startLogErrorsCnt = builder.startLogErrorsCnt; - this.defaultScannerCaching = tableName.isSystemTable() ? conn.connConf.getMetaScannerCaching() : - conn.connConf.getScannerCaching(); + this.defaultScannerCaching = tableName.isSystemTable() ? conn.connConf.getMetaScannerCaching() + : conn.connConf.getScannerCaching(); this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize(); } @@ -210,15 +213,15 @@ class RawAsyncTableImpl implements AsyncTable { } private static CompletableFuture voidMutate(HBaseRpcController controller, - HRegionLocation loc, ClientService.Interface stub, REQ req, - Converter reqConvert) { + HRegionLocation loc, ClientService.Interface stub, REQ req, + Converter reqConvert) { return mutate(controller, loc, stub, req, reqConvert, (c, resp) -> { return null; }); } private static Result toResult(HBaseRpcController controller, MutateResponse resp) - throws IOException { + throws IOException { if (!resp.hasResult()) { return null; } @@ -231,33 +234,33 @@ class RawAsyncTableImpl implements AsyncTable { } private CompletableFuture noncedMutate(long nonceGroup, long nonce, - HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req, - NoncedConverter reqConvert, - Converter respConverter) { + HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req, + NoncedConverter reqConvert, + Converter respConverter) { return mutate(controller, loc, stub, req, (info, src) -> reqConvert.convert(info, src, nonceGroup, nonce), respConverter); } private SingleRequestCallerBuilder newCaller(byte[] row, int priority, long rpcTimeoutNs) { return conn.callerFactory. single().table(tableName).row(row).priority(priority) - .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS) - .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS) - .pause(pauseNs, TimeUnit.NANOSECONDS).pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS) - .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt); + .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS) + .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS) + .pause(pauseNs, TimeUnit.NANOSECONDS).pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS) + .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt); } private SingleRequestCallerBuilder - newCaller(R row, long rpcTimeoutNs) { + newCaller(R row, long rpcTimeoutNs) { return newCaller(row.getRow(), row.getPriority(), rpcTimeoutNs); } private CompletableFuture get(Get get, int replicaId) { return this. newCaller(get, readRpcTimeoutNs) - .action((controller, loc, stub) -> RawAsyncTableImpl - . call(controller, loc, stub, get, - RequestConverter::buildGetRequest, (s, c, req, done) -> s.get(c, req, done), - (c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner()))) - .replicaId(replicaId).call(); + .action((controller, loc, stub) -> RawAsyncTableImpl + . call(controller, loc, stub, get, + RequestConverter::buildGetRequest, (s, c, req, done) -> s.get(c, req, done), + (c, resp) -> ProtobufUtil.toResult(resp.getResult(), c.cellScanner()))) + .replicaId(replicaId).call(); } private TableOperationSpanBuilder newTableOperationSpanBuilder() { @@ -266,8 +269,7 @@ class RawAsyncTableImpl implements AsyncTable { @Override public CompletableFuture get(Get get) { - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(get); + final Supplier supplier = newTableOperationSpanBuilder().setOperation(get); return tracedFuture( () -> timelineConsistentRead(conn.getLocator(), tableName, get, get.getRow(), RegionLocateType.CURRENT, replicaId -> get(get, replicaId), readRpcTimeoutNs, @@ -278,20 +280,18 @@ class RawAsyncTableImpl implements AsyncTable { @Override public CompletableFuture put(Put put) { validatePut(put, conn.connConf.getMaxKeyValueSize()); - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(put); + final Supplier supplier = newTableOperationSpanBuilder().setOperation(put); return tracedFuture(() -> this. newCaller(put, writeRpcTimeoutNs) - .action((controller, loc, stub) -> RawAsyncTableImpl. voidMutate(controller, loc, stub, - put, RequestConverter::buildMutateRequest)) - .call(), supplier); + .action((controller, loc, stub) -> RawAsyncTableImpl. voidMutate(controller, loc, stub, + put, RequestConverter::buildMutateRequest)) + .call(), + supplier); } @Override public CompletableFuture delete(Delete delete) { - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(delete); - return tracedFuture( - () -> this. newCaller(delete, writeRpcTimeoutNs) + final Supplier supplier = newTableOperationSpanBuilder().setOperation(delete); + return tracedFuture(() -> this. newCaller(delete, writeRpcTimeoutNs) .action((controller, loc, stub) -> RawAsyncTableImpl. voidMutate(controller, loc, stub, delete, RequestConverter::buildMutateRequest)) .call(), @@ -301,32 +301,30 @@ class RawAsyncTableImpl implements AsyncTable { @Override public CompletableFuture append(Append append) { checkHasFamilies(append); - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(append); + final Supplier supplier = newTableOperationSpanBuilder().setOperation(append); return tracedFuture(() -> { long nonceGroup = conn.getNonceGenerator().getNonceGroup(); long nonce = conn.getNonceGenerator().newNonce(); return this. newCaller(append, rpcTimeoutNs) - .action((controller, loc, stub) -> this. noncedMutate(nonceGroup, nonce, - controller, loc, stub, append, RequestConverter::buildMutateRequest, - RawAsyncTableImpl::toResult)) - .call(); + .action((controller, loc, stub) -> this. noncedMutate(nonceGroup, nonce, + controller, loc, stub, append, RequestConverter::buildMutateRequest, + RawAsyncTableImpl::toResult)) + .call(); }, supplier); } @Override public CompletableFuture increment(Increment increment) { checkHasFamilies(increment); - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(increment); + final Supplier supplier = newTableOperationSpanBuilder().setOperation(increment); return tracedFuture(() -> { long nonceGroup = conn.getNonceGenerator().getNonceGroup(); long nonce = conn.getNonceGenerator().newNonce(); return this. newCaller(increment, rpcTimeoutNs) - .action((controller, loc, stub) -> this. noncedMutate(nonceGroup, nonce, - controller, loc, stub, increment, RequestConverter::buildMutateRequest, - RawAsyncTableImpl::toResult)) - .call(); + .action((controller, loc, stub) -> this. noncedMutate(nonceGroup, + nonce, controller, loc, stub, increment, RequestConverter::buildMutateRequest, + RawAsyncTableImpl::toResult)) + .call(); }, supplier); } @@ -351,8 +349,8 @@ class RawAsyncTableImpl implements AsyncTable { @Override public CheckAndMutateBuilder qualifier(byte[] qualifier) { - this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using" + - " an empty byte array, or just do not call this method if you want a null qualifier"); + this.qualifier = Preconditions.checkNotNull(qualifier, "qualifier is null. Consider using" + + " an empty byte array, or just do not call this method if you want a null qualifier"); return this; } @@ -377,8 +375,8 @@ class RawAsyncTableImpl implements AsyncTable { } private void preCheck() { - Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by" + - " calling ifNotExists/ifEquals/ifMatches before executing the request"); + Preconditions.checkNotNull(op, "condition is null. You need to specify the condition by" + + " calling ifNotExists/ifEquals/ifMatches before executing the request"); } @Override @@ -386,15 +384,15 @@ class RawAsyncTableImpl implements AsyncTable { validatePut(put, conn.connConf.getMaxKeyValueSize()); preCheck(); final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(put); + .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(put); return tracedFuture( () -> RawAsyncTableImpl.this. newCaller(row, put.getPriority(), rpcTimeoutNs) - .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, put, - (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value, - null, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE), - (c, r) -> r.getProcessed())) - .call(), + .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, put, + (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value, + null, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE), + (c, r) -> r.getProcessed())) + .call(), supplier); } @@ -402,10 +400,10 @@ class RawAsyncTableImpl implements AsyncTable { public CompletableFuture thenDelete(Delete delete) { preCheck(); final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(delete); - return tracedFuture( - () -> RawAsyncTableImpl.this. newCaller(row, delete.getPriority(), rpcTimeoutNs) + .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(delete); + return tracedFuture(() -> RawAsyncTableImpl.this + . newCaller(row, delete.getPriority(), rpcTimeoutNs) .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, delete, (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value, null, timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE), @@ -419,10 +417,9 @@ class RawAsyncTableImpl implements AsyncTable { preCheck(); validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize()); final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(mutations); - return tracedFuture( - () -> RawAsyncTableImpl.this + .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(mutations); + return tracedFuture(() -> RawAsyncTableImpl.this . newCaller(row, mutations.getMaxPriority(), rpcTimeoutNs) .action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller, loc, stub, mutations, @@ -440,7 +437,7 @@ class RawAsyncTableImpl implements AsyncTable { } private final class CheckAndMutateWithFilterBuilderImpl - implements CheckAndMutateWithFilterBuilder { + implements CheckAndMutateWithFilterBuilder { private final byte[] row; @@ -463,26 +460,25 @@ class RawAsyncTableImpl implements AsyncTable { public CompletableFuture thenPut(Put put) { validatePut(put, conn.connConf.getMaxKeyValueSize()); final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(put); - return tracedFuture( - () -> RawAsyncTableImpl.this. newCaller(row, put.getPriority(), rpcTimeoutNs) - .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, - stub, put, - (rn, p) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null, - filter, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE), - (c, r) -> r.getProcessed())) - .call(), + .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(put); + return tracedFuture(() -> RawAsyncTableImpl.this + . newCaller(row, put.getPriority(), rpcTimeoutNs) + .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, put, + (rn, p) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null, filter, + timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE), + (c, r) -> r.getProcessed())) + .call(), supplier); } @Override public CompletableFuture thenDelete(Delete delete) { final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(delete); - return tracedFuture( - () -> RawAsyncTableImpl.this. newCaller(row, delete.getPriority(), rpcTimeoutNs) + .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(delete); + return tracedFuture(() -> RawAsyncTableImpl.this + . newCaller(row, delete.getPriority(), rpcTimeoutNs) .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, delete, (rn, d) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null, filter, timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE), @@ -495,10 +491,9 @@ class RawAsyncTableImpl implements AsyncTable { public CompletableFuture thenMutate(RowMutations mutations) { validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize()); final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) - .setContainerOperations(mutations); - return tracedFuture( - () -> RawAsyncTableImpl.this + .setOperation(HBaseSemanticAttributes.Operation.CHECK_AND_MUTATE) + .setContainerOperations(mutations); + return tracedFuture(() -> RawAsyncTableImpl.this . newCaller(row, mutations.getMaxPriority(), rpcTimeoutNs) .action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller, loc, stub, mutations, @@ -517,14 +512,12 @@ class RawAsyncTableImpl implements AsyncTable { @Override public CompletableFuture checkAndMutate(CheckAndMutate checkAndMutate) { - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(checkAndMutate) - .setContainerOperations(checkAndMutate.getAction()); + final Supplier supplier = newTableOperationSpanBuilder().setOperation(checkAndMutate) + .setContainerOperations(checkAndMutate.getAction()); return tracedFuture(() -> { - if (checkAndMutate.getAction() instanceof Put || - checkAndMutate.getAction() instanceof Delete || - checkAndMutate.getAction() instanceof Increment || - checkAndMutate.getAction() instanceof Append) { + if (checkAndMutate.getAction() instanceof Put || checkAndMutate.getAction() instanceof Delete + || checkAndMutate.getAction() instanceof Increment + || checkAndMutate.getAction() instanceof Append) { Mutation mutation = (Mutation) checkAndMutate.getAction(); if (mutation instanceof Put) { validatePut((Put) mutation, conn.connConf.getMaxKeyValueSize()); @@ -532,37 +525,38 @@ class RawAsyncTableImpl implements AsyncTable { long nonceGroup = conn.getNonceGenerator().getNonceGroup(); long nonce = conn.getNonceGenerator().newNonce(); return RawAsyncTableImpl.this - . newCaller(checkAndMutate.getRow(), mutation.getPriority(), - rpcTimeoutNs) - .action( - (controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, mutation, - (rn, m) -> RequestConverter.buildMutateRequest(rn, checkAndMutate.getRow(), - checkAndMutate.getFamily(), checkAndMutate.getQualifier(), - checkAndMutate.getCompareOp(), checkAndMutate.getValue(), - checkAndMutate.getFilter(), checkAndMutate.getTimeRange(), m, nonceGroup, nonce), - (c, r) -> ResponseConverter.getCheckAndMutateResult(r, c.cellScanner()))) - .call(); + . newCaller(checkAndMutate.getRow(), mutation.getPriority(), + rpcTimeoutNs) + .action( + (controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, mutation, + (rn, m) -> RequestConverter.buildMutateRequest(rn, checkAndMutate.getRow(), + checkAndMutate.getFamily(), checkAndMutate.getQualifier(), + checkAndMutate.getCompareOp(), checkAndMutate.getValue(), + checkAndMutate.getFilter(), checkAndMutate.getTimeRange(), m, nonceGroup, nonce), + (c, r) -> ResponseConverter.getCheckAndMutateResult(r, c.cellScanner()))) + .call(); } else if (checkAndMutate.getAction() instanceof RowMutations) { RowMutations rowMutations = (RowMutations) checkAndMutate.getAction(); validatePutsInRowMutations(rowMutations, conn.connConf.getMaxKeyValueSize()); long nonceGroup = conn.getNonceGenerator().getNonceGroup(); long nonce = conn.getNonceGenerator().newNonce(); return RawAsyncTableImpl.this - . newCaller(checkAndMutate.getRow(), rowMutations.getMaxPriority(), - rpcTimeoutNs) - .action((controller, loc, stub) -> RawAsyncTableImpl.this - . mutateRow(controller, loc, stub, - rowMutations, - (rn, rm) -> RequestConverter.buildMultiRequest(rn, checkAndMutate.getRow(), - checkAndMutate.getFamily(), checkAndMutate.getQualifier(), - checkAndMutate.getCompareOp(), checkAndMutate.getValue(), - checkAndMutate.getFilter(), checkAndMutate.getTimeRange(), rm, nonceGroup, nonce), - resp -> resp)) - .call(); + . newCaller(checkAndMutate.getRow(), + rowMutations.getMaxPriority(), rpcTimeoutNs) + .action((controller, loc, stub) -> RawAsyncTableImpl.this + . mutateRow(controller, loc, stub, + rowMutations, + (rn, rm) -> RequestConverter.buildMultiRequest(rn, checkAndMutate.getRow(), + checkAndMutate.getFamily(), checkAndMutate.getQualifier(), + checkAndMutate.getCompareOp(), checkAndMutate.getValue(), + checkAndMutate.getFilter(), checkAndMutate.getTimeRange(), rm, nonceGroup, + nonce), + resp -> resp)) + .call(); } else { CompletableFuture future = new CompletableFuture<>(); future.completeExceptionally(new DoNotRetryIOException( - "CheckAndMutate doesn't support " + checkAndMutate.getAction().getClass().getName())); + "CheckAndMutate doesn't support " + checkAndMutate.getAction().getClass().getName())); return future; } }, supplier); @@ -570,12 +564,10 @@ class RawAsyncTableImpl implements AsyncTable { @Override public List> - checkAndMutate(List checkAndMutates) { - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(checkAndMutates) - .setContainerOperations(checkAndMutates); - return tracedFutures( - () -> batch(checkAndMutates, rpcTimeoutNs).stream() + checkAndMutate(List checkAndMutates) { + final Supplier supplier = newTableOperationSpanBuilder().setOperation(checkAndMutates) + .setContainerOperations(checkAndMutates); + return tracedFutures(() -> batch(checkAndMutates, rpcTimeoutNs).stream() .map(f -> f.thenApply(r -> (CheckAndMutateResult) r)).collect(toList()), supplier); } @@ -584,8 +576,8 @@ class RawAsyncTableImpl implements AsyncTable { // so here I write a new method as I do not want to change the abstraction of call method. @SuppressWarnings("unchecked") private CompletableFuture mutateRow(HBaseRpcController controller, - HRegionLocation loc, ClientService.Interface stub, RowMutations mutation, - Converter reqConvert, Function respConverter) { + HRegionLocation loc, ClientService.Interface stub, RowMutations mutation, + Converter reqConvert, Function respConverter) { CompletableFuture future = new CompletableFuture<>(); try { byte[] regionName = loc.getRegion().getRegionName(); @@ -599,14 +591,16 @@ class RawAsyncTableImpl implements AsyncTable { } else { try { org.apache.hadoop.hbase.client.MultiResponse multiResp = - ResponseConverter.getResults(req, resp, controller.cellScanner()); + ResponseConverter.getResults(req, resp, controller.cellScanner()); ConnectionUtils.updateStats(conn.getStatisticsTracker(), conn.getConnectionMetrics(), loc.getServerName(), multiResp); Throwable ex = multiResp.getException(regionName); if (ex != null) { - future.completeExceptionally(ex instanceof IOException ? ex : - new IOException( - "Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex)); + future + .completeExceptionally(ex instanceof IOException ? ex + : new IOException( + "Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), + ex)); } else { future.complete( respConverter.apply((RES) multiResp.getResults().get(regionName).result.get(0))); @@ -628,11 +622,9 @@ class RawAsyncTableImpl implements AsyncTable { validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize()); long nonceGroup = conn.getNonceGenerator().getNonceGroup(); long nonce = conn.getNonceGenerator().newNonce(); - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(mutations) - .setContainerOperations(mutations); - return tracedFuture( - () -> this + final Supplier supplier = + newTableOperationSpanBuilder().setOperation(mutations).setContainerOperations(mutations); + return tracedFuture(() -> this . newCaller(mutations.getRow(), mutations.getMaxPriority(), writeRpcTimeoutNs) .action((controller, loc, stub) -> this. mutateRow(controller, loc, stub, mutations, (rn, rm) -> RequestConverter.buildMultiRequest(rn, rm, nonceGroup, nonce), @@ -656,8 +648,8 @@ class RawAsyncTableImpl implements AsyncTable { @Override public void scan(Scan scan, AdvancedScanResultConsumer consumer) { new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, retryTimer, - pauseNs, pauseForCQTBENs, maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt) - .start(); + pauseNs, pauseForCQTBENs, maxAttempts, scanTimeoutNs, readRpcTimeoutNs, startLogErrorsCnt) + .start(); } private long resultSize2CacheSize(long maxResultSize) { @@ -671,74 +663,66 @@ class RawAsyncTableImpl implements AsyncTable { scan.getMaxResultSize() > 0 ? scan.getMaxResultSize() : defaultScannerMaxResultSize); final Scan scanCopy = ReflectionUtils.newInstance(scan.getClass(), scan); final AsyncTableResultScanner scanner = - new AsyncTableResultScanner(tableName, scanCopy, maxCacheSize); + new AsyncTableResultScanner(tableName, scanCopy, maxCacheSize); scan(scan, scanner); return scanner; } @Override public CompletableFuture> scanAll(Scan scan) { - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(scan); - return tracedFuture(() -> { - CompletableFuture> future = new CompletableFuture<>(); - List scanResults = new ArrayList<>(); - scan(scan, new AdvancedScanResultConsumer() { + CompletableFuture> future = new CompletableFuture<>(); + List scanResults = new ArrayList<>(); + scan(scan, new AdvancedScanResultConsumer() { - @Override - public void onNext(Result[] results, ScanController controller) { - scanResults.addAll(Arrays.asList(results)); - } + @Override + public void onNext(Result[] results, ScanController controller) { + scanResults.addAll(Arrays.asList(results)); + } - @Override - public void onError(Throwable error) { - future.completeExceptionally(error); - } + @Override + public void onError(Throwable error) { + future.completeExceptionally(error); + } - @Override - public void onComplete() { - future.complete(scanResults); - } - }); - return future; - }, supplier); + @Override + public void onComplete() { + future.complete(scanResults); + } + }); + return future; } @Override public List> get(List gets) { - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(gets) - .setContainerOperations(HBaseSemanticAttributes.Operation.GET); + final Supplier supplier = newTableOperationSpanBuilder().setOperation(gets) + .setContainerOperations(HBaseSemanticAttributes.Operation.GET); return tracedFutures(() -> batch(gets, readRpcTimeoutNs), supplier); } @Override public List> put(List puts) { - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(puts) - .setContainerOperations(HBaseSemanticAttributes.Operation.PUT); + final Supplier supplier = newTableOperationSpanBuilder().setOperation(puts) + .setContainerOperations(HBaseSemanticAttributes.Operation.PUT); return tracedFutures(() -> voidMutate(puts), supplier); } @Override public List> delete(List deletes) { - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(deletes) - .setContainerOperations(HBaseSemanticAttributes.Operation.DELETE); + final Supplier supplier = newTableOperationSpanBuilder().setOperation(deletes) + .setContainerOperations(HBaseSemanticAttributes.Operation.DELETE); return tracedFutures(() -> voidMutate(deletes), supplier); } @Override public List> batch(List actions) { - final Supplier supplier = newTableOperationSpanBuilder() - .setOperation(actions) - .setContainerOperations(actions); + final Supplier supplier = + newTableOperationSpanBuilder().setOperation(actions).setContainerOperations(actions); return tracedFutures(() -> batch(actions, rpcTimeoutNs), supplier); } private List> voidMutate(List actions) { return this. batch(actions, writeRpcTimeoutNs).stream() - .map(f -> f. thenApply(r -> null)).collect(toList()); + .map(f -> f. thenApply(r -> null)).collect(toList()); } private List> batch(List actions, long rpcTimeoutNs) { @@ -758,10 +742,10 @@ class RawAsyncTableImpl implements AsyncTable { } } return conn.callerFactory.batch().table(tableName).actions(actions) - .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS) - .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) - .pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts) - .startLogErrorsCnt(startLogErrorsCnt).call(); + .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS) + .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) + .pauseForCQTBE(pauseForCQTBENs, TimeUnit.NANOSECONDS).maxAttempts(maxAttempts) + .startLogErrorsCnt(startLogErrorsCnt).call(); } @Override @@ -790,9 +774,9 @@ class RawAsyncTableImpl implements AsyncTable { } private CompletableFuture coprocessorService(Function stubMaker, - ServiceCaller callable, RegionInfo region, byte[] row) { + ServiceCaller callable, RegionInfo region, byte[] row) { RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName, - region, row, rpcTimeoutNs, operationTimeoutNs); + region, row, rpcTimeoutNs, operationTimeoutNs); final Span span = Span.current(); S stub = stubMaker.apply(channel); CompletableFuture future = new CompletableFuture<>(); @@ -816,7 +800,7 @@ class RawAsyncTableImpl implements AsyncTable { @Override public CompletableFuture coprocessorService(Function stubMaker, - ServiceCaller callable, byte[] row) { + ServiceCaller callable, byte[] row) { return coprocessorService(stubMaker, callable, null, row); } @@ -838,9 +822,9 @@ class RawAsyncTableImpl implements AsyncTable { } private void onLocateComplete(Function stubMaker, - ServiceCaller callable, CoprocessorCallback callback, List locs, - byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished, - AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) { + ServiceCaller callable, CoprocessorCallback callback, List locs, + byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished, + AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) { final Span span = Span.current(); if (error != null) { callback.onError(error); @@ -853,10 +837,8 @@ class RawAsyncTableImpl implements AsyncTable { if (locateFinished(region, endKey, endKeyInclusive)) { locateFinished.set(true); } else { - addListener( - conn.getLocator().getRegionLocation(tableName, region.getEndKey(), RegionLocateType.CURRENT, - operationTimeoutNs), - (l, e) -> { + addListener(conn.getLocator().getRegionLocation(tableName, region.getEndKey(), + RegionLocateType.CURRENT, operationTimeoutNs), (l, e) -> { try (Scope ignored = span.makeCurrent()) { onLocateComplete(stubMaker, callable, callback, locs, endKey, endKeyInclusive, locateFinished, unfinishedRequest, l, e); @@ -878,7 +860,7 @@ class RawAsyncTableImpl implements AsyncTable { } private final class CoprocessorServiceBuilderImpl - implements CoprocessorServiceBuilder { + implements CoprocessorServiceBuilder { private final Function stubMaker; @@ -895,7 +877,7 @@ class RawAsyncTableImpl implements AsyncTable { private boolean endKeyInclusive; public CoprocessorServiceBuilderImpl(Function stubMaker, - ServiceCaller callable, CoprocessorCallback callback) { + ServiceCaller callable, CoprocessorCallback callback) { this.stubMaker = Preconditions.checkNotNull(stubMaker, "stubMaker is null"); this.callable = Preconditions.checkNotNull(callable, "callable is null"); this.callback = Preconditions.checkNotNull(callback, "callback is null"); @@ -904,9 +886,9 @@ class RawAsyncTableImpl implements AsyncTable { @Override public CoprocessorServiceBuilderImpl fromRow(byte[] startKey, boolean inclusive) { this.startKey = Preconditions.checkNotNull(startKey, - "startKey is null. Consider using" + - " an empty byte array, or just do not call this method if you want to start selection" + - " from the first region"); + "startKey is null. Consider using" + + " an empty byte array, or just do not call this method if you want to start selection" + + " from the first region"); this.startKeyInclusive = inclusive; return this; } @@ -914,9 +896,9 @@ class RawAsyncTableImpl implements AsyncTable { @Override public CoprocessorServiceBuilderImpl toRow(byte[] endKey, boolean inclusive) { this.endKey = Preconditions.checkNotNull(endKey, - "endKey is null. Consider using" + - " an empty byte array, or just do not call this method if you want to continue" + - " selection to the last region"); + "endKey is null. Consider using" + + " an empty byte array, or just do not call this method if you want to continue" + + " selection to the last region"); this.endKeyInclusive = inclusive; return this; } @@ -924,14 +906,12 @@ class RawAsyncTableImpl implements AsyncTable { @Override public void execute() { final Span span = newTableOperationSpanBuilder() - .setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC) - .build(); + .setOperation(HBaseSemanticAttributes.Operation.COPROC_EXEC).build(); try (Scope ignored = span.makeCurrent()) { - final RegionLocateType regionLocateType = startKeyInclusive - ? RegionLocateType.CURRENT - : RegionLocateType.AFTER; + final RegionLocateType regionLocateType = + startKeyInclusive ? RegionLocateType.CURRENT : RegionLocateType.AFTER; final CompletableFuture future = conn.getLocator() - .getRegionLocation(tableName, startKey, regionLocateType, operationTimeoutNs); + .getRegionLocation(tableName, startKey, regionLocateType, operationTimeoutNs); addListener(future, (loc, error) -> { try (Scope ignored1 = span.makeCurrent()) { onLocateComplete(stubMaker, callable, callback, new ArrayList<>(), endKey, @@ -944,8 +924,8 @@ class RawAsyncTableImpl implements AsyncTable { @Override public CoprocessorServiceBuilder coprocessorService( - Function stubMaker, ServiceCaller callable, - CoprocessorCallback callback) { + Function stubMaker, ServiceCaller callable, + CoprocessorCallback callback) { return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java index 0881dd253d7..09f4cd92c7a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultScanner.java @@ -1,5 +1,4 @@ -/** - * +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -24,9 +23,8 @@ import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; - -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; +import org.apache.yetus.audience.InterfaceAudience; /** * Interface for client-side scanning. Go to {@link Table} to obtain instances. @@ -50,7 +48,8 @@ public interface ResultScanner extends Closeable, Iterable { return true; } try { - return (next = ResultScanner.this.next()) != null; + next = ResultScanner.this.next(); + return next != null; } catch (IOException e) { throw new UncheckedIOException(e); } @@ -89,7 +88,6 @@ public interface ResultScanner extends Closeable, Iterable { * @param nbRows number of rows to return * @return Between zero and nbRows rowResults. Scan is done if returned array is of zero-length * (We never return null). - * @throws IOException */ default Result[] next(int nbRows) throws IOException { List resultSets = new ArrayList<>(nbRows); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java index 04df972d7a1..31d154f03a4 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java @@ -28,6 +28,8 @@ import static org.apache.hadoop.hbase.client.trace.hamcrest.TraceTestUtil.buildT import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.fail; @@ -36,6 +38,7 @@ import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; + import io.opentelemetry.api.trace.SpanKind; import io.opentelemetry.api.trace.StatusCode; import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule; @@ -44,8 +47,10 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; @@ -75,8 +80,10 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; + import org.apache.hbase.thirdparty.com.google.common.io.Closeables; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; @@ -97,7 +104,7 @@ public class TestAsyncTableTracing { @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestAsyncTableTracing.class); + HBaseClassTestRule.forClass(TestAsyncTableTracing.class); private static Configuration CONF = HBaseConfiguration.create(); @@ -105,7 +112,7 @@ public class TestAsyncTableTracing { private AsyncConnectionImpl conn; - private AsyncTable table; + private AsyncTable table; @Rule public OpenTelemetryRule traceRule = OpenTelemetryRule.create(); @@ -122,18 +129,18 @@ public class TestAsyncTableTracing { RpcCallback done = invocation.getArgument(2); if (!req.hasScannerId()) { done.run(ScanResponse.newBuilder().setScannerId(1).setTtl(800) - .setMoreResultsInRegion(true).setMoreResults(true).build()); + .setMoreResultsInRegion(true).setMoreResults(true).build()); } else { if (req.hasCloseScanner() && req.getCloseScanner()) { done.run(ScanResponse.getDefaultInstance()); } else { Cell cell = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) - .setType(Cell.Type.Put).setRow(Bytes.toBytes(scanNextCalled.incrementAndGet())) - .setFamily(Bytes.toBytes("cf")).setQualifier(Bytes.toBytes("cq")) - .setValue(Bytes.toBytes("v")).build(); + .setType(Cell.Type.Put).setRow(Bytes.toBytes(scanNextCalled.incrementAndGet())) + .setFamily(Bytes.toBytes("cf")).setQualifier(Bytes.toBytes("cq")) + .setValue(Bytes.toBytes("v")).build(); Result result = Result.create(Arrays.asList(cell)); ScanResponse.Builder builder = ScanResponse.newBuilder().setScannerId(1).setTtl(800) - .addResults(ProtobufUtil.toResult(result)); + .addResults(ProtobufUtil.toResult(result)); if (req.getLimitOfRows() == 1) { builder.setMoreResultsInRegion(false).setMoreResults(false); } else { @@ -175,13 +182,13 @@ public class TestAsyncTableTracing { case INCREMENT: ColumnValue value = req.getColumnValue(0); QualifierValue qvalue = value.getQualifierValue(0); - Cell cell = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) - .setType(Cell.Type.Put).setRow(req.getRow().toByteArray()) - .setFamily(value.getFamily().toByteArray()) - .setQualifier(qvalue.getQualifier().toByteArray()) - .setValue(qvalue.getValue().toByteArray()).build(); + Cell cell = + CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setType(Cell.Type.Put) + .setRow(req.getRow().toByteArray()).setFamily(value.getFamily().toByteArray()) + .setQualifier(qvalue.getQualifier().toByteArray()) + .setValue(qvalue.getValue().toByteArray()).build(); resp = MutateResponse.newBuilder() - .setResult(ProtobufUtil.toResult(Result.create(Arrays.asList(cell)))).build(); + .setResult(ProtobufUtil.toResult(Result.create(Arrays.asList(cell)))).build(); break; default: resp = MutateResponse.getDefaultInstance(); @@ -202,25 +209,24 @@ public class TestAsyncTableTracing { } }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); final User user = UserProvider.instantiate(CONF).getCurrent(); - conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", - user) { + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test", user) { @Override AsyncRegionLocator getLocator() { AsyncRegionLocator locator = mock(AsyncRegionLocator.class); Answer> answer = - new Answer>() { + new Answer>() { - @Override - public CompletableFuture answer(InvocationOnMock invocation) - throws Throwable { - TableName tableName = invocation.getArgument(0); - RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); - ServerName serverName = ServerName.valueOf("rs", 16010, 12345); - HRegionLocation loc = new HRegionLocation(info, serverName); - return CompletableFuture.completedFuture(loc); - } - }; + @Override + public CompletableFuture answer(InvocationOnMock invocation) + throws Throwable { + TableName tableName = invocation.getArgument(0); + RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); + ServerName serverName = ServerName.valueOf("rs", 16010, 12345); + HRegionLocation loc = new HRegionLocation(info, serverName); + return CompletableFuture.completedFuture(loc); + } + }; doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class), any(RegionLocateType.class), anyLong()); doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class), @@ -249,26 +255,19 @@ public class TestAsyncTableTracing { // n.b. this method implementation must match the one of the same name found in // TestHTableTracing final TableName tableName = table.getName(); - final Matcher spanLocator = allOf( - hasName(containsString(tableOperation)), hasEnded()); + final Matcher spanLocator = + allOf(hasName(containsString(tableOperation)), hasEnded()); final String expectedName = tableOperation + " " + tableName.getNameWithNamespaceInclAsString(); - Waiter.waitFor(CONF, 1000, new MatcherPredicate<>( - "waiting for span to emit", - () -> traceRule.getSpans(), hasItem(spanLocator))); - List candidateSpans = traceRule.getSpans() - .stream() - .filter(spanLocator::matches) - .collect(Collectors.toList()); + Waiter.waitFor(CONF, 1000, new MatcherPredicate<>("waiting for span to emit", + () -> traceRule.getSpans(), hasItem(spanLocator))); + List candidateSpans = + traceRule.getSpans().stream().filter(spanLocator::matches).collect(Collectors.toList()); assertThat(candidateSpans, hasSize(1)); SpanData data = candidateSpans.iterator().next(); - assertThat(data, allOf( - hasName(expectedName), - hasKind(SpanKind.CLIENT), - hasStatusWithCode(StatusCode.OK), - buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(tableName), - matcher)); + assertThat(data, + allOf(hasName(expectedName), hasKind(SpanKind.CLIENT), hasStatusWithCode(StatusCode.OK), + buildConnectionAttributesMatcher(conn), buildTableAttributesMatcher(tableName), matcher)); } @Test @@ -306,16 +305,16 @@ public class TestAsyncTableTracing { @Test public void testIncrement() { table - .increment( - new Increment(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1)) - .join(); + .increment( + new Increment(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1)) + .join(); assertTrace("INCREMENT"); } @Test public void testIncrementColumnValue1() { table.incrementColumnValue(Bytes.toBytes(0), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1) - .join(); + .join(); assertTrace("INCREMENT"); } @@ -329,38 +328,37 @@ public class TestAsyncTableTracing { @Test public void testCheckAndMutate() { table.checkAndMutate(CheckAndMutate.newBuilder(Bytes.toBytes(0)) - .ifEquals(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")) - .build(new Delete(Bytes.toBytes(0)))).join(); + .ifEquals(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")) + .build(new Delete(Bytes.toBytes(0)))).join(); assertTrace("CHECK_AND_MUTATE"); } @Test public void testCheckAndMutateList() { CompletableFuture - .allOf(table.checkAndMutate(Arrays.asList(CheckAndMutate.newBuilder(Bytes.toBytes(0)) - .ifEquals(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")) - .build(new Delete(Bytes.toBytes(0))))).toArray(new CompletableFuture[0])) - .join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf( - "db.hbase.container_operations", "CHECK_AND_MUTATE", "DELETE"))); + .allOf(table.checkAndMutate(Arrays.asList(CheckAndMutate.newBuilder(Bytes.toBytes(0)) + .ifEquals(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")) + .build(new Delete(Bytes.toBytes(0))))).toArray(new CompletableFuture[0])) + .join(); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", + "CHECK_AND_MUTATE", "DELETE"))); } @Test public void testCheckAndMutateAll() { table.checkAndMutateAll(Arrays.asList(CheckAndMutate.newBuilder(Bytes.toBytes(0)) - .ifEquals(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")) - .build(new Delete(Bytes.toBytes(0))))).join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf( - "db.hbase.container_operations", "CHECK_AND_MUTATE", "DELETE"))); + .ifEquals(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")) + .build(new Delete(Bytes.toBytes(0))))).join(); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", + "CHECK_AND_MUTATE", "DELETE"))); } private void testCheckAndMutateBuilder(Row op) { AsyncTable.CheckAndMutateBuilder builder = - table.checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) - .qualifier(Bytes.toBytes("cq")) - .ifEquals(Bytes.toBytes("v")); + table.checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")).qualifier(Bytes.toBytes("cq")) + .ifEquals(Bytes.toBytes("v")); if (op instanceof Put) { Put put = (Put) op; builder.thenPut(put).join(); @@ -378,8 +376,8 @@ public class TestAsyncTableTracing { @Test public void testCheckAndMutateBuilderThenPut() { - Put put = new Put(Bytes.toBytes(0)) - .addColumn(Bytes.toBytes("f"), Bytes.toBytes("cq"), Bytes.toBytes("v")); + Put put = new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("f"), Bytes.toBytes("cq"), + Bytes.toBytes("v")); testCheckAndMutateBuilder(put); } @@ -390,17 +388,18 @@ public class TestAsyncTableTracing { @Test public void testCheckAndMutateBuilderThenMutations() throws IOException { - RowMutations mutations = new RowMutations(Bytes.toBytes(0)) - .add((Mutation) (new Put(Bytes.toBytes(0)) - .addColumn(Bytes.toBytes("f"), Bytes.toBytes("cq"), Bytes.toBytes("v")))) - .add((Mutation) new Delete(Bytes.toBytes(0))); + RowMutations mutations = + new RowMutations(Bytes.toBytes(0)) + .add((Mutation) (new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("f"), + Bytes.toBytes("cq"), Bytes.toBytes("v")))) + .add((Mutation) new Delete(Bytes.toBytes(0))); testCheckAndMutateBuilder(mutations); } private void testCheckAndMutateWithFilterBuilder(Row op) { // use of `PrefixFilter` is completely arbitrary here. AsyncTable.CheckAndMutateWithFilterBuilder builder = - table.checkAndMutate(Bytes.toBytes(0), new PrefixFilter(Bytes.toBytes(0))); + table.checkAndMutate(Bytes.toBytes(0), new PrefixFilter(Bytes.toBytes(0))); if (op instanceof Put) { Put put = (Put) op; builder.thenPut(put).join(); @@ -418,8 +417,8 @@ public class TestAsyncTableTracing { @Test public void testCheckAndMutateWithFilterBuilderThenPut() { - Put put = new Put(Bytes.toBytes(0)) - .addColumn(Bytes.toBytes("f"), Bytes.toBytes("cq"), Bytes.toBytes("v")); + Put put = new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("f"), Bytes.toBytes("cq"), + Bytes.toBytes("v")); testCheckAndMutateWithFilterBuilder(put); } @@ -430,19 +429,21 @@ public class TestAsyncTableTracing { @Test public void testCheckAndMutateWithFilterBuilderThenMutations() throws IOException { - RowMutations mutations = new RowMutations(Bytes.toBytes(0)) - .add((Mutation) new Put(Bytes.toBytes(0)) - .addColumn(Bytes.toBytes("f"), Bytes.toBytes("cq"), Bytes.toBytes("v"))) - .add((Mutation) new Delete(Bytes.toBytes(0))); + RowMutations mutations = + new RowMutations(Bytes.toBytes(0)) + .add((Mutation) new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("f"), + Bytes.toBytes("cq"), Bytes.toBytes("v"))) + .add((Mutation) new Delete(Bytes.toBytes(0))); testCheckAndMutateWithFilterBuilder(mutations); } @Test public void testMutateRow() throws IOException { - final RowMutations mutations = new RowMutations(Bytes.toBytes(0)) - .add((Mutation) new Put(Bytes.toBytes(0)) - .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))) - .add((Mutation) new Delete(Bytes.toBytes(0))); + final RowMutations mutations = + new RowMutations(Bytes.toBytes(0)) + .add((Mutation) new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), + Bytes.toBytes("cq"), Bytes.toBytes("v"))) + .add((Mutation) new Delete(Bytes.toBytes(0))); table.mutateRow(mutations).join(); assertTrace("BATCH", hasAttributes( containsEntryWithStringValuesOf("db.hbase.container_operations", "DELETE", "PUT"))); @@ -454,88 +455,137 @@ public class TestAsyncTableTracing { assertTrace("SCAN"); } + @Test + public void testScan() throws Throwable { + final CountDownLatch doneSignal = new CountDownLatch(1); + final AtomicInteger count = new AtomicInteger(); + final AtomicReference throwable = new AtomicReference<>(); + final Scan scan = new Scan().setCaching(1).setMaxResultSize(1).setLimit(1); + table.scan(scan, new ScanResultConsumer() { + @Override + public boolean onNext(Result result) { + if (result.getRow() != null) { + count.incrementAndGet(); + } + return true; + } + + @Override + public void onError(Throwable error) { + throwable.set(error); + doneSignal.countDown(); + } + + @Override + public void onComplete() { + doneSignal.countDown(); + } + }); + doneSignal.await(); + if (throwable.get() != null) { + throw throwable.get(); + } + assertThat("user code did not run. check test setup.", count.get(), greaterThan(0)); + assertTrace("SCAN"); + } + + @Test + public void testGetScanner() { + final Scan scan = new Scan().setCaching(1).setMaxResultSize(1).setLimit(1); + try (ResultScanner scanner = table.getScanner(scan)) { + int count = 0; + for (Result result : scanner) { + if (result.getRow() != null) { + count++; + } + } + // do something with it. + assertThat(count, greaterThanOrEqualTo(0)); + } + assertTrace("SCAN"); + } + @Test public void testExistsList() { CompletableFuture - .allOf( - table.exists(Arrays.asList(new Get(Bytes.toBytes(0)))).toArray(new CompletableFuture[0])) - .join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "GET"))); + .allOf( + table.exists(Arrays.asList(new Get(Bytes.toBytes(0)))).toArray(new CompletableFuture[0])) + .join(); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "GET"))); } @Test public void testExistsAll() { table.existsAll(Arrays.asList(new Get(Bytes.toBytes(0)))).join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "GET"))); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "GET"))); } @Test public void testGetList() { CompletableFuture - .allOf(table.get(Arrays.asList(new Get(Bytes.toBytes(0)))).toArray(new CompletableFuture[0])) - .join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "GET"))); + .allOf( + table.get(Arrays.asList(new Get(Bytes.toBytes(0)))).toArray(new CompletableFuture[0])) + .join(); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "GET"))); } @Test public void testGetAll() { table.getAll(Arrays.asList(new Get(Bytes.toBytes(0)))).join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "GET"))); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "GET"))); } @Test public void testPutList() { CompletableFuture - .allOf(table.put(Arrays.asList(new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), - Bytes.toBytes("cq"), Bytes.toBytes("v")))).toArray(new CompletableFuture[0])) - .join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "PUT"))); + .allOf(table.put(Arrays.asList(new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), + Bytes.toBytes("cq"), Bytes.toBytes("v")))).toArray(new CompletableFuture[0])) + .join(); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "PUT"))); } @Test public void testPutAll() { table.putAll(Arrays.asList(new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v")))).join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "PUT"))); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "PUT"))); } @Test public void testDeleteList() { - CompletableFuture - .allOf( - table.delete(Arrays.asList(new Delete(Bytes.toBytes(0)))).toArray(new CompletableFuture[0])) - .join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "DELETE"))); + CompletableFuture.allOf( + table.delete(Arrays.asList(new Delete(Bytes.toBytes(0)))).toArray(new CompletableFuture[0])) + .join(); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "DELETE"))); } @Test public void testDeleteAll() { table.deleteAll(Arrays.asList(new Delete(Bytes.toBytes(0)))).join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "DELETE"))); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "DELETE"))); } @Test public void testBatch() { - CompletableFuture - .allOf( - table.batch(Arrays.asList(new Delete(Bytes.toBytes(0)))).toArray(new CompletableFuture[0])) - .join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "DELETE"))); + CompletableFuture.allOf( + table.batch(Arrays.asList(new Delete(Bytes.toBytes(0)))).toArray(new CompletableFuture[0])) + .join(); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "DELETE"))); } @Test public void testBatchAll() { table.batchAll(Arrays.asList(new Delete(Bytes.toBytes(0)))).join(); - assertTrace("BATCH", hasAttributes( - containsEntryWithStringValuesOf("db.hbase.container_operations", "DELETE"))); + assertTrace("BATCH", + hasAttributes(containsEntryWithStringValuesOf("db.hbase.container_operations", "DELETE"))); } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/SpanDataMatchers.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/SpanDataMatchers.java index 01aa61805a2..c7a9d9029fc 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/SpanDataMatchers.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/trace/hamcrest/SpanDataMatchers.java @@ -17,15 +17,19 @@ */ package org.apache.hadoop.hbase.client.trace.hamcrest; +import static org.apache.hadoop.hbase.client.trace.hamcrest.AttributesMatchers.containsEntry; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; + import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.trace.SpanKind; import io.opentelemetry.api.trace.StatusCode; import io.opentelemetry.sdk.trace.data.EventData; import io.opentelemetry.sdk.trace.data.SpanData; import io.opentelemetry.sdk.trace.data.StatusData; +import io.opentelemetry.semconv.trace.attributes.SemanticAttributes; import java.time.Duration; +import java.util.Objects; import org.hamcrest.Description; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; @@ -36,21 +40,22 @@ import org.hamcrest.TypeSafeMatcher; */ public final class SpanDataMatchers { - private SpanDataMatchers() { } + private SpanDataMatchers() { + } public static Matcher hasAttributes(Matcher matcher) { - return new FeatureMatcher( - matcher, "SpanData having attributes that ", "attributes" - ) { - @Override protected Attributes featureValueOf(SpanData item) { + return new FeatureMatcher(matcher, "SpanData having attributes that ", + "attributes") { + @Override + protected Attributes featureValueOf(SpanData item) { return item.getAttributes(); } }; } public static Matcher hasDuration(Matcher matcher) { - return new FeatureMatcher( - matcher, "SpanData having duration that ", "duration") { + return new FeatureMatcher(matcher, "SpanData having duration that ", + "duration") { @Override protected Duration featureValueOf(SpanData item) { return Duration.ofNanos(item.getEndEpochNanos() - item.getStartEpochNanos()); @@ -60,28 +65,49 @@ public final class SpanDataMatchers { public static Matcher hasEnded() { return new TypeSafeMatcher() { - @Override protected boolean matchesSafely(SpanData item) { + @Override + protected boolean matchesSafely(SpanData item) { return item.hasEnded(); } - @Override public void describeTo(Description description) { + + @Override + public void describeTo(Description description) { description.appendText("SpanData that hasEnded"); } }; } public static Matcher hasEvents(Matcher> matcher) { - return new FeatureMatcher>( - matcher, "SpanData having events that", "events") { - @Override protected Iterable featureValueOf(SpanData item) { + return new FeatureMatcher>(matcher, + "SpanData having events that", "events") { + @Override + protected Iterable featureValueOf(SpanData item) { return item.getEvents(); } }; } + public static Matcher hasExceptionWithType(Matcher matcher) { + return hasException(containsEntry(is(SemanticAttributes.EXCEPTION_TYPE), matcher)); + } + + public static Matcher hasException(Matcher matcher) { + return new FeatureMatcher(matcher, + "SpanData having Exception with Attributes that", "exception attributes") { + @Override + protected Attributes featureValueOf(SpanData actual) { + return actual.getEvents().stream() + .filter(e -> Objects.equals(SemanticAttributes.EXCEPTION_EVENT_NAME, e.getName())) + .map(EventData::getAttributes).findFirst().orElse(null); + } + }; + } + public static Matcher hasKind(SpanKind kind) { - return new FeatureMatcher( - equalTo(kind), "SpanData with kind that", "SpanKind") { - @Override protected SpanKind featureValueOf(SpanData item) { + return new FeatureMatcher(equalTo(kind), "SpanData with kind that", + "SpanKind") { + @Override + protected SpanKind featureValueOf(SpanData item) { return item.getKind(); } }; @@ -93,7 +119,8 @@ public final class SpanDataMatchers { public static Matcher hasName(Matcher matcher) { return new FeatureMatcher(matcher, "SpanKind with a name that", "name") { - @Override protected String featureValueOf(SpanData item) { + @Override + protected String featureValueOf(SpanData item) { return item.getName(); } }; @@ -109,9 +136,9 @@ public final class SpanDataMatchers { public static Matcher hasParentSpanId(Matcher matcher) { return new FeatureMatcher(matcher, "SpanKind with a parentSpanId that", - "parentSpanId" - ) { - @Override protected String featureValueOf(SpanData item) { + "parentSpanId") { + @Override + protected String featureValueOf(SpanData item) { return item.getParentSpanId(); } }; @@ -120,13 +147,15 @@ public final class SpanDataMatchers { public static Matcher hasStatusWithCode(StatusCode statusCode) { final Matcher matcher = is(equalTo(statusCode)); return new TypeSafeMatcher() { - @Override protected boolean matchesSafely(SpanData item) { + @Override + protected boolean matchesSafely(SpanData item) { final StatusData statusData = item.getStatus(); - return statusData != null - && statusData.getStatusCode() != null - && matcher.matches(statusData.getStatusCode()); + return statusData != null && statusData.getStatusCode() != null + && matcher.matches(statusData.getStatusCode()); } - @Override public void describeTo(Description description) { + + @Override + public void describeTo(Description description) { description.appendText("SpanData with StatusCode that ").appendDescriptionOf(matcher); } }; @@ -137,9 +166,10 @@ public final class SpanDataMatchers { } public static Matcher hasTraceId(Matcher matcher) { - return new FeatureMatcher( - matcher, "SpanData with a traceId that ", "traceId") { - @Override protected String featureValueOf(SpanData item) { + return new FeatureMatcher(matcher, "SpanData with a traceId that ", + "traceId") { + @Override + protected String featureValueOf(SpanData item) { return item.getTraceId(); } }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/StartMiniClusterOption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/StartMiniClusterOption.java index 7a9bd68c6d8..5ffd8f09121 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/StartMiniClusterOption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/StartMiniClusterOption.java @@ -1,5 +1,4 @@ -/** - * +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -20,7 +19,6 @@ package org.apache.hadoop.hbase; import java.util.Arrays; import java.util.List; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hbase.master.HMaster; import org.apache.yetus.audience.InterfaceAudience; @@ -28,10 +26,9 @@ import org.apache.yetus.audience.InterfaceAudience; /** * Options for starting up a mini cluster (including an hbase, dfs and zookeeper clusters) in test. * The options include HDFS options to build mini dfs cluster, Zookeeper options to build mini zk - * cluster, and mostly HBase options to build mini hbase cluster. - * - * To create an object, use a {@link Builder}. - * Example usage: + * cluster, and mostly HBase options to build mini hbase cluster. To create an object, use a + * {@link Builder}. Example usage: + * *
  *    StartMiniClusterOption option = StartMiniClusterOption.builder().
  *        .numMasters(3).rsClass(MyRegionServer.class).createWALDir(true).build();
@@ -42,8 +39,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Public
 public final class StartMiniClusterOption {
   /**
-   * Number of masters to start up.  We'll start this many hbase masters.  If numMasters > 1, you
-   * can find the active/primary master with {@link MiniHBaseCluster#getMaster()}.
+   * Number of masters to start up. We'll start this many hbase masters. If numMasters > 1, you can
+   * find the active/primary master with {@link MiniHBaseCluster#getMaster()}.
    */
   private final int numMasters;
 
@@ -60,9 +57,8 @@ public final class StartMiniClusterOption {
   private final Class masterClass;
 
   /**
-   * Number of region servers to start up.
-   * If this value is > 1, then make sure config "hbase.regionserver.info.port" is -1
-   * (i.e. no ui per regionserver) otherwise bind errors.
+   * Number of region servers to start up. If this value is > 1, then make sure config
+   * "hbase.regionserver.info.port" is -1 (i.e. no ui per regionserver) otherwise bind errors.
    */
   private final int numRegionServers;
   /**
@@ -93,13 +89,13 @@ public final class StartMiniClusterOption {
   private final int numZkServers;
 
   /**
-   * Whether to create a new root or data directory path.  If true, the newly created data directory
-   * will be configured as HBase rootdir.  This will overwrite existing root directory config.
+   * Whether to create a new root or data directory path. If true, the newly created data directory
+   * will be configured as HBase rootdir. This will overwrite existing root directory config.
    */
   private final boolean createRootDir;
 
   /**
-   * Whether to create a new WAL directory.  If true, the newly created directory will be configured
+   * Whether to create a new WAL directory. If true, the newly created directory will be configured
    * as HBase wal.dir which is separate from HBase rootdir.
    */
   private final boolean createWALDir;
@@ -172,9 +168,9 @@ public final class StartMiniClusterOption {
   public String toString() {
     return "StartMiniClusterOption{" + "numMasters=" + numMasters + ", masterClass=" + masterClass
         + ", numRegionServers=" + numRegionServers + ", rsPorts=" + StringUtils.join(rsPorts)
-        + ", rsClass=" + rsClass + ", numDataNodes=" + numDataNodes
-        + ", dataNodeHosts=" + Arrays.toString(dataNodeHosts) + ", numZkServers=" + numZkServers
-        + ", createRootDir=" + createRootDir + ", createWALDir=" + createWALDir + '}';
+        + ", rsClass=" + rsClass + ", numDataNodes=" + numDataNodes + ", dataNodeHosts="
+        + Arrays.toString(dataNodeHosts) + ", numZkServers=" + numZkServers + ", createRootDir="
+        + createRootDir + ", createWALDir=" + createWALDir + '}';
   }
 
   /**
@@ -185,10 +181,9 @@ public final class StartMiniClusterOption {
   }
 
   /**
-   * Builder pattern for creating an {@link StartMiniClusterOption}.
-   *
-   * The default values of its fields should be considered public and constant. Changing the default
-   * values may cause other tests fail.
+   * Builder pattern for creating an {@link StartMiniClusterOption}. The default values of its
+   * fields should be considered public and constant. Changing the default values may cause other
+   * tests fail.
    */
   public static final class Builder {
     private int numMasters = 1;
@@ -210,7 +205,7 @@ public final class StartMiniClusterOption {
       if (dataNodeHosts != null && dataNodeHosts.length != 0) {
         numDataNodes = dataNodeHosts.length;
       }
-      return new StartMiniClusterOption(numMasters,numAlwaysStandByMasters, masterClass,
+      return new StartMiniClusterOption(numMasters, numAlwaysStandByMasters, masterClass,
           numRegionServers, rsPorts, rsClass, numDataNodes, dataNodeHosts, numZkServers,
           createRootDir, createWALDir);
     }
@@ -260,6 +255,10 @@ public final class StartMiniClusterOption {
       return this;
     }
 
+    public Builder numWorkers(int numWorkers) {
+      return numDataNodes(numWorkers).numRegionServers(numWorkers);
+    }
+
     public Builder createRootDir(boolean createRootDir) {
       this.createRootDir = createRootDir;
       return this;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestAsyncTableScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestAsyncTableScan.java
index f971a5384d7..25c8eaf6645 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestAsyncTableScan.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestAsyncTableScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,30 +17,91 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.endsWith;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.hasProperty;
+import static org.hamcrest.Matchers.isA;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.fail;
 
+import io.opentelemetry.sdk.trace.data.SpanData;
 import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.util.Arrays;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
-
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ConnectionRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MatcherPredicate;
+import org.apache.hadoop.hbase.MiniClusterRule;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
+import org.apache.hadoop.hbase.trace.OpenTelemetryClassRule;
+import org.apache.hadoop.hbase.trace.OpenTelemetryTestRule;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.Pair;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.hamcrest.Matcher;
+import org.junit.ClassRule;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
 
 public abstract class AbstractTestAsyncTableScan {
 
-  protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static final OpenTelemetryClassRule otelClassRule = OpenTelemetryClassRule.create();
+  protected static final MiniClusterRule miniClusterRule = MiniClusterRule.newBuilder()
+      .setMiniClusterOption(StartMiniClusterOption.builder().numWorkers(3).build()).build();
+
+  protected static final ConnectionRule connectionRule =
+      ConnectionRule.createAsyncConnectionRule(miniClusterRule::createAsyncConnection);
+
+  private static final class Setup extends ExternalResource {
+    @Override
+    protected void before() throws Throwable {
+      final HBaseTestingUtility testingUtil = miniClusterRule.getTestingUtility();
+      final AsyncConnection conn = connectionRule.getAsyncConnection();
+
+      byte[][] splitKeys = new byte[8][];
+      for (int i = 111; i < 999; i += 111) {
+        splitKeys[i / 111 - 1] = Bytes.toBytes(String.format("%03d", i));
+      }
+      testingUtil.createTable(TABLE_NAME, FAMILY, splitKeys);
+      testingUtil.waitTableAvailable(TABLE_NAME);
+      conn.getTable(TABLE_NAME)
+          .putAll(IntStream.range(0, COUNT)
+              .mapToObj(i -> new Put(Bytes.toBytes(String.format("%03d", i)))
+                  .addColumn(FAMILY, CQ1, Bytes.toBytes(i))
+                  .addColumn(FAMILY, CQ2, Bytes.toBytes(i * i)))
+              .collect(Collectors.toList()))
+          .get();
+    }
+  }
+
+  @ClassRule
+  public static final TestRule classRule = RuleChain.outerRule(otelClassRule)
+      .around(miniClusterRule).around(connectionRule).around(new Setup());
+
+  @Rule
+  public final OpenTelemetryTestRule otelTestRule = new OpenTelemetryTestRule(otelClassRule);
+
+  @Rule
+  public final TestName testName = new TestName();
 
   protected static TableName TABLE_NAME = TableName.valueOf("async");
 
@@ -52,53 +113,29 @@ public abstract class AbstractTestAsyncTableScan {
 
   protected static int COUNT = 1000;
 
-  protected static AsyncConnection ASYNC_CONN;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    TEST_UTIL.startMiniCluster(3);
-    byte[][] splitKeys = new byte[8][];
-    for (int i = 111; i < 999; i += 111) {
-      splitKeys[i / 111 - 1] = Bytes.toBytes(String.format("%03d", i));
-    }
-    TEST_UTIL.createTable(TABLE_NAME, FAMILY, splitKeys);
-    TEST_UTIL.waitTableAvailable(TABLE_NAME);
-    ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
-    ASYNC_CONN.getTable(TABLE_NAME).putAll(IntStream.range(0, COUNT)
-        .mapToObj(i -> new Put(Bytes.toBytes(String.format("%03d", i)))
-            .addColumn(FAMILY, CQ1, Bytes.toBytes(i)).addColumn(FAMILY, CQ2, Bytes.toBytes(i * i)))
-        .collect(Collectors.toList())).get();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    ASYNC_CONN.close();
-    TEST_UTIL.shutdownMiniCluster();
-  }
-
-  protected static Scan createNormalScan() {
+  private static Scan createNormalScan() {
     return new Scan();
   }
 
-  protected static Scan createBatchScan() {
+  private static Scan createBatchScan() {
     return new Scan().setBatch(1);
   }
 
   // set a small result size for testing flow control
-  protected static Scan createSmallResultSizeScan() {
+  private static Scan createSmallResultSizeScan() {
     return new Scan().setMaxResultSize(1);
   }
 
-  protected static Scan createBatchSmallResultSizeScan() {
+  private static Scan createBatchSmallResultSizeScan() {
     return new Scan().setBatch(1).setMaxResultSize(1);
   }
 
-  protected static AsyncTable getRawTable() {
-    return ASYNC_CONN.getTable(TABLE_NAME);
+  private static AsyncTable getRawTable() {
+    return connectionRule.getAsyncConnection().getTable(TABLE_NAME);
   }
 
-  protected static AsyncTable getTable() {
-    return ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
+  private static AsyncTable getTable() {
+    return connectionRule.getAsyncConnection().getTable(TABLE_NAME, ForkJoinPool.commonPool());
   }
 
   private static List>> getScanCreator() {
@@ -132,8 +169,18 @@ public abstract class AbstractTestAsyncTableScan {
 
   protected abstract List doScan(Scan scan, int closeAfter) throws Exception;
 
+  /**
+   * Used by implementation classes to assert the correctness of spans produced under test.
+   */
+  protected abstract void assertTraceContinuity();
+
+  /**
+   * Used by implementation classes to assert the correctness of spans having errors.
+   */
+  protected abstract void assertTraceError(final Matcher exceptionTypeNameMatcher);
+
   protected final List convertFromBatchResult(List results) {
-    assertTrue(results.size() % 2 == 0);
+    assertEquals(0, results.size() % 2);
     return IntStream.range(0, results.size() / 2).mapToObj(i -> {
       try {
         return Result
@@ -144,16 +191,22 @@ public abstract class AbstractTestAsyncTableScan {
     }).collect(Collectors.toList());
   }
 
+  protected static void waitForSpan(final Matcher parentSpanMatcher) {
+    final Configuration conf = miniClusterRule.getTestingUtility().getConfiguration();
+    Waiter.waitFor(conf, TimeUnit.SECONDS.toMillis(5), new MatcherPredicate<>(
+        "Span for test failed to complete.", otelClassRule::getSpans, hasItem(parentSpanMatcher)));
+  }
+
   @Test
   public void testScanAll() throws Exception {
     List results = doScan(createScan(), -1);
     // make sure all scanners are closed at RS side
-    TEST_UTIL.getHBaseCluster().getRegionServerThreads().stream().map(t -> t.getRegionServer())
-        .forEach(
-          rs -> assertEquals(
-            "The scanner count of " + rs.getServerName() + " is " +
-              rs.getRSRpcServices().getScannersCount(),
-            0, rs.getRSRpcServices().getScannersCount()));
+    miniClusterRule.getTestingUtility().getHBaseCluster().getRegionServerThreads().stream()
+        .map(JVMClusterUtil.RegionServerThread::getRegionServer)
+        .forEach(rs -> assertEquals(
+          "The scanner count of " + rs.getServerName() + " is "
+              + rs.getRSRpcServices().getScannersCount(),
+          0, rs.getRSRpcServices().getScannersCount()));
     assertEquals(COUNT, results.size());
     IntStream.range(0, COUNT).forEach(i -> {
       Result result = results.get(i);
@@ -170,49 +223,66 @@ public abstract class AbstractTestAsyncTableScan {
 
   @Test
   public void testReversedScanAll() throws Exception {
-    List results = doScan(createScan().setReversed(true), -1);
+    List results =
+        TraceUtil.trace(() -> doScan(createScan().setReversed(true), -1), testName.getMethodName());
     assertEquals(COUNT, results.size());
     IntStream.range(0, COUNT).forEach(i -> assertResultEquals(results.get(i), COUNT - i - 1));
+    assertTraceContinuity();
   }
 
   @Test
   public void testScanNoStopKey() throws Exception {
     int start = 345;
-    List results =
-      doScan(createScan().withStartRow(Bytes.toBytes(String.format("%03d", start))), -1);
+    List results = TraceUtil.trace(
+      () -> doScan(createScan().withStartRow(Bytes.toBytes(String.format("%03d", start))), -1),
+      testName.getMethodName());
     assertEquals(COUNT - start, results.size());
     IntStream.range(0, COUNT - start).forEach(i -> assertResultEquals(results.get(i), start + i));
+    assertTraceContinuity();
   }
 
   @Test
   public void testReverseScanNoStopKey() throws Exception {
     int start = 765;
-    List results = doScan(
-      createScan().withStartRow(Bytes.toBytes(String.format("%03d", start))).setReversed(true), -1);
+    final Scan scan =
+        createScan().withStartRow(Bytes.toBytes(String.format("%03d", start))).setReversed(true);
+    List results = TraceUtil.trace(() -> doScan(scan, -1), testName.getMethodName());
     assertEquals(start + 1, results.size());
     IntStream.range(0, start + 1).forEach(i -> assertResultEquals(results.get(i), start - i));
+    assertTraceContinuity();
   }
 
   @Test
-  public void testScanWrongColumnFamily() throws Exception {
-    try {
-      doScan(createScan().addFamily(Bytes.toBytes("WrongColumnFamily")), -1);
-    } catch (Exception e) {
-      assertTrue(e instanceof NoSuchColumnFamilyException ||
-        e.getCause() instanceof NoSuchColumnFamilyException);
+  public void testScanWrongColumnFamily() {
+    final Exception e = assertThrows(Exception.class,
+      () -> TraceUtil.trace(
+        () -> doScan(createScan().addFamily(Bytes.toBytes("WrongColumnFamily")), -1),
+        testName.getMethodName()));
+    // hamcrest generic enforcement for `anyOf` is a pain; skip it
+    // but -- don't we always unwrap ExecutionExceptions -- bug?
+    if (e instanceof NoSuchColumnFamilyException) {
+      final NoSuchColumnFamilyException ex = (NoSuchColumnFamilyException) e;
+      assertThat(ex, isA(NoSuchColumnFamilyException.class));
+    } else if (e instanceof ExecutionException) {
+      final ExecutionException ex = (ExecutionException) e;
+      assertThat(ex, allOf(isA(ExecutionException.class),
+        hasProperty("cause", isA(NoSuchColumnFamilyException.class))));
+    } else {
+      fail("Found unexpected Exception " + e);
     }
+    assertTraceError(endsWith(NoSuchColumnFamilyException.class.getName()));
   }
 
   private void testScan(int start, boolean startInclusive, int stop, boolean stopInclusive,
-    int limit) throws Exception {
+      int limit) throws Exception {
     testScan(start, startInclusive, stop, stopInclusive, limit, -1);
   }
 
   private void testScan(int start, boolean startInclusive, int stop, boolean stopInclusive,
       int limit, int closeAfter) throws Exception {
     Scan scan =
-      createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
-          .withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive);
+        createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
+            .withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive);
     if (limit > 0) {
       scan.setLimit(limit);
     }
@@ -232,9 +302,9 @@ public abstract class AbstractTestAsyncTableScan {
 
   private void testReversedScan(int start, boolean startInclusive, int stop, boolean stopInclusive,
       int limit) throws Exception {
-    Scan scan =
-      createScan().withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
-          .withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive).setReversed(true);
+    Scan scan = createScan()
+        .withStartRow(Bytes.toBytes(String.format("%03d", start)), startInclusive)
+        .withStopRow(Bytes.toBytes(String.format("%03d", stop)), stopInclusive).setReversed(true);
     if (limit > 0) {
       scan.setLimit(limit);
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/LimitedScanResultConsumer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/LimitedScanResultConsumer.java
new file mode 100644
index 00000000000..28fbc5ab28f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/LimitedScanResultConsumer.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+/**
+ * Advise the scanning infrastructure to collect up to {@code limit} results.
+ */
+class LimitedScanResultConsumer extends SimpleScanResultConsumerImpl {
+
+  private final int limit;
+
+  public LimitedScanResultConsumer(int limit) {
+    this.limit = limit;
+  }
+
+  @Override
+  public synchronized boolean onNext(Result result) {
+    return super.onNext(result) && results.size() < limit;
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SimpleScanResultConsumer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SimpleScanResultConsumer.java
index fce6773f7a6..b019fe75420 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SimpleScanResultConsumer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SimpleScanResultConsumer.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,59 +17,15 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
-
-import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 
-final class SimpleScanResultConsumer implements ScanResultConsumer {
+/**
+ * A simplistic {@link ScanResultConsumer} for use in tests.
+ */
+public interface SimpleScanResultConsumer extends ScanResultConsumer {
 
-  private ScanMetrics scanMetrics;
+  List getAll() throws Exception;
 
-  private final List results = new ArrayList<>();
-
-  private Throwable error;
-
-  private boolean finished = false;
-
-  @Override
-  public void onScanMetricsCreated(ScanMetrics scanMetrics) {
-    this.scanMetrics = scanMetrics;
-  }
-
-  @Override
-  public synchronized boolean onNext(Result result) {
-    results.add(result);
-    return true;
-  }
-
-  @Override
-  public synchronized void onError(Throwable error) {
-    this.error = error;
-    finished = true;
-    notifyAll();
-  }
-
-  @Override
-  public synchronized void onComplete() {
-    finished = true;
-    notifyAll();
-  }
-
-  public synchronized List getAll() throws Exception {
-    while (!finished) {
-      wait();
-    }
-    if (error != null) {
-      Throwables.propagateIfPossible(error, Exception.class);
-      throw new Exception(error);
-    }
-    return results;
-  }
-
-  public ScanMetrics getScanMetrics() {
-    return scanMetrics;
-  }
+  ScanMetrics getScanMetrics();
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SimpleScanResultConsumerImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SimpleScanResultConsumerImpl.java
new file mode 100644
index 00000000000..a7d7f7661ff
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/SimpleScanResultConsumerImpl.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+class SimpleScanResultConsumerImpl implements SimpleScanResultConsumer {
+
+  private ScanMetrics scanMetrics;
+
+  protected final List results = new ArrayList<>();
+
+  private Throwable error;
+
+  private boolean finished = false;
+
+  @Override
+  public void onScanMetricsCreated(ScanMetrics scanMetrics) {
+    this.scanMetrics = scanMetrics;
+  }
+
+  @Override
+  public synchronized boolean onNext(Result result) {
+    results.add(result);
+    return true;
+  }
+
+  @Override
+  public synchronized void onError(Throwable error) {
+    this.error = error;
+    finished = true;
+    notifyAll();
+  }
+
+  @Override
+  public synchronized void onComplete() {
+    finished = true;
+    notifyAll();
+  }
+
+  @Override
+  public synchronized List getAll() throws Exception {
+    while (!finished) {
+      wait();
+    }
+    if (error != null) {
+      Throwables.propagateIfPossible(error, Exception.class);
+      throw new Exception(error);
+    }
+    return results;
+  }
+
+  @Override
+  public ScanMetrics getScanMetrics() {
+    return scanMetrics;
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScan.java
index c1797f3833c..9efb5447a66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScan.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,24 +17,41 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.util.ArrayList;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasEnded;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasExceptionWithType;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasName;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasParentSpanId;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasStatusWithCode;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.startsWith;
+
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.sdk.trace.data.SpanData;
 import java.util.List;
+import java.util.Objects;
 import java.util.concurrent.ForkJoinPool;
 import java.util.function.Supplier;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.trace.StringTraceRenderer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.hamcrest.Matcher;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @RunWith(Parameterized.class)
 @Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncTableScan extends AbstractTestAsyncTableScan {
+  private static final Logger logger = LoggerFactory.getLogger(TestAsyncTableScan.class);
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -59,7 +76,7 @@ public class TestAsyncTableScan extends AbstractTestAsyncTableScan {
   @Override
   protected List doScan(Scan scan, int closeAfter) throws Exception {
     AsyncTable table =
-      ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
+        connectionRule.getAsyncConnection().getTable(TABLE_NAME, ForkJoinPool.commonPool());
     List results;
     if (closeAfter > 0) {
       // these tests batch settings with the sample data result in each result being
@@ -68,11 +85,13 @@ public class TestAsyncTableScan extends AbstractTestAsyncTableScan {
       if (scan.getBatch() > 0) {
         closeAfter = closeAfter * 2;
       }
-      LimitedScanResultConsumer consumer = new LimitedScanResultConsumer(closeAfter);
+      TracedScanResultConsumer consumer =
+          new TracedScanResultConsumer(new LimitedScanResultConsumer(closeAfter));
       table.scan(scan, consumer);
       results = consumer.getAll();
     } else {
-      SimpleScanResultConsumer consumer = new SimpleScanResultConsumer();
+      TracedScanResultConsumer consumer =
+          new TracedScanResultConsumer(new SimpleScanResultConsumerImpl());
       table.scan(scan, consumer);
       results = consumer.getAll();
     }
@@ -82,49 +101,77 @@ public class TestAsyncTableScan extends AbstractTestAsyncTableScan {
     return results;
   }
 
-  private static class LimitedScanResultConsumer implements ScanResultConsumer {
+  @Override
+  protected void assertTraceContinuity() {
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher =
+        allOf(hasName(parentSpanName), hasStatusWithCode(StatusCode.OK), hasEnded());
+    waitForSpan(parentSpanMatcher);
 
-    private final int limit;
-
-    public LimitedScanResultConsumer(int limit) {
-      this.limit = limit;
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (logger.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(logger::debug);
     }
 
-    private final List results = new ArrayList<>();
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
 
-    private Throwable error;
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.OK), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+    final String scanOperationSpanId = spans.stream().filter(scanOperationSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
 
-    private boolean finished = false;
+    final Matcher onScanMetricsCreatedMatcher =
+        hasName("TracedScanResultConsumer#onScanMetricsCreated");
+    assertThat(spans, hasItem(onScanMetricsCreatedMatcher));
+    spans.stream().filter(onScanMetricsCreatedMatcher::matches).forEach(span -> assertThat(span,
+      allOf(onScanMetricsCreatedMatcher, hasParentSpanId(scanOperationSpanId), hasEnded())));
 
-    @Override
-    public synchronized boolean onNext(Result result) {
-      results.add(result);
-      return results.size() < limit;
-    }
+    final Matcher onNextMatcher = hasName("TracedScanResultConsumer#onNext");
+    assertThat(spans, hasItem(onNextMatcher));
+    spans.stream().filter(onNextMatcher::matches)
+        .forEach(span -> assertThat(span, allOf(onNextMatcher, hasParentSpanId(scanOperationSpanId),
+          hasStatusWithCode(StatusCode.OK), hasEnded())));
 
-    @Override
-    public synchronized void onError(Throwable error) {
-      this.error = error;
-      finished = true;
-      notifyAll();
-    }
-
-    @Override
-    public synchronized void onComplete() {
-      finished = true;
-      notifyAll();
-    }
-
-    public synchronized List getAll() throws Exception {
-      while (!finished) {
-        wait();
-      }
-      if (error != null) {
-        Throwables.propagateIfPossible(error, Exception.class);
-        throw new Exception(error);
-      }
-      return results;
-    }
+    final Matcher onCompleteMatcher = hasName("TracedScanResultConsumer#onComplete");
+    assertThat(spans, hasItem(onCompleteMatcher));
+    spans.stream().filter(onCompleteMatcher::matches)
+        .forEach(span -> assertThat(span, allOf(onCompleteMatcher,
+          hasParentSpanId(scanOperationSpanId), hasStatusWithCode(StatusCode.OK), hasEnded())));
   }
 
+  @Override
+  protected void assertTraceError(Matcher exceptionTypeNameMatcher) {
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher = allOf(hasName(parentSpanName), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (logger.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(logger::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.ERROR),
+          hasExceptionWithType(exceptionTypeNameMatcher), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+    final String scanOperationSpanId = spans.stream().filter(scanOperationSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher onErrorMatcher = hasName("TracedScanResultConsumer#onError");
+    assertThat(spans, hasItem(onErrorMatcher));
+    spans.stream().filter(onErrorMatcher::matches)
+        .forEach(span -> assertThat(span, allOf(onErrorMatcher,
+          hasParentSpanId(scanOperationSpanId), hasStatusWithCode(StatusCode.OK), hasEnded())));
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java
index 96c2d40138c..79e1c6bc4b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanAll.java
@@ -17,21 +17,40 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasEnded;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasExceptionWithType;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasName;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasParentSpanId;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasStatusWithCode;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.startsWith;
+
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.sdk.trace.data.SpanData;
 import java.util.List;
+import java.util.Objects;
 import java.util.function.Supplier;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.trace.StringTraceRenderer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.hamcrest.Matcher;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @RunWith(Parameterized.class)
 @Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncTableScanAll extends AbstractTestAsyncTableScan {
+  private static final Logger logger = LoggerFactory.getLogger(TestAsyncTableScanAll.class);
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -72,4 +91,50 @@ public class TestAsyncTableScanAll extends AbstractTestAsyncTableScan {
     }
     return results;
   }
+
+  @Override
+  protected void assertTraceContinuity() {
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher =
+        allOf(hasName(parentSpanName), hasStatusWithCode(StatusCode.OK), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (logger.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(logger::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.OK), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+  }
+
+  @Override
+  protected void assertTraceError(Matcher exceptionTypeNameMatcher) {
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher = allOf(hasName(parentSpanName), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (logger.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(logger::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.ERROR),
+          hasExceptionWithType(exceptionTypeNameMatcher), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanMetrics.java
index f5aa1967690..eca1c287726 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanMetrics.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -123,7 +123,7 @@ public class TestAsyncTableScanMetrics {
 
   private static Pair, ScanMetrics> doScanWithAsyncTableScan(Scan scan)
       throws Exception {
-    SimpleScanResultConsumer consumer = new SimpleScanResultConsumer();
+    SimpleScanResultConsumerImpl consumer = new SimpleScanResultConsumerImpl();
     CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool()).scan(scan, consumer);
     return Pair.newPair(consumer.getAll(), consumer.getScanMetrics());
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanner.java
index 2e990f763da..4c0f755324f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableScanner.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,23 +17,42 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasEnded;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasExceptionWithType;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasName;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasParentSpanId;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasStatusWithCode;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.startsWith;
+
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.sdk.trace.data.SpanData;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 import java.util.concurrent.ForkJoinPool;
 import java.util.function.Supplier;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.trace.StringTraceRenderer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.hamcrest.Matcher;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @RunWith(Parameterized.class)
 @Category({ LargeTests.class, ClientTests.class })
 public class TestAsyncTableScanner extends AbstractTestAsyncTableScan {
+  private static final Logger logger = LoggerFactory.getLogger(TestAsyncTableScanner.class);
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -63,7 +82,8 @@ public class TestAsyncTableScanner extends AbstractTestAsyncTableScan {
 
   @Override
   protected List doScan(Scan scan, int closeAfter) throws Exception {
-    AsyncTable table = ASYNC_CONN.getTable(TABLE_NAME, ForkJoinPool.commonPool());
+    AsyncTable table =
+        connectionRule.getAsyncConnection().getTable(TABLE_NAME, ForkJoinPool.commonPool());
     List results = new ArrayList<>();
     // these tests batch settings with the sample data result in each result being
     // split in two. so we must allow twice the expected results in order to reach
@@ -84,4 +104,49 @@ public class TestAsyncTableScanner extends AbstractTestAsyncTableScan {
     }
     return results;
   }
+
+  @Override
+  protected void assertTraceContinuity() {
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher =
+        allOf(hasName(parentSpanName), hasStatusWithCode(StatusCode.OK), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (logger.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(logger::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    assertThat(spans,
+      hasItem(allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+        hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.OK), hasEnded())));
+  }
+
+  @Override
+  protected void assertTraceError(Matcher exceptionTypeNameMatcher) {
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher = allOf(hasName(parentSpanName), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (logger.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(logger::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.ERROR),
+          hasExceptionWithType(exceptionTypeNameMatcher), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableScan.java
index 26c201e1986..eb4eb56438f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableScan.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRawAsyncTableScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,22 +17,42 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasEnded;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasExceptionWithType;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasName;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasParentSpanId;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasStatusWithCode;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.startsWith;
+
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.sdk.trace.data.SpanData;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 import java.util.function.Supplier;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.client.trace.StringTraceRenderer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.hamcrest.Matcher;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @RunWith(Parameterized.class)
 @Category({ LargeTests.class, ClientTests.class })
 public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
+  private static final Logger logger = LoggerFactory.getLogger(TestRawAsyncTableScan.class);
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -56,8 +76,8 @@ public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
 
   @Override
   protected List doScan(Scan scan, int closeAfter) throws Exception {
-    BufferingScanResultConsumer scanConsumer = new BufferingScanResultConsumer();
-    ASYNC_CONN.getTable(TABLE_NAME).scan(scan, scanConsumer);
+    TracedAdvancedScanResultConsumer scanConsumer = new TracedAdvancedScanResultConsumer();
+    connectionRule.getAsyncConnection().getTable(TABLE_NAME).scan(scan, scanConsumer);
     List results = new ArrayList<>();
     // these tests batch settings with the sample data result in each result being
     // split in two. so we must allow twice the expected results in order to reach
@@ -76,4 +96,79 @@ public class TestRawAsyncTableScan extends AbstractTestAsyncTableScan {
     }
     return results;
   }
+
+  @Override
+  protected void assertTraceContinuity() {
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher =
+        allOf(hasName(parentSpanName), hasStatusWithCode(StatusCode.OK), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (logger.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(logger::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.OK), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+    final String scanOperationSpanId = spans.stream().filter(scanOperationSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    // RawAsyncTableImpl never invokes the callback to `onScanMetricsCreated` -- bug?
+    final Matcher onScanMetricsCreatedMatcher =
+        hasName("TracedAdvancedScanResultConsumer#onScanMetricsCreated");
+    assertThat(spans, not(hasItem(onScanMetricsCreatedMatcher)));
+
+    final Matcher onNextMatcher = hasName("TracedAdvancedScanResultConsumer#onNext");
+    assertThat(spans, hasItem(onNextMatcher));
+    spans.stream().filter(onNextMatcher::matches)
+        .forEach(span -> assertThat(span, hasParentSpanId(scanOperationSpanId)));
+    assertThat(spans, hasItem(allOf(onNextMatcher, hasParentSpanId(scanOperationSpanId),
+      hasStatusWithCode(StatusCode.OK), hasEnded())));
+
+    final Matcher onCompleteMatcher =
+        hasName("TracedAdvancedScanResultConsumer#onComplete");
+    assertThat(spans, hasItem(onCompleteMatcher));
+    spans.stream().filter(onCompleteMatcher::matches)
+        .forEach(span -> assertThat(span, allOf(onCompleteMatcher,
+          hasParentSpanId(scanOperationSpanId), hasStatusWithCode(StatusCode.OK), hasEnded())));
+  }
+
+  @Override
+  protected void assertTraceError(Matcher exceptionTypeNameMatcher) {
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher = allOf(hasName(parentSpanName), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (logger.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(logger::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.ERROR),
+          hasExceptionWithType(exceptionTypeNameMatcher), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+    final String scanOperationSpanId = spans.stream().filter(scanOperationSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher onCompleteMatcher = hasName("TracedAdvancedScanResultConsumer#onError");
+    assertThat(spans, hasItem(onCompleteMatcher));
+    spans.stream().filter(onCompleteMatcher::matches)
+        .forEach(span -> assertThat(span, allOf(onCompleteMatcher,
+          hasParentSpanId(scanOperationSpanId), hasStatusWithCode(StatusCode.OK), hasEnded())));
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerTracing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerTracing.java
new file mode 100644
index 00000000000..787790897b0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultScannerTracing.java
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasEnded;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasName;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasParentSpanId;
+import static org.apache.hadoop.hbase.client.trace.hamcrest.SpanDataMatchers.hasStatusWithCode;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.nullValue;
+import static org.hamcrest.Matchers.startsWith;
+
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.sdk.trace.data.SpanData;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ConnectionRule;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MatcherPredicate;
+import org.apache.hadoop.hbase.MiniClusterRule;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.trace.StringTraceRenderer;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.trace.OpenTelemetryClassRule;
+import org.apache.hadoop.hbase.trace.OpenTelemetryTestRule;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.hamcrest.Matcher;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ LargeTests.class, ClientTests.class })
+public class TestResultScannerTracing {
+  private static final Logger LOG = LoggerFactory.getLogger(TestResultScannerTracing.class);
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestResultScannerTracing.class);
+
+  private static final TableName TABLE_NAME =
+      TableName.valueOf(TestResultScannerTracing.class.getSimpleName());
+  private static final byte[] FAMILY = Bytes.toBytes("f");
+  private static final byte[] CQ = Bytes.toBytes("q");
+  private static final int COUNT = 1000;
+
+  private static final OpenTelemetryClassRule otelClassRule = OpenTelemetryClassRule.create();
+  private static final MiniClusterRule miniClusterRule = MiniClusterRule.newBuilder()
+      .setMiniClusterOption(StartMiniClusterOption.builder().numRegionServers(3).build()).build();
+
+  private static final ConnectionRule connectionRule =
+      ConnectionRule.createConnectionRule(miniClusterRule::createConnection);
+
+  private static final class Setup extends ExternalResource {
+
+    private Connection conn;
+
+    @Override
+    protected void before() throws Throwable {
+      final HBaseTestingUtility testUtil = miniClusterRule.getTestingUtility();
+      conn = testUtil.getConnection();
+
+      byte[][] splitKeys = new byte[8][];
+      for (int i = 111; i < 999; i += 111) {
+        splitKeys[i / 111 - 1] = Bytes.toBytes(String.format("%03d", i));
+      }
+      testUtil.createTable(TABLE_NAME, FAMILY, splitKeys);
+      testUtil.waitTableAvailable(TABLE_NAME);
+      try (final Table table = conn.getTable(TABLE_NAME)) {
+        table.put(
+          IntStream.range(0, COUNT).mapToObj(i -> new Put(Bytes.toBytes(String.format("%03d", i)))
+              .addColumn(FAMILY, CQ, Bytes.toBytes(i))).collect(Collectors.toList()));
+      }
+    }
+
+    @Override
+    protected void after() {
+      try (Admin admin = conn.getAdmin()) {
+        if (!admin.tableExists(TABLE_NAME)) {
+          return;
+        }
+        admin.disableTable(TABLE_NAME);
+        admin.deleteTable(TABLE_NAME);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  @ClassRule
+  public static final TestRule classRule = RuleChain.outerRule(otelClassRule)
+      .around(miniClusterRule).around(connectionRule).around(new Setup());
+
+  @Rule
+  public final OpenTelemetryTestRule otelTestRule = new OpenTelemetryTestRule(otelClassRule);
+
+  @Rule
+  public final TestName testName = new TestName();
+
+  @Before
+  public void before() throws Exception {
+    final Connection conn = connectionRule.getConnection();
+    try (final RegionLocator locator = conn.getRegionLocator(TABLE_NAME)) {
+      locator.clearRegionLocationCache();
+    }
+  }
+
+  private static void waitForSpan(final Matcher parentSpanMatcher) {
+    final Configuration conf = miniClusterRule.getTestingUtility().getConfiguration();
+    Waiter.waitFor(conf, TimeUnit.SECONDS.toMillis(5), new MatcherPredicate<>(
+        "Span for test failed to complete.", otelClassRule::getSpans, hasItem(parentSpanMatcher)));
+  }
+
+  private Scan buildDefaultScan() {
+    return new Scan().withStartRow(Bytes.toBytes(String.format("%03d", 1)))
+        .withStopRow(Bytes.toBytes(String.format("%03d", 998)));
+  }
+
+  private void assertDefaultScan(final Scan scan) {
+    assertThat(scan.isReversed(), is(false));
+    assertThat(scan.isAsyncPrefetch(), nullValue());
+  }
+
+  private Scan buildAsyncPrefetchScan() {
+    return new Scan().withStartRow(Bytes.toBytes(String.format("%03d", 1)))
+        .withStopRow(Bytes.toBytes(String.format("%03d", 998))).setAsyncPrefetch(true);
+  }
+
+  private void assertAsyncPrefetchScan(final Scan scan) {
+    assertThat(scan.isReversed(), is(false));
+    assertThat(scan.isAsyncPrefetch(), is(true));
+  }
+
+  private Scan buildReversedScan() {
+    return new Scan().withStartRow(Bytes.toBytes(String.format("%03d", 998)))
+        .withStopRow(Bytes.toBytes(String.format("%03d", 1))).setReversed(true);
+  }
+
+  private void assertReversedScan(final Scan scan) {
+    assertThat(scan.isReversed(), is(true));
+    assertThat(scan.isAsyncPrefetch(), nullValue());
+  }
+
+  private void doScan(final Supplier spanSupplier, final Consumer scanAssertions)
+      throws Exception {
+    final Connection conn = connectionRule.getConnection();
+    final Scan scan = spanSupplier.get();
+    scanAssertions.accept(scan);
+    try (final Table table = conn.getTable(TABLE_NAME);
+        final ResultScanner scanner = table.getScanner(scan)) {
+      final List results = new ArrayList<>(COUNT);
+      scanner.forEach(results::add);
+      assertThat(results, not(emptyIterable()));
+    }
+  }
+
+  @Test
+  public void testNormalScan() throws Exception {
+    TraceUtil.trace(() -> doScan(this::buildDefaultScan, this::assertDefaultScan),
+      testName.getMethodName());
+
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher =
+        allOf(hasName(parentSpanName), hasStatusWithCode(StatusCode.OK), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (LOG.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(LOG::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.OK), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+    final String scanOperationSpanId = spans.stream().filter(scanOperationSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher childMetaScanSpanMatcher = allOf(hasName(startsWith("SCAN hbase:meta")),
+      hasParentSpanId(scanOperationSpanId), hasStatusWithCode(StatusCode.OK), hasEnded());
+    assertThat("expected a scan of hbase:meta", spans, hasItem(childMetaScanSpanMatcher));
+  }
+
+  @Test
+  public void testAsyncPrefetchScan() throws Exception {
+    TraceUtil.trace(() -> doScan(this::buildAsyncPrefetchScan, this::assertAsyncPrefetchScan),
+      testName.getMethodName());
+
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher =
+        allOf(hasName(parentSpanName), hasStatusWithCode(StatusCode.OK), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (LOG.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(LOG::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.OK), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+    final String scanOperationSpanId = spans.stream().filter(scanOperationSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher childMetaScanSpanMatcher = allOf(hasName(startsWith("SCAN hbase:meta")),
+      hasParentSpanId(scanOperationSpanId), hasStatusWithCode(StatusCode.OK), hasEnded());
+    assertThat("expected a scan of hbase:meta", spans, hasItem(childMetaScanSpanMatcher));
+  }
+
+  @Test
+  public void testReversedScan() throws Exception {
+    TraceUtil.trace(() -> doScan(this::buildReversedScan, this::assertReversedScan),
+      testName.getMethodName());
+
+    final String parentSpanName = testName.getMethodName();
+    final Matcher parentSpanMatcher =
+        allOf(hasName(parentSpanName), hasStatusWithCode(StatusCode.OK), hasEnded());
+    waitForSpan(parentSpanMatcher);
+
+    final List spans =
+        otelClassRule.getSpans().stream().filter(Objects::nonNull).collect(Collectors.toList());
+    if (LOG.isDebugEnabled()) {
+      StringTraceRenderer stringTraceRenderer = new StringTraceRenderer(spans);
+      stringTraceRenderer.render(LOG::debug);
+    }
+
+    final String parentSpanId = spans.stream().filter(parentSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher scanOperationSpanMatcher =
+        allOf(hasName(startsWith("SCAN " + TABLE_NAME.getNameWithNamespaceInclAsString())),
+          hasParentSpanId(parentSpanId), hasStatusWithCode(StatusCode.OK), hasEnded());
+    assertThat(spans, hasItem(scanOperationSpanMatcher));
+    final String scanOperationSpanId = spans.stream().filter(scanOperationSpanMatcher::matches)
+        .map(SpanData::getSpanId).findAny().orElseThrow(AssertionError::new);
+
+    final Matcher childMetaScanSpanMatcher = allOf(hasName(startsWith("SCAN hbase:meta")),
+      hasParentSpanId(scanOperationSpanId), hasStatusWithCode(StatusCode.OK), hasEnded());
+    assertThat("expected a scan of hbase:meta", spans, hasItem(childMetaScanSpanMatcher));
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TracedAdvancedScanResultConsumer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TracedAdvancedScanResultConsumer.java
new file mode 100644
index 00000000000..ba6f75a591b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TracedAdvancedScanResultConsumer.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+
+/**
+ * A drop-in replacement for {@link BufferingScanResultConsumer} that adds tracing spans to its
+ * implementation of the {@link AdvancedScanResultConsumer} API.
+ */
+public class TracedAdvancedScanResultConsumer implements AdvancedScanResultConsumer {
+
+  private final BufferingScanResultConsumer delegate = new BufferingScanResultConsumer();
+
+  @Override
+  public void onScanMetricsCreated(ScanMetrics scanMetrics) {
+    TraceUtil.trace(() -> delegate.onScanMetricsCreated(scanMetrics),
+      "TracedAdvancedScanResultConsumer#onScanMetricsCreated");
+  }
+
+  @Override
+  public void onNext(Result[] results, ScanController controller) {
+    TraceUtil.trace(() -> delegate.onNext(results, controller),
+      "TracedAdvancedScanResultConsumer#onNext");
+  }
+
+  @Override
+  public void onError(Throwable error) {
+    TraceUtil.trace(() -> delegate.onError(error), "TracedAdvancedScanResultConsumer#onError");
+  }
+
+  @Override
+  public void onComplete() {
+    TraceUtil.trace(delegate::onComplete, "TracedAdvancedScanResultConsumer#onComplete");
+  }
+
+  public Result take() throws IOException, InterruptedException {
+    return delegate.take();
+  }
+
+  public ScanMetrics getScanMetrics() {
+    return delegate.getScanMetrics();
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TracedScanResultConsumer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TracedScanResultConsumer.java
new file mode 100644
index 00000000000..cfb5907c1fd
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TracedScanResultConsumer.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.util.List;
+import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+
+/**
+ * A wrapper over {@link SimpleScanResultConsumer} that adds tracing of spans to its implementation.
+ */
+class TracedScanResultConsumer implements SimpleScanResultConsumer {
+
+  private final SimpleScanResultConsumer delegate;
+
+  public TracedScanResultConsumer(final SimpleScanResultConsumer delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public void onScanMetricsCreated(ScanMetrics scanMetrics) {
+    TraceUtil.trace(() -> delegate.onScanMetricsCreated(scanMetrics),
+      "TracedScanResultConsumer#onScanMetricsCreated");
+  }
+
+  @Override
+  public boolean onNext(Result result) {
+    return TraceUtil.trace(() -> delegate.onNext(result), "TracedScanResultConsumer#onNext");
+  }
+
+  @Override
+  public void onError(Throwable error) {
+    TraceUtil.trace(() -> delegate.onError(error), "TracedScanResultConsumer#onError");
+  }
+
+  @Override
+  public void onComplete() {
+    TraceUtil.trace(delegate::onComplete, "TracedScanResultConsumer#onComplete");
+  }
+
+  @Override
+  public List getAll() throws Exception {
+    return delegate.getAll();
+  }
+
+  @Override
+  public ScanMetrics getScanMetrics() {
+    return delegate.getScanMetrics();
+  }
+}
diff --git a/pom.xml b/pom.xml
index 626a8aa67b2..0c47060ec6b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -692,7 +692,8 @@
     -enableassertions -Xmx${surefire.cygwinXmx}
       -Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true
       "-Djava.library.path=${hadoop.library.path};${java.library.path}"
-      -Dorg.apache.hbase.thirdparty.io.netty.leakDetection.level=advanced
+      -Dorg.apache.hbase.thirdparty.io.netty.leakDetection.level=advanced
+      -Dio.opentelemetry.context.enableStrictContext=true
     
     ${hbase-surefire.argLine}
     0.7.5.201505241946