diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index baae6cf6a95..f251a8f5dd8 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -46,9 +46,6 @@ import com.google.protobuf.RpcChannel;
/**
* The asynchronous administrative API for HBase.
- *
- * This feature is still under development, so marked as IA.Private. Will change to public when
- * done. Use it with caution.
* @since 2.0.0
*/
@InterfaceAudience.Public
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 320d970b013..d5df7857556 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
@@ -282,7 +282,7 @@ class AsyncConnectionImpl implements AsyncConnection {
return new AsyncAdminBuilderBase(connConf) {
@Override
public AsyncAdmin build() {
- return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this);
+ return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);
}
};
}
@@ -292,7 +292,8 @@ class AsyncConnectionImpl implements AsyncConnection {
return new AsyncAdminBuilderBase(connConf) {
@Override
public AsyncAdmin build() {
- RawAsyncHBaseAdmin rawAdmin = new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, this);
+ RawAsyncHBaseAdmin rawAdmin =
+ new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);
return new AsyncHBaseAdmin(rawAdmin, pool);
}
};
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 04005eb3001..250a38c7ad0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hbase.client;
+import com.google.protobuf.RpcChannel;
+
import java.util.Collection;
import java.util.EnumSet;
import java.util.List;
@@ -28,8 +30,6 @@ import java.util.concurrent.ExecutorService;
import java.util.function.Function;
import java.util.regex.Pattern;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -45,17 +45,19 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.yetus.audience.InterfaceAudience;
-import com.google.protobuf.RpcChannel;
-
/**
- * The implementation of AsyncAdmin.
+ * Just a wrapper of {@link RawAsyncHBaseAdmin}. The difference is that users need to provide a
+ * thread pool when constructing this class, and the callback methods registered to the returned
+ * {@link CompletableFuture} will be executed in this thread pool. So usually it is safe for users
+ * to do anything they want in the callbacks without breaking the rpc framework.
* @since 2.0.0
+ * @see RawAsyncHBaseAdmin
+ * @see AsyncConnection#getAdmin(ExecutorService)
+ * @see AsyncConnection#getAdminBuilder(ExecutorService)
*/
@InterfaceAudience.Private
public class AsyncHBaseAdmin implements AsyncAdmin {
- private static final Log LOG = LogFactory.getLog(AsyncHBaseAdmin.class);
-
private final RawAsyncHBaseAdmin rawAdmin;
private final ExecutorService pool;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 1d807978b57..d77cd156895 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;
import org.apache.hadoop.hbase.shaded.io.netty.util.TimerTask;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -248,7 +249,15 @@ import com.google.protobuf.RpcChannel;
/**
* The implementation of AsyncAdmin.
+ *
+ * The word 'Raw' means that this is a low level class. The returned {@link CompletableFuture} will
+ * be finished inside the rpc framework thread, which means that the callbacks registered to the
+ * {@link CompletableFuture} will also be executed inside the rpc framework thread. So users who use
+ * this class should not try to do time consuming tasks in the callbacks.
* @since 2.0.0
+ * @see AsyncHBaseAdmin
+ * @see AsyncConnection#getAdmin()
+ * @see AsyncConnection#getAdminBuilder()
*/
@InterfaceAudience.Private
public class RawAsyncHBaseAdmin implements AsyncAdmin {
@@ -258,6 +267,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private final AsyncConnectionImpl connection;
+ private final HashedWheelTimer retryTimer;
+
private final RawAsyncTable metaTable;
private final long rpcTimeoutNs;
@@ -272,8 +283,10 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private final NonceGenerator ng;
- RawAsyncHBaseAdmin(AsyncConnectionImpl connection, AsyncAdminBuilderBase builder) {
+ RawAsyncHBaseAdmin(AsyncConnectionImpl connection, HashedWheelTimer retryTimer,
+ AsyncAdminBuilderBase builder) {
this.connection = connection;
+ this.retryTimer = retryTimer;
this.metaTable = connection.getRawTable(META_TABLE_NAME);
this.rpcTimeoutNs = builder.rpcTimeoutNs;
this.operationTimeoutNs = builder.operationTimeoutNs;
@@ -489,7 +502,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
Preconditions.checkNotNull(tableName, "table name is null");
return this. procedureCall(request,
(s, c, req, done) -> s.createTable(c, req, done), (resp) -> resp.getProcId(),
- new CreateTableProcedureBiConsumer(this, tableName));
+ new CreateTableProcedureBiConsumer(tableName));
}
@Override
@@ -497,7 +510,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(RequestConverter
.buildDeleteTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()),
(s, c, req, done) -> s.deleteTable(c, req, done), (resp) -> resp.getProcId(),
- new DeleteTableProcedureBiConsumer(this, tableName));
+ new DeleteTableProcedureBiConsumer(tableName));
}
@Override
@@ -505,7 +518,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(
RequestConverter.buildTruncateTableRequest(tableName, preserveSplits, ng.getNonceGroup(),
ng.newNonce()), (s, c, req, done) -> s.truncateTable(c, req, done),
- (resp) -> resp.getProcId(), new TruncateTableProcedureBiConsumer(this, tableName));
+ (resp) -> resp.getProcId(), new TruncateTableProcedureBiConsumer(tableName));
}
@Override
@@ -513,7 +526,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(RequestConverter
.buildEnableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()),
(s, c, req, done) -> s.enableTable(c, req, done), (resp) -> resp.getProcId(),
- new EnableTableProcedureBiConsumer(this, tableName));
+ new EnableTableProcedureBiConsumer(tableName));
}
@Override
@@ -521,7 +534,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(RequestConverter
.buildDisableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce()),
(s, c, req, done) -> s.disableTable(c, req, done), (resp) -> resp.getProcId(),
- new DisableTableProcedureBiConsumer(this, tableName));
+ new DisableTableProcedureBiConsumer(tableName));
}
@Override
@@ -634,7 +647,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(
RequestConverter.buildAddColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
ng.newNonce()), (s, c, req, done) -> s.addColumn(c, req, done), (resp) -> resp.getProcId(),
- new AddColumnFamilyProcedureBiConsumer(this, tableName));
+ new AddColumnFamilyProcedureBiConsumer(tableName));
}
@Override
@@ -642,7 +655,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(
RequestConverter.buildDeleteColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
ng.newNonce()), (s, c, req, done) -> s.deleteColumn(c, req, done),
- (resp) -> resp.getProcId(), new DeleteColumnFamilyProcedureBiConsumer(this, tableName));
+ (resp) -> resp.getProcId(), new DeleteColumnFamilyProcedureBiConsumer(tableName));
}
@Override
@@ -651,7 +664,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(
RequestConverter.buildModifyColumnRequest(tableName, columnFamily, ng.getNonceGroup(),
ng.newNonce()), (s, c, req, done) -> s.modifyColumn(c, req, done),
- (resp) -> resp.getProcId(), new ModifyColumnFamilyProcedureBiConsumer(this, tableName));
+ (resp) -> resp.getProcId(), new ModifyColumnFamilyProcedureBiConsumer(tableName));
}
@Override
@@ -659,7 +672,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(
RequestConverter.buildCreateNamespaceRequest(descriptor),
(s, c, req, done) -> s.createNamespace(c, req, done), (resp) -> resp.getProcId(),
- new CreateNamespaceProcedureBiConsumer(this, descriptor.getName()));
+ new CreateNamespaceProcedureBiConsumer(descriptor.getName()));
}
@Override
@@ -667,7 +680,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(
RequestConverter.buildModifyNamespaceRequest(descriptor),
(s, c, req, done) -> s.modifyNamespace(c, req, done), (resp) -> resp.getProcId(),
- new ModifyNamespaceProcedureBiConsumer(this, descriptor.getName()));
+ new ModifyNamespaceProcedureBiConsumer(descriptor.getName()));
}
@Override
@@ -675,7 +688,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return this. procedureCall(
RequestConverter.buildDeleteNamespaceRequest(name),
(s, c, req, done) -> s.deleteNamespace(c, req, done), (resp) -> resp.getProcId(),
- new DeleteNamespaceProcedureBiConsumer(this, name));
+ new DeleteNamespaceProcedureBiConsumer(name));
}
@Override
@@ -717,11 +730,11 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
public CompletableFuture> getTableRegions(TableName tableName) {
if (tableName.equals(META_TABLE_NAME)) {
return connection.getLocator().getRegionLocation(tableName, null, null, operationTimeoutNs)
- .thenApply(loc -> Arrays.asList(loc.getRegionInfo()));
+ .thenApply(loc -> Collections.singletonList(loc.getRegion()));
} else {
return AsyncMetaTableAccessor.getTableHRegionLocations(metaTable, Optional.of(tableName))
.thenApply(
- locs -> locs.stream().map(loc -> loc.getRegionInfo()).collect(Collectors.toList()));
+ locs -> locs.stream().map(loc -> loc.getRegion()).collect(Collectors.toList()));
}
}
@@ -771,7 +784,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return;
}
- RegionInfo regionInfo = location.getRegionInfo();
+ RegionInfo regionInfo = location.getRegion();
this. newAdminCaller()
.serverName(serverName)
.action(
@@ -886,7 +899,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
.toStringBinary(regionName)));
return;
}
- compact(location.getServerName(), location.getRegionInfo(), major, columnFamily)
+ compact(location.getServerName(), location.getRegion(), major, columnFamily)
.whenComplete((ret, err2) -> {
if (err2 != null) {
future.completeExceptionally(err2);
@@ -929,8 +942,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
/**
* Compact column family of a table, Asynchronous operation even if CompletableFuture.get()
*/
- private CompletableFuture compact(final TableName tableName, byte[] columnFamily,
- final boolean major, CompactType compactType) {
+ private CompletableFuture compact(TableName tableName, byte[] columnFamily, boolean major,
+ CompactType compactType) {
if (CompactType.MOB.equals(compactType)) {
// TODO support MOB compact.
return failedFuture(new UnsupportedOperationException("MOB compact does not support"));
@@ -941,23 +954,18 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
future.completeExceptionally(err);
return;
}
- List> compactFutures = new ArrayList<>();
- for (HRegionLocation location : locations) {
- if (location.getRegionInfo() == null || location.getRegionInfo().isOffline()) continue;
- if (location.getServerName() == null) continue;
- compactFutures
- .add(compact(location.getServerName(), location.getRegionInfo(), major, columnFamily));
- }
+ CompletableFuture>[] compactFutures = locations.stream().filter(l -> l.getRegion() != null)
+ .filter(l -> !l.getRegion().isOffline()).filter(l -> l.getServerName() != null)
+ .map(l -> compact(l.getServerName(), l.getRegion(), major, columnFamily))
+ .toArray(CompletableFuture>[]::new);
// future complete unless all of the compact futures are completed.
- CompletableFuture
- .allOf(compactFutures.toArray(new CompletableFuture>[compactFutures.size()]))
- .whenComplete((ret, err2) -> {
- if (err2 != null) {
- future.completeExceptionally(err2);
- } else {
- future.complete(ret);
- }
- });
+ CompletableFuture.allOf(compactFutures).whenComplete((ret, err2) -> {
+ if (err2 != null) {
+ future.completeExceptionally(err2);
+ } else {
+ future.complete(ret);
+ }
+ });
});
return future;
}
@@ -994,7 +1002,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
result.completeExceptionally(err);
return;
}
- RegionInfo regionInfo = location.getRegionInfo();
+ RegionInfo regionInfo = location.getRegion();
if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
result.completeExceptionally(new IllegalArgumentException(
"Can't invoke merge on non-default regions directly"));
@@ -1094,7 +1102,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
this. procedureCall(request,
(s, c, req, done) -> s.mergeTableRegions(c, req, done), (resp) -> resp.getProcId(),
- new MergeTableRegionProcedureBiConsumer(this, tableName)).whenComplete((ret, err2) -> {
+ new MergeTableRegionProcedureBiConsumer(tableName)).whenComplete((ret, err2) -> {
if (err2 != null) {
future.completeExceptionally(err2);
} else {
@@ -1171,11 +1179,11 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
.whenComplete((loc, err) -> {
if (err != null) {
result.completeExceptionally(err);
- } else if (loc == null || loc.getRegionInfo() == null) {
+ } else if (loc == null || loc.getRegion() == null) {
result.completeExceptionally(new IllegalArgumentException(
"Region does not found: rowKey=" + Bytes.toStringBinary(splitPoint)));
} else {
- splitRegion(loc.getRegionInfo().getRegionName(), splitPoint)
+ splitRegion(loc.getRegion().getRegionName(), splitPoint)
.whenComplete((ret, err2) -> {
if (err2 != null) {
result.completeExceptionally(err2);
@@ -1270,7 +1278,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
this.procedureCall(request,
(s, c, req, done) -> s.splitRegion(c, req, done), (resp) -> resp.getProcId(),
- new SplitTableRegionProcedureBiConsumer(this, tableName)).whenComplete((ret, err2) -> {
+ new SplitTableRegionProcedureBiConsumer(tableName)).whenComplete((ret, err2) -> {
if (err2 != null) {
future.completeExceptionally(err2);
} else {
@@ -2161,7 +2169,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
return;
}
LOG.info("location is " + location);
- if (!location.isPresent() || location.get().getRegionInfo() == null) {
+ if (!location.isPresent() || location.get().getRegion() == null) {
LOG.info("unknown location is " + location);
returnedFuture.completeExceptionally(new UnknownRegionException(
"Invalid region name or encoded region name: "
@@ -2200,7 +2208,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
if (err != null) {
future.completeExceptionally(err);
} else {
- future.complete(location.getRegionInfo());
+ future.complete(location.getRegion());
}
});
return future;
@@ -2239,11 +2247,6 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
}
private abstract class ProcedureBiConsumer implements BiConsumer {
- protected final AsyncAdmin admin;
-
- ProcedureBiConsumer(AsyncAdmin admin) {
- this.admin = admin;
- }
abstract void onFinished();
@@ -2262,8 +2265,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private abstract class TableProcedureBiConsumer extends ProcedureBiConsumer {
protected final TableName tableName;
- TableProcedureBiConsumer(final AsyncAdmin admin, final TableName tableName) {
- super(admin);
+ TableProcedureBiConsumer(TableName tableName) {
this.tableName = tableName;
}
@@ -2288,8 +2290,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private abstract class NamespaceProcedureBiConsumer extends ProcedureBiConsumer {
protected final String namespaceName;
- NamespaceProcedureBiConsumer(final AsyncAdmin admin, final String namespaceName) {
- super(admin);
+ NamespaceProcedureBiConsumer(String namespaceName) {
this.namespaceName = namespaceName;
}
@@ -2312,8 +2313,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class CreateTableProcedureBiConsumer extends TableProcedureBiConsumer {
- CreateTableProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ CreateTableProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2324,8 +2325,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class DeleteTableProcedureBiConsumer extends TableProcedureBiConsumer {
- DeleteTableProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ DeleteTableProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2342,8 +2343,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class TruncateTableProcedureBiConsumer extends TableProcedureBiConsumer {
- TruncateTableProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ TruncateTableProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2354,8 +2355,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class EnableTableProcedureBiConsumer extends TableProcedureBiConsumer {
- EnableTableProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ EnableTableProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2366,8 +2367,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class DisableTableProcedureBiConsumer extends TableProcedureBiConsumer {
- DisableTableProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ DisableTableProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2378,8 +2379,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class AddColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
- AddColumnFamilyProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ AddColumnFamilyProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2390,8 +2391,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class DeleteColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
- DeleteColumnFamilyProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ DeleteColumnFamilyProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2402,8 +2403,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class ModifyColumnFamilyProcedureBiConsumer extends TableProcedureBiConsumer {
- ModifyColumnFamilyProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ ModifyColumnFamilyProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2414,8 +2415,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class CreateNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
- CreateNamespaceProcedureBiConsumer(AsyncAdmin admin, String namespaceName) {
- super(admin, namespaceName);
+ CreateNamespaceProcedureBiConsumer(String namespaceName) {
+ super(namespaceName);
}
@Override
@@ -2426,8 +2427,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class DeleteNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
- DeleteNamespaceProcedureBiConsumer(AsyncAdmin admin, String namespaceName) {
- super(admin, namespaceName);
+ DeleteNamespaceProcedureBiConsumer(String namespaceName) {
+ super(namespaceName);
}
@Override
@@ -2438,8 +2439,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class ModifyNamespaceProcedureBiConsumer extends NamespaceProcedureBiConsumer {
- ModifyNamespaceProcedureBiConsumer(AsyncAdmin admin, String namespaceName) {
- super(admin, namespaceName);
+ ModifyNamespaceProcedureBiConsumer(String namespaceName) {
+ super(namespaceName);
}
@Override
@@ -2450,8 +2451,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class MergeTableRegionProcedureBiConsumer extends TableProcedureBiConsumer {
- MergeTableRegionProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ MergeTableRegionProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2462,8 +2463,8 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
private class SplitTableRegionProcedureBiConsumer extends TableProcedureBiConsumer {
- SplitTableRegionProcedureBiConsumer(AsyncAdmin admin, TableName tableName) {
- super(admin, tableName);
+ SplitTableRegionProcedureBiConsumer(TableName tableName) {
+ super(tableName);
}
@Override
@@ -2497,12 +2498,12 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
if (error != null) {
LOG.warn("failed to get the procedure result procId=" + procId,
ConnectionUtils.translateException(error));
- connection.RETRY_TIMER.newTimeout(t -> getProcedureResult(procId, future), pauseNs,
+ retryTimer.newTimeout(t -> getProcedureResult(procId, future), pauseNs,
TimeUnit.NANOSECONDS);
return;
}
if (response.getState() == GetProcedureResultResponse.State.RUNNING) {
- connection.RETRY_TIMER.newTimeout(t -> getProcedureResult(procId, future), pauseNs,
+ retryTimer.newTimeout(t -> getProcedureResult(procId, future), pauseNs,
TimeUnit.NANOSECONDS);
return;
}
@@ -2700,9 +2701,9 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
List regionStates = new ArrayList<>();
List> futures = new ArrayList<>();
locations.stream().filter(loc -> loc.getServerName() != null)
- .filter(loc -> loc.getRegionInfo() != null)
- .filter(loc -> !loc.getRegionInfo().isOffline())
- .map(loc -> loc.getRegionInfo().getRegionName()).forEach(region -> {
+ .filter(loc -> loc.getRegion() != null)
+ .filter(loc -> !loc.getRegion().isOffline())
+ .map(loc -> loc.getRegion().getRegionName()).forEach(region -> {
futures.add(getCompactionStateForRegion(region).whenComplete((regionState, err2) -> {
// If any region compaction state is MAJOR_AND_MINOR
// the table compaction state is MAJOR_AND_MINOR, too.
@@ -2770,7 +2771,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
(controller, stub) -> this
. adminCall(
controller, stub, RequestConverter.buildGetRegionInfoRequest(location
- .getRegionInfo().getRegionName(), true), (s, c, req, done) -> s
+ .getRegion().getRegionName(), true), (s, c, req, done) -> s
.getRegionInfo(controller, req, done), resp -> resp))
.serverName(serverName).call().whenComplete((resp2, err2) -> {
if (err2 != null) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java
index 56be6575c92..08565e07af4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java
@@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
/**
* Test provides some very generic helpers which might be used across the tests
- * *** for running UTs in hbase-server
*/
public abstract class GenericTestUtils {