HBASE-19241 Improve javadoc for AsyncAdmin and cleanup warnings for the implementation classes

This commit is contained in:
zhangduo 2017-11-13 16:40:10 +08:00
parent 1ba7cc2164
commit 5c312667ed
5 changed files with 95 additions and 95 deletions

View File

@ -46,9 +46,6 @@ import com.google.protobuf.RpcChannel;
/**
* The asynchronous administrative API for HBase.
* <p>
* 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

View File

@ -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);
}
};

View File

@ -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;

View File

@ -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.
* <p>
* 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.<CreateTableRequest, CreateTableResponse> 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.<DeleteTableRequest, DeleteTableResponse> 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.<TruncateTableRequest, TruncateTableResponse> 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.<EnableTableRequest, EnableTableResponse> 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.<DisableTableRequest, DisableTableResponse> 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.<AddColumnRequest, AddColumnResponse> 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.<DeleteColumnRequest, DeleteColumnResponse> 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.<ModifyColumnRequest, ModifyColumnResponse> 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.<CreateNamespaceRequest, CreateNamespaceResponse> 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.<ModifyNamespaceRequest, ModifyNamespaceResponse> 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.<DeleteNamespaceRequest, DeleteNamespaceResponse> 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<List<RegionInfo>> 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.<Void> 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<Void> compact(final TableName tableName, byte[] columnFamily,
final boolean major, CompactType compactType) {
private CompletableFuture<Void> 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,17 +954,12 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
future.completeExceptionally(err);
return;
}
List<CompletableFuture<Void>> 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) -> {
CompletableFuture.allOf(compactFutures).whenComplete((ret, err2) -> {
if (err2 != null) {
future.completeExceptionally(err2);
} else {
@ -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.<MergeTableRegionsRequest, MergeTableRegionsResponse> 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.<SplitTableRegionRequest, SplitTableRegionResponse>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<Void, Throwable> {
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<CompactionState> regionStates = new ArrayList<>();
List<CompletableFuture<CompactionState>> 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
.<GetRegionInfoRequest, GetRegionInfoResponse, GetRegionInfoResponse> 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) {

View File

@ -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 {