From c060e01498ea3001b40b4bf06efaba0f65071832 Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Wed, 1 Mar 2023 21:16:14 -0500 Subject: [PATCH 1/6] Connection and Request Attributes PR feedback: prefer emptyMap, no wildcard imports default attributes in Call constructor fix RpcClient in TestRpcBasedRegistryHedgedReads more test fixes request attributes support in tablebuilder cleanup checkstyle, banned imports PR feedback support setting a single req attribute remove plural attribute setter fix tests --- .../AsyncAdminRequestRetryingCaller.java | 3 +- .../client/AsyncBatchRpcRetryingCaller.java | 7 +- .../hbase/client/AsyncClientScanner.java | 31 +- .../hbase/client/AsyncConnectionImpl.java | 13 +- .../AsyncMasterRequestRpcRetryingCaller.java | 3 +- .../hbase/client/AsyncRpcRetryingCaller.java | 4 +- .../client/AsyncRpcRetryingCallerFactory.java | 31 +- ...syncScanSingleRegionRpcRetryingCaller.java | 4 +- .../AsyncServerRequestRpcRetryingCaller.java | 3 +- .../AsyncSingleRequestRpcRetryingCaller.java | 6 +- .../hadoop/hbase/client/AsyncTable.java | 10 + .../hbase/client/AsyncTableBuilder.java | 5 + .../hbase/client/AsyncTableBuilderBase.java | 14 + .../hadoop/hbase/client/AsyncTableImpl.java | 6 + .../hbase/client/ConnectionFactory.java | 44 ++- .../hbase/client/RawAsyncTableImpl.java | 17 +- .../org/apache/hadoop/hbase/client/Table.java | 9 + .../hadoop/hbase/client/TableBuilder.java | 5 + .../hadoop/hbase/client/TableBuilderBase.java | 14 + .../hadoop/hbase/ipc/AbstractRpcClient.java | 36 +- .../hadoop/hbase/ipc/BlockingRpcClient.java | 8 +- .../hbase/ipc/BlockingRpcConnection.java | 2 +- .../org/apache/hadoop/hbase/ipc/Call.java | 7 +- .../ipc/DelegatingHBaseRpcController.java | 11 + .../hadoop/hbase/ipc/HBaseRpcController.java | 11 + .../hbase/ipc/HBaseRpcControllerImpl.java | 14 + .../org/apache/hadoop/hbase/ipc/IPCUtil.java | 11 + .../hadoop/hbase/ipc/NettyRpcClient.java | 11 +- .../hadoop/hbase/ipc/NettyRpcConnection.java | 2 +- .../hadoop/hbase/ipc/RpcClientFactory.java | 13 +- .../hadoop/hbase/ipc/RpcConnection.java | 15 +- .../TestRpcBasedRegistryHedgedReads.java | 3 +- .../hbase/ipc/TestTLSHandshadeFailure.java | 4 +- .../mapreduce/TestHFileOutputFormat2.java | 5 +- .../TestMultiTableInputFormatBase.java | 3 +- .../mapreduce/TestTableInputFormatBase.java | 4 +- .../src/main/protobuf/RPC.proto | 2 + .../org/apache/hadoop/hbase/ipc/RpcCall.java | 3 + .../apache/hadoop/hbase/ipc/ServerCall.java | 8 + .../hadoop/hbase/client/DummyAsyncTable.java | 6 + .../hbase/client/TestClientTimeouts.java | 5 +- .../TestRequestAndConnectionAttributes.java | 317 ++++++++++++++++++ .../hadoop/hbase/ipc/TestRpcClientLeaks.java | 5 +- .../ipc/TestRpcServerSlowConnectionSetup.java | 2 +- .../namequeues/TestNamedQueueRecorder.java | 7 +- .../region/TestRegionProcedureStore.java | 6 + .../thrift2/client/ThriftConnection.java | 9 +- 47 files changed, 674 insertions(+), 85 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java index d3bec8b3cfbf..f7fa7e9f03fa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdminRequestRetryingCaller.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.util.FutureUtils.addListener; import java.io.IOException; +import java.util.Collections; import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -44,7 +45,7 @@ public AsyncAdminRequestRetryingCaller(Timer retryTimer, AsyncConnectionImpl con long pauseNs, long pauseNsForServerOverloaded, int maxAttempts, long operationTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt, ServerName serverName, Callable callable) { super(retryTimer, conn, priority, pauseNs, pauseNsForServerOverloaded, maxAttempts, - operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt); + operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, Collections.emptyMap()); this.serverName = serverName; this.callable = callable; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java index 063d3df34d0e..d376cfc6a696 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBatchRpcRetryingCaller.java @@ -114,6 +114,8 @@ class AsyncBatchRpcRetryingCaller { private final HBaseServerExceptionPauseManager pauseManager; + private final Map requestAttributes; + // we can not use HRegionLocation as the map key because the hashCode and equals method of // HRegionLocation only consider serverName. private static final class RegionRequest { @@ -149,7 +151,8 @@ public int getPriority() { public AsyncBatchRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn, TableName tableName, List actions, long pauseNs, long pauseNsForServerOverloaded, - int maxAttempts, long operationTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt) { + int maxAttempts, long operationTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt, + Map requestAttributes) { this.retryTimer = retryTimer; this.conn = conn; this.tableName = tableName; @@ -180,6 +183,7 @@ public AsyncBatchRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn, } this.action2Errors = new IdentityHashMap<>(); this.startNs = System.nanoTime(); + this.requestAttributes = requestAttributes; } private static boolean hasIncrementOrAppend(Row action) { @@ -392,6 +396,7 @@ private void sendToServer(ServerName serverName, ServerRequest serverReq, int tr HBaseRpcController controller = conn.rpcControllerFactory.newController(); resetController(controller, Math.min(rpcTimeoutNs, remainingNs), calcPriority(serverReq.getPriority(), tableName)); + controller.setRequestAttributes(requestAttributes); if (!cells.isEmpty()) { controller.setCellScanner(createCellScanner(cells)); } 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 ed381df7e0da..b61f5b80c9e7 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 @@ -32,6 +32,7 @@ import io.opentelemetry.api.trace.StatusCode; import io.opentelemetry.context.Scope; import java.io.IOException; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -92,9 +93,12 @@ class AsyncClientScanner { private final Span span; + private final Map requestAttributes; + public AsyncClientScanner(Scan scan, AdvancedScanResultConsumer consumer, TableName tableName, AsyncConnectionImpl conn, Timer retryTimer, long pauseNs, long pauseNsForServerOverloaded, - int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt) { + int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt, + Map requestAttributes) { if (scan.getStartRow() == null) { scan.withStartRow(EMPTY_START_ROW, scan.includeStartRow()); } @@ -113,6 +117,7 @@ public AsyncClientScanner(Scan scan, AdvancedScanResultConsumer consumer, TableN this.rpcTimeoutNs = rpcTimeoutNs; this.startLogErrorsCnt = startLogErrorsCnt; this.resultCache = createScanResultCache(scan); + this.requestAttributes = requestAttributes; if (scan.isScanMetricsEnabled()) { this.scanMetrics = new ScanMetrics(); consumer.onScanMetricsCreated(scanMetrics); @@ -191,15 +196,17 @@ private CompletableFuture callOpenScanner(HBaseRpcControlle } 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) - .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS) - .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt) - .start(resp.controller, resp.resp), (hasMore, error) -> { + 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) + .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS) + .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt) + .setRequestAttributes(requestAttributes).start(resp.controller, resp.resp), + (hasMore, error) -> { try (Scope ignored = span.makeCurrent()) { if (error != null) { try { @@ -231,8 +238,8 @@ private CompletableFuture openScanner(int replicaId) { .priority(scan.getPriority()).rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS) .operationTimeout(scanTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS) - .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt).action(this::callOpenScanner) - .call(); + .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt) + .setRequestAttributes(requestAttributes).action(this::callOpenScanner).call(); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java index 19193e234dcb..a56391503f02 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java @@ -30,6 +30,9 @@ import io.opentelemetry.api.trace.Span; import java.io.IOException; +import java.net.SocketAddress; +import java.util.Collections; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -121,7 +124,12 @@ public class AsyncConnectionImpl implements AsyncConnection { private final ClusterStatusListener clusterStatusListener; public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId, - User user) { + SocketAddress localAddress, User user) { + this(conf, registry, clusterId, localAddress, user, Collections.emptyMap()); + } + + public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId, + SocketAddress localAddress, User user, Map connectionAttributes) { this.conf = conf; this.user = user; this.metricsScope = MetricsConnection.getScope(conf, clusterId, this); @@ -137,7 +145,8 @@ public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, Stri } else { this.metrics = Optional.empty(); } - this.rpcClient = RpcClientFactory.createClient(conf, clusterId, metrics.orElse(null)); + this.rpcClient = RpcClientFactory.createClient(conf, clusterId, localAddress, + metrics.orElse(null), connectionAttributes); this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); this.rpcTimeout = (int) Math.min(Integer.MAX_VALUE, TimeUnit.NANOSECONDS.toMillis(connConf.getRpcTimeoutNs())); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java index c02b80c666ae..42585ea1c919 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMasterRequestRpcRetryingCaller.java @@ -19,6 +19,7 @@ import static org.apache.hadoop.hbase.util.FutureUtils.addListener; +import java.util.Collections; import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -47,7 +48,7 @@ public AsyncMasterRequestRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl Callable callable, int priority, long pauseNs, long pauseNsForServerOverloaded, int maxRetries, long operationTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt) { super(retryTimer, conn, priority, pauseNs, pauseNsForServerOverloaded, maxRetries, - operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt); + operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, Collections.emptyMap()); this.callable = callable; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java index f89aef214391..5b1727f9f66e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCaller.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.OptionalLong; import java.util.concurrent.CompletableFuture; @@ -78,7 +79,7 @@ public abstract class AsyncRpcRetryingCaller { public AsyncRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn, int priority, long pauseNs, long pauseNsForServerOverloaded, int maxAttempts, long operationTimeoutNs, - long rpcTimeoutNs, int startLogErrorsCnt) { + long rpcTimeoutNs, int startLogErrorsCnt, Map requestAttributes) { this.retryTimer = retryTimer; this.conn = conn; this.priority = priority; @@ -89,6 +90,7 @@ public AsyncRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn, int pr this.future = new CompletableFuture<>(); this.controller = conn.rpcControllerFactory.newController(); this.controller.setPriority(priority); + this.controller.setRequestAttributes(requestAttributes); this.exceptions = new ArrayList<>(); this.startNs = System.nanoTime(); this.pauseManager = diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java index 2d8e7b7aabe9..1ea2a1ad7dd4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRpcRetryingCallerFactory.java @@ -23,7 +23,9 @@ import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkArgument; import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import org.apache.hadoop.hbase.HRegionLocation; @@ -83,6 +85,8 @@ public class SingleRequestCallerBuilder extends BuilderBase { private int priority = PRIORITY_UNSET; + private Map requestAttributes = Collections.emptyMap(); + public SingleRequestCallerBuilder table(TableName tableName) { this.tableName = tableName; return this; @@ -144,6 +148,12 @@ public SingleRequestCallerBuilder priority(int priority) { return this; } + public SingleRequestCallerBuilder + setRequestAttributes(Map requestAttributes) { + this.requestAttributes = requestAttributes; + return this; + } + private void preCheck() { checkArgument(replicaId >= 0, "invalid replica id %s", replicaId); checkNotNull(tableName, "tableName is null"); @@ -157,7 +167,7 @@ public AsyncSingleRequestRpcRetryingCaller build() { preCheck(); return new AsyncSingleRequestRpcRetryingCaller<>(retryTimer, conn, tableName, row, replicaId, locateType, callable, priority, pauseNs, pauseNsForServerOverloaded, maxAttempts, - operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt); + operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, requestAttributes); } /** @@ -201,6 +211,8 @@ public class ScanSingleRegionCallerBuilder extends BuilderBase { private int priority = PRIORITY_UNSET; + private Map requestAttributes = Collections.emptyMap(); + public ScanSingleRegionCallerBuilder id(long scannerId) { this.scannerId = scannerId; return this; @@ -278,6 +290,12 @@ public ScanSingleRegionCallerBuilder startLogErrorsCnt(int startLogErrorsCnt) { return this; } + public ScanSingleRegionCallerBuilder + setRequestAttributes(Map requestAttributes) { + this.requestAttributes = requestAttributes; + return this; + } + private void preCheck() { checkArgument(scannerId != null, "invalid scannerId %d", scannerId); checkNotNull(scan, "scan is null"); @@ -293,7 +311,7 @@ public AsyncScanSingleRegionRpcRetryingCaller build() { return new AsyncScanSingleRegionRpcRetryingCaller(retryTimer, conn, scan, scanMetrics, scannerId, resultCache, consumer, stub, loc, isRegionServerRemote, priority, scannerLeaseTimeoutPeriodNs, pauseNs, pauseNsForServerOverloaded, maxAttempts, - scanTimeoutNs, rpcTimeoutNs, startLogErrorsCnt); + scanTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, requestAttributes); } /** @@ -322,6 +340,8 @@ public class BatchCallerBuilder extends BuilderBase { private long rpcTimeoutNs = -1L; + private Map requestAttributes = Collections.emptyMap(); + public BatchCallerBuilder table(TableName tableName) { this.tableName = tableName; return this; @@ -362,10 +382,15 @@ public BatchCallerBuilder startLogErrorsCnt(int startLogErrorsCnt) { return this; } + public BatchCallerBuilder setRequestAttributes(Map requestAttributes) { + this.requestAttributes = requestAttributes; + return this; + } + public AsyncBatchRpcRetryingCaller build() { return new AsyncBatchRpcRetryingCaller<>(retryTimer, conn, tableName, actions, pauseNs, pauseNsForServerOverloaded, maxAttempts, operationTimeoutNs, rpcTimeoutNs, - startLogErrorsCnt); + startLogErrorsCnt, requestAttributes); } public List> call() { 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 a046f0c7b6e5..dd6eed6712df 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 @@ -31,6 +31,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.OptionalLong; import java.util.concurrent.CompletableFuture; @@ -316,7 +317,7 @@ public AsyncScanSingleRegionRpcRetryingCaller(Timer retryTimer, AsyncConnectionI AdvancedScanResultConsumer consumer, Interface stub, HRegionLocation loc, boolean isRegionServerRemote, int priority, long scannerLeaseTimeoutPeriodNs, long pauseNs, long pauseNsForServerOverloaded, int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs, - int startLogErrorsCnt) { + int startLogErrorsCnt, Map requestAttributes) { this.retryTimer = retryTimer; this.conn = conn; this.scan = scan; @@ -341,6 +342,7 @@ public AsyncScanSingleRegionRpcRetryingCaller(Timer retryTimer, AsyncConnectionI this.priority = priority; this.controller = conn.rpcControllerFactory.newController(); this.controller.setPriority(priority); + this.controller.setRequestAttributes(requestAttributes); this.exceptions = new ArrayList<>(); this.pauseManager = new HBaseServerExceptionPauseManager(pauseNs, pauseNsForServerOverloaded, scanTimeoutNs); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncServerRequestRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncServerRequestRpcRetryingCaller.java index 40cd3b87e928..d4484ba87bf1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncServerRequestRpcRetryingCaller.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncServerRequestRpcRetryingCaller.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.util.FutureUtils.addListener; import java.io.IOException; +import java.util.Collections; import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; @@ -49,7 +50,7 @@ public AsyncServerRequestRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl long pauseNs, long pauseNsForServerOverloaded, int maxAttempts, long operationTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt, ServerName serverName, Callable callable) { super(retryTimer, conn, HConstants.NORMAL_QOS, pauseNs, pauseNsForServerOverloaded, maxAttempts, - operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt); + operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, Collections.emptyMap()); this.serverName = serverName; this.callable = callable; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java index 9c115af97b5b..a0d536aef5f7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncSingleRequestRpcRetryingCaller.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.util.FutureUtils.addListener; import java.io.IOException; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.HRegionLocation; @@ -57,9 +58,10 @@ CompletableFuture call(HBaseRpcController controller, HRegionLocation loc, public AsyncSingleRequestRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn, TableName tableName, byte[] row, int replicaId, RegionLocateType locateType, Callable callable, int priority, long pauseNs, long pauseNsForServerOverloaded, - int maxAttempts, long operationTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt) { + int maxAttempts, long operationTimeoutNs, long rpcTimeoutNs, int startLogErrorsCnt, + Map requestAttributes) { super(retryTimer, conn, priority, pauseNs, pauseNsForServerOverloaded, maxAttempts, - operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt); + operationTimeoutNs, rpcTimeoutNs, startLogErrorsCnt, requestAttributes); this.tableName = tableName; this.row = row; this.replicaId = replicaId; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java index 2bd6b50df22f..da1b0e7b62f4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java @@ -23,9 +23,11 @@ import com.google.protobuf.RpcChannel; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.TableName; @@ -110,6 +112,14 @@ public interface AsyncTable { */ long getScanTimeout(TimeUnit unit); + /** + * Get the map of request attributes + * @return a map of request attributes supplied by the client + */ + default Map getRequestAttributes() { + throw new NotImplementedException("Add an implementation!"); + } + /** * Test for the existence of columns in the table, as specified by the Get. *

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java index f6db89f82bf5..007f7ad48685 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilder.java @@ -137,6 +137,11 @@ default AsyncTableBuilder setMaxRetries(int maxRetries) { */ AsyncTableBuilder setStartLogErrorsCnt(int startLogErrorsCnt); + /** + * Set a request attribute + */ + AsyncTableBuilder setRequestAttribute(String key, byte[] value); + /** * Create the {@link AsyncTable} instance. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilderBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilderBase.java index 624d6e1dbb0a..02e9da0770b4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilderBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableBuilderBase.java @@ -19,6 +19,9 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.retries2Attempts; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; @@ -50,6 +53,8 @@ abstract class AsyncTableBuilderBase protected int startLogErrorsCnt; + protected Map requestAttributes = Collections.emptyMap(); + AsyncTableBuilderBase(TableName tableName, AsyncConnectionConfiguration connConf) { this.tableName = tableName; this.operationTimeoutNs = tableName.isSystemTable() @@ -121,4 +126,13 @@ public AsyncTableBuilderBase setStartLogErrorsCnt(int startLogErrorsCnt) { this.startLogErrorsCnt = startLogErrorsCnt; return this; } + + @Override + public AsyncTableBuilder setRequestAttribute(String key, byte[] value) { + if (this.requestAttributes.isEmpty()) { + this.requestAttributes = new HashMap<>(); + } + this.requestAttributes.put(key, value); + return this; + } } 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 40dffcd62adf..e75a9411efb5 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 @@ -25,6 +25,7 @@ import io.opentelemetry.context.Scope; import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; @@ -100,6 +101,11 @@ public long getScanTimeout(TimeUnit unit) { return rawTable.getScanTimeout(unit); } + @Override + public Map getRequestAttributes() { + return rawTable.getRequestAttributes(); + } + private CompletableFuture wrap(CompletableFuture future) { return FutureUtils.wrapFuture(future, pool); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index 1fb0c7fecaf7..da9e5ea2b87c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -22,6 +22,8 @@ import java.io.IOException; import java.lang.reflect.Constructor; import java.security.PrivilegedExceptionAction; +import java.util.Collections; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import org.apache.hadoop.conf.Configuration; @@ -101,7 +103,7 @@ protected ConnectionFactory() { */ public static Connection createConnection() throws IOException { Configuration conf = HBaseConfiguration.create(); - return createConnection(conf, null, AuthUtil.loginClient(conf)); + return createConnection(conf, null, AuthUtil.loginClient(conf), Collections.emptyMap()); } /** @@ -128,7 +130,7 @@ public static Connection createConnection() throws IOException { * @return Connection object for conf */ public static Connection createConnection(Configuration conf) throws IOException { - return createConnection(conf, null, AuthUtil.loginClient(conf)); + return createConnection(conf, null, AuthUtil.loginClient(conf), Collections.emptyMap()); } /** @@ -157,7 +159,7 @@ public static Connection createConnection(Configuration conf) throws IOException */ public static Connection createConnection(Configuration conf, ExecutorService pool) throws IOException { - return createConnection(conf, pool, AuthUtil.loginClient(conf)); + return createConnection(conf, pool, AuthUtil.loginClient(conf), Collections.emptyMap()); } /** @@ -185,7 +187,7 @@ public static Connection createConnection(Configuration conf, ExecutorService po * @return Connection object for conf */ public static Connection createConnection(Configuration conf, User user) throws IOException { - return createConnection(conf, null, user); + return createConnection(conf, null, user, Collections.emptyMap()); } /** @@ -211,10 +213,11 @@ public static Connection createConnection(Configuration conf, User user) throws * @param conf configuration * @param user the user the connection is for * @param pool the thread pool to use for batch operations + * @param connectionAttributes attributes to be sent along to server during connection establish * @return Connection object for conf */ public static Connection createConnection(Configuration conf, ExecutorService pool, - final User user) throws IOException { + final User user, Map connectionAttributes) throws IOException { return TraceUtil.trace(() -> { String className = conf.get(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL, ConnectionImplementation.class.getName()); @@ -226,11 +229,11 @@ public static Connection createConnection(Configuration conf, ExecutorService po } try { // Default HCM#HCI is not accessible; make it so before invoking. - Constructor constructor = - clazz.getDeclaredConstructor(Configuration.class, ExecutorService.class, User.class); + Constructor constructor = clazz.getDeclaredConstructor(Configuration.class, + ExecutorService.class, User.class, Map.class); constructor.setAccessible(true); - return user.runAs((PrivilegedExceptionAction< - Connection>) () -> (Connection) constructor.newInstance(conf, pool, user)); + return user.runAs((PrivilegedExceptionAction) () -> (Connection) constructor + .newInstance(conf, pool, user, connectionAttributes)); } catch (Exception e) { throw new IOException(e); } @@ -284,6 +287,27 @@ public static CompletableFuture createAsyncConnection(Configura */ public static CompletableFuture createAsyncConnection(Configuration conf, final User user) { + return createAsyncConnection(conf, user, null); + } + + /** + * Create a new AsyncConnection instance using the passed {@code conf} and {@code user}. + * AsyncConnection encapsulates all housekeeping for a connection to the cluster. All tables and + * interfaces created from returned connection share zookeeper connection, meta cache, and + * connections to region servers and masters. + *

+ * The caller is responsible for calling {@link AsyncConnection#close()} on the returned + * connection instance. + *

+ * Usually you should only create one AsyncConnection instance in your code and use it everywhere + * as it is thread safe. + * @param conf configuration + * @param user the user the asynchronous connection is for + * @param connectionAttributes attributes to be sent along to server during connection establish + * @return AsyncConnection object wrapped by CompletableFuture + */ + public static CompletableFuture createAsyncConnection(Configuration conf, + final User user, Map connectionAttributes) { return TraceUtil.tracedFuture(() -> { CompletableFuture future = new CompletableFuture<>(); ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf); @@ -303,7 +327,7 @@ public static CompletableFuture createAsyncConnection(Configura try { future.complete( user.runAs((PrivilegedExceptionAction) () -> ReflectionUtils - .newInstance(clazz, conf, registry, clusterId, user))); + .newInstance(clazz, conf, registry, clusterId, null, user, connectionAttributes))); } catch (Exception e) { registry.close(); future.completeExceptionally(e); 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 7ffd9eabd7db..a5dc77d6e8e5 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 @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -118,6 +119,8 @@ class RawAsyncTableImpl implements AsyncTable { private final int startLogErrorsCnt; + private final Map requestAttributes; + RawAsyncTableImpl(AsyncConnectionImpl conn, Timer retryTimer, AsyncTableBuilderBase builder) { this.conn = conn; this.retryTimer = retryTimer; @@ -144,6 +147,7 @@ class RawAsyncTableImpl implements AsyncTable { ? conn.connConf.getMetaScannerCaching() : conn.connConf.getScannerCaching(); this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize(); + this.requestAttributes = builder.requestAttributes; } @Override @@ -248,7 +252,8 @@ private SingleRequestCallerBuilder newCaller(byte[] row, int priority, lo .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS) .pause(pauseNs, TimeUnit.NANOSECONDS) .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS) - .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt); + .maxAttempts(maxAttempts).setRequestAttributes(requestAttributes) + .startLogErrorsCnt(startLogErrorsCnt).setRequestAttributes(requestAttributes); } private SingleRequestCallerBuilder @@ -646,7 +651,7 @@ private Scan setDefaultScanConfig(Scan scan) { public void scan(Scan scan, AdvancedScanResultConsumer consumer) { new AsyncClientScanner(setDefaultScanConfig(scan), consumer, tableName, conn, retryTimer, pauseNs, pauseNsForServerOverloaded, maxAttempts, scanTimeoutNs, readRpcTimeoutNs, - startLogErrorsCnt).start(); + startLogErrorsCnt, requestAttributes).start(); } private long resultSize2CacheSize(long maxResultSize) { @@ -742,7 +747,8 @@ private List> batch(List actions, long r .operationTimeout(operationTimeoutNs, TimeUnit.NANOSECONDS) .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS) - .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt).call(); + .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt) + .setRequestAttributes(requestAttributes).call(); } @Override @@ -770,6 +776,11 @@ public long getScanTimeout(TimeUnit unit) { return unit.convert(scanTimeoutNs, TimeUnit.NANOSECONDS); } + @Override + public Map getRequestAttributes() { + return requestAttributes; + } + private CompletableFuture coprocessorService(Function stubMaker, ServiceCaller callable, RegionInfo region, byte[] row) { RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName, diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java index 00c531c47633..fb67917d80a1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java @@ -979,6 +979,7 @@ default long getOperationTimeout(TimeUnit unit) { } /** +<<<<<<< HEAD * Get timeout (millisecond) of each operation for in Table instance. * @deprecated since 2.0 and will be removed in 3.0 version use * {@link #getOperationTimeout(TimeUnit)} instead @@ -1002,4 +1003,12 @@ default int getOperationTimeout() { default void setOperationTimeout(int operationTimeout) { throw new NotImplementedException("Add an implementation!"); } + + /** + * Get the attributes to be submitted with requests + * @return map of request attributes + */ + default Map getRequestAttributes() { + throw new NotImplementedException("Add an implementation!"); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java index 75e16e89a5de..eee985555b34 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilder.java @@ -55,6 +55,11 @@ public interface TableBuilder { */ TableBuilder setWriteRpcTimeout(int timeout); + /** + * Set a request attribute + */ + TableBuilder setRequestAttribute(String key, byte[] value); + /** * Create the {@link Table} instance. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilderBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilderBase.java index 43d7a1894ea1..f797d3c6cad3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilderBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableBuilderBase.java @@ -17,6 +17,9 @@ */ package org.apache.hadoop.hbase.client; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; @@ -38,6 +41,8 @@ abstract class TableBuilderBase implements TableBuilder { protected final int scanReadRpcTimeout; protected int scanTimeout; + protected Map requestAttributes = Collections.emptyMap(); + TableBuilderBase(TableName tableName, ConnectionConfiguration connConf) { if (tableName == null) { throw new IllegalArgumentException("Given table name is null"); @@ -78,4 +83,13 @@ public TableBuilderBase setWriteRpcTimeout(int timeout) { this.writeRpcTimeout = timeout; return this; } + + @Override + public TableBuilderBase setRequestAttribute(String key, byte[] value) { + if (this.requestAttributes.isEmpty()) { + this.requestAttributes = new HashMap<>(); + } + this.requestAttributes.put(key, value); + return this; + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java index 23d14c272d2b..5e42558671b7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.net.SocketAddress; import java.util.Collection; +import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -106,6 +107,7 @@ public abstract class AbstractRpcClient implements RpcC private boolean running = true; // if client runs protected final Configuration conf; + protected final Map connectionAttributes; protected final String clusterId; protected final SocketAddress localAddr; protected final MetricsConnection metrics; @@ -154,7 +156,7 @@ public AtomicInteger load(Address key) throws Exception { * @param metrics the connection metrics */ public AbstractRpcClient(Configuration conf, String clusterId, SocketAddress localAddr, - MetricsConnection metrics) { + MetricsConnection metrics, Map connectionAttributes) { this.userProvider = UserProvider.instantiate(conf); this.localAddr = localAddr; this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true); @@ -167,6 +169,7 @@ public AbstractRpcClient(Configuration conf, String clusterId, SocketAddress loc this.minIdleTimeBeforeClose = conf.getInt(IDLE_TIME, 120000); // 2 minutes this.conf = conf; + this.connectionAttributes = connectionAttributes; this.codec = getCodec(); this.compressor = getCompressor(conf); this.fallbackAllowed = conf.getBoolean(IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY, @@ -416,23 +419,24 @@ private Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcCon } final AtomicInteger counter = concurrentCounterCache.getUnchecked(addr); - Call call = new Call(nextCallId(), md, param, hrc.cellScanner(), returnType, - hrc.getCallTimeout(), hrc.getPriority(), new RpcCallback() { - @Override - public void run(Call call) { - try (Scope scope = call.span.makeCurrent()) { - counter.decrementAndGet(); - onCallFinished(call, hrc, addr, callback); - } finally { - if (hrc.failed()) { - TraceUtil.setError(span, hrc.getFailed()); - } else { - span.setStatus(StatusCode.OK); + Call call = + new Call(nextCallId(), md, param, hrc.cellScanner(), returnType, hrc.getCallTimeout(), + hrc.getPriority(), hrc.getRequestAttributes(), new RpcCallback() { + @Override + public void run(Call call) { + try (Scope scope = call.span.makeCurrent()) { + counter.decrementAndGet(); + onCallFinished(call, hrc, addr, callback); + } finally { + if (hrc.failed()) { + TraceUtil.setError(span, hrc.getFailed()); + } else { + span.setStatus(StatusCode.OK); + } + span.end(); } - span.end(); } - } - }, cs); + }, cs); ConnectionId remoteId = new ConnectionId(ticket, md.getService().getName(), addr); int count = counter.incrementAndGet(); try { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java index 7fffdad935fc..3da00c5395d3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcClient.java @@ -19,6 +19,8 @@ import java.io.IOException; import java.net.SocketAddress; +import java.util.Collections; +import java.util.Map; import javax.net.SocketFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; @@ -41,7 +43,7 @@ public class BlockingRpcClient extends AbstractRpcClient * SocketFactory */ BlockingRpcClient(Configuration conf) { - this(conf, HConstants.CLUSTER_ID_DEFAULT, null, null); + this(conf, HConstants.CLUSTER_ID_DEFAULT, null, null, Collections.emptyMap()); } /** @@ -53,8 +55,8 @@ public class BlockingRpcClient extends AbstractRpcClient * @param metrics the connection metrics */ public BlockingRpcClient(Configuration conf, String clusterId, SocketAddress localAddr, - MetricsConnection metrics) { - super(conf, clusterId, localAddr, metrics); + MetricsConnection metrics, Map connectionAttributes) { + super(conf, clusterId, localAddr, metrics, connectionAttributes); this.socketFactory = NetUtils.getDefaultSocketFactory(conf); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java index f0e71ae68cb2..1ed08321a125 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java @@ -227,7 +227,7 @@ public void cleanup(IOException e) { BlockingRpcConnection(BlockingRpcClient rpcClient, ConnectionId remoteId) throws IOException { super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId, rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor, - rpcClient.metrics); + rpcClient.metrics, rpcClient.connectionAttributes); this.rpcClient = rpcClient; this.connectionHeaderPreamble = getConnectionHeaderPreamble(); ConnectionHeader header = getConnectionHeader(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java index 3c0e24e57145..669fc73a3bfa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java @@ -19,6 +19,7 @@ import io.opentelemetry.api.trace.Span; import java.io.IOException; +import java.util.Map; import java.util.Optional; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; @@ -56,14 +57,15 @@ class Call { final Descriptors.MethodDescriptor md; final int timeout; // timeout in millisecond for this call; 0 means infinite. final int priority; + final Map attributes; final MetricsConnection.CallStats callStats; private final RpcCallback callback; final Span span; Timeout timeoutTask; Call(int id, final Descriptors.MethodDescriptor md, Message param, final CellScanner cells, - final Message responseDefaultType, int timeout, int priority, RpcCallback callback, - MetricsConnection.CallStats callStats) { + final Message responseDefaultType, int timeout, int priority, Map attributes, + RpcCallback callback, MetricsConnection.CallStats callStats) { this.param = param; this.md = md; this.cells = cells; @@ -73,6 +75,7 @@ class Call { this.id = id; this.timeout = timeout; this.priority = priority; + this.attributes = attributes; this.callback = callback; this.span = Span.current(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java index 9bee88d599f7..c752f4c18355 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/DelegatingHBaseRpcController.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.ipc; import java.io.IOException; +import java.util.Map; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; @@ -112,6 +113,16 @@ public boolean hasCallTimeout() { return delegate.hasCallTimeout(); } + @Override + public Map getRequestAttributes() { + return delegate.getRequestAttributes(); + } + + @Override + public void setRequestAttributes(Map requestAttributes) { + delegate.setRequestAttributes(requestAttributes); + } + @Override public void setFailed(IOException e) { delegate.setFailed(e); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java index c60de7658f3d..cd303a5eda77 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcController.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.ipc; import java.io.IOException; +import java.util.Map; import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HBaseInterfaceAudience; @@ -71,6 +72,16 @@ public interface HBaseRpcController extends RpcController, CellScannable { boolean hasCallTimeout(); + /** + * Get the map of request attributes + */ + Map getRequestAttributes(); + + /** + * Set the map of request attributes + */ + void setRequestAttributes(Map requestAttributes); + /** * Set failed with an exception to pass on. For use in async rpc clients * @param e exception to set with diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java index f3e19aeadaee..6f8c65d6bdce 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java @@ -19,7 +19,9 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellUtil; @@ -70,6 +72,8 @@ public class HBaseRpcControllerImpl implements HBaseRpcController { */ private CellScanner cellScanner; + private Map requestAttributes = Collections.emptyMap(); + public HBaseRpcControllerImpl() { this(null, (CellScanner) null); } @@ -166,6 +170,16 @@ public boolean hasCallTimeout() { return callTimeout != null; } + @Override + public Map getRequestAttributes() { + return requestAttributes; + } + + @Override + public void setRequestAttributes(Map requestAttributes) { + this.requestAttributes = requestAttributes; + } + @Override public synchronized String errorText() { if (!done || exception == null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java index b509dcbd27b7..d6df6c974ccf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java @@ -25,6 +25,7 @@ import java.net.ConnectException; import java.net.SocketTimeoutException; import java.nio.channels.ClosedChannelException; +import java.util.Map; import java.util.concurrent.TimeoutException; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -44,10 +45,12 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream; import org.apache.hbase.thirdparty.com.google.protobuf.Message; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf; import org.apache.hbase.thirdparty.io.netty.channel.EventLoop; import org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocal; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; @@ -126,6 +129,14 @@ static RequestHeader buildRequestHeader(Call call, CellBlockMeta cellBlockMeta) if (call.priority != HConstants.PRIORITY_UNSET) { builder.setPriority(call.priority); } + if (call.attributes != null && !call.attributes.isEmpty()) { + HBaseProtos.NameBytesPair.Builder attributeBuilder = HBaseProtos.NameBytesPair.newBuilder(); + for (Map.Entry attribute : call.attributes.entrySet()) { + attributeBuilder.setName(attribute.getKey()); + attributeBuilder.setValue(UnsafeByteOperations.unsafeWrap(attribute.getValue())); + builder.addAttribute(attributeBuilder.build()); + } + } builder.setTimeout(call.timeout); return builder.build(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java index 606749315f41..68d28e6ef602 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcClient.java @@ -19,6 +19,8 @@ import java.io.IOException; import java.net.SocketAddress; +import java.util.Collections; +import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; @@ -56,7 +58,12 @@ public class NettyRpcClient extends AbstractRpcClient { public NettyRpcClient(Configuration configuration, String clusterId, SocketAddress localAddress, MetricsConnection metrics) { - super(configuration, clusterId, localAddress, metrics); + this(configuration, clusterId, localAddress, metrics, Collections.emptyMap()); + } + + public NettyRpcClient(Configuration configuration, String clusterId, SocketAddress localAddress, + MetricsConnection metrics, Map connectionAttributes) { + super(configuration, clusterId, localAddress, metrics, connectionAttributes); Pair> groupAndChannelClass = NettyRpcClientConfigHelper.getEventLoopConfig(conf); if (groupAndChannelClass == null) { @@ -76,7 +83,7 @@ public NettyRpcClient(Configuration configuration, String clusterId, SocketAddre /** Used in test only. */ public NettyRpcClient(Configuration configuration) { - this(configuration, HConstants.CLUSTER_ID_DEFAULT, null, null); + this(configuration, HConstants.CLUSTER_ID_DEFAULT, null, null, Collections.emptyMap()); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java index 56f7b9fd0486..598304ce0465 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java @@ -104,7 +104,7 @@ class NettyRpcConnection extends RpcConnection { NettyRpcConnection(NettyRpcClient rpcClient, ConnectionId remoteId) throws IOException { super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId, rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor, - rpcClient.metrics); + rpcClient.metrics, rpcClient.connectionAttributes); this.rpcClient = rpcClient; this.eventLoop = rpcClient.group.next(); byte[] connectionHeaderPreamble = getConnectionHeaderPreamble(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java index 9b69b5234050..f1df572675c7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.ipc; import java.net.SocketAddress; +import java.util.Collections; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.MetricsConnection; import org.apache.hadoop.hbase.util.ReflectionUtils; @@ -59,7 +61,7 @@ public static RpcClient createClient(Configuration conf, String clusterId) { */ public static RpcClient createClient(Configuration conf, String clusterId, MetricsConnection metrics) { - return createClient(conf, clusterId, null, metrics); + return createClient(conf, clusterId, null, metrics, Collections.emptyMap()); } private static String getRpcClientClass(Configuration conf) { @@ -81,10 +83,11 @@ private static String getRpcClientClass(Configuration conf) { * @return newly created RpcClient */ public static RpcClient createClient(Configuration conf, String clusterId, - SocketAddress localAddr, MetricsConnection metrics) { + SocketAddress localAddr, MetricsConnection metrics, Map connectionAttributes) { String rpcClientClass = getRpcClientClass(conf); - return ReflectionUtils.instantiateWithCustomCtor(rpcClientClass, new Class[] { - Configuration.class, String.class, SocketAddress.class, MetricsConnection.class }, - new Object[] { conf, clusterId, localAddr, metrics }); + return ReflectionUtils.instantiateWithCustomCtor( + rpcClientClass, new Class[] { Configuration.class, String.class, SocketAddress.class, + MetricsConnection.class, Map.class }, + new Object[] { conf, clusterId, localAddr, metrics, connectionAttributes }); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java index 912fa4fb0654..31698a1a1e8e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.UnknownHostException; +import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; @@ -39,11 +40,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer; import org.apache.hbase.thirdparty.io.netty.util.Timeout; import org.apache.hbase.thirdparty.io.netty.util.TimerTask; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation; @@ -70,6 +73,7 @@ abstract class RpcConnection { protected final CompressionCodec compressor; protected final MetricsConnection metrics; + private final Map connectionAttributes; protected final HashedWheelTimer timeoutTimer; @@ -86,12 +90,13 @@ abstract class RpcConnection { protected RpcConnection(Configuration conf, HashedWheelTimer timeoutTimer, ConnectionId remoteId, String clusterId, boolean isSecurityEnabled, Codec codec, CompressionCodec compressor, - MetricsConnection metrics) throws IOException { + MetricsConnection metrics, Map connectionAttributes) throws IOException { this.timeoutTimer = timeoutTimer; this.codec = codec; this.compressor = compressor; this.conf = conf; this.metrics = metrics; + this.connectionAttributes = connectionAttributes; User ticket = remoteId.getTicket(); this.securityInfo = SecurityInfo.getInfo(remoteId.getServiceName()); this.useSasl = isSecurityEnabled; @@ -169,6 +174,14 @@ protected final ConnectionHeader getConnectionHeader() { if (this.compressor != null) { builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName()); } + if (connectionAttributes != null && !connectionAttributes.isEmpty()) { + HBaseProtos.NameBytesPair.Builder attributeBuilder = HBaseProtos.NameBytesPair.newBuilder(); + for (Map.Entry attribute : connectionAttributes.entrySet()) { + attributeBuilder.setName(attribute.getKey()); + attributeBuilder.setValue(UnsafeByteOperations.unsafeWrap(attribute.getValue())); + builder.addAttribute(attributeBuilder.build()); + } + } builder.setVersionInfo(ProtobufUtil.getVersionInfo()); boolean isCryptoAESEnable = conf.getBoolean(CRYPTO_AES_ENABLED_KEY, CRYPTO_AES_ENABLED_DEFAULT); // if Crypto AES enable, setup Cipher transformation diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcBasedRegistryHedgedReads.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcBasedRegistryHedgedReads.java index 43737a71ca0d..0e7a25a8581c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcBasedRegistryHedgedReads.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRpcBasedRegistryHedgedReads.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.net.SocketAddress; import java.util.Collections; +import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -95,7 +96,7 @@ public class TestRpcBasedRegistryHedgedReads { public static final class RpcClientImpl implements RpcClient { public RpcClientImpl(Configuration configuration, String clusterId, SocketAddress localAddress, - MetricsConnection metrics) { + MetricsConnection metrics, Map attributes) { } @Override diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestTLSHandshadeFailure.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestTLSHandshadeFailure.java index 7375388e4a04..10948358ff92 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestTLSHandshadeFailure.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestTLSHandshadeFailure.java @@ -30,6 +30,7 @@ import java.io.IOException; import java.net.ServerSocket; import java.net.Socket; +import java.util.Collections; import java.util.Random; import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; @@ -148,7 +149,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { Address.fromParts("127.0.0.1", server.getLocalPort())); NettyRpcConnection conn = client.createConnection(id); BlockingRpcCallback done = new BlockingRpcCallback<>(); - Call call = new Call(1, null, null, null, null, 0, 0, done, new CallStats()); + Call call = + new Call(1, null, null, null, null, 0, 0, Collections.emptyMap(), done, new CallStats()); HBaseRpcController hrc = new HBaseRpcControllerImpl(); conn.sendRequest(call, hrc); done.get(); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index e0c596e14cad..116af5ccd266 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -1605,12 +1605,11 @@ private static class ConfigurationCaptorConnection implements Connection { private final Connection delegate; - public ConfigurationCaptorConnection(Configuration conf, ExecutorService es, User user) + public ConfigurationCaptorConnection(Configuration conf, ExecutorService es, User user, Map connectionAttributes) throws IOException { Configuration confForDelegate = new Configuration(conf); confForDelegate.unset(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL); - delegate = createConnection(confForDelegate, es, user); - + delegate = createConnection(confForDelegate, es, user, connectionAttributes); final String uuid = conf.get(UUID_KEY); if (uuid != null) { confs.computeIfAbsent(UUID.fromString(uuid), u -> new CopyOnWriteArrayList<>()).add(conf); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java index 32da37f1f4ff..493cc08b494e 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java @@ -122,7 +122,8 @@ public static class MRSplitsConnection implements Connection { private final Configuration configuration; static final AtomicInteger creations = new AtomicInteger(0); - MRSplitsConnection(Configuration conf, ExecutorService pool, User user) throws IOException { + MRSplitsConnection(Configuration conf, ExecutorService pool, User user, + Map connectionAttributes) throws IOException { this.configuration = conf; creations.incrementAndGet(); } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java index 38157c5cb0a2..83e55b5466c2 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java @@ -211,8 +211,8 @@ private static class ConnectionForMergeTesting implements Connection { SIZE_MAP.put(Bytes.toBytes("p"), 200L * 1024L * 1024L); } - ConnectionForMergeTesting(Configuration conf, ExecutorService pool, User user) - throws IOException { + ConnectionForMergeTesting(Configuration conf, ExecutorService pool, User user, + Map connectionAttributes) throws IOException { } @Override diff --git a/hbase-protocol-shaded/src/main/protobuf/RPC.proto b/hbase-protocol-shaded/src/main/protobuf/RPC.proto index d1b73f7e0197..792fb69695ed 100644 --- a/hbase-protocol-shaded/src/main/protobuf/RPC.proto +++ b/hbase-protocol-shaded/src/main/protobuf/RPC.proto @@ -92,6 +92,7 @@ message ConnectionHeader { optional VersionInfo version_info = 5; // the transformation for rpc AES encryption with Apache Commons Crypto optional string rpc_crypto_cipher_transformation = 6; + repeated NameBytesPair attribute = 7; } // This is sent by rpc server to negotiate the data if necessary @@ -148,6 +149,7 @@ message RequestHeader { // See HConstants. optional uint32 priority = 6; optional uint32 timeout = 7; + repeated NameBytesPair attribute = 8; } message ResponseHeader { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java index 197ddb71d7e6..cc97a39c7ee4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCall.java @@ -27,6 +27,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor; import org.apache.hbase.thirdparty.com.google.protobuf.Message; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; /** @@ -82,6 +83,8 @@ public interface RpcCall extends RpcCallContext { /** Returns The request header of this call. */ RequestHeader getHeader(); + ConnectionHeader getConnectionHeader(); + /** Returns Port of remote address in this call */ int getRemotePort(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java index 2c7a8b637414..a211d08514d1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.ipc; import com.google.errorprone.annotations.RestrictedApi; + import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.StatusCode; import io.opentelemetry.context.Scope; @@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.io.ByteBufferListOutputStream; import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup; import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; @@ -50,6 +52,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader; @@ -207,6 +210,11 @@ public RequestHeader getHeader() { return this.header; } + @Override + public RPCProtos.ConnectionHeader getConnectionHeader() { + return this.connection.connectionHeader; + } + @Override public int getPriority() { return this.header.getPriority(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncTable.java index 3aa99575a9e3..996103116c5b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncTable.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -75,6 +76,11 @@ public long getScanTimeout(TimeUnit unit) { return 0; } + @Override + public Map getRequestAttributes() { + return null; + } + @Override public CompletableFuture get(Get get) { return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index c7a1b48676cd..82c6d2d03870 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -23,6 +23,7 @@ import java.net.SocketAddress; import java.net.SocketTimeoutException; import java.util.Random; +import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; @@ -139,8 +140,8 @@ public void testAdminTimeout() throws Exception { */ public static class RandomTimeoutRpcClient extends BlockingRpcClient { public RandomTimeoutRpcClient(Configuration conf, String clusterId, SocketAddress localAddr, - MetricsConnection metrics) { - super(conf, clusterId, localAddr, metrics); + MetricsConnection metrics, Map connectionAttributes) { + super(conf, clusterId, localAddr, metrics, connectionAttributes); } // Return my own instance, one that does random timeouts diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java new file mode 100644 index 000000000000..b376bfc18557 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java @@ -0,0 +1,317 @@ +/* + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.AuthUtil; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionObserver; +import org.apache.hadoop.hbase.ipc.RpcCall; +import org.apache.hadoop.hbase.ipc.RpcServer; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; + +@Category({ ClientTests.class, MediumTests.class }) +public class TestRequestAndConnectionAttributes { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRequestAndConnectionAttributes.class); + + private static final Map CONNECTION_ATTRIBUTES = new HashMap<>(); + static { + CONNECTION_ATTRIBUTES.put("clientId", Bytes.toBytes("foo")); + } + private static final Map REQUEST_ATTRIBUTES = new HashMap<>(); + private static final ExecutorService EXECUTOR_SERVICE = Executors.newFixedThreadPool(100); + private static final AtomicBoolean REQUEST_ATTRIBUTES_VALIDATED = new AtomicBoolean(false); + private static final byte[] REQUEST_ATTRIBUTES_TEST_TABLE_CF = Bytes.toBytes("0"); + private static final TableName REQUEST_ATTRIBUTES_TEST_TABLE = + TableName.valueOf("testRequestAttributes"); + + private static HBaseTestingUtil TEST_UTIL = null; + + @BeforeClass + public static void setUp() throws Exception { + TEST_UTIL = new HBaseTestingUtil(); + TEST_UTIL.startMiniCluster(1); + TEST_UTIL.createTable(REQUEST_ATTRIBUTES_TEST_TABLE, + new byte[][] { REQUEST_ATTRIBUTES_TEST_TABLE_CF }, 1, HConstants.DEFAULT_BLOCKSIZE, + AttributesCoprocessor.class.getName()); + } + + @AfterClass + public static void afterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Before + public void setup() { + REQUEST_ATTRIBUTES_VALIDATED.getAndSet(false); + } + + @Test + public void testConnectionAttributes() throws IOException { + TableName tableName = TableName.valueOf("testConnectionAttributes"); + TEST_UTIL.createTable(tableName, new byte[][] { Bytes.toBytes("0") }, 1, + HConstants.DEFAULT_BLOCKSIZE, AttributesCoprocessor.class.getName()); + + Configuration conf = TEST_UTIL.getConfiguration(); + try (Connection conn = ConnectionFactory.createConnection(conf, null, + AuthUtil.loginClient(conf), CONNECTION_ATTRIBUTES); Table table = conn.getTable(tableName)) { + Result result = table.get(new Get(Bytes.toBytes(0))); + assertEquals(CONNECTION_ATTRIBUTES.size(), result.size()); + for (Map.Entry attr : CONNECTION_ATTRIBUTES.entrySet()) { + byte[] val = result.getValue(Bytes.toBytes("c"), Bytes.toBytes(attr.getKey())); + assertEquals(Bytes.toStringBinary(attr.getValue()), Bytes.toStringBinary(val)); + } + } + } + + @Test + public void testRequestAttributesGet() throws IOException { + addRandomRequestAttributes(); + + Configuration conf = TEST_UTIL.getConfiguration(); + try ( + Connection conn = ConnectionFactory.createConnection(conf, null, AuthUtil.loginClient(conf), + CONNECTION_ATTRIBUTES); + Table table = configureRequestAttributes( + conn.getTableBuilder(REQUEST_ATTRIBUTES_TEST_TABLE, EXECUTOR_SERVICE)).build()) { + + table.get(new Get(Bytes.toBytes(0))); + } + + assertTrue(REQUEST_ATTRIBUTES_VALIDATED.get()); + } + + @Test + public void testRequestAttributesMultiGet() throws IOException { + assertFalse(REQUEST_ATTRIBUTES_VALIDATED.get()); + addRandomRequestAttributes(); + + Configuration conf = TEST_UTIL.getConfiguration(); + try ( + Connection conn = ConnectionFactory.createConnection(conf, null, AuthUtil.loginClient(conf), + CONNECTION_ATTRIBUTES); + Table table = configureRequestAttributes( + conn.getTableBuilder(REQUEST_ATTRIBUTES_TEST_TABLE, EXECUTOR_SERVICE)).build()) { + List gets = ImmutableList.of(new Get(Bytes.toBytes(0)), new Get(Bytes.toBytes(1))); + table.get(gets); + } + + assertTrue(REQUEST_ATTRIBUTES_VALIDATED.get()); + } + + @Test + public void testRequestAttributesExists() throws IOException { + assertFalse(REQUEST_ATTRIBUTES_VALIDATED.get()); + addRandomRequestAttributes(); + + Configuration conf = TEST_UTIL.getConfiguration(); + try ( + Connection conn = ConnectionFactory.createConnection(conf, null, AuthUtil.loginClient(conf), + CONNECTION_ATTRIBUTES); + Table table = configureRequestAttributes( + conn.getTableBuilder(REQUEST_ATTRIBUTES_TEST_TABLE, EXECUTOR_SERVICE)).build()) { + + table.exists(new Get(Bytes.toBytes(0))); + } + + assertTrue(REQUEST_ATTRIBUTES_VALIDATED.get()); + } + + @Test + public void testRequestAttributesScan() throws IOException { + assertFalse(REQUEST_ATTRIBUTES_VALIDATED.get()); + addRandomRequestAttributes(); + + Configuration conf = TEST_UTIL.getConfiguration(); + try ( + Connection conn = ConnectionFactory.createConnection(conf, null, AuthUtil.loginClient(conf), + CONNECTION_ATTRIBUTES); + Table table = configureRequestAttributes( + conn.getTableBuilder(REQUEST_ATTRIBUTES_TEST_TABLE, EXECUTOR_SERVICE)).build()) { + ResultScanner scanner = table.getScanner(new Scan()); + scanner.next(); + } + assertTrue(REQUEST_ATTRIBUTES_VALIDATED.get()); + } + + @Test + public void testRequestAttributesPut() throws IOException { + assertFalse(REQUEST_ATTRIBUTES_VALIDATED.get()); + addRandomRequestAttributes(); + + Configuration conf = TEST_UTIL.getConfiguration(); + try ( + Connection conn = ConnectionFactory.createConnection(conf, null, AuthUtil.loginClient(conf), + CONNECTION_ATTRIBUTES); + Table table = configureRequestAttributes( + conn.getTableBuilder(REQUEST_ATTRIBUTES_TEST_TABLE, EXECUTOR_SERVICE)).build()) { + Put put = new Put(Bytes.toBytes("a")); + put.addColumn(REQUEST_ATTRIBUTES_TEST_TABLE_CF, Bytes.toBytes("c"), Bytes.toBytes("v")); + table.put(put); + } + assertTrue(REQUEST_ATTRIBUTES_VALIDATED.get()); + } + + @Test + public void testRequestAttributesMultiPut() throws IOException { + assertFalse(REQUEST_ATTRIBUTES_VALIDATED.get()); + addRandomRequestAttributes(); + + Configuration conf = TEST_UTIL.getConfiguration(); + try ( + Connection conn = ConnectionFactory.createConnection(conf, null, AuthUtil.loginClient(conf), + CONNECTION_ATTRIBUTES); + Table table = configureRequestAttributes( + conn.getTableBuilder(REQUEST_ATTRIBUTES_TEST_TABLE, EXECUTOR_SERVICE)).build()) { + Put put = new Put(Bytes.toBytes("a")); + put.addColumn(REQUEST_ATTRIBUTES_TEST_TABLE_CF, Bytes.toBytes("c"), Bytes.toBytes("v")); + table.put(put); + } + assertTrue(REQUEST_ATTRIBUTES_VALIDATED.get()); + } + + @Test + public void testNoRequestAttributes() throws IOException { + assertFalse(REQUEST_ATTRIBUTES_VALIDATED.get()); + TableName tableName = TableName.valueOf("testNoRequestAttributesScan"); + TEST_UTIL.createTable(tableName, new byte[][] { Bytes.toBytes("0") }, 1, + HConstants.DEFAULT_BLOCKSIZE, AttributesCoprocessor.class.getName()); + + REQUEST_ATTRIBUTES.clear(); + Configuration conf = TEST_UTIL.getConfiguration(); + try (Connection conn = ConnectionFactory.createConnection(conf, null, + AuthUtil.loginClient(conf), CONNECTION_ATTRIBUTES)) { + TableBuilder tableBuilder = conn.getTableBuilder(tableName, null); + try (Table table = tableBuilder.build()) { + table.get(new Get(Bytes.toBytes(0))); + assertTrue(REQUEST_ATTRIBUTES_VALIDATED.get()); + } + } + } + + private void addRandomRequestAttributes() { + REQUEST_ATTRIBUTES.clear(); + int j = Math.max(2, (int) (10 * Math.random())); + for (int i = 0; i < j; i++) { + REQUEST_ATTRIBUTES.put(String.valueOf(i), Bytes.toBytes(UUID.randomUUID().toString())); + } + } + + private static TableBuilder configureRequestAttributes(TableBuilder tableBuilder) { + REQUEST_ATTRIBUTES.forEach(tableBuilder::setRequestAttribute); + return tableBuilder; + } + + public static class AttributesCoprocessor implements RegionObserver, RegionCoprocessor { + + @Override + public Optional getRegionObserver() { + return Optional.of(this); + } + + @Override + public void preGetOp(ObserverContext c, Get get, + List result) throws IOException { + validateRequestAttributes(); + + // for connection attrs test + RpcCall rpcCall = RpcServer.getCurrentCall().get(); + for (HBaseProtos.NameBytesPair attr : rpcCall.getHeader().getAttributeList()) { + result.add(c.getEnvironment().getCellBuilder().clear().setRow(get.getRow()) + .setFamily(Bytes.toBytes("r")).setQualifier(Bytes.toBytes(attr.getName())) + .setValue(attr.getValue().toByteArray()).setType(Cell.Type.Put).setTimestamp(1).build()); + } + for (HBaseProtos.NameBytesPair attr : rpcCall.getConnectionHeader().getAttributeList()) { + result.add(c.getEnvironment().getCellBuilder().clear().setRow(get.getRow()) + .setFamily(Bytes.toBytes("c")).setQualifier(Bytes.toBytes(attr.getName())) + .setValue(attr.getValue().toByteArray()).setType(Cell.Type.Put).setTimestamp(1).build()); + } + result.sort(CellComparator.getInstance()); + c.bypass(); + } + + @Override + public boolean preScannerNext(ObserverContext c, + InternalScanner s, List result, int limit, boolean hasNext) throws IOException { + validateRequestAttributes(); + return hasNext; + } + + @Override + public void prePut(ObserverContext c, Put put, WALEdit edit) + throws IOException { + validateRequestAttributes(); + } + + private void validateRequestAttributes() { + RpcCall rpcCall = RpcServer.getCurrentCall().get(); + List attrs = rpcCall.getHeader().getAttributeList(); + if (attrs.size() != REQUEST_ATTRIBUTES.size()) { + return; + } + for (HBaseProtos.NameBytesPair attr : attrs) { + if (!REQUEST_ATTRIBUTES.containsKey(attr.getName())) { + return; + } + if (!Arrays.equals(REQUEST_ATTRIBUTES.get(attr.getName()), attr.getValue().toByteArray())) { + return; + } + } + REQUEST_ATTRIBUTES_VALIDATED.getAndSet(true); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java index f7a636755a25..7aa7678c228f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcClientLeaks.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.net.Socket; import java.net.SocketAddress; +import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import org.apache.hadoop.conf.Configuration; @@ -71,8 +72,8 @@ public MyRpcClientImpl(Configuration conf) { } public MyRpcClientImpl(Configuration conf, String clusterId, SocketAddress address, - MetricsConnection metrics) { - super(conf, clusterId, address, metrics); + MetricsConnection metrics, Map connectionAttributes) { + super(conf, clusterId, address, metrics, connectionAttributes); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerSlowConnectionSetup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerSlowConnectionSetup.java index e14b710647d1..80b3845d6688 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerSlowConnectionSetup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerSlowConnectionSetup.java @@ -124,7 +124,7 @@ public void test() throws IOException, InterruptedException { int callId = 10; Call call = new Call(callId, TestProtobufRpcProto.getDescriptor().findMethodByName("ping"), EmptyRequestProto.getDefaultInstance(), null, EmptyResponseProto.getDefaultInstance(), 1000, - HConstants.NORMAL_QOS, null, MetricsConnection.newCallStats()); + HConstants.NORMAL_QOS, null, null, MetricsConnection.newCallStats()); RequestHeader requestHeader = IPCUtil.buildRequestHeader(call, null); dos.writeInt(IPCUtil.getTotalSizeWhenWrittenDelimited(requestHeader, call.param)); requestHeader.writeDelimitedTo(dos); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java index ef91779e699c..6b4bc8fd39fa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namequeues/TestNamedQueueRecorder.java @@ -632,6 +632,7 @@ private static RpcCall getRpcCall(String userName, int forcedParamIndex) { return getRpcCall(userName, Optional.of(forcedParamIndex)); } + @SuppressWarnings("checkstyle:methodlength") private static RpcCall getRpcCall(String userName, Optional forcedParamIndex) { RpcCall rpcCall = new RpcCall() { @Override @@ -666,7 +667,6 @@ public long getStartTime() { @Override public void setStartTime(long startTime) { - } @Override @@ -694,6 +694,11 @@ public RPCProtos.RequestHeader getHeader() { return null; } + @Override + public RPCProtos.ConnectionHeader getConnectionHeader() { + return null; + } + @Override public int getRemotePort() { return 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java index d26870b77dfd..dd49d00ac3a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure2/store/region/TestRegionProcedureStore.java @@ -158,6 +158,7 @@ public void testInsertWithRpcCall() throws Exception { RpcServer.setCurrentCall(null); } + @SuppressWarnings("checkstyle:methodlength") private RpcCall newRpcCallWithDeadline() { return new RpcCall() { @Override @@ -220,6 +221,11 @@ public RPCProtos.RequestHeader getHeader() { return null; } + @Override + public RPCProtos.ConnectionHeader getConnectionHeader() { + return null; + } + @Override public int getRemotePort() { return 0; diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java index 8a5f41960958..92a6282c86c1 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java @@ -88,8 +88,8 @@ public class ThriftConnection implements Connection { private int operationTimeout; private int connectTimeout; - public ThriftConnection(Configuration conf, ExecutorService pool, final User user) - throws IOException { + public ThriftConnection(Configuration conf, ExecutorService pool, final User user, + Map connectionAttributes) throws IOException { this.conf = conf; this.user = user; this.host = conf.get(Constants.HBASE_THRIFT_SERVER_NAME); @@ -321,6 +321,11 @@ public TableBuilder setWriteRpcTimeout(int timeout) { return this; } + @Override + public TableBuilder setRequestAttribute(String key, byte[] value) { + return this; + } + @Override public Table build() { try { From 37dec7ec2a31af090bc07745529235049c57d0b2 Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Mon, 24 Jul 2023 11:29:41 -0400 Subject: [PATCH 2/6] branch-2 spotless more branch-2 compatibility --- .../hbase/client/AsyncConnectionImpl.java | 11 ++++---- .../hbase/client/ConnectionFactory.java | 6 ++--- .../org/apache/hadoop/hbase/client/Table.java | 1 - .../hadoop/hbase/ipc/RpcClientFactory.java | 25 +++++++++++++++---- .../mapreduce/TestHFileOutputFormat2.java | 4 +-- .../apache/hadoop/hbase/ipc/ServerCall.java | 2 -- .../hbase/regionserver/HRegionServer.java | 2 +- .../hbase/client/TestClientTimeouts.java | 2 +- .../TestRequestAndConnectionAttributes.java | 6 ++--- 9 files changed, 35 insertions(+), 24 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java index a56391503f02..679efabaf050 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java @@ -30,7 +30,6 @@ import io.opentelemetry.api.trace.Span; import java.io.IOException; -import java.net.SocketAddress; import java.util.Collections; import java.util.Map; import java.util.Optional; @@ -124,12 +123,12 @@ public class AsyncConnectionImpl implements AsyncConnection { private final ClusterStatusListener clusterStatusListener; public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId, - SocketAddress localAddress, User user) { - this(conf, registry, clusterId, localAddress, user, Collections.emptyMap()); + User user) { + this(conf, registry, clusterId, user, Collections.emptyMap()); } public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId, - SocketAddress localAddress, User user, Map connectionAttributes) { + User user, Map connectionAttributes) { this.conf = conf; this.user = user; this.metricsScope = MetricsConnection.getScope(conf, clusterId, this); @@ -145,8 +144,8 @@ public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, Stri } else { this.metrics = Optional.empty(); } - this.rpcClient = RpcClientFactory.createClient(conf, clusterId, localAddress, - metrics.orElse(null), connectionAttributes); + this.rpcClient = + RpcClientFactory.createClient(conf, clusterId, metrics.orElse(null), connectionAttributes); this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); this.rpcTimeout = (int) Math.min(Integer.MAX_VALUE, TimeUnit.NANOSECONDS.toMillis(connConf.getRpcTimeoutNs())); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index da9e5ea2b87c..2a4826251330 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -210,9 +210,9 @@ public static Connection createConnection(Configuration conf, User user) throws * } * * - * @param conf configuration - * @param user the user the connection is for - * @param pool the thread pool to use for batch operations + * @param conf configuration + * @param user the user the connection is for + * @param pool the thread pool to use for batch operations * @param connectionAttributes attributes to be sent along to server during connection establish * @return Connection object for conf */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java index fb67917d80a1..ae0f8099a3af 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Table.java @@ -979,7 +979,6 @@ default long getOperationTimeout(TimeUnit unit) { } /** -<<<<<<< HEAD * Get timeout (millisecond) of each operation for in Table instance. * @deprecated since 2.0 and will be removed in 3.0 version use * {@link #getOperationTimeout(TimeUnit)} instead diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java index f1df572675c7..fa65739c165e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientFactory.java @@ -61,7 +61,7 @@ public static RpcClient createClient(Configuration conf, String clusterId) { */ public static RpcClient createClient(Configuration conf, String clusterId, MetricsConnection metrics) { - return createClient(conf, clusterId, null, metrics, Collections.emptyMap()); + return createClient(conf, clusterId, metrics, Collections.emptyMap()); } private static String getRpcClientClass(Configuration conf) { @@ -76,10 +76,25 @@ private static String getRpcClientClass(Configuration conf) { /** * Creates a new RpcClient by the class defined in the configuration or falls back to * RpcClientImpl - * @param conf configuration - * @param clusterId the cluster id - * @param localAddr client socket bind address. - * @param metrics the connection metrics + * @param conf configuration + * @param clusterId the cluster id + * @param metrics the connection metrics + * @param connectionAttributes attributes to be sent along to server during connection establish + * @return newly created RpcClient + */ + public static RpcClient createClient(Configuration conf, String clusterId, + MetricsConnection metrics, Map connectionAttributes) { + return createClient(conf, clusterId, null, metrics, connectionAttributes); + } + + /** + * Creates a new RpcClient by the class defined in the configuration or falls back to + * RpcClientImpl + * @param conf configuration + * @param clusterId the cluster id + * @param localAddr client socket bind address. + * @param metrics the connection metrics + * @param connectionAttributes attributes to be sent along to server during connection establish * @return newly created RpcClient */ public static RpcClient createClient(Configuration conf, String clusterId, diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index 116af5ccd266..50e435715388 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -1605,8 +1605,8 @@ private static class ConfigurationCaptorConnection implements Connection { private final Connection delegate; - public ConfigurationCaptorConnection(Configuration conf, ExecutorService es, User user, Map connectionAttributes) - throws IOException { + public ConfigurationCaptorConnection(Configuration conf, ExecutorService es, User user, + Map connectionAttributes) throws IOException { Configuration confForDelegate = new Configuration(conf); confForDelegate.unset(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL); delegate = createConnection(confForDelegate, es, user, connectionAttributes); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java index a211d08514d1..bfd9e2091502 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.ipc; import com.google.errorprone.annotations.RestrictedApi; - import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.StatusCode; import io.opentelemetry.context.Scope; @@ -37,7 +36,6 @@ import org.apache.hadoop.hbase.io.ByteBufferListOutputStream; import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos; import org.apache.hadoop.hbase.trace.TraceUtil; import org.apache.hadoop.hbase.util.ByteBufferUtils; import org.apache.hadoop.hbase.util.Bytes; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 94fefa26c578..74b8d1bef927 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -948,7 +948,7 @@ private void preRegistrationInitialization() { // Setup RPC client for master communication this.rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(this.rpcServices.isa.getAddress(), 0), - clusterConnection.getConnectionMetrics()); + clusterConnection.getConnectionMetrics(), Collections.emptyMap()); span.setStatus(StatusCode.OK); } catch (Throwable t) { // Call stop if error or process will stick around for ever since server diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java index 82c6d2d03870..0449ef6fe917 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientTimeouts.java @@ -22,8 +22,8 @@ import java.net.SocketAddress; import java.net.SocketTimeoutException; -import java.util.Random; import java.util.Map; +import java.util.Random; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.conf.Configuration; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java index b376bfc18557..bf427b166a43 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -79,11 +79,11 @@ public class TestRequestAndConnectionAttributes { private static final TableName REQUEST_ATTRIBUTES_TEST_TABLE = TableName.valueOf("testRequestAttributes"); - private static HBaseTestingUtil TEST_UTIL = null; + private static HBaseTestingUtility TEST_UTIL = null; @BeforeClass public static void setUp() throws Exception { - TEST_UTIL = new HBaseTestingUtil(); + TEST_UTIL = new HBaseTestingUtility(); TEST_UTIL.startMiniCluster(1); TEST_UTIL.createTable(REQUEST_ATTRIBUTES_TEST_TABLE, new byte[][] { REQUEST_ATTRIBUTES_TEST_TABLE_CF }, 1, HConstants.DEFAULT_BLOCKSIZE, From 88aa4aa6d52efef7307f5a6337a4d966ad2b0d0c Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Mon, 24 Jul 2023 16:43:24 -0400 Subject: [PATCH 3/6] branch-2 connection and request attributes impl --- .../hadoop/hbase/client/AsyncProcess.java | 3 +- .../hadoop/hbase/client/AsyncProcessTask.java | 21 +++- .../hbase/client/AsyncRequestFutureImpl.java | 5 +- .../hbase/client/BufferedMutatorImpl.java | 4 +- .../CancellableRegionServerCallable.java | 6 +- .../client/ClientAsyncPrefetchScanner.java | 6 +- .../hadoop/hbase/client/ClientScanner.java | 6 +- .../hbase/client/ClientServiceCallable.java | 11 +- .../hbase/client/ClientSimpleScanner.java | 8 +- .../hbase/client/ConnectionFactory.java | 2 +- .../client/ConnectionImplementation.java | 36 ++++-- .../apache/hadoop/hbase/client/HTable.java | 106 +++++++++++------- .../hbase/client/HTableMultiplexer.java | 3 +- .../hbase/client/MultiServerCallable.java | 5 +- .../client/NoncedRegionServerCallable.java | 5 +- .../client/RegionCoprocessorRpcChannel.java | 4 +- .../hbase/client/ReversedClientScanner.java | 8 +- .../hbase/client/ReversedScannerCallable.java | 8 +- .../RpcRetryingCallerWithReadReplicas.java | 4 +- .../hadoop/hbase/client/ScannerCallable.java | 8 +- .../hbase/client/SecureBulkLoadClient.java | 30 ++--- .../hbase/ipc/HBaseRpcControllerImpl.java | 13 +++ .../hadoop/hbase/client/TestAsyncProcess.java | 8 +- .../TestAsyncProcessWithRegionException.java | 2 +- .../hbase/client/TestClientScanner.java | 4 +- .../client/TestReversedScannerCallable.java | 13 ++- .../hbase/client/TestScannerCallable.java | 9 +- ...onServerBulkLoadWithOldSecureEndpoint.java | 28 ++--- .../hbase/tool/LoadIncrementalHFiles.java | 2 +- .../client/HConnectionTestingUtility.java | 3 +- .../hadoop/hbase/client/TestCISleep.java | 3 +- .../hbase/client/TestReplicaWithCluster.java | 33 +++--- .../TestRequestAndConnectionAttributes.java | 21 ++++ .../quotas/SpaceQuotaHelperForTests.java | 4 +- .../TestHRegionServerBulkLoad.java | 5 +- ...estHRegionServerBulkLoadWithOldClient.java | 28 ++--- ...estLoadIncrementalHFilesSplitRecovery.java | 3 +- 37 files changed, 305 insertions(+), 163 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java index 7a4ed3f5b365..6518039f183c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java @@ -434,7 +434,8 @@ private int checkRpcTimeout(int rpcTimeout) { AsyncRequestFutureImpl createAsyncRequestFuture(AsyncProcessTask task, List actions, long nonceGroup) { - return new AsyncRequestFutureImpl<>(task, actions, nonceGroup, this); + return new AsyncRequestFutureImpl<>(task, actions, nonceGroup, this, + task.getRequestAttributes()); } /** Wait until the async does not have more than max tasks in progress. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java index 49b04b357535..fde132cb8a93 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hbase.client; +import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutorService; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.coprocessor.Batch; @@ -63,6 +65,7 @@ public static class Builder { private int operationTimeout; private CancellableRegionServerCallable callable; private Object[] results; + private Map requestAttributes = Collections.emptyMap(); private Builder() { } @@ -124,9 +127,14 @@ Builder setCallable(CancellableRegionServerCallable callable) { return this; } + Builder setRequestAttributes(Map setRequestAttributes) { + this.requestAttributes = requestAttributes; + return this; + } + public AsyncProcessTask build() { return new AsyncProcessTask<>(pool, tableName, rows, submittedRows, callback, callable, - needResults, rpcTimeout, operationTimeout, results); + needResults, rpcTimeout, operationTimeout, results, requestAttributes); } } @@ -140,16 +148,18 @@ public AsyncProcessTask build() { private final int rpcTimeout; private final int operationTimeout; private final Object[] results; + private final Map requestAttributes; AsyncProcessTask(AsyncProcessTask task) { this(task.getPool(), task.getTableName(), task.getRowAccess(), task.getSubmittedRows(), task.getCallback(), task.getCallable(), task.getNeedResults(), task.getRpcTimeout(), - task.getOperationTimeout(), task.getResults()); + task.getOperationTimeout(), task.getResults(), task.getRequestAttributes()); } AsyncProcessTask(ExecutorService pool, TableName tableName, RowAccess rows, SubmittedRows size, Batch.Callback callback, CancellableRegionServerCallable callable, - boolean needResults, int rpcTimeout, int operationTimeout, Object[] results) { + boolean needResults, int rpcTimeout, int operationTimeout, Object[] results, + Map requestAttributes) { this.pool = pool; this.tableName = tableName; this.rows = rows; @@ -160,6 +170,7 @@ public AsyncProcessTask build() { this.rpcTimeout = rpcTimeout; this.operationTimeout = operationTimeout; this.results = results; + this.requestAttributes = requestAttributes; } public int getOperationTimeout() { @@ -190,6 +201,10 @@ CancellableRegionServerCallable getCallable() { return callable; } + public Map getRequestAttributes() { + return requestAttributes; + } + Object[] getResults() { return results; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java index 827dd51e4214..62f41cb3a7a4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java @@ -323,7 +323,7 @@ public String toString() { } public AsyncRequestFutureImpl(AsyncProcessTask task, List actions, long nonceGroup, - AsyncProcess asyncProcess) { + AsyncProcess asyncProcess, Map requestAttributes) { this.pool = task.getPool(); this.callback = task.getCallback(); this.nonceGroup = nonceGroup; @@ -1316,7 +1316,8 @@ private ConnectionImplementation.ServerErrorTracker createServerErrorTracker() { private MultiServerCallable createCallable(final ServerName server, TableName tableName, final MultiAction multi) { return new MultiServerCallable(asyncProcess.connection, tableName, server, multi, - asyncProcess.rpcFactory.newController(), rpcTimeout, tracker, multi.getPriority()); + asyncProcess.rpcFactory.newController(), rpcTimeout, tracker, multi.getPriority(), + asyncProcess.getRequestAttributes()); } private void updateResult(int index, Object result) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java index e0795c7c4c67..93fd12f7aab8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java @@ -142,7 +142,9 @@ public class BufferedMutatorImpl implements BufferedMutator { RpcControllerFactory rpcFactory, BufferedMutatorParams params) { this(conn, params, // puts need to track errors globally due to how the APIs currently work. - new AsyncProcess(conn, conn.getConfiguration(), rpcCallerFactory, rpcFactory)); + // todo rmattingly support buffered mutator request attributes + new AsyncProcess(conn, conn.getConfiguration(), rpcCallerFactory, rpcFactory, + Collections.emptyMap())); } private void checkClose() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java index 800a3ea65a9e..c120cdbe1a59 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CancellableRegionServerCallable.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.io.InterruptedIOException; +import java.util.Map; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; @@ -41,8 +42,9 @@ abstract class CancellableRegionServerCallable extends ClientServiceCallable< private final int rpcTimeout; CancellableRegionServerCallable(Connection connection, TableName tableName, byte[] row, - RpcController rpcController, int rpcTimeout, RetryingTimeTracker tracker, int priority) { - super(connection, tableName, row, rpcController, priority); + RpcController rpcController, int rpcTimeout, RetryingTimeTracker tracker, int priority, + Map requestAttributes) { + super(connection, tableName, row, rpcController, priority, requestAttributes); this.rpcTimeout = rpcTimeout; this.tracker = tracker; } 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 77ec41ec4e9d..769931b7083f 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 @@ -23,6 +23,7 @@ import io.opentelemetry.context.Scope; import java.io.IOException; import java.io.InterruptedIOException; +import java.util.Map; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; @@ -64,9 +65,10 @@ public class ClientAsyncPrefetchScanner extends ClientSimpleScanner { public ClientAsyncPrefetchScanner(Configuration configuration, Scan scan, TableName name, ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, ExecutorService pool, int scanReadRpcTimeout, - int scannerTimeout, int replicaCallTimeoutMicroSecondScan) throws IOException { + int scannerTimeout, int replicaCallTimeoutMicroSecondScan, + Map requestAttributes) throws IOException { super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool, - scanReadRpcTimeout, scannerTimeout, replicaCallTimeoutMicroSecondScan); + scanReadRpcTimeout, scannerTimeout, replicaCallTimeoutMicroSecondScan, requestAttributes); exceptionsQueue = new ConcurrentLinkedQueue<>(); final Context context = Context.current(); final Runnable runnable = context.wrap(new PrefetchRunnable()); 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 f240798c58de..33cfedc362ad 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 @@ -27,6 +27,7 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayDeque; +import java.util.Map; import java.util.Queue; import java.util.concurrent.ExecutorService; import org.apache.commons.lang3.mutable.MutableBoolean; @@ -90,6 +91,7 @@ public abstract class ClientScanner extends AbstractClientScanner { protected final int primaryOperationTimeout; private int retries; protected final ExecutorService pool; + protected final Map requestAttributes; /** * Create a new ClientScanner for the specified table Note that the passed {@link Scan}'s start @@ -102,7 +104,8 @@ public abstract class ClientScanner extends AbstractClientScanner { public ClientScanner(final Configuration conf, final Scan scan, final TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, RpcControllerFactory controllerFactory, ExecutorService pool, int scanReadRpcTimeout, - int scannerTimeout, int primaryOperationTimeout) throws IOException { + int scannerTimeout, int primaryOperationTimeout, Map requestAttributes) + throws IOException { if (LOG.isTraceEnabled()) { LOG.trace( "Scan table=" + tableName + ", startRow=" + Bytes.toStringBinary(scan.getStartRow())); @@ -123,6 +126,7 @@ public ClientScanner(final Configuration conf, final Scan scan, final TableName } this.readRpcTimeout = scanReadRpcTimeout; this.scannerTimeout = scannerTimeout; + this.requestAttributes = requestAttributes; // check if application wants to collect scan metrics initScanMetrics(scan); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java index c9220540e453..5c63ff11ffcb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java @@ -18,8 +18,10 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import java.util.Map; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; @@ -34,9 +36,14 @@ public abstract class ClientServiceCallable extends RegionServerCallable { + protected Map requestAttributes; + public ClientServiceCallable(Connection connection, TableName tableName, byte[] row, - RpcController rpcController, int priority) { - super(connection, tableName, row, rpcController, priority); + RpcController rpcController, int priority, Map requestAttributes) { + super(connection, tableName, row, + HBaseRpcControllerImpl.configureRequestAttributes(rpcController, requestAttributes), + priority); + this.requestAttributes = requestAttributes; } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java index 51d1bb1b2280..81091ad3010c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientSimpleScanner.java @@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForScan; import java.io.IOException; +import java.util.Map; import java.util.concurrent.ExecutorService; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; @@ -37,9 +38,10 @@ public class ClientSimpleScanner extends ClientScanner { public ClientSimpleScanner(Configuration configuration, Scan scan, TableName name, ClusterConnection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, ExecutorService pool, int scanReadRpcTimeout, - int scannerTimeout, int replicaCallTimeoutMicroSecondScan) throws IOException { + int scannerTimeout, int replicaCallTimeoutMicroSecondScan, + Map requestAttributes) throws IOException { super(configuration, scan, name, connection, rpcCallerFactory, rpcControllerFactory, pool, - scanReadRpcTimeout, scannerTimeout, replicaCallTimeoutMicroSecondScan); + scanReadRpcTimeout, scannerTimeout, replicaCallTimeoutMicroSecondScan, requestAttributes); } @Override @@ -59,6 +61,6 @@ protected ScannerCallable createScannerCallable() { scan.withStartRow(createClosestRowAfter(scan.getStartRow()), true); } return new ScannerCallable(getConnection(), getTable(), scan, this.scanMetrics, - this.rpcControllerFactory, getScanReplicaId()); + this.rpcControllerFactory, getScanReplicaId(), requestAttributes); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index 2a4826251330..9b4880c6aa4f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -327,7 +327,7 @@ public static CompletableFuture createAsyncConnection(Configura try { future.complete( user.runAs((PrivilegedExceptionAction) () -> ReflectionUtils - .newInstance(clazz, conf, registry, clusterId, null, user, connectionAttributes))); + .newInstance(clazz, conf, registry, clusterId, user, connectionAttributes))); } catch (Exception e) { registry.close(); future.completeExceptionally(e); 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 bcb295a2628b..1257cba1030f 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 @@ -39,6 +39,7 @@ import java.util.Collections; import java.util.Date; import java.util.List; +import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -284,7 +285,16 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { * @param conf Configuration object */ ConnectionImplementation(Configuration conf, ExecutorService pool, User user) throws IOException { - this(conf, pool, user, null); + this(conf, pool, user, null, Collections.emptyMap()); + } + + /** + * constructor + * @param conf Configuration object + */ + ConnectionImplementation(Configuration conf, ExecutorService pool, User user, + Map connectionAttributes) throws IOException { + this(conf, pool, user, null, connectionAttributes); } /** @@ -292,6 +302,14 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { */ ConnectionImplementation(Configuration conf, ExecutorService pool, User user, ConnectionRegistry registry) throws IOException { + this(conf, pool, user, registry, Collections.emptyMap()); + } + + /** + * Constructor, for creating cluster connection with provided ConnectionRegistry. + */ + ConnectionImplementation(Configuration conf, ExecutorService pool, User user, + ConnectionRegistry registry, Map connectionAttributes) throws IOException { this.conf = conf; this.user = user; if (user != null && user.isLoginFromKeytab()) { @@ -348,11 +366,13 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { } this.metaCache = new MetaCache(this.metrics); - this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId, this.metrics); + this.rpcClient = RpcClientFactory.createClient(this.conf, this.clusterId, this.metrics, + connectionAttributes); this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, connectionConfig, interceptor, this.stats, this.metrics); - this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory); + this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory, + Collections.emptyMap()); // Do we publish the status? if (shouldListen) { @@ -476,7 +496,7 @@ public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) { @Override public Table build() { return new HTable(ConnectionImplementation.this, this, rpcCallerFactory, - rpcControllerFactory, pool); + rpcControllerFactory, pool, requestAttributes); } }; } @@ -1030,10 +1050,10 @@ private RegionLocations locateRegionInMeta(TableName tableName, byte[] row, bool 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(), connectionConfig.getMetaReadRpcTimeout(), - connectionConfig.getMetaScanTimeout(), metaReplicaCallTimeoutScanInMicroSecond)) { + ReversedClientScanner rcs = new ReversedClientScanner(conf, s, TableName.META_TABLE_NAME, + this, rpcCallerFactory, rpcControllerFactory, getMetaLookupPool(), + connectionConfig.getMetaReadRpcTimeout(), connectionConfig.getMetaScanTimeout(), + metaReplicaCallTimeoutScanInMicroSecond, Collections.emptyMap())) { boolean tableNotFound = true; for (;;) { Result regionInfoRow = rcs.next(); 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 44761c2fbf16..96fa640e8e4b 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 @@ -130,6 +130,8 @@ public class HTable implements Table { private final RpcRetryingCallerFactory rpcCallerFactory; private final RpcControllerFactory rpcControllerFactory; + private final Map requestAttributes; + // Marked Private @since 1.0 @InterfaceAudience.Private public static ThreadPoolExecutor getDefaultExecutor(Configuration conf) { @@ -165,7 +167,8 @@ public static ThreadPoolExecutor getDefaultExecutor(Configuration conf) { @InterfaceAudience.Private protected HTable(final ConnectionImplementation connection, final TableBuilderBase builder, final RpcRetryingCallerFactory rpcCallerFactory, - final RpcControllerFactory rpcControllerFactory, final ExecutorService pool) { + final RpcControllerFactory rpcControllerFactory, final ExecutorService pool, + final Map requestAttributes) { this.connection = Preconditions.checkNotNull(connection, "connection is null"); this.configuration = connection.getConfiguration(); this.connConfiguration = connection.getConnectionConfiguration(); @@ -197,6 +200,7 @@ protected HTable(final ConnectionImplementation connection, final TableBuilderBa this.scanTimeout = builder.scanTimeout; this.scannerCaching = connConfiguration.getScannerCaching(); this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize(); + this.requestAttributes = requestAttributes; // puts need to track errors globally due to how the APIs currently work. multiAp = this.connection.getAsyncProcess(); @@ -319,16 +323,16 @@ public ResultScanner getScanner(Scan scan) throws IOException { if (scan.isReversed()) { return new ReversedClientScanner(getConfiguration(), scan, getName(), connection, rpcCallerFactory, rpcControllerFactory, pool, scanReadRpcTimeout, scanTimeout, - replicaTimeout); + replicaTimeout, requestAttributes); } else { if (async) { return new ClientAsyncPrefetchScanner(getConfiguration(), scan, getName(), connection, rpcCallerFactory, rpcControllerFactory, pool, scanReadRpcTimeout, scanTimeout, - replicaTimeout); + replicaTimeout, requestAttributes); } else { return new ClientSimpleScanner(getConfiguration(), scan, getName(), connection, rpcCallerFactory, rpcControllerFactory, pool, scanReadRpcTimeout, scanTimeout, - replicaTimeout); + replicaTimeout, requestAttributes); } } } @@ -375,18 +379,19 @@ private Result get(Get get, final boolean checkExistenceOnly) throws IOException if (get.getConsistency() == Consistency.STRONG) { final Get configuredGet = get; - 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.GetResponse response = doGet(request); - return response == null - ? null - : ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); - } - }; + ClientServiceCallable callable = + new ClientServiceCallable(this.connection, getName(), get.getRow(), + this.rpcControllerFactory.newController(), get.getPriority(), requestAttributes) { + @Override + protected Result rpcCall() throws Exception { + ClientProtos.GetRequest request = RequestConverter + .buildGetRequest(getLocation().getRegionInfo().getRegionName(), configuredGet); + ClientProtos.GetResponse response = doGet(request); + return response == null + ? null + : ProtobufUtil.toResult(response.getResult(), getRpcControllerCellScanner()); + } + }; return rpcCallerFactory. newCaller(readRpcTimeoutMs).callWithRetries(callable, this.operationTimeoutMs); } @@ -453,10 +458,10 @@ public void batch(final List actions, final Object[] results) 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(); + AsyncProcessTask task = AsyncProcessTask.newBuilder().setPool(pool).setTableName(tableName) + .setRowAccess(actions).setResults(results).setRpcTimeout(rpcTimeout) + .setOperationTimeout(operationTimeoutMs).setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .setRequestAttributes(requestAttributes).build(); final Span span = new TableOperationSpanBuilder(connection).setTableName(tableName) .setOperation(HBaseSemanticAttributes.Operation.BATCH).setContainerOperations(actions) .build(); @@ -476,12 +481,20 @@ public void batch(final List actions, final Object[] results, int @Override public void batchCallback(final List actions, final Object[] results, final Batch.Callback callback) throws IOException, InterruptedException { - doBatchWithCallback(actions, results, callback, connection, pool, tableName); + doBatchWithCallback(actions, results, callback, connection, pool, tableName, requestAttributes); } public static void doBatchWithCallback(List actions, Object[] results, Batch.Callback callback, ClusterConnection connection, ExecutorService pool, TableName tableName) throws InterruptedIOException, RetriesExhaustedWithDetailsException { + doBatchWithCallback(actions, results, callback, connection, pool, tableName, + Collections.emptyMap()); + } + + public static void doBatchWithCallback(List actions, Object[] results, + Batch.Callback callback, ClusterConnection connection, ExecutorService pool, + TableName tableName, Map requestAttributes) + 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, @@ -489,7 +502,8 @@ public static void doBatchWithCallback(List actions, Object[] AsyncProcessTask task = AsyncProcessTask.newBuilder(callback).setPool(pool).setTableName(tableName) .setRowAccess(actions).setResults(results).setOperationTimeout(operationTimeout) - .setRpcTimeout(writeTimeout).setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).build(); + .setRpcTimeout(writeTimeout).setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .setRequestAttributes(requestAttributes).build(); final Span span = new TableOperationSpanBuilder(connection).setTableName(tableName) .setOperation(HBaseSemanticAttributes.Operation.BATCH).setContainerOperations(actions) .build(); @@ -512,7 +526,7 @@ public void delete(final Delete delete) throws IOException { TraceUtil.trace(() -> { ClientServiceCallable callable = new ClientServiceCallable(this.connection, getName(), delete.getRow(), - this.rpcControllerFactory.newController(), delete.getPriority()) { + this.rpcControllerFactory.newController(), delete.getPriority(), requestAttributes) { @Override protected Void rpcCall() throws Exception { MutateRequest request = RequestConverter @@ -553,16 +567,17 @@ public void put(final Put put) throws IOException { 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.buildMutateRequest(getLocation().getRegionInfo().getRegionName(), put); - doMutate(request); - return null; - } - }; + ClientServiceCallable callable = + new ClientServiceCallable(this.connection, getName(), put.getRow(), + this.rpcControllerFactory.newController(), put.getPriority(), requestAttributes) { + @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); }, supplier); @@ -592,7 +607,7 @@ public Result mutateRow(final RowMutations rm) throws IOException { CancellableRegionServerCallable callable = new CancellableRegionServerCallable(this.connection, getName(), rm.getRow(), rpcControllerFactory.newController(), writeRpcTimeoutMs, - new RetryingTimeTracker().start(), rm.getMaxPriority()) { + new RetryingTimeTracker().start(), rm.getMaxPriority(), requestAttributes) { @Override protected MultiResponse rpcCall() throws Exception { MultiRequest request = RequestConverter.buildMultiRequest( @@ -614,7 +629,8 @@ protected MultiResponse rpcCall() throws Exception { AsyncProcessTask task = AsyncProcessTask.newBuilder().setPool(pool).setTableName(tableName) .setRowAccess(rm.getMutations()).setCallable(callable).setRpcTimeout(writeRpcTimeoutMs) .setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).setResults(results).build(); + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).setResults(results) + .setRequestAttributes(requestAttributes).build(); AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); if (ars.hasError()) { @@ -640,7 +656,7 @@ public Result append(final Append append) throws IOException { checkHasFamilies(append); NoncedRegionServerCallable callable = new NoncedRegionServerCallable(this.connection, getName(), append.getRow(), - this.rpcControllerFactory.newController(), append.getPriority()) { + this.rpcControllerFactory.newController(), append.getPriority(), requestAttributes) { @Override protected Result rpcCall() throws Exception { MutateRequest request = @@ -666,7 +682,7 @@ public Result increment(final Increment increment) throws IOException { checkHasFamilies(increment); NoncedRegionServerCallable callable = new NoncedRegionServerCallable(this.connection, getName(), increment.getRow(), - this.rpcControllerFactory.newController(), increment.getPriority()) { + this.rpcControllerFactory.newController(), increment.getPriority(), requestAttributes) { @Override protected Result rpcCall() throws Exception { MutateRequest request = @@ -706,7 +722,7 @@ public long incrementColumnValue(final byte[] row, final byte[] family, final by NoncedRegionServerCallable callable = new NoncedRegionServerCallable(this.connection, getName(), row, - this.rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) { + this.rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET, requestAttributes) { @Override protected Long rpcCall() throws Exception { MutateRequest request = RequestConverter.buildIncrementRequest( @@ -819,7 +835,7 @@ private CheckAndMutateResult doCheckAndMutate(final byte[] row, final byte[] fam CancellableRegionServerCallable callable = new CancellableRegionServerCallable(connection, getName(), rm.getRow(), rpcControllerFactory.newController(), writeRpcTimeoutMs, new RetryingTimeTracker().start(), - rm.getMaxPriority()) { + rm.getMaxPriority(), requestAttributes) { @Override protected MultiResponse rpcCall() throws Exception { MultiRequest request = @@ -849,7 +865,7 @@ protected MultiResponse rpcCall() throws Exception { // TODO any better timeout? .setRpcTimeout(Math.max(readRpcTimeoutMs, writeRpcTimeoutMs)) .setOperationTimeout(operationTimeoutMs).setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) - .build(); + .setRequestAttributes(requestAttributes).build(); AsyncRequestFuture ars = multiAp.submit(task); ars.waitUntilDone(); if (ars.hasError()) { @@ -913,7 +929,7 @@ private CheckAndMutateResult doCheckAndMutate(final byte[] row, final byte[] fam long nonce = getNonce(); ClientServiceCallable callable = new ClientServiceCallable(this.connection, getName(), row, - this.rpcControllerFactory.newController(), mutation.getPriority()) { + this.rpcControllerFactory.newController(), mutation.getPriority(), requestAttributes) { @Override protected CheckAndMutateResult rpcCall() throws Exception { MutateRequest request = @@ -1334,7 +1350,8 @@ public void batchCoprocessorService( AsyncProcessTask.newBuilder(resultsCallback).setPool(context.wrap(pool)) .setTableName(tableName).setRowAccess(execs).setResults(results) .setRpcTimeout(readRpcTimeoutMs).setOperationTimeout(operationTimeoutMs) - .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL).build(); + .setSubmittedRows(AsyncProcessTask.SubmittedRows.ALL) + .setRequestAttributes(requestAttributes).build(); AsyncRequestFuture future = asyncProcess.submit(task); future.waitUntilDone(); @@ -1433,6 +1450,11 @@ public boolean thenMutate(RowMutations mutation) throws IOException { } } + @Override + public Map getRequestAttributes() { + return requestAttributes; + } + private class CheckAndMutateWithFilterBuilderImpl implements CheckAndMutateWithFilterBuilder { private final byte[] row; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java index e81be5bcb253..e5bc90612d7f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java @@ -434,7 +434,8 @@ public FlushWorker(Configuration conf, ClusterConnection conn, HRegionLocation a HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); // Specify 0 retries in AsyncProcess because we need to reassign puts to different queues // if regions are moved. - this.ap = new AsyncProcess(conn, conf, rpcCallerFactory, rpcControllerFactory, 0); + this.ap = new AsyncProcess(conn, conf, rpcCallerFactory, rpcControllerFactory, 0, + Collections.emptyMap()); this.executor = executor; this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000); this.pool = pool; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java index 86b499ea5673..6ba0832b26e5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java @@ -52,8 +52,9 @@ class MultiServerCallable extends CancellableRegionServerCallable MultiServerCallable(final ClusterConnection connection, final TableName tableName, final ServerName location, final MultiAction multi, RpcController rpcController, int rpcTimeout, - RetryingTimeTracker tracker, int priority) { - super(connection, tableName, null, rpcController, rpcTimeout, tracker, priority); + RetryingTimeTracker tracker, int priority, Map requestAttributes) { + super(connection, tableName, null, rpcController, rpcTimeout, tracker, priority, + requestAttributes); this.multiAction = multi; // RegionServerCallable has HRegionLocation field, but this is a multi-region request. // Using region info from parent HRegionLocation would be a mistake for this class; so diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java index 7d0e2979f773..1355c529e2dc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NoncedRegionServerCallable.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.client; +import java.util.Map; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.yetus.audience.InterfaceAudience; @@ -45,8 +46,8 @@ public abstract class NoncedRegionServerCallable extends ClientServiceCallabl * @param row The row we want in tableName. */ public NoncedRegionServerCallable(Connection connection, TableName tableName, byte[] row, - HBaseRpcController rpcController, int priority) { - super(connection, tableName, row, rpcController, priority); + HBaseRpcController rpcController, int priority, Map requestAttributes) { + super(connection, tableName, row, rpcController, priority, requestAttributes); this.nonce = getConnection().getNonceGenerator().newNonce(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java index 0f541c559441..ab0edd8d83da 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java @@ -23,6 +23,7 @@ import io.opentelemetry.context.Context; import io.opentelemetry.context.Scope; import java.io.IOException; +import java.util.Collections; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; @@ -78,7 +79,8 @@ protected Message callExecService(final RpcController controller, final Context context = Context.current(); ClientServiceCallable callable = new ClientServiceCallable(this.conn, this.table, this.row, - this.conn.getRpcControllerFactory().newController(), HConstants.PRIORITY_UNSET) { + this.conn.getRpcControllerFactory().newController(), HConstants.PRIORITY_UNSET, + Collections.emptyMap()) { @Override protected CoprocessorServiceResponse rpcCall() throws Exception { try (Scope ignored = context.makeCurrent()) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java index 8b6e6e46d175..68a8e7b74067 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedClientScanner.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForReverseScan; import java.io.IOException; +import java.util.Map; import java.util.concurrent.ExecutorService; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; @@ -39,9 +40,10 @@ public class ReversedClientScanner extends ClientScanner { public ReversedClientScanner(Configuration conf, Scan scan, TableName tableName, ClusterConnection connection, RpcRetryingCallerFactory rpcFactory, RpcControllerFactory controllerFactory, ExecutorService pool, int scanReadRpcTimeout, - int scannerTimeout, int primaryOperationTimeout) throws IOException { + int scannerTimeout, int primaryOperationTimeout, Map requestAttributes) + throws IOException { super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, - scanReadRpcTimeout, scannerTimeout, primaryOperationTimeout); + scanReadRpcTimeout, scannerTimeout, primaryOperationTimeout, requestAttributes); } @Override @@ -56,6 +58,6 @@ protected boolean setNewStartKey() { @Override protected ReversedScannerCallable createScannerCallable() { return new ReversedScannerCallable(getConnection(), getTable(), scan, this.scanMetrics, - this.rpcControllerFactory, getScanReplicaId()); + this.rpcControllerFactory, getScanReplicaId(), requestAttributes); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java index 59dcf79bbfc7..ba215450e5af 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ReversedScannerCallable.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.InterruptedIOException; +import java.util.Map; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -54,8 +55,9 @@ public class ReversedScannerCallable extends ScannerCallable { * @param replicaId the replica id */ public ReversedScannerCallable(ClusterConnection connection, TableName tableName, Scan scan, - ScanMetrics scanMetrics, RpcControllerFactory rpcFactory, int replicaId) { - super(connection, tableName, scan, scanMetrics, rpcFactory, replicaId); + ScanMetrics scanMetrics, RpcControllerFactory rpcFactory, int replicaId, + Map requestAttributes) { + super(connection, tableName, scan, scanMetrics, rpcFactory, replicaId, requestAttributes); } @Override @@ -162,7 +164,7 @@ private Pair locateLastRegionInRange(byte[] startKey, b @Override public ScannerCallable getScannerCallableForReplica(int id) { ReversedScannerCallable r = new ReversedScannerCallable(getConnection(), getTableName(), - this.getScan(), this.scanMetrics, rpcControllerFactory, id); + this.getScan(), this.scanMetrics, rpcControllerFactory, id, requestAttributes); r.setCaching(this.getCaching()); return r; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java index 30718742ef76..205f611577a6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java @@ -95,8 +95,8 @@ class ReplicaRegionServerCallable extends CancellableRegionServerCallable requestAttributes) { super(connection, tableName, scan.getStartRow(), rpcControllerFactory.newController(), - scan.getPriority()); + scan.getPriority(), requestAttributes); this.id = id; this.scan = scan; this.scanMetrics = scanMetrics; @@ -423,7 +425,7 @@ public void setCaching(int caching) { public ScannerCallable getScannerCallableForReplica(int id) { ScannerCallable s = new ScannerCallable(this.getConnection(), getTableName(), this.getScan(), - this.scanMetrics, this.rpcControllerFactory, id); + this.scanMetrics, this.rpcControllerFactory, id, requestAttributes); s.setCaching(this.caching); return s; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java index 825a58e7bdd5..d056f2fbee7d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SecureBulkLoadClient.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hbase.HConstants.PRIORITY_UNSET; import java.io.IOException; +import java.util.Collections; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; @@ -56,7 +57,7 @@ public String prepareBulkLoad(final Connection conn) throws IOException { try { ClientServiceCallable callable = new ClientServiceCallable(conn, table.getName(), HConstants.EMPTY_START_ROW, - this.rpcControllerFactory.newController(), PRIORITY_UNSET) { + this.rpcControllerFactory.newController(), PRIORITY_UNSET, Collections.emptyMap()) { @Override protected String rpcCall() throws Exception { byte[] regionName = getLocation().getRegionInfo().getRegionName(); @@ -78,19 +79,20 @@ protected String rpcCall() throws Exception { public void cleanupBulkLoad(final Connection conn, final String bulkToken) throws IOException { try { - ClientServiceCallable callable = new ClientServiceCallable(conn, table.getName(), - HConstants.EMPTY_START_ROW, this.rpcControllerFactory.newController(), PRIORITY_UNSET) { - @Override - protected Void rpcCall() throws Exception { - byte[] regionName = getLocation().getRegionInfo().getRegionName(); - RegionSpecifier region = - RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName); - CleanupBulkLoadRequest request = - CleanupBulkLoadRequest.newBuilder().setRegion(region).setBulkToken(bulkToken).build(); - getStub().cleanupBulkLoad(null, request); - return null; - } - }; + ClientServiceCallable callable = + new ClientServiceCallable(conn, table.getName(), HConstants.EMPTY_START_ROW, + this.rpcControllerFactory.newController(), PRIORITY_UNSET, Collections.emptyMap()) { + @Override + protected Void rpcCall() throws Exception { + byte[] regionName = getLocation().getRegionInfo().getRegionName(); + RegionSpecifier region = + RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName); + CleanupBulkLoadRequest request = + CleanupBulkLoadRequest.newBuilder().setRegion(region).setBulkToken(bulkToken).build(); + getStub().cleanupBulkLoad(null, request); + return null; + } + }; RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null). newCaller() .callWithRetries(callable, Integer.MAX_VALUE); } catch (Throwable throwable) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java index 6f8c65d6bdce..1b7a8ab81f18 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java @@ -31,6 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; /** * Get instances via {@link RpcControllerFactory} on client-side. @@ -281,4 +282,16 @@ public String toString() { + exception + ", regionInfo=" + regionInfo + ", priority=" + priority + ", cellScanner=" + cellScanner + '}'; } + + public static RpcController configureRequestAttributes(RpcController rpcController, + Map requestAttributes) { // todo rmattingly delete? + if ( + !requestAttributes.isEmpty() && rpcController != null + && rpcController instanceof HBaseRpcController + ) { + HBaseRpcController controller = (HBaseRpcController) rpcController; + controller.setRequestAttributes(requestAttributes); + } + return rpcController; + } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java index b0ddd83efd37..61da7ee27e9c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java @@ -185,14 +185,14 @@ public TableName getTableName() { public MyAsyncProcess(ClusterConnection hc, Configuration conf) { super(hc, conf, new RpcRetryingCallerFactory(conf, hc.getConnectionConfiguration()), - new RpcControllerFactory(conf)); + new RpcControllerFactory(conf), Collections.emptyMap()); service = Executors.newFixedThreadPool(5); this.conf = conf; } public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) { super(hc, conf, new RpcRetryingCallerFactory(conf, hc.getConnectionConfiguration()), - new RpcControllerFactory(conf)); + new RpcControllerFactory(conf), Collections.emptyMap()); service = new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, new SynchronousQueue<>(), new CountingThreadFactory(nbThreads)); } @@ -273,7 +273,7 @@ static class MyAsyncRequestFutureImpl extends AsyncRequestFutureImpl { public MyAsyncRequestFutureImpl(AsyncProcessTask task, List actions, long nonceGroup, AsyncProcess asyncProcess) { - super(task, actions, nonceGroup, asyncProcess); + super(task, actions, nonceGroup, asyncProcess, Collections.emptyMap()); } @Override @@ -1705,7 +1705,7 @@ public Future submit(Runnable runnable) { static class AsyncProcessForThrowableCheck extends AsyncProcess { public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf) { super(hc, conf, new RpcRetryingCallerFactory(conf, hc.getConnectionConfiguration()), - new RpcControllerFactory(conf)); + new RpcControllerFactory(conf), Collections.emptyMap()); } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java index e63215c8ea54..0f38124ff4a5 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java @@ -199,7 +199,7 @@ private static class MyAsyncProcess extends AsyncProcess { MyAsyncProcess(ClusterConnection hc, Configuration conf) { super(hc, conf, new RpcRetryingCallerFactory(conf, hc.getConnectionConfiguration()), - new RpcControllerFactory(conf)); + new RpcControllerFactory(conf), Collections.emptyMap()); } public AsyncRequestFuture submit(TableName tableName, List rows) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java index 16f4f687dfe1..0025a4fdbdb4 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientScanner.java @@ -30,6 +30,7 @@ import static org.mockito.Mockito.when; import java.io.IOException; +import java.util.Collections; import java.util.Iterator; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -111,7 +112,8 @@ public MockClientScanner(final Configuration conf, final Scan scan, final TableN throws IOException { super(conf, scan, tableName, connection, rpcFactory, controllerFactory, pool, HConstants.DEFAULT_HBASE_RPC_TIMEOUT, - HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, primaryOperationTimeout); + HConstants.DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, primaryOperationTimeout, + Collections.emptyMap()); } @Override diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java index e2782f6dab45..e9e1d7efa830 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestReversedScannerCallable.java @@ -24,6 +24,7 @@ import static org.mockito.Mockito.when; import java.io.IOException; +import java.util.Collections; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; @@ -82,8 +83,8 @@ public void setUp() throws Exception { public void testPrepareAlwaysUsesCache() throws Exception { when(connection.locateRegion(TABLE_NAME, ROW, true, true, 0)).thenReturn(regionLocations); - ReversedScannerCallable callable = - new ReversedScannerCallable(connection, TABLE_NAME, DEFAULT_SCAN, null, rpcFactory, 0); + ReversedScannerCallable callable = new ReversedScannerCallable(connection, TABLE_NAME, + DEFAULT_SCAN, null, rpcFactory, 0, Collections.emptyMap()); callable.prepare(false); callable.prepare(true); @@ -94,8 +95,8 @@ public void testPrepareAlwaysUsesCache() throws Exception { public void testHandleDisabledTable() throws IOException { when(connection.isTableDisabled(TABLE_NAME)).thenReturn(true); - ReversedScannerCallable callable = - new ReversedScannerCallable(connection, TABLE_NAME, DEFAULT_SCAN, null, rpcFactory, 0); + ReversedScannerCallable callable = new ReversedScannerCallable(connection, TABLE_NAME, + DEFAULT_SCAN, null, rpcFactory, 0, Collections.emptyMap()); assertThrows(TableNotEnabledException.class, () -> callable.prepare(true)); } @@ -116,8 +117,8 @@ public void testUpdateSearchKeyCacheLocation() throws IOException { .thenReturn(regionLocations); Scan scan = new Scan().setReversed(true); - ReversedScannerCallable callable = - new ReversedScannerCallable(connection, TABLE_NAME, scan, null, rpcFactory, 0); + ReversedScannerCallable callable = new ReversedScannerCallable(connection, TABLE_NAME, scan, + null, rpcFactory, 0, Collections.emptyMap()); callable.prepare(false); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScannerCallable.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScannerCallable.java index 83dd0e4efd06..e675b06b574b 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScannerCallable.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScannerCallable.java @@ -23,6 +23,7 @@ import static org.mockito.Mockito.when; import java.io.IOException; +import java.util.Collections; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionLocation; @@ -78,8 +79,8 @@ public void setUp() throws Exception { public void testPrepareAlwaysUsesCache() throws Exception { when(connection.locateRegion(TABLE_NAME, ROW, true, true, 0)).thenReturn(regionLocations); - ScannerCallable callable = - new ScannerCallable(connection, TABLE_NAME, DEFAULT_SCAN, null, rpcFactory, 0); + ScannerCallable callable = new ScannerCallable(connection, TABLE_NAME, DEFAULT_SCAN, null, + rpcFactory, 0, Collections.emptyMap()); callable.prepare(false); callable.prepare(true); @@ -90,8 +91,8 @@ public void testPrepareAlwaysUsesCache() throws Exception { public void testHandleDisabledTable() throws IOException { when(connection.isTableDisabled(TABLE_NAME)).thenReturn(true); - ScannerCallable callable = - new ScannerCallable(connection, TABLE_NAME, DEFAULT_SCAN, null, rpcFactory, 0); + ScannerCallable callable = new ScannerCallable(connection, TABLE_NAME, DEFAULT_SCAN, null, + rpcFactory, 0, Collections.emptyMap()); assertThrows(TableNotEnabledException.class, () -> callable.prepare(true)); } diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java index 5ffd7b92b79c..9d760a848564 100644 --- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java +++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldSecureEndpoint.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.fs.FileSystem; @@ -112,19 +113,20 @@ public void doAnAction() throws Exception { Table table = conn.getTable(tableName); final String bulkToken = new SecureBulkLoadEndpointClient(table).prepareBulkLoad(tableName); RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration()); - ClientServiceCallable callable = new ClientServiceCallable(conn, tableName, - Bytes.toBytes("aaa"), rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) { - @Override - protected Void rpcCall() throws Exception { - LOG.debug("Going to connect to server " + getLocation() + " for row " - + Bytes.toStringBinary(getRow())); - try (Table table = conn.getTable(getTableName())) { - new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths, null, bulkToken, - getLocation().getRegionInfo().getStartKey()); + ClientServiceCallable callable = + new ClientServiceCallable(conn, tableName, Bytes.toBytes("aaa"), + rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET, Collections.emptyMap()) { + @Override + protected Void rpcCall() throws Exception { + LOG.debug("Going to connect to server " + getLocation() + " for row " + + Bytes.toStringBinary(getRow())); + try (Table table = conn.getTable(getTableName())) { + new SecureBulkLoadEndpointClient(table).bulkLoadHFiles(famPaths, null, bulkToken, + getLocation().getRegionInfo().getStartKey()); + } + return null; } - return null; - } - }; + }; RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(conf, conn.getConnectionConfiguration()); RpcRetryingCaller caller = factory. newCaller(); @@ -134,7 +136,7 @@ protected Void rpcCall() throws Exception { if (numBulkLoads.get() % 5 == 0) { // 5 * 50 = 250 open file handles! callable = new ClientServiceCallable(conn, tableName, Bytes.toBytes("aaa"), - rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) { + rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET, Collections.emptyMap()) { @Override protected Void rpcCall() throws Exception { LOG.debug("compacting " + getLocation() + " for row " + Bytes.toStringBinary(getRow())); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java index 76b528cc9504..875556b11d81 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java @@ -549,7 +549,7 @@ protected ClientServiceCallable buildClientServiceCallable(Connection co lqis.stream().map(lqi -> Pair.newPair(lqi.getFamily(), lqi.getFilePath().toString())) .collect(Collectors.toList()); return new ClientServiceCallable(conn, tableName, first, - rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) { + rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET, Collections.emptyMap()) { @Override protected byte[] rpcCall() throws Exception { SecureBulkLoadClient secureClient = null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java index a1ff19c2faa6..9be9014d2c45 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import java.util.Collections; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; @@ -126,7 +127,7 @@ public static ClusterConnection getMockedConnectionAndDecorate(final Configurati Mockito.when(c.getNonceGenerator()).thenReturn(ng); AsyncProcess asyncProcess = new AsyncProcess(c, conf, RpcRetryingCallerFactory.instantiate(conf, connectionConfiguration, c.getConnectionMetrics()), - RpcControllerFactory.instantiate(conf)); + RpcControllerFactory.instantiate(conf), Collections.emptyMap()); Mockito.when(c.getAsyncProcess()).thenReturn(asyncProcess); Mockito.when(c.getNewRpcRetryingCallerFactory(conf)) .thenReturn(RpcRetryingCallerFactory.instantiate(conf, connectionConfiguration, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java index 7a37b22bdef1..ce7a872ae784 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCISleep.java @@ -21,6 +21,7 @@ import static org.junit.Assert.fail; import java.net.SocketTimeoutException; +import java.util.Collections; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; @@ -95,7 +96,7 @@ public void testCallableSleep() throws Exception { ClientServiceCallable regionServerCallable = new ClientServiceCallable(TEST_UTIL.getConnection(), tableName, FAM_NAM, new RpcControllerFactory(TEST_UTIL.getConfiguration()).newController(), - HConstants.PRIORITY_UNSET) { + HConstants.PRIORITY_UNSET, Collections.emptyMap()) { @Override protected Object rpcCall() throws Exception { return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index 1646a3b81a38..ce79a9771f72 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.concurrent.CountDownLatch; @@ -488,22 +489,24 @@ public void testBulkLoad() throws IOException { table = conn.getTable(hdt.getTableName()); final String bulkToken = new SecureBulkLoadClient(HTU.getConfiguration(), table).prepareBulkLoad(conn); - ClientServiceCallable callable = new ClientServiceCallable(conn, hdt.getTableName(), - TestHRegionServerBulkLoad.rowkey(0), - new RpcControllerFactory(HTU.getConfiguration()).newController(), HConstants.PRIORITY_UNSET) { - @Override - protected Void rpcCall() throws Exception { - LOG.debug("Going to connect to server " + getLocation() + " for row " - + Bytes.toStringBinary(getRow())); - SecureBulkLoadClient secureClient = null; - byte[] regionName = getLocation().getRegionInfo().getRegionName(); - try (Table table = conn.getTable(getTableName())) { - secureClient = new SecureBulkLoadClient(HTU.getConfiguration(), table); - secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName, true, null, bulkToken); + ClientServiceCallable callable = + new ClientServiceCallable(conn, hdt.getTableName(), TestHRegionServerBulkLoad.rowkey(0), + new RpcControllerFactory(HTU.getConfiguration()).newController(), HConstants.PRIORITY_UNSET, + Collections.emptyMap()) { + @Override + protected Void rpcCall() throws Exception { + LOG.debug("Going to connect to server " + getLocation() + " for row " + + Bytes.toStringBinary(getRow())); + SecureBulkLoadClient secureClient = null; + byte[] regionName = getLocation().getRegionInfo().getRegionName(); + try (Table table = conn.getTable(getTableName())) { + secureClient = new SecureBulkLoadClient(HTU.getConfiguration(), table); + secureClient.secureBulkLoadHFiles(getStub(), famPaths, regionName, true, null, + bulkToken); + } + return null; } - return null; - } - }; + }; RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(HTU.getConfiguration(), conn.getConnectionConfiguration()); RpcRetryingCaller caller = factory.newCaller(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java index bf427b166a43..793fdc5a1f91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAndConnectionAttributes.java @@ -224,6 +224,27 @@ public void testRequestAttributesMultiPut() throws IOException { assertTrue(REQUEST_ATTRIBUTES_VALIDATED.get()); } + @Test + public void testRequestAttributesCheckAndMutate() throws IOException { + assertFalse(REQUEST_ATTRIBUTES_VALIDATED.get()); + addRandomRequestAttributes(); + + Configuration conf = TEST_UTIL.getConfiguration(); + try ( + Connection conn = ConnectionFactory.createConnection(conf, null, AuthUtil.loginClient(conf), + CONNECTION_ATTRIBUTES); + Table table = configureRequestAttributes( + conn.getTableBuilder(REQUEST_ATTRIBUTES_TEST_TABLE, EXECUTOR_SERVICE)).build()) { + Put put = new Put(Bytes.toBytes("a")); + put.addColumn(REQUEST_ATTRIBUTES_TEST_TABLE_CF, Bytes.toBytes("c"), Bytes.toBytes("v")); + CheckAndMutate checkAndMutate = CheckAndMutate.newBuilder(Bytes.toBytes("a")) + .ifEquals(REQUEST_ATTRIBUTES_TEST_TABLE_CF, Bytes.toBytes("c"), Bytes.toBytes("v")) + .build(put); + table.checkAndMutate(checkAndMutate); + } + assertTrue(REQUEST_ATTRIBUTES_VALIDATED.get()); + } + @Test public void testNoRequestAttributes() throws IOException { assertFalse(REQUEST_ATTRIBUTES_VALIDATED.get()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java index 6e0c4b1a1b0e..e8693077ed5c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map.Entry; import java.util.Objects; @@ -354,7 +355,8 @@ ClientServiceCallable generateFileToLoad(TableName tn, int numFiles, int n Table table = conn.getTable(tn); final String bulkToken = new SecureBulkLoadClient(conf, table).prepareBulkLoad(conn); return new ClientServiceCallable(conn, tn, Bytes.toBytes("row"), - new RpcControllerFactory(conf).newController(), HConstants.PRIORITY_UNSET) { + new RpcControllerFactory(conf).newController(), HConstants.PRIORITY_UNSET, + Collections.emptyMap()) { @Override public Void rpcCall() throws Exception { SecureBulkLoadClient secureClient = null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java index 751d782f601a..c548b50a6c2a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java @@ -25,6 +25,7 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -215,7 +216,7 @@ public void doAnAction() throws Exception { new SecureBulkLoadClient(UTIL.getConfiguration(), table).prepareBulkLoad(conn); ClientServiceCallable callable = new ClientServiceCallable(conn, tableName, Bytes.toBytes("aaa"), new RpcControllerFactory(UTIL.getConfiguration()).newController(), - HConstants.PRIORITY_UNSET) { + HConstants.PRIORITY_UNSET, Collections.emptyMap()) { @Override public Void rpcCall() throws Exception { LOG.debug("Going to connect to server " + getLocation() + " for row " @@ -240,7 +241,7 @@ public Void rpcCall() throws Exception { // 5 * 50 = 250 open file handles! callable = new ClientServiceCallable(conn, tableName, Bytes.toBytes("aaa"), new RpcControllerFactory(UTIL.getConfiguration()).newController(), - HConstants.PRIORITY_UNSET) { + HConstants.PRIORITY_UNSET, Collections.emptyMap()) { @Override protected Void rpcCall() throws Exception { LOG.debug("compacting " + getLocation() + " for row " + Bytes.toStringBinary(getRow())); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java index b4e00a90df2a..3fade4cabdfd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoadWithOldClient.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.fs.FileSystem; @@ -100,18 +101,19 @@ public void doAnAction() throws Exception { // bulk load HFiles final ClusterConnection conn = (ClusterConnection) UTIL.getAdmin().getConnection(); RpcControllerFactory rpcControllerFactory = new RpcControllerFactory(UTIL.getConfiguration()); - ClientServiceCallable callable = new ClientServiceCallable(conn, tableName, - Bytes.toBytes("aaa"), rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) { - @Override - protected Void rpcCall() throws Exception { - LOG.info("Non-secure old client"); - byte[] regionName = getLocation().getRegionInfo().getRegionName(); - BulkLoadHFileRequest request = - RequestConverter.buildBulkLoadHFileRequest(famPaths, regionName, true, null, null); - getStub().bulkLoadHFile(null, request); - return null; - } - }; + ClientServiceCallable callable = + new ClientServiceCallable(conn, tableName, Bytes.toBytes("aaa"), + rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET, Collections.emptyMap()) { + @Override + protected Void rpcCall() throws Exception { + LOG.info("Non-secure old client"); + byte[] regionName = getLocation().getRegionInfo().getRegionName(); + BulkLoadHFileRequest request = + RequestConverter.buildBulkLoadHFileRequest(famPaths, regionName, true, null, null); + getStub().bulkLoadHFile(null, request); + return null; + } + }; RpcRetryingCallerFactory factory = new RpcRetryingCallerFactory(conf, conn.getConnectionConfiguration()); RpcRetryingCaller caller = factory. newCaller(); @@ -121,7 +123,7 @@ protected Void rpcCall() throws Exception { if (numBulkLoads.get() % 5 == 0) { // 5 * 50 = 250 open file handles! callable = new ClientServiceCallable(conn, tableName, Bytes.toBytes("aaa"), - rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET) { + rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET, Collections.emptyMap()) { @Override protected Void rpcCall() throws Exception { LOG.debug("compacting " + getLocation() + " for row " + Bytes.toStringBinary(getRow())); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java index e76f6ce5f16a..485bf3a81a74 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestLoadIncrementalHFilesSplitRecovery.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collection; +import java.util.Collections; import java.util.Deque; import java.util.List; import java.util.Map; @@ -341,7 +342,7 @@ protected ClientServiceCallable buildClientServiceCallable(Connection co calls.getAndIncrement(); return new ClientServiceCallable(conn, tableName, first, new RpcControllerFactory(util.getConfiguration()).newController(), - HConstants.PRIORITY_UNSET) { + HConstants.PRIORITY_UNSET, Collections.emptyMap()) { @Override public byte[] rpcCall() throws Exception { throw new IOException("Error calling something on RegionServer"); From 19f53ba18c43b92f3c0650195e772bb9d765727b Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Thu, 27 Jul 2023 11:44:49 -0400 Subject: [PATCH 4/6] pr feedback --- .../apache/hadoop/hbase/client/AsyncProcess.java | 3 +-- .../hadoop/hbase/client/AsyncProcessTask.java | 2 +- .../hadoop/hbase/client/AsyncRequestFutureImpl.java | 6 ++++-- .../hadoop/hbase/client/BufferedMutatorImpl.java | 3 +-- .../hadoop/hbase/client/ClientServiceCallable.java | 7 ++----- .../hbase/client/ConnectionImplementation.java | 3 +-- .../java/org/apache/hadoop/hbase/client/HTable.java | 13 +++---------- .../hadoop/hbase/client/HTableMultiplexer.java | 3 +-- .../hbase/client/RegionCoprocessorRpcChannel.java | 9 ++++++--- .../hadoop/hbase/client/RegionServerCallable.java | 9 ++++++--- .../client/RpcRetryingCallerWithReadReplicas.java | 8 ++++++-- .../hadoop/hbase/client/TestAsyncProcess.java | 8 ++++---- .../client/TestAsyncProcessWithRegionException.java | 2 +- .../hadoop/hbase/client/TestClientNoCluster.java | 6 +++--- .../apache/hadoop/hbase/util/MultiHConnection.java | 4 +++- .../hbase/client/HConnectionTestingUtility.java | 3 +-- 16 files changed, 44 insertions(+), 45 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java index 6518039f183c..7a4ed3f5b365 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java @@ -434,8 +434,7 @@ private int checkRpcTimeout(int rpcTimeout) { AsyncRequestFutureImpl createAsyncRequestFuture(AsyncProcessTask task, List actions, long nonceGroup) { - return new AsyncRequestFutureImpl<>(task, actions, nonceGroup, this, - task.getRequestAttributes()); + return new AsyncRequestFutureImpl<>(task, actions, nonceGroup, this); } /** Wait until the async does not have more than max tasks in progress. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java index fde132cb8a93..97cdcb9d73cc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcessTask.java @@ -127,7 +127,7 @@ Builder setCallable(CancellableRegionServerCallable callable) { return this; } - Builder setRequestAttributes(Map setRequestAttributes) { + Builder setRequestAttributes(Map requestAttributes) { this.requestAttributes = requestAttributes; return this; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java index 62f41cb3a7a4..dc150724504e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java @@ -284,6 +284,7 @@ public void run() { private final int operationTimeout; private final int rpcTimeout; private final AsyncProcess asyncProcess; + private final Map requestAttributes; /** * For {@link AsyncRequestFutureImpl#manageError(int, Row, Retry, Throwable, ServerName)}. Only @@ -323,7 +324,7 @@ public String toString() { } public AsyncRequestFutureImpl(AsyncProcessTask task, List actions, long nonceGroup, - AsyncProcess asyncProcess, Map requestAttributes) { + AsyncProcess asyncProcess) { this.pool = task.getPool(); this.callback = task.getCallback(); this.nonceGroup = nonceGroup; @@ -398,6 +399,7 @@ public AsyncRequestFutureImpl(AsyncProcessTask task, List actions, long if (task.getCallable() == null) { tracker = new RetryingTimeTracker().start(); } + this.requestAttributes = task.getRequestAttributes(); } protected Set getCallsInProgress() { @@ -1317,7 +1319,7 @@ private MultiServerCallable createCallable(final ServerName server, TableName ta final MultiAction multi) { return new MultiServerCallable(asyncProcess.connection, tableName, server, multi, asyncProcess.rpcFactory.newController(), rpcTimeout, tracker, multi.getPriority(), - asyncProcess.getRequestAttributes()); + requestAttributes); } private void updateResult(int index, Object result) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java index 93fd12f7aab8..4c3c94a35c2e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java @@ -143,8 +143,7 @@ public class BufferedMutatorImpl implements BufferedMutator { this(conn, params, // puts need to track errors globally due to how the APIs currently work. // todo rmattingly support buffered mutator request attributes - new AsyncProcess(conn, conn.getConfiguration(), rpcCallerFactory, rpcFactory, - Collections.emptyMap())); + new AsyncProcess(conn, conn.getConfiguration(), rpcCallerFactory, rpcFactory)); } private void checkClose() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java index 5c63ff11ffcb..a669456911e5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java @@ -36,14 +36,11 @@ public abstract class ClientServiceCallable extends RegionServerCallable { - protected Map requestAttributes; - public ClientServiceCallable(Connection connection, TableName tableName, byte[] row, RpcController rpcController, int priority, Map requestAttributes) { super(connection, tableName, row, - HBaseRpcControllerImpl.configureRequestAttributes(rpcController, requestAttributes), - priority); - this.requestAttributes = requestAttributes; + HBaseRpcControllerImpl.configureRequestAttributes(rpcController, requestAttributes), priority, + requestAttributes); } @Override 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 1257cba1030f..7d8e7038e869 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 @@ -371,8 +371,7 @@ public class ConnectionImplementation implements ClusterConnection, Closeable { this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf, connectionConfig, interceptor, this.stats, this.metrics); - this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory, - Collections.emptyMap()); + this.asyncProcess = new AsyncProcess(this, conf, rpcCallerFactory, rpcControllerFactory); // Do we publish the status? if (shouldListen) { 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 96fa640e8e4b..eec0e912d036 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 @@ -400,7 +400,7 @@ protected Result rpcCall() throws Exception { RpcRetryingCallerWithReadReplicas callable = new RpcRetryingCallerWithReadReplicas(rpcControllerFactory, tableName, this.connection, get, pool, connConfiguration.getRetriesNumber(), operationTimeoutMs, readRpcTimeoutMs, - connConfiguration.getPrimaryCallTimeoutMicroSecond()); + connConfiguration.getPrimaryCallTimeoutMicroSecond(), requestAttributes); return callable.call(operationTimeoutMs); } @@ -484,13 +484,6 @@ public void batchCallback(final List actions, final Object[] doBatchWithCallback(actions, results, callback, connection, pool, tableName, requestAttributes); } - public static void doBatchWithCallback(List actions, Object[] results, - Batch.Callback callback, ClusterConnection connection, ExecutorService pool, - TableName tableName) throws InterruptedIOException, RetriesExhaustedWithDetailsException { - doBatchWithCallback(actions, results, callback, connection, pool, tableName, - Collections.emptyMap()); - } - public static void doBatchWithCallback(List actions, Object[] results, Batch.Callback callback, ClusterConnection connection, ExecutorService pool, TableName tableName, Map requestAttributes) @@ -1118,7 +1111,7 @@ public void clearRegionCache() { @Override public CoprocessorRpcChannel coprocessorService(byte[] row) { - return new RegionCoprocessorRpcChannel(connection, tableName, row); + return new RegionCoprocessorRpcChannel(connection, tableName, row, requestAttributes); } @Override @@ -1149,7 +1142,7 @@ public void coprocessorService(final Class service, by 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, requestAttributes); Future future = wrappedPool.submit(() -> { T instance = org.apache.hadoop.hbase.protobuf.ProtobufUtil.newServiceStub(service, channel); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java index e5bc90612d7f..e81be5bcb253 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableMultiplexer.java @@ -434,8 +434,7 @@ public FlushWorker(Configuration conf, ClusterConnection conn, HRegionLocation a HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT); // Specify 0 retries in AsyncProcess because we need to reassign puts to different queues // if regions are moved. - this.ap = new AsyncProcess(conn, conf, rpcCallerFactory, rpcControllerFactory, 0, - Collections.emptyMap()); + this.ap = new AsyncProcess(conn, conf, rpcCallerFactory, rpcControllerFactory, 0); this.executor = executor; this.maxRetryInQueue = conf.getInt(TABLE_MULTIPLEXER_MAX_RETRIES_IN_QUEUE, 10000); this.pool = pool; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java index ab0edd8d83da..b12fcef3ea9a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorRpcChannel.java @@ -23,7 +23,7 @@ import io.opentelemetry.context.Context; import io.opentelemetry.context.Scope; import java.io.IOException; -import java.util.Collections; +import java.util.Map; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; @@ -51,6 +51,7 @@ class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel { private byte[] lastRegion; private final int operationTimeout; private final RpcRetryingCallerFactory rpcCallerFactory; + private final Map requestAttributes; /** * Constructor @@ -58,12 +59,14 @@ class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel { * @param table to connect to * @param row to locate region with */ - RegionCoprocessorRpcChannel(ClusterConnection conn, TableName table, byte[] row) { + RegionCoprocessorRpcChannel(ClusterConnection conn, TableName table, byte[] row, + Map requestAttributes) { this.table = table; this.row = row; this.conn = conn; this.operationTimeout = conn.getConnectionConfiguration().getOperationTimeout(); this.rpcCallerFactory = conn.getRpcRetryingCallerFactory(); + this.requestAttributes = requestAttributes; } @Override @@ -80,7 +83,7 @@ protected Message callExecService(final RpcController controller, ClientServiceCallable callable = new ClientServiceCallable(this.conn, this.table, this.row, this.conn.getRpcControllerFactory().newController(), HConstants.PRIORITY_UNSET, - Collections.emptyMap()) { + requestAttributes) { @Override protected CoprocessorServiceResponse rpcCall() throws Exception { try (Scope ignored = context.makeCurrent()) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java index a97c6da80ae3..28a3e2661bec 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import java.util.Map; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -65,6 +66,7 @@ public abstract class RegionServerCallable implements RetryingCallable */ protected final RpcController rpcController; private int priority = HConstants.NORMAL_QOS; + protected final Map requestAttributes; /** * @param connection Connection to use. @@ -73,18 +75,19 @@ public abstract class RegionServerCallable implements RetryingCallable * @param row The row we want in tableName. */ public RegionServerCallable(Connection connection, TableName tableName, byte[] row, - RpcController rpcController) { - this(connection, tableName, row, rpcController, HConstants.NORMAL_QOS); + RpcController rpcController, Map requestAttributes) { + this(connection, tableName, row, rpcController, HConstants.NORMAL_QOS, requestAttributes); } public RegionServerCallable(Connection connection, TableName tableName, byte[] row, - RpcController rpcController, int priority) { + RpcController rpcController, int priority, Map requestAttributes) { super(); this.connection = connection; this.tableName = tableName; this.row = row; this.rpcController = rpcController; this.priority = priority; + this.requestAttributes = requestAttributes; } protected RpcController getRpcController() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java index 205f611577a6..d24312b043c1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerWithReadReplicas.java @@ -23,6 +23,7 @@ import java.io.InterruptedIOException; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -67,10 +68,12 @@ public class RpcRetryingCallerWithReadReplicas { private final int retries; private final RpcControllerFactory rpcControllerFactory; private final RpcRetryingCallerFactory rpcRetryingCallerFactory; + private final Map requestAttributes; public RpcRetryingCallerWithReadReplicas(RpcControllerFactory rpcControllerFactory, TableName tableName, ClusterConnection cConnection, final Get get, ExecutorService pool, - int retries, int operationTimeout, int rpcTimeout, int timeBeforeReplicas) { + int retries, int operationTimeout, int rpcTimeout, int timeBeforeReplicas, + Map requestAttributes) { this.rpcControllerFactory = rpcControllerFactory; this.tableName = tableName; this.cConnection = cConnection; @@ -83,6 +86,7 @@ public RpcRetryingCallerWithReadReplicas(RpcControllerFactory rpcControllerFacto this.timeBeforeReplicas = timeBeforeReplicas; this.rpcRetryingCallerFactory = new RpcRetryingCallerFactory(conf, cConnection.getConnectionConfiguration()); + this.requestAttributes = requestAttributes; } /** @@ -96,7 +100,7 @@ public ReplicaRegionServerCallable(int id, HRegionLocation location) { super(RpcRetryingCallerWithReadReplicas.this.cConnection, RpcRetryingCallerWithReadReplicas.this.tableName, get.getRow(), rpcControllerFactory.newController(), rpcTimeout, new RetryingTimeTracker(), PRIORITY_UNSET, - Collections.emptyMap()); + RpcRetryingCallerWithReadReplicas.this.requestAttributes); this.id = id; this.location = location; } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java index 61da7ee27e9c..b0ddd83efd37 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java @@ -185,14 +185,14 @@ public TableName getTableName() { public MyAsyncProcess(ClusterConnection hc, Configuration conf) { super(hc, conf, new RpcRetryingCallerFactory(conf, hc.getConnectionConfiguration()), - new RpcControllerFactory(conf), Collections.emptyMap()); + new RpcControllerFactory(conf)); service = Executors.newFixedThreadPool(5); this.conf = conf; } public MyAsyncProcess(ClusterConnection hc, Configuration conf, AtomicInteger nbThreads) { super(hc, conf, new RpcRetryingCallerFactory(conf, hc.getConnectionConfiguration()), - new RpcControllerFactory(conf), Collections.emptyMap()); + new RpcControllerFactory(conf)); service = new ThreadPoolExecutor(1, 20, 60, TimeUnit.SECONDS, new SynchronousQueue<>(), new CountingThreadFactory(nbThreads)); } @@ -273,7 +273,7 @@ static class MyAsyncRequestFutureImpl extends AsyncRequestFutureImpl { public MyAsyncRequestFutureImpl(AsyncProcessTask task, List actions, long nonceGroup, AsyncProcess asyncProcess) { - super(task, actions, nonceGroup, asyncProcess, Collections.emptyMap()); + super(task, actions, nonceGroup, asyncProcess); } @Override @@ -1705,7 +1705,7 @@ public Future submit(Runnable runnable) { static class AsyncProcessForThrowableCheck extends AsyncProcess { public AsyncProcessForThrowableCheck(ClusterConnection hc, Configuration conf) { super(hc, conf, new RpcRetryingCallerFactory(conf, hc.getConnectionConfiguration()), - new RpcControllerFactory(conf), Collections.emptyMap()); + new RpcControllerFactory(conf)); } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java index 0f38124ff4a5..e63215c8ea54 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcessWithRegionException.java @@ -199,7 +199,7 @@ private static class MyAsyncProcess extends AsyncProcess { MyAsyncProcess(ClusterConnection hc, Configuration conf) { super(hc, conf, new RpcRetryingCallerFactory(conf, hc.getConnectionConfiguration()), - new RpcControllerFactory(conf), Collections.emptyMap()); + new RpcControllerFactory(conf)); } public AsyncRequestFuture submit(TableName tableName, List rows) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index e6b27d53d346..7d68c714d17d 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -403,9 +403,9 @@ static class ManyServersManyRegionsConnection extends ConnectionImplementation { final AtomicLong sequenceids = new AtomicLong(0); private final Configuration conf; - ManyServersManyRegionsConnection(Configuration conf, ExecutorService pool, User user) - throws IOException { - super(conf, pool, user); + ManyServersManyRegionsConnection(Configuration conf, ExecutorService pool, User user, + Map requestAttributes) throws IOException { + super(conf, pool, user, requestAttributes); int serverCount = conf.getInt("hbase.test.servers", 10); this.serversByClient = new HashMap<>(serverCount); this.meta = diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java index a7dd67d6ce1c..d910f7aa2a7a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MultiHConnection.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.util; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; @@ -117,7 +118,8 @@ public void processBatchCallback(List actions, TableName tabl ClusterConnection conn = (ClusterConnection) connections[ThreadLocalRandom.current().nextInt(noOfConnections)]; - HTable.doBatchWithCallback(actions, results, callback, conn, batchPool, tableName); + HTable.doBatchWithCallback(actions, results, callback, conn, batchPool, tableName, + Collections.emptyMap()); } // Copied from ConnectionImplementation.getBatchPool() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java index 9be9014d2c45..a1ff19c2faa6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; -import java.util.Collections; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; @@ -127,7 +126,7 @@ public static ClusterConnection getMockedConnectionAndDecorate(final Configurati Mockito.when(c.getNonceGenerator()).thenReturn(ng); AsyncProcess asyncProcess = new AsyncProcess(c, conf, RpcRetryingCallerFactory.instantiate(conf, connectionConfiguration, c.getConnectionMetrics()), - RpcControllerFactory.instantiate(conf), Collections.emptyMap()); + RpcControllerFactory.instantiate(conf)); Mockito.when(c.getAsyncProcess()).thenReturn(asyncProcess); Mockito.when(c.getNewRpcRetryingCallerFactory(conf)) .thenReturn(RpcRetryingCallerFactory.instantiate(conf, connectionConfiguration, From af77aad2c75e952fd8011a7cdeb292b3620aa313 Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Thu, 27 Jul 2023 17:00:01 -0400 Subject: [PATCH 5/6] fix ConnectionImplementation constructor, mostly relates to tests --- .../hadoop/hbase/client/ConnectionUtils.java | 6 +- .../hadoop/hbase/client/TestAsyncProcess.java | 2 +- .../hbase/client/TestClientNoCluster.java | 15 ++--- .../TestConnectionImplementationTracing.java | 4 +- .../hbase/client/TestHTableTracing.java | 56 +++++++++---------- .../client/TestRegionLocatorTracing.java | 4 +- .../client/HConnectionTestingUtility.java | 3 +- .../regionserver/TestWALEntrySinkFilter.java | 3 +- 8 files changed, 51 insertions(+), 42 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java index 9b44c682b4a5..b9682a942da1 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java @@ -27,6 +27,7 @@ import java.net.UnknownHostException; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -140,8 +141,9 @@ public static void setupMasterlessConnection(Configuration conf) { * region re-lookups. */ static class MasterlessConnection extends ConnectionImplementation { - MasterlessConnection(Configuration conf, ExecutorService pool, User user) throws IOException { - super(conf, pool, user); + MasterlessConnection(Configuration conf, ExecutorService pool, User user, + Map requestAttributes) throws IOException { + super(conf, pool, user, requestAttributes); } @Override diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java index b0ddd83efd37..8181cc5601eb 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java @@ -526,7 +526,7 @@ public CompletableFuture getClusterId() { final AtomicInteger nbThreads = new AtomicInteger(0); protected MyConnectionImpl(Configuration conf) throws IOException { - super(setupConf(conf), null, null); + super(setupConf(conf), null, null, Collections.emptyMap()); } private static Configuration setupConf(Configuration conf) { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index 7d68c714d17d..70a127d27b8c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -329,9 +329,9 @@ public void testConnectionClosedOnRegionLocate() throws IOException { static class RegionServerStoppedOnScannerOpenConnection extends ConnectionImplementation { final ClientService.BlockingInterface stub; - RegionServerStoppedOnScannerOpenConnection(Configuration conf, ExecutorService pool, User user) - throws IOException { - super(conf, pool, user); + RegionServerStoppedOnScannerOpenConnection(Configuration conf, ExecutorService pool, User user, + Map requestAttributes) throws IOException { + super(conf, pool, user, requestAttributes); // Mock up my stub so open scanner returns a scanner id and then on next, we throw // exceptions for three times and then after that, we return no more to scan. this.stub = Mockito.mock(ClientService.BlockingInterface.class); @@ -360,8 +360,9 @@ public BlockingInterface getClient(ServerName sn) throws IOException { static class RpcTimeoutConnection extends ConnectionImplementation { final ClientService.BlockingInterface stub; - RpcTimeoutConnection(Configuration conf, ExecutorService pool, User user) throws IOException { - super(conf, pool, user); + RpcTimeoutConnection(Configuration conf, ExecutorService pool, User user, + Map requestAttributes) throws IOException { + super(conf, pool, user, requestAttributes); // Mock up my stub so an exists call -- which turns into a get -- throws an exception this.stub = Mockito.mock(ClientService.BlockingInterface.class); try { @@ -384,8 +385,8 @@ public BlockingInterface getClient(ServerName sn) throws IOException { */ static class RpcTimeoutAsyncConnection extends AsyncConnectionImpl { RpcTimeoutAsyncConnection(Configuration configuration, ConnectionRegistry registry, - String clusterId, User user) { - super(configuration, registry, clusterId, user); + String clusterId, User user, Map connectionAttributes) { + super(configuration, registry, clusterId, user, connectionAttributes); } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementationTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementationTracing.java index fba41ecbe5bf..a207186785a8 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementationTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionImplementationTracing.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import java.util.Collections; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.security.UserProvider; @@ -44,7 +45,8 @@ public class TestConnectionImplementationTracing extends TestTracingBase { @Before public void setUp() throws Exception { super.setUp(); - conn = new ConnectionImplementation(conf, null, UserProvider.instantiate(conf).getCurrent()); + conn = new ConnectionImplementation(conf, null, UserProvider.instantiate(conf).getCurrent(), + Collections.emptyMap()); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableTracing.java index a2193553a65b..3a0f54a2af1d 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestHTableTracing.java @@ -44,6 +44,7 @@ import io.opentelemetry.sdk.trace.data.SpanData; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.atomic.AtomicInteger; @@ -191,35 +192,34 @@ public GetResponse answer(InvocationOnMock invocation) throws Throwable { } }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class)); - conn = - spy(new ConnectionImplementation(conf, null, UserProvider.instantiate(conf).getCurrent()) { - @Override - public RegionLocator getRegionLocator(TableName tableName) throws IOException { - RegionLocator locator = mock(HRegionLocator.class); - Answer answer = new Answer() { - - @Override - public HRegionLocation answer(InvocationOnMock invocation) throws Throwable { - TableName tableName = TableName.META_TABLE_NAME; - RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); - ServerName serverName = MASTER_HOST; - HRegionLocation loc = new HRegionLocation(info, serverName); - return loc; - } - }; - doAnswer(answer).when(locator).getRegionLocation(any(byte[].class), anyInt(), - anyBoolean()); - doAnswer(answer).when(locator).getRegionLocation(any(byte[].class)); - doAnswer(answer).when(locator).getRegionLocation(any(byte[].class), anyInt()); - doAnswer(answer).when(locator).getRegionLocation(any(byte[].class), anyBoolean()); - return locator; - } + conn = spy(new ConnectionImplementation(conf, null, UserProvider.instantiate(conf).getCurrent(), + Collections.emptyMap()) { + @Override + public RegionLocator getRegionLocator(TableName tableName) throws IOException { + RegionLocator locator = mock(HRegionLocator.class); + Answer answer = new Answer() { + + @Override + public HRegionLocation answer(InvocationOnMock invocation) throws Throwable { + TableName tableName = TableName.META_TABLE_NAME; + RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build(); + ServerName serverName = MASTER_HOST; + HRegionLocation loc = new HRegionLocation(info, serverName); + return loc; + } + }; + doAnswer(answer).when(locator).getRegionLocation(any(byte[].class), anyInt(), anyBoolean()); + doAnswer(answer).when(locator).getRegionLocation(any(byte[].class)); + doAnswer(answer).when(locator).getRegionLocation(any(byte[].class), anyInt()); + doAnswer(answer).when(locator).getRegionLocation(any(byte[].class), anyBoolean()); + return locator; + } - @Override - public ClientService.BlockingInterface getClient(ServerName serverName) throws IOException { - return stub; - } - }); + @Override + public ClientService.BlockingInterface getClient(ServerName serverName) throws IOException { + return stub; + } + }); // this setup of AsyncProcess is for MultiResponse AsyncProcess asyncProcess = mock(AsyncProcess.class); AsyncRequestFuture asyncRequestFuture = mock(AsyncRequestFuture.class); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionLocatorTracing.java index a1355554f6f8..1e9860b283b4 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionLocatorTracing.java @@ -32,6 +32,7 @@ import io.opentelemetry.sdk.trace.data.SpanData; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; @@ -59,7 +60,8 @@ public class TestRegionLocatorTracing extends TestTracingBase { @Before public void setUp() throws Exception { super.setUp(); - conn = new ConnectionImplementation(conf, null, UserProvider.instantiate(conf).getCurrent()); + conn = new ConnectionImplementation(conf, null, UserProvider.instantiate(conf).getCurrent(), + Collections.emptyMap()); } @After diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java index a1ff19c2faa6..bf19dbf3d9bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import java.util.Collections; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; @@ -149,7 +150,7 @@ public static ClusterConnection getMockedConnectionAndDecorate(final Configurati */ public static ClusterConnection getSpiedConnection(final Configuration conf) throws IOException { ConnectionImplementation connection = - Mockito.spy(new ConnectionImplementation(conf, null, null)); + Mockito.spy(new ConnectionImplementation(conf, null, null, Collections.emptyMap())); return connection; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java index fe59e12f73ea..bb27f1c2fe9c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; @@ -217,7 +218,7 @@ public static class DevNullAsyncConnection implements AsyncConnection { private final Configuration conf; public DevNullAsyncConnection(Configuration conf, ConnectionRegistry registry, String clusterId, - User user) { + User user, Map connectionAttributes) { this.conf = conf; } From 1b61ad4069ba2cc581f5ad966288e134abead600 Mon Sep 17 00:00:00 2001 From: Ray Mattingly Date: Fri, 28 Jul 2023 16:16:35 -0400 Subject: [PATCH 6/6] cleanup todos --- .../hadoop/hbase/client/BufferedMutatorImpl.java | 1 - .../hadoop/hbase/client/ClientServiceCallable.java | 5 +---- .../hadoop/hbase/client/RegionServerCallable.java | 1 + .../hadoop/hbase/ipc/HBaseRpcControllerImpl.java | 13 ------------- 4 files changed, 2 insertions(+), 18 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java index 4c3c94a35c2e..e0795c7c4c67 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java @@ -142,7 +142,6 @@ public class BufferedMutatorImpl implements BufferedMutator { RpcControllerFactory rpcFactory, BufferedMutatorParams params) { this(conn, params, // puts need to track errors globally due to how the APIs currently work. - // todo rmattingly support buffered mutator request attributes new AsyncProcess(conn, conn.getConfiguration(), rpcCallerFactory, rpcFactory)); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java index a669456911e5..7b7af0f1f5c5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientServiceCallable.java @@ -21,7 +21,6 @@ import java.util.Map; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; @@ -38,9 +37,7 @@ public abstract class ClientServiceCallable public ClientServiceCallable(Connection connection, TableName tableName, byte[] row, RpcController rpcController, int priority, Map requestAttributes) { - super(connection, tableName, row, - HBaseRpcControllerImpl.configureRequestAttributes(rpcController, requestAttributes), priority, - requestAttributes); + super(connection, tableName, row, rpcController, priority, requestAttributes); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java index 28a3e2661bec..da5d291a0998 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionServerCallable.java @@ -122,6 +122,7 @@ public T call(int callTimeout) throws IOException { hrc.setPriority(tableName); hrc.setPriority(priority); hrc.setCallTimeout(callTimeout); + hrc.setRequestAttributes(requestAttributes); } } return rpcCall(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java index 1b7a8ab81f18..6f8c65d6bdce 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRpcControllerImpl.java @@ -31,7 +31,6 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; -import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; /** * Get instances via {@link RpcControllerFactory} on client-side. @@ -282,16 +281,4 @@ public String toString() { + exception + ", regionInfo=" + regionInfo + ", priority=" + priority + ", cellScanner=" + cellScanner + '}'; } - - public static RpcController configureRequestAttributes(RpcController rpcController, - Map requestAttributes) { // todo rmattingly delete? - if ( - !requestAttributes.isEmpty() && rpcController != null - && rpcController instanceof HBaseRpcController - ) { - HBaseRpcController controller = (HBaseRpcController) rpcController; - controller.setRequestAttributes(requestAttributes); - } - return rpcController; - } }