From ba61a0c522f3a4c9f52580263c3b9c113ae6e3fc Mon Sep 17 00:00:00 2001 From: Kodey Converse Date: Fri, 25 Jul 2025 09:51:43 -0400 Subject: [PATCH 1/3] HBASE-29850 Add an ability to generate request attributes per request --- .../hbase/client/AsyncTableBuilder.java | 9 ++ .../hbase/client/AsyncTableBuilderBase.java | 10 ++ .../hbase/client/RawAsyncTableImpl.java | 16 +- .../client/RequestAttributesFactory.java | 47 ++++++ .../hbase/client/TestRequestAttributes.java | 140 +++++++++++++++++- 5 files changed, 218 insertions(+), 4 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestAttributesFactory.java 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 007f7ad48685..be55177e3aac 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 @@ -142,6 +142,15 @@ default AsyncTableBuilder setMaxRetries(int maxRetries) { */ AsyncTableBuilder setRequestAttribute(String key, byte[] value); + /** + * Set a factory for creating request attributes per request. This will be called with any + * attributes set by {@link #setRequestAttribute(String, byte[])}. + */ + default AsyncTableBuilder + setRequestAttributesFactory(RequestAttributesFactory requestAttributesFactory) { + throw new UnsupportedOperationException("Not implemented"); + } + /** * 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 428e7358195e..372d7b9c1d0f 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 @@ -55,6 +55,9 @@ abstract class AsyncTableBuilderBase protected Map requestAttributes = Collections.emptyMap(); + protected RequestAttributesFactory requestAttributesFactory = + RequestAttributesFactory.PASSTHROUGH; + AsyncTableBuilderBase(TableName tableName, AsyncConnectionConfiguration connConf) { this.tableName = tableName; this.operationTimeoutNs = tableName.isSystemTable() @@ -135,4 +138,11 @@ public AsyncTableBuilder setRequestAttribute(String key, byte[] value) { requestAttributes.put(key, value); return this; } + + @Override + public AsyncTableBuilder + setRequestAttributesFactory(RequestAttributesFactory requestAttributesFactory) { + this.requestAttributesFactory = requestAttributesFactory; + return this; + } } 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 553b4afa55ea..d3a61d2f1ff1 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 @@ -123,6 +123,8 @@ class RawAsyncTableImpl implements AsyncTable { private final Map requestAttributes; + private final RequestAttributesFactory requestAttributesFactory; + RawAsyncTableImpl(AsyncConnectionImpl conn, Timer retryTimer, AsyncTableBuilderBase builder) { this.conn = conn; this.retryTimer = retryTimer; @@ -150,6 +152,14 @@ class RawAsyncTableImpl implements AsyncTable { : conn.connConf.getScannerCaching(); this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize(); this.requestAttributes = builder.requestAttributes; + this.requestAttributesFactory = builder.requestAttributesFactory; + } + + private Map createRequestAttributes() { + Map attributes = requestAttributesFactory.create(requestAttributes); + Preconditions.checkState(attributes != null, + "RequestAttributesFactory.create() must not return null"); + return attributes; } @Override @@ -216,7 +226,7 @@ private SingleRequestCallerBuilder newCaller(byte[] row, int priority, lo .pause(pauseNs, TimeUnit.NANOSECONDS) .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS) .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt) - .setRequestAttributes(requestAttributes); + .setRequestAttributes(createRequestAttributes()); } private SingleRequestCallerBuilder @@ -616,7 +626,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, requestAttributes).start(); + startLogErrorsCnt, createRequestAttributes()).start(); } private long resultSize2CacheSize(long maxResultSize) { @@ -713,7 +723,7 @@ private List> batch(List actions, long r .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS) .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt) - .setRequestAttributes(requestAttributes).call(); + .setRequestAttributes(createRequestAttributes()).call(); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestAttributesFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestAttributesFactory.java new file mode 100644 index 000000000000..246e8588d535 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestAttributesFactory.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import java.util.Map; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A factory for creating request attributes. This is called each time a new call is started, + * allowing for dynamic attributes based on the current context or existing attributes. + *

+ * The {@link #create} method is guaranteed to be called on the same thread that initiates the + * client call (e.g., {@link AsyncTable#get}, {@link AsyncTable#put}, {@link AsyncTable#scan}, + * etc.). + */ +@InterfaceAudience.Public +public interface RequestAttributesFactory { + + /** + * A factory that returns the input attributes unchanged. + */ + RequestAttributesFactory PASSTHROUGH = (requestAttributes) -> requestAttributes; + + /** + * Creates a new map of request attributes based on the existing attributes for the table. + *

+ * This method is guaranteed to be called on the same thread that initiates the client call. + * @param requestAttributes The existing attributes configured on the table + * @return The new map of request attributes. Must not be null. + */ + Map create(Map requestAttributes); +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAttributes.java index 9d6dc33a46a3..ccaffaad6108 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAttributes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAttributes.java @@ -17,6 +17,9 @@ */ package org.apache.hadoop.hbase.client; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.util.Arrays; import java.util.HashMap; @@ -24,8 +27,11 @@ import java.util.Map; import java.util.Optional; import java.util.UUID; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.AuthUtil; import org.apache.hadoop.hbase.Cell; @@ -72,8 +78,18 @@ public class TestRequestAttributes { private static final byte[] ROW_KEY6 = Bytes.toBytes("6"); private static final byte[] ROW_KEY7 = Bytes.toBytes("7"); private static final byte[] ROW_KEY8 = Bytes.toBytes("8"); + private static final byte[] ROW_KEY_FACTORY_GET = Bytes.toBytes("F1"); + private static final byte[] ROW_KEY_FACTORY_SCAN = Bytes.toBytes("F2"); + private static final byte[] ROW_KEY_FACTORY_PUT = Bytes.toBytes("F3"); + private static final byte[] ROW_KEY_FACTORY_AUGMENT = Bytes.toBytes("F4"); + private static final byte[] ROW_KEY_FACTORY_PER_REQUEST = Bytes.toBytes("F5"); + private static final String FACTORY_KEY = "factoryKey"; + private static final byte[] FACTORY_VALUE = Bytes.toBytes("factoryValue"); + private static final String STATIC_KEY = "staticKey"; + private static final byte[] STATIC_VALUE = Bytes.toBytes("staticValue"); private static final Map CONNECTION_ATTRIBUTES = new HashMap<>(); private static final Map REQUEST_ATTRIBUTES_SCAN = addRandomRequestAttributes(); + private static final Map REQUEST_ATTRIBUTES_FACTORY_SCAN = new HashMap<>(); private static final Map> ROW_KEY_TO_REQUEST_ATTRIBUTES = new HashMap<>(); static { @@ -88,6 +104,25 @@ public class TestRequestAttributes { ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY6, addRandomRequestAttributes()); ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY7, addRandomRequestAttributes()); ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY8, new HashMap()); + + Map factoryGetAttrs = new HashMap<>(); + factoryGetAttrs.put(FACTORY_KEY, FACTORY_VALUE); + ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_FACTORY_GET, factoryGetAttrs); + + REQUEST_ATTRIBUTES_FACTORY_SCAN.put(FACTORY_KEY, FACTORY_VALUE); + + Map factoryPutAttrs = new HashMap<>(); + factoryPutAttrs.put(FACTORY_KEY, FACTORY_VALUE); + ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_FACTORY_PUT, factoryPutAttrs); + + Map factoryAugmentAttrs = new HashMap<>(); + factoryAugmentAttrs.put(STATIC_KEY, STATIC_VALUE); + factoryAugmentAttrs.put(FACTORY_KEY, FACTORY_VALUE); + ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_FACTORY_AUGMENT, factoryAugmentAttrs); + + Map factoryPerRequestAttrs = new HashMap<>(); + factoryPerRequestAttrs.put(FACTORY_KEY, FACTORY_VALUE); + ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_FACTORY_PER_REQUEST, factoryPerRequestAttrs); } private static final ExecutorService EXECUTOR_SERVICE = Executors.newFixedThreadPool(100); private static final byte[] FAMILY = Bytes.toBytes("0"); @@ -237,6 +272,106 @@ public void testNoRequestAttributes() throws IOException { } } + @Test + public void testAsyncRequestAttributesFactoryGet() + throws IOException, ExecutionException, InterruptedException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { + Map newAttrs = new HashMap<>(attrs); + newAttrs.put(FACTORY_KEY, FACTORY_VALUE); + return newAttrs; + }).build(); + table.get(new Get(ROW_KEY_FACTORY_GET)).get(); + } + } + + @Test + public void testAsyncRequestAttributesFactoryScan() + throws IOException, ExecutionException, InterruptedException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { + Map newAttrs = new HashMap<>(attrs); + newAttrs.put(FACTORY_KEY, FACTORY_VALUE); + return newAttrs; + }).build(); + List results = table + .scanAll( + new Scan().withStartRow(ROW_KEY_FACTORY_SCAN).withStopRow(ROW_KEY_FACTORY_SCAN, true)) + .get(); + } + } + + @Test + public void testAsyncRequestAttributesFactoryPut() + throws IOException, ExecutionException, InterruptedException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { + Map newAttrs = new HashMap<>(attrs); + newAttrs.put(FACTORY_KEY, FACTORY_VALUE); + return newAttrs; + }).build(); + Put put = new Put(ROW_KEY_FACTORY_PUT); + put.addColumn(FAMILY, Bytes.toBytes("c"), Bytes.toBytes("v")); + table.put(put).get(); + } + } + + @Test + public void testAsyncRequestAttributesFactoryAugmentsStaticAttributes() + throws IOException, ExecutionException, InterruptedException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME) + .setRequestAttribute(STATIC_KEY, STATIC_VALUE).setRequestAttributesFactory(attrs -> { + Map newAttrs = new HashMap<>(attrs); + newAttrs.put(FACTORY_KEY, FACTORY_VALUE); + return newAttrs; + }).build(); + table.get(new Get(ROW_KEY_FACTORY_AUGMENT)).get(); + } + } + + @Test + public void testAsyncRequestAttributesFactoryCalledPerRequest() + throws IOException, ExecutionException, InterruptedException { + Configuration conf = TEST_UTIL.getConfiguration(); + AtomicInteger callCount = new AtomicInteger(0); + try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { + callCount.incrementAndGet(); + Map newAttrs = new HashMap<>(attrs); + newAttrs.put(FACTORY_KEY, FACTORY_VALUE); + return newAttrs; + }).build(); + table.get(new Get(ROW_KEY_FACTORY_PER_REQUEST)).get(); + table.get(new Get(ROW_KEY_FACTORY_PER_REQUEST)).get(); + table.get(new Get(ROW_KEY_FACTORY_PER_REQUEST)).get(); + } + assertTrue("Factory should be called at least 3 times", callCount.get() >= 3); + } + + @Test + public void testAsyncRequestAttributesFactoryCalledOnInitiatingThread() + throws IOException, ExecutionException, InterruptedException { + Configuration conf = TEST_UTIL.getConfiguration(); + Thread testThread = Thread.currentThread(); + AtomicReference factoryThread = new AtomicReference<>(); + try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { + factoryThread.set(Thread.currentThread()); + Map newAttrs = new HashMap<>(attrs); + newAttrs.put(FACTORY_KEY, FACTORY_VALUE); + return newAttrs; + }).build(); + table.get(new Get(ROW_KEY_FACTORY_GET)).get(); + } + assertEquals("Factory should be called on the initiating thread", testThread, + factoryThread.get()); + } + private static Map addRandomRequestAttributes() { Map requestAttributes = new HashMap<>(); int j = Math.max(2, (int) (10 * Math.random())); @@ -324,7 +459,10 @@ public void preGetOp(ObserverContext c, @Override public boolean preScannerNext(ObserverContext c, InternalScanner s, List result, int limit, boolean hasNext) throws IOException { - if (!isValidRequestAttributes(REQUEST_ATTRIBUTES_SCAN)) { + if ( + !isValidRequestAttributes(REQUEST_ATTRIBUTES_SCAN) + && !isValidRequestAttributes(REQUEST_ATTRIBUTES_FACTORY_SCAN) + ) { throw new IOException("Incorrect request attributes"); } return hasNext; From d708dba82cdfa9d80cffc72261ff0235b777ff5f Mon Sep 17 00:00:00 2001 From: Kodey Converse Date: Fri, 30 Jan 2026 06:39:49 -0500 Subject: [PATCH 2/3] HBASE-29850 Deprecate existing request attributes setter --- .../hadoop/hbase/client/AsyncTable.java | 6 +- .../hbase/client/AsyncTableBuilder.java | 15 +++- .../hbase/client/AsyncTableBuilderBase.java | 25 ++++-- .../client/FixedRequestAttributesFactory.java | 86 +++++++++++++++++++ .../hbase/client/RawAsyncTableImpl.java | 23 ++--- .../client/RequestAttributesFactory.java | 25 +++--- 6 files changed, 136 insertions(+), 44 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/client/FixedRequestAttributesFactory.java 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 259183a41414..a2de4d7eac83 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 @@ -114,8 +114,10 @@ public interface AsyncTable { long getScanTimeout(TimeUnit unit); /** - * Get the map of request attributes - * @return a map of request attributes supplied by the client + * Returns the request attributes for this call. The attributes are generated by the configured + * {@link RequestAttributesFactory}, so each invocation may return different values if a dynamic + * factory is in use. + * @return the request attributes */ default Map getRequestAttributes() { return Collections.emptyMap(); 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 be55177e3aac..8b7300daec9a 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 @@ -138,13 +138,22 @@ default AsyncTableBuilder setMaxRetries(int maxRetries) { AsyncTableBuilder setStartLogErrorsCnt(int startLogErrorsCnt); /** - * Set a request attribute + * Sets a request attribute. Ignored if a factory is set via + * {@link #setRequestAttributesFactory(RequestAttributesFactory)}. + * @param key the attribute key + * @param value the attribute value + * @deprecated Since 3.0.0, will be removed in 4.0.0. Please use + * {@link #setRequestAttributesFactory(RequestAttributesFactory)} instead. */ + @Deprecated AsyncTableBuilder setRequestAttribute(String key, byte[] value); /** - * Set a factory for creating request attributes per request. This will be called with any - * attributes set by {@link #setRequestAttribute(String, byte[])}. + * Sets the factory for creating request attributes. Use {@link FixedRequestAttributesFactory} for + * attributes that do not change, or implement {@link RequestAttributesFactory} for dynamic + * attributes. + * @param requestAttributesFactory the factory to use + * @return this builder */ default AsyncTableBuilder setRequestAttributesFactory(RequestAttributesFactory requestAttributesFactory) { 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 372d7b9c1d0f..65e9f878971b 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,9 +19,6 @@ 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; @@ -53,10 +50,9 @@ abstract class AsyncTableBuilderBase protected int startLogErrorsCnt; - protected Map requestAttributes = Collections.emptyMap(); + protected FixedRequestAttributesFactory.Builder fixedRequestAttributesFactoryBuilder = null; - protected RequestAttributesFactory requestAttributesFactory = - RequestAttributesFactory.PASSTHROUGH; + protected RequestAttributesFactory requestAttributesFactory = null; AsyncTableBuilderBase(TableName tableName, AsyncConnectionConfiguration connConf) { this.tableName = tableName; @@ -132,10 +128,11 @@ public AsyncTableBuilderBase setStartLogErrorsCnt(int startLogErrorsCnt) { @Override public AsyncTableBuilder setRequestAttribute(String key, byte[] value) { - if (requestAttributes.isEmpty()) { - requestAttributes = new HashMap<>(); + if (fixedRequestAttributesFactoryBuilder == null) { + fixedRequestAttributesFactoryBuilder = + FixedRequestAttributesFactory.newBuilder(); } - requestAttributes.put(key, value); + fixedRequestAttributesFactoryBuilder.setAttribute(key, value); return this; } @@ -145,4 +142,14 @@ public AsyncTableBuilder setRequestAttribute(String key, byte[] value) { this.requestAttributesFactory = requestAttributesFactory; return this; } + + protected RequestAttributesFactory buildRequestAttributesFactory() { + if (requestAttributesFactory != null) { + return requestAttributesFactory; + } else if (fixedRequestAttributesFactoryBuilder != null) { + return fixedRequestAttributesFactoryBuilder.build(); + } else { + return FixedRequestAttributesFactory.EMPTY; + } + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FixedRequestAttributesFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FixedRequestAttributesFactory.java new file mode 100644 index 000000000000..8f2b1f562c94 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FixedRequestAttributesFactory.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * A {@link RequestAttributesFactory} that returns a fixed set of attributes for every call. Use + * this when attributes do not need to change for the lifetime of the {@link AsyncTable}. + * @see AsyncTableBuilder#setRequestAttributesFactory(RequestAttributesFactory) + */ +@InterfaceAudience.Public +public final class FixedRequestAttributesFactory implements RequestAttributesFactory { + + /** + * A factory that always returns an empty map. + */ + public static final RequestAttributesFactory EMPTY = Collections::emptyMap; + + /** + * Builder for creating {@link FixedRequestAttributesFactory} instances. + */ + public static final class Builder { + private final Map requestAttributes = new HashMap<>(); + + /** + * Sets a request attribute. If value is null, the attribute is removed. + * @param key the attribute key + * @param value the attribute value, or null to remove + * @return this builder + */ + public Builder setAttribute(String key, byte[] value) { + if (value == null) { + requestAttributes.remove(key); + } else { + requestAttributes.put(key, value); + } + return this; + } + + /** + * Builds a {@link FixedRequestAttributesFactory} with the configured attributes. + * @return the factory + */ + public FixedRequestAttributesFactory build() { + return new FixedRequestAttributesFactory(requestAttributes); + } + } + + /** + * Returns a new builder. + * @return a new builder instance + */ + public static Builder newBuilder() { + return new Builder(); + } + + private final Map requestAttributes; + + private FixedRequestAttributesFactory(Map requestAttributes) { + this.requestAttributes = Map.copyOf(requestAttributes); + } + + @Override + public Map create() { + return requestAttributes; + } +} 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 d3a61d2f1ff1..1cd38f135907 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 @@ -121,8 +121,6 @@ class RawAsyncTableImpl implements AsyncTable { private final int startLogErrorsCnt; - private final Map requestAttributes; - private final RequestAttributesFactory requestAttributesFactory; RawAsyncTableImpl(AsyncConnectionImpl conn, Timer retryTimer, AsyncTableBuilderBase builder) { @@ -151,15 +149,7 @@ class RawAsyncTableImpl implements AsyncTable { ? conn.connConf.getMetaScannerCaching() : conn.connConf.getScannerCaching(); this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize(); - this.requestAttributes = builder.requestAttributes; - this.requestAttributesFactory = builder.requestAttributesFactory; - } - - private Map createRequestAttributes() { - Map attributes = requestAttributesFactory.create(requestAttributes); - Preconditions.checkState(attributes != null, - "RequestAttributesFactory.create() must not return null"); - return attributes; + this.requestAttributesFactory = builder.buildRequestAttributesFactory(); } @Override @@ -226,7 +216,7 @@ private SingleRequestCallerBuilder newCaller(byte[] row, int priority, lo .pause(pauseNs, TimeUnit.NANOSECONDS) .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS) .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt) - .setRequestAttributes(createRequestAttributes()); + .setRequestAttributes(getRequestAttributes()); } private SingleRequestCallerBuilder @@ -626,7 +616,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, createRequestAttributes()).start(); + startLogErrorsCnt, getRequestAttributes()).start(); } private long resultSize2CacheSize(long maxResultSize) { @@ -723,7 +713,7 @@ private List> batch(List actions, long r .rpcTimeout(rpcTimeoutNs, TimeUnit.NANOSECONDS).pause(pauseNs, TimeUnit.NANOSECONDS) .pauseForServerOverloaded(pauseNsForServerOverloaded, TimeUnit.NANOSECONDS) .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt) - .setRequestAttributes(createRequestAttributes()).call(); + .setRequestAttributes(getRequestAttributes()).call(); } @Override @@ -753,7 +743,10 @@ public long getScanTimeout(TimeUnit unit) { @Override public Map getRequestAttributes() { - return requestAttributes; + Map attributes = requestAttributesFactory.create(); + Preconditions.checkState(attributes != null, + "RequestAttributesFactory.create() must not return null"); + return attributes; } private CompletableFuture coprocessorService(Function stubMaker, diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestAttributesFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestAttributesFactory.java index 246e8588d535..90277bee9d5d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestAttributesFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RequestAttributesFactory.java @@ -21,27 +21,22 @@ import org.apache.yetus.audience.InterfaceAudience; /** - * A factory for creating request attributes. This is called each time a new call is started, - * allowing for dynamic attributes based on the current context or existing attributes. + * Factory for creating request attributes. Called each time a client call is started, allowing + * dynamic attributes per call. Useful for propagating {@link ThreadLocal} context as request + * attributes. *

- * The {@link #create} method is guaranteed to be called on the same thread that initiates the - * client call (e.g., {@link AsyncTable#get}, {@link AsyncTable#put}, {@link AsyncTable#scan}, - * etc.). + * For a fixed set of attributes that does not change, use {@link FixedRequestAttributesFactory}. + * @see AsyncTableBuilder#setRequestAttributesFactory(RequestAttributesFactory) */ @InterfaceAudience.Public public interface RequestAttributesFactory { /** - * A factory that returns the input attributes unchanged. - */ - RequestAttributesFactory PASSTHROUGH = (requestAttributes) -> requestAttributes; - - /** - * Creates a new map of request attributes based on the existing attributes for the table. + * Creates request attributes for a client call (e.g., {@link AsyncTable#get}, + * {@link AsyncTable#put}, {@link AsyncTable#scan}). *

- * This method is guaranteed to be called on the same thread that initiates the client call. - * @param requestAttributes The existing attributes configured on the table - * @return The new map of request attributes. Must not be null. + * Guaranteed to be called on the same thread that initiates the client call. + * @return the request attributes, must not be null */ - Map create(Map requestAttributes); + Map create(); } From 28b63679d3d7f22bcf9d21ed48168ee92b23474c Mon Sep 17 00:00:00 2001 From: Kodey Converse Date: Wed, 4 Feb 2026 10:42:22 -0500 Subject: [PATCH 3/3] HBASE-29850 Add request attributes factory to remaining builders --- .../client/AsyncBufferedMutatorBuilder.java | 21 +- .../AsyncBufferedMutatorBuilderImpl.java | 7 + .../hbase/client/AsyncTableBuilderBase.java | 5 +- .../hbase/client/BufferedMutatorParams.java | 61 ++++- .../client/ConnectionOverAsyncConnection.java | 9 +- .../client/FixedRequestAttributesFactory.java | 17 +- .../hbase/client/RawAsyncTableImpl.java | 2 +- .../hadoop/hbase/client/TableBuilder.java | 16 +- .../hadoop/hbase/client/TableBuilderBase.java | 30 ++- .../client/TestBufferedMutatorParams.java | 3 +- .../TestFixedRequestAttributesFactory.java | 140 +++++++++++ .../hbase/client/TestRequestAttributes.java | 229 ++++++++++++++---- 12 files changed, 460 insertions(+), 80 deletions(-) create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestFixedRequestAttributesFactory.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilder.java index 57c609ebb038..6fbc61ad6c99 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilder.java @@ -40,15 +40,32 @@ public interface AsyncBufferedMutatorBuilder { AsyncBufferedMutatorBuilder setRpcTimeout(long timeout, TimeUnit unit); /** - * Set a rpc request attribute. + * Sets a request attribute. Ignored if a factory is set via + * {@link #setRequestAttributesFactory(RequestAttributesFactory)}. + * @deprecated Since 3.0.0, will be removed in 4.0.0. Please use + * {@link #setRequestAttributesFactory(RequestAttributesFactory)} instead. */ + @Deprecated AsyncBufferedMutatorBuilder setRequestAttribute(String key, byte[] value); /** - * Set multiple rpc request attributes. + * Sets multiple request attributes. Ignored if a factory is set via + * @deprecated Since 3.0.0, will be removed in 4.0.0. Please use + * {@link #setRequestAttributesFactory(RequestAttributesFactory)} instead. */ + @Deprecated AsyncBufferedMutatorBuilder setRequestAttributes(Map requestAttributes); + /** + * Sets the factory for creating request attributes. Use {@link FixedRequestAttributesFactory} for + * attributes that do not change, or implement {@link RequestAttributesFactory} for dynamic + * attributes. + */ + default AsyncBufferedMutatorBuilder + setRequestAttributesFactory(RequestAttributesFactory requestAttributesFactory) { + throw new UnsupportedOperationException("Not implemented"); + } + /** * Set the base pause time for retrying. We use an exponential policy to generate sleep time when * retrying. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java index 7fa860dc3d4e..21ae30467d32 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java @@ -96,6 +96,13 @@ public AsyncBufferedMutatorBuilder setRequestAttributes(Map requ return this; } + @Override + public AsyncBufferedMutatorBuilder + setRequestAttributesFactory(RequestAttributesFactory requestAttributesFactory) { + tableBuilder.setRequestAttributesFactory(requestAttributesFactory); + return this; + } + @Override public AsyncBufferedMutatorBuilder setWriteBufferSize(long writeBufferSize) { Preconditions.checkArgument(writeBufferSize > 0, "writeBufferSize %d must be > 0", 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 65e9f878971b..3e629a4dfd80 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 @@ -129,8 +129,7 @@ public AsyncTableBuilderBase setStartLogErrorsCnt(int startLogErrorsCnt) { @Override public AsyncTableBuilder setRequestAttribute(String key, byte[] value) { if (fixedRequestAttributesFactoryBuilder == null) { - fixedRequestAttributesFactoryBuilder = - FixedRequestAttributesFactory.newBuilder(); + fixedRequestAttributesFactoryBuilder = FixedRequestAttributesFactory.newBuilder(); } fixedRequestAttributesFactoryBuilder.setAttribute(key, value); return this; @@ -143,7 +142,7 @@ public AsyncTableBuilder setRequestAttribute(String key, byte[] value) { return this; } - protected RequestAttributesFactory buildRequestAttributesFactory() { + RequestAttributesFactory getRequestAttributesFactory() { if (requestAttributesFactory != null) { return requestAttributesFactory; } else if (fixedRequestAttributesFactoryBuilder != null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java index 550d13033cb2..261bb84fec14 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorParams.java @@ -17,15 +17,11 @@ */ package org.apache.hadoop.hbase.client; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.concurrent.ExecutorService; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hbase.thirdparty.com.google.common.collect.Maps; - /** * Parameters for instantiating a {@link BufferedMutator}. */ @@ -44,7 +40,8 @@ public class BufferedMutatorParams implements Cloneable { private int rpcTimeout = UNSET; private int operationTimeout = UNSET; private int maxMutations = UNSET; - protected Map requestAttributes = Collections.emptyMap(); + protected FixedRequestAttributesFactory.Builder fixedRequestAttributesFactoryBuilder = null; + protected RequestAttributesFactory requestAttributesFactory = null; private BufferedMutator.ExceptionListener listener = new BufferedMutator.ExceptionListener() { @Override public void onException(RetriesExhaustedWithDetailsException exception, @@ -109,16 +106,55 @@ public int getMaxMutations() { return maxMutations; } + /** + * Sets a request attribute. Ignored if a factory is set via + * {@link #setRequestAttributesFactory(RequestAttributesFactory)}. + * @deprecated Since 3.0.0, will be removed in 4.0.0. Please use + * {@link #setRequestAttributesFactory(RequestAttributesFactory)} instead. + */ + @Deprecated public BufferedMutatorParams setRequestAttribute(String key, byte[] value) { - if (requestAttributes.isEmpty()) { - requestAttributes = new HashMap<>(); + if (fixedRequestAttributesFactoryBuilder == null) { + fixedRequestAttributesFactoryBuilder = FixedRequestAttributesFactory.newBuilder(); } - requestAttributes.put(key, value); + fixedRequestAttributesFactoryBuilder.setAttribute(key, value); return this; } + /** + * Returns *only* the request attributes added by {@link #setRequestAttribute(String, byte[])}. + * @deprecated Since 3.0.0, will be removed in 4.0.0. Please use + * {@link #getRequestAttributesFactory()} instead. + */ public Map getRequestAttributes() { - return requestAttributes; + if (fixedRequestAttributesFactoryBuilder == null) { + return null; + } + return fixedRequestAttributesFactoryBuilder.getAttributes(); + } + + /** + * Sets the factory for creating request attributes. Use {@link FixedRequestAttributesFactory} for + * attributes that do not change, or implement {@link RequestAttributesFactory} for dynamic + * attributes. + */ + public BufferedMutatorParams + setRequestAttributesFactory(RequestAttributesFactory requestAttributesFactory) { + this.requestAttributesFactory = requestAttributesFactory; + return this; + } + + /** + * Returns the request attributes factory. + */ + public RequestAttributesFactory getRequestAttributesFactory() { + if (requestAttributesFactory != null) { + return requestAttributesFactory; + } else if (fixedRequestAttributesFactoryBuilder != null) { + return fixedRequestAttributesFactoryBuilder.build(); + } else { + return FixedRequestAttributesFactory.EMPTY; + } } /** @@ -243,7 +279,12 @@ public BufferedMutatorParams clone() { clone.writeBufferPeriodicFlushTimerTickMs = this.writeBufferPeriodicFlushTimerTickMs; clone.maxKeyValueSize = this.maxKeyValueSize; clone.maxMutations = this.maxMutations; - clone.requestAttributes = Maps.newHashMap(this.requestAttributes); + clone.requestAttributesFactory = this.requestAttributesFactory; + if (fixedRequestAttributesFactoryBuilder != null) { + clone.fixedRequestAttributesFactoryBuilder = FixedRequestAttributesFactory.newBuilder(); + fixedRequestAttributesFactoryBuilder.getAttributes() + .forEach(clone.fixedRequestAttributesFactoryBuilder::setAttribute); + } clone.pool = this.pool; clone.listener = this.listener; clone.implementationClassName = this.implementationClassName; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java index 471cfa874458..c3858b0468cf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java @@ -110,10 +110,7 @@ public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws I if (params.getMaxMutations() != BufferedMutatorParams.UNSET) { builder.setMaxMutations(params.getMaxMutations()); } - if (!params.getRequestAttributes().isEmpty()) { - - builder.setRequestAttributes(params.getRequestAttributes()); - } + builder.setRequestAttributesFactory(params.getRequestAttributesFactory()); return new BufferedMutatorOverAsyncBufferedMutator(builder.build(), params.getListener()); } @@ -200,8 +197,8 @@ public Table build() { conn.getTableBuilder(tableName).setRpcTimeout(rpcTimeout, TimeUnit.MILLISECONDS) .setReadRpcTimeout(readRpcTimeout, TimeUnit.MILLISECONDS) .setWriteRpcTimeout(writeRpcTimeout, TimeUnit.MILLISECONDS) - .setOperationTimeout(operationTimeout, TimeUnit.MILLISECONDS); - requestAttributes.forEach(tableBuilder::setRequestAttribute); + .setOperationTimeout(operationTimeout, TimeUnit.MILLISECONDS) + .setRequestAttributesFactory(getRequestAttributesFactory()); return new TableOverAsyncTable(conn, tableBuilder.build(), poolSupplier); } }; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FixedRequestAttributesFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FixedRequestAttributesFactory.java index 8f2b1f562c94..59690eb7ab38 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FixedRequestAttributesFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/FixedRequestAttributesFactory.java @@ -18,13 +18,13 @@ package org.apache.hadoop.hbase.client; import java.util.Collections; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; import org.apache.yetus.audience.InterfaceAudience; /** * A {@link RequestAttributesFactory} that returns a fixed set of attributes for every call. Use - * this when attributes do not need to change for the lifetime of the {@link AsyncTable}. + * this when attributes are fixed and do not change. * @see AsyncTableBuilder#setRequestAttributesFactory(RequestAttributesFactory) */ @InterfaceAudience.Public @@ -39,7 +39,7 @@ public final class FixedRequestAttributesFactory implements RequestAttributesFac * Builder for creating {@link FixedRequestAttributesFactory} instances. */ public static final class Builder { - private final Map requestAttributes = new HashMap<>(); + private final Map requestAttributes = new LinkedHashMap<>(); /** * Sets a request attribute. If value is null, the attribute is removed. @@ -56,12 +56,19 @@ public Builder setAttribute(String key, byte[] value) { return this; } + /** + * Gets the accumulated request attributes. + */ + public Map getAttributes() { + return Collections.unmodifiableMap(requestAttributes); + } + /** * Builds a {@link FixedRequestAttributesFactory} with the configured attributes. * @return the factory */ public FixedRequestAttributesFactory build() { - return new FixedRequestAttributesFactory(requestAttributes); + return new FixedRequestAttributesFactory(new LinkedHashMap<>(requestAttributes)); } } @@ -76,7 +83,7 @@ public static Builder newBuilder() { private final Map requestAttributes; private FixedRequestAttributesFactory(Map requestAttributes) { - this.requestAttributes = Map.copyOf(requestAttributes); + this.requestAttributes = Collections.unmodifiableMap(requestAttributes); } @Override 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 1cd38f135907..5cddf5131d5a 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 @@ -149,7 +149,7 @@ class RawAsyncTableImpl implements AsyncTable { ? conn.connConf.getMetaScannerCaching() : conn.connConf.getScannerCaching(); this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize(); - this.requestAttributesFactory = builder.buildRequestAttributesFactory(); + this.requestAttributesFactory = builder.getRequestAttributesFactory(); } @Override 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 eee985555b34..50cb550b5ef2 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 @@ -56,10 +56,24 @@ public interface TableBuilder { TableBuilder setWriteRpcTimeout(int timeout); /** - * Set a request attribute + * Sets a request attribute. Ignored if a factory is set via + * {@link #setRequestAttributesFactory(RequestAttributesFactory)}. + * @deprecated Since 3.0.0, will be removed in 4.0.0. Please use + * {@link #setRequestAttributesFactory(RequestAttributesFactory)} instead. */ + @Deprecated TableBuilder setRequestAttribute(String key, byte[] value); + /** + * Sets the factory for creating request attributes. Use {@link FixedRequestAttributesFactory} for + * attributes that do not change, or implement {@link RequestAttributesFactory} for dynamic + * attributes. + */ + default TableBuilder + setRequestAttributesFactory(RequestAttributesFactory requestAttributesFactory) { + throw new UnsupportedOperationException("Not implemented"); + } + /** * 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 dc3111b0c79d..4e3100ac9b3d 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,9 +17,6 @@ */ 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; @@ -39,7 +36,9 @@ abstract class TableBuilderBase implements TableBuilder { protected int writeRpcTimeout; - protected Map requestAttributes = Collections.emptyMap(); + protected FixedRequestAttributesFactory.Builder fixedRequestAttributesFactoryBuilder = null; + + protected RequestAttributesFactory requestAttributesFactory = null; TableBuilderBase(TableName tableName, ConnectionConfiguration connConf) { if (tableName == null) { @@ -81,10 +80,27 @@ public TableBuilderBase setWriteRpcTimeout(int timeout) { @Override public TableBuilderBase setRequestAttribute(String key, byte[] value) { - if (this.requestAttributes.isEmpty()) { - this.requestAttributes = new HashMap<>(); + if (fixedRequestAttributesFactoryBuilder == null) { + fixedRequestAttributesFactoryBuilder = FixedRequestAttributesFactory.newBuilder(); } - this.requestAttributes.put(key, value); + fixedRequestAttributesFactoryBuilder.setAttribute(key, value); + return this; + } + + @Override + public TableBuilderBase + setRequestAttributesFactory(RequestAttributesFactory requestAttributesFactory) { + this.requestAttributesFactory = requestAttributesFactory; return this; } + + RequestAttributesFactory getRequestAttributesFactory() { + if (requestAttributesFactory != null) { + return requestAttributesFactory; + } else if (fixedRequestAttributesFactoryBuilder != null) { + return fixedRequestAttributesFactoryBuilder.build(); + } else { + return FixedRequestAttributesFactory.EMPTY; + } + } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutatorParams.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutatorParams.java index b6c52a0cd0d6..9600bcd3cc6c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutatorParams.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestBufferedMutatorParams.java @@ -182,7 +182,8 @@ private void cloneTest(BufferedMutatorParams some, BufferedMutatorParams clone) clone.getWriteBufferPeriodicFlushTimerTickMs()); assertEquals(some.getMaxKeyValueSize(), clone.getMaxKeyValueSize()); assertTrue(some.getMaxMutations() == clone.getMaxMutations()); - assertEquals(some.requestAttributes, clone.requestAttributes); + assertEquals(some.getRequestAttributesFactory().create(), + clone.getRequestAttributesFactory().create()); assertTrue(some.getListener() == clone.getListener()); assertTrue(some.getPool() == clone.getPool()); assertEquals(some.getImplementationClassName(), clone.getImplementationClassName()); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestFixedRequestAttributesFactory.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestFixedRequestAttributesFactory.java new file mode 100644 index 000000000000..90c2b3d3b3fe --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestFixedRequestAttributesFactory.java @@ -0,0 +1,140 @@ +/* + * 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.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import java.util.Map; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ClientTests.class, SmallTests.class }) +public class TestFixedRequestAttributesFactory { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFixedRequestAttributesFactory.class); + + @Test + public void testEmptyFactory() { + Map attrs = FixedRequestAttributesFactory.EMPTY.create(); + assertTrue(attrs.isEmpty()); + } + + @Test + public void testBuilderSetAttribute() { + byte[] value = Bytes.toBytes("value1"); + FixedRequestAttributesFactory factory = + FixedRequestAttributesFactory.newBuilder().setAttribute("key1", value).build(); + + Map attrs = factory.create(); + assertEquals(1, attrs.size()); + assertArrayEquals(value, attrs.get("key1")); + } + + @Test + public void testBuilderMultipleAttributes() { + byte[] value1 = Bytes.toBytes("value1"); + byte[] value2 = Bytes.toBytes("value2"); + FixedRequestAttributesFactory factory = FixedRequestAttributesFactory.newBuilder() + .setAttribute("key1", value1).setAttribute("key2", value2).build(); + + Map attrs = factory.create(); + assertEquals(2, attrs.size()); + assertArrayEquals(value1, attrs.get("key1")); + assertArrayEquals(value2, attrs.get("key2")); + } + + @Test + public void testBuilderOverrideAttribute() { + byte[] value1 = Bytes.toBytes("value1"); + byte[] value2 = Bytes.toBytes("value2"); + FixedRequestAttributesFactory factory = FixedRequestAttributesFactory.newBuilder() + .setAttribute("key1", value1).setAttribute("key1", value2).build(); + + Map attrs = factory.create(); + assertEquals(1, attrs.size()); + assertArrayEquals(value2, attrs.get("key1")); + } + + @Test + public void testBuilderNullValueRemovesAttribute() { + byte[] value1 = Bytes.toBytes("value1"); + FixedRequestAttributesFactory factory = FixedRequestAttributesFactory.newBuilder() + .setAttribute("key1", value1).setAttribute("key1", null).build(); + + Map attrs = factory.create(); + assertTrue(attrs.isEmpty()); + } + + @Test + public void testBuilderNullValueOnNonExistentKey() { + FixedRequestAttributesFactory factory = + FixedRequestAttributesFactory.newBuilder().setAttribute("key1", null).build(); + + Map attrs = factory.create(); + assertTrue(attrs.isEmpty()); + } + + @Test + public void testCreateReturnsSameInstance() { + FixedRequestAttributesFactory factory = FixedRequestAttributesFactory.newBuilder() + .setAttribute("key1", Bytes.toBytes("value1")).build(); + + Map attrs1 = factory.create(); + Map attrs2 = factory.create(); + assertSame(attrs1, attrs2); + } + + @Test(expected = UnsupportedOperationException.class) + public void testReturnedMapIsUnmodifiable() { + FixedRequestAttributesFactory factory = FixedRequestAttributesFactory.newBuilder() + .setAttribute("key1", Bytes.toBytes("value1")).build(); + + Map attrs = factory.create(); + attrs.put("key2", Bytes.toBytes("value2")); + } + + @Test + public void testBuilderIsolation() { + FixedRequestAttributesFactory.Builder builder = FixedRequestAttributesFactory.newBuilder(); + builder.setAttribute("key1", Bytes.toBytes("value1")); + FixedRequestAttributesFactory factory = builder.build(); + + builder.setAttribute("key2", Bytes.toBytes("value2")); + + Map attrs = factory.create(); + assertEquals(1, attrs.size()); + } + + @Test + public void testNewBuilderReturnsFreshInstance() { + FixedRequestAttributesFactory.Builder builder1 = FixedRequestAttributesFactory.newBuilder(); + FixedRequestAttributesFactory.Builder builder2 = FixedRequestAttributesFactory.newBuilder(); + assertNotSame(builder1, builder2); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAttributes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAttributes.java index ccaffaad6108..5c2452e314be 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAttributes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRequestAttributes.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Optional; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -81,15 +82,20 @@ public class TestRequestAttributes { private static final byte[] ROW_KEY_FACTORY_GET = Bytes.toBytes("F1"); private static final byte[] ROW_KEY_FACTORY_SCAN = Bytes.toBytes("F2"); private static final byte[] ROW_KEY_FACTORY_PUT = Bytes.toBytes("F3"); - private static final byte[] ROW_KEY_FACTORY_AUGMENT = Bytes.toBytes("F4"); private static final byte[] ROW_KEY_FACTORY_PER_REQUEST = Bytes.toBytes("F5"); + private static final byte[] ROW_KEY_TABLE_FACTORY_GET = Bytes.toBytes("TF1"); + private static final byte[] ROW_KEY_TABLE_FACTORY_SCAN = Bytes.toBytes("TF2"); + private static final byte[] ROW_KEY_TABLE_FACTORY_PUT = Bytes.toBytes("TF3"); + private static final byte[] ROW_KEY_BM_FACTORY = Bytes.toBytes("BM1"); + private static final byte[] ROW_KEY_ASYNC_BM_FACTORY = Bytes.toBytes("ABM1"); private static final String FACTORY_KEY = "factoryKey"; private static final byte[] FACTORY_VALUE = Bytes.toBytes("factoryValue"); - private static final String STATIC_KEY = "staticKey"; - private static final byte[] STATIC_VALUE = Bytes.toBytes("staticValue"); + private static final String IGNORED_KEY = "ignoredKey"; + private static final byte[] IGNORED_VALUE = Bytes.toBytes("ignoredValue"); private static final Map CONNECTION_ATTRIBUTES = new HashMap<>(); private static final Map REQUEST_ATTRIBUTES_SCAN = addRandomRequestAttributes(); private static final Map REQUEST_ATTRIBUTES_FACTORY_SCAN = new HashMap<>(); + private static final Map REQUEST_ATTRIBUTES_TABLE_FACTORY_SCAN = new HashMap<>(); private static final Map> ROW_KEY_TO_REQUEST_ATTRIBUTES = new HashMap<>(); static { @@ -115,14 +121,27 @@ public class TestRequestAttributes { factoryPutAttrs.put(FACTORY_KEY, FACTORY_VALUE); ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_FACTORY_PUT, factoryPutAttrs); - Map factoryAugmentAttrs = new HashMap<>(); - factoryAugmentAttrs.put(STATIC_KEY, STATIC_VALUE); - factoryAugmentAttrs.put(FACTORY_KEY, FACTORY_VALUE); - ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_FACTORY_AUGMENT, factoryAugmentAttrs); - Map factoryPerRequestAttrs = new HashMap<>(); factoryPerRequestAttrs.put(FACTORY_KEY, FACTORY_VALUE); ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_FACTORY_PER_REQUEST, factoryPerRequestAttrs); + + Map tableFactoryGetAttrs = new HashMap<>(); + tableFactoryGetAttrs.put(FACTORY_KEY, FACTORY_VALUE); + ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_TABLE_FACTORY_GET, tableFactoryGetAttrs); + + REQUEST_ATTRIBUTES_TABLE_FACTORY_SCAN.put(FACTORY_KEY, FACTORY_VALUE); + + Map tableFactoryPutAttrs = new HashMap<>(); + tableFactoryPutAttrs.put(FACTORY_KEY, FACTORY_VALUE); + ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_TABLE_FACTORY_PUT, tableFactoryPutAttrs); + + Map bmFactoryAttrs = new HashMap<>(); + bmFactoryAttrs.put(FACTORY_KEY, FACTORY_VALUE); + ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_BM_FACTORY, bmFactoryAttrs); + + Map asyncBmFactoryAttrs = new HashMap<>(); + asyncBmFactoryAttrs.put(FACTORY_KEY, FACTORY_VALUE); + ROW_KEY_TO_REQUEST_ATTRIBUTES.put(ROW_KEY_ASYNC_BM_FACTORY, asyncBmFactoryAttrs); } private static final ExecutorService EXECUTOR_SERVICE = Executors.newFixedThreadPool(100); private static final byte[] FAMILY = Bytes.toBytes("0"); @@ -277,10 +296,10 @@ public void testAsyncRequestAttributesFactoryGet() throws IOException, ExecutionException, InterruptedException { Configuration conf = TEST_UTIL.getConfiguration(); try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { - AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { - Map newAttrs = new HashMap<>(attrs); - newAttrs.put(FACTORY_KEY, FACTORY_VALUE); - return newAttrs; + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; }).build(); table.get(new Get(ROW_KEY_FACTORY_GET)).get(); } @@ -291,12 +310,12 @@ public void testAsyncRequestAttributesFactoryScan() throws IOException, ExecutionException, InterruptedException { Configuration conf = TEST_UTIL.getConfiguration(); try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { - AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { - Map newAttrs = new HashMap<>(attrs); - newAttrs.put(FACTORY_KEY, FACTORY_VALUE); - return newAttrs; + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; }).build(); - List results = table + table .scanAll( new Scan().withStartRow(ROW_KEY_FACTORY_SCAN).withStopRow(ROW_KEY_FACTORY_SCAN, true)) .get(); @@ -308,10 +327,10 @@ public void testAsyncRequestAttributesFactoryPut() throws IOException, ExecutionException, InterruptedException { Configuration conf = TEST_UTIL.getConfiguration(); try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { - AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { - Map newAttrs = new HashMap<>(attrs); - newAttrs.put(FACTORY_KEY, FACTORY_VALUE); - return newAttrs; + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; }).build(); Put put = new Put(ROW_KEY_FACTORY_PUT); put.addColumn(FAMILY, Bytes.toBytes("c"), Bytes.toBytes("v")); @@ -319,32 +338,17 @@ public void testAsyncRequestAttributesFactoryPut() } } - @Test - public void testAsyncRequestAttributesFactoryAugmentsStaticAttributes() - throws IOException, ExecutionException, InterruptedException { - Configuration conf = TEST_UTIL.getConfiguration(); - try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { - AsyncTable table = conn.getTableBuilder(TABLE_NAME) - .setRequestAttribute(STATIC_KEY, STATIC_VALUE).setRequestAttributesFactory(attrs -> { - Map newAttrs = new HashMap<>(attrs); - newAttrs.put(FACTORY_KEY, FACTORY_VALUE); - return newAttrs; - }).build(); - table.get(new Get(ROW_KEY_FACTORY_AUGMENT)).get(); - } - } - @Test public void testAsyncRequestAttributesFactoryCalledPerRequest() throws IOException, ExecutionException, InterruptedException { Configuration conf = TEST_UTIL.getConfiguration(); AtomicInteger callCount = new AtomicInteger(0); try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { - AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(() -> { callCount.incrementAndGet(); - Map newAttrs = new HashMap<>(attrs); - newAttrs.put(FACTORY_KEY, FACTORY_VALUE); - return newAttrs; + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; }).build(); table.get(new Get(ROW_KEY_FACTORY_PER_REQUEST)).get(); table.get(new Get(ROW_KEY_FACTORY_PER_REQUEST)).get(); @@ -360,11 +364,11 @@ public void testAsyncRequestAttributesFactoryCalledOnInitiatingThread() Thread testThread = Thread.currentThread(); AtomicReference factoryThread = new AtomicReference<>(); try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { - AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(attrs -> { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory(() -> { factoryThread.set(Thread.currentThread()); - Map newAttrs = new HashMap<>(attrs); - newAttrs.put(FACTORY_KEY, FACTORY_VALUE); - return newAttrs; + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; }).build(); table.get(new Get(ROW_KEY_FACTORY_GET)).get(); } @@ -372,6 +376,142 @@ public void testAsyncRequestAttributesFactoryCalledOnInitiatingThread() factoryThread.get()); } + @Test + public void testAsyncFixedRequestAttributesFactory() + throws IOException, ExecutionException, InterruptedException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setRequestAttributesFactory( + FixedRequestAttributesFactory.newBuilder().setAttribute(FACTORY_KEY, FACTORY_VALUE).build()) + .build(); + table.get(new Get(ROW_KEY_FACTORY_GET)).get(); + } + } + + @Test + public void testTableRequestAttributesFactoryGet() throws IOException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (Connection conn = ConnectionFactory.createConnection(conf)) { + Table table = conn.getTableBuilder(TABLE_NAME, null).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; + }).build(); + table.get(new Get(ROW_KEY_TABLE_FACTORY_GET)); + } + } + + @Test + public void testTableRequestAttributesFactoryScan() throws IOException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (Connection conn = ConnectionFactory.createConnection(conf)) { + Table table = conn.getTableBuilder(TABLE_NAME, null).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; + }).build(); + ResultScanner scanner = table.getScanner(new Scan().withStartRow(ROW_KEY_TABLE_FACTORY_SCAN) + .withStopRow(ROW_KEY_TABLE_FACTORY_SCAN, true)); + scanner.next(); + } + } + + @Test + public void testTableRequestAttributesFactoryPut() throws IOException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (Connection conn = ConnectionFactory.createConnection(conf)) { + Table table = conn.getTableBuilder(TABLE_NAME, null).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; + }).build(); + Put put = new Put(ROW_KEY_TABLE_FACTORY_PUT); + put.addColumn(FAMILY, Bytes.toBytes("c"), Bytes.toBytes("v")); + table.put(put); + } + } + + @Test + public void testTableRequestAttributesFactoryOverridesStaticAttributes() throws IOException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (Connection conn = ConnectionFactory.createConnection(conf)) { + Table table = conn.getTableBuilder(TABLE_NAME, null) + .setRequestAttribute(IGNORED_KEY, IGNORED_VALUE).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; + }).build(); + table.get(new Get(ROW_KEY_TABLE_FACTORY_GET)); + } + } + + @Test + public void testTableFixedRequestAttributesFactory() throws IOException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (Connection conn = ConnectionFactory.createConnection(conf)) { + Table table = conn.getTableBuilder(TABLE_NAME, null).setRequestAttributesFactory( + FixedRequestAttributesFactory.newBuilder().setAttribute(FACTORY_KEY, FACTORY_VALUE).build()) + .build(); + table.get(new Get(ROW_KEY_TABLE_FACTORY_GET)); + } + } + + @Test + public void testBufferedMutatorRequestAttributesFactory() throws IOException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (Connection conn = ConnectionFactory.createConnection(conf)) { + BufferedMutatorParams params = + new BufferedMutatorParams(TABLE_NAME).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; + }); + BufferedMutator bufferedMutator = conn.getBufferedMutator(params); + Put put = new Put(ROW_KEY_BM_FACTORY); + put.addColumn(FAMILY, Bytes.toBytes("c"), Bytes.toBytes("v")); + bufferedMutator.mutate(put); + bufferedMutator.flush(); + } + } + + @Test + public void testBufferedMutatorRequestAttributesFactoryOverridesStaticAttributes() + throws IOException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (Connection conn = ConnectionFactory.createConnection(conf)) { + BufferedMutatorParams params = new BufferedMutatorParams(TABLE_NAME) + .setRequestAttribute(IGNORED_KEY, IGNORED_VALUE).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; + }); + BufferedMutator bufferedMutator = conn.getBufferedMutator(params); + Put put = new Put(ROW_KEY_BM_FACTORY); + put.addColumn(FAMILY, Bytes.toBytes("c"), Bytes.toBytes("v")); + bufferedMutator.mutate(put); + bufferedMutator.flush(); + } + } + + @Test + public void testAsyncBufferedMutatorRequestAttributesFactory() + throws IOException, ExecutionException, InterruptedException { + Configuration conf = TEST_UTIL.getConfiguration(); + try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(conf).get()) { + AsyncBufferedMutator bufferedMutator = + conn.getBufferedMutatorBuilder(TABLE_NAME).setRequestAttributesFactory(() -> { + Map attrs = new HashMap<>(); + attrs.put(FACTORY_KEY, FACTORY_VALUE); + return attrs; + }).build(); + Put put = new Put(ROW_KEY_ASYNC_BM_FACTORY); + put.addColumn(FAMILY, Bytes.toBytes("c"), Bytes.toBytes("v")); + CompletableFuture future = bufferedMutator.mutate(put); + bufferedMutator.flush(); + future.get(); + } + } + private static Map addRandomRequestAttributes() { Map requestAttributes = new HashMap<>(); int j = Math.max(2, (int) (10 * Math.random())); @@ -462,6 +602,7 @@ public boolean preScannerNext(ObserverContext