@InterfaceAudience.Private class RawAsyncHBaseAdmin extends Object implements AsyncAdmin
The word 'Raw' means that this is a low level class. The returned CompletableFuture
will
be finished inside the rpc framework thread, which means that the callbacks registered to the
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.
AsyncHBaseAdmin
,
AsyncConnection.getAdmin()
,
AsyncConnection.getAdminBuilder()
Modifier and Type | Field and Description |
---|---|
private AsyncConnectionImpl |
connection |
static String |
FLUSH_TABLE_PROCEDURE_SIGNATURE |
private static org.slf4j.Logger |
LOG |
private int |
maxAttempts |
private AsyncTable<AdvancedScanResultConsumer> |
metaTable |
private NonceGenerator |
ng |
private long |
operationTimeoutNs |
private long |
pauseNs |
private org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer |
retryTimer |
private long |
rpcTimeoutNs |
private int |
startLogErrorsCnt |
Constructor and Description |
---|
RawAsyncHBaseAdmin(AsyncConnectionImpl connection,
org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer retryTimer,
AsyncAdminBuilderBase builder) |
Modifier and Type | Method and Description |
---|---|
CompletableFuture<Boolean> |
abortProcedure(long procId,
boolean mayInterruptIfRunning)
Abort a procedure
Do not use.
|
CompletableFuture<Void> |
addColumnFamily(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Add a column family to an existing table.
|
CompletableFuture<Void> |
addReplicationPeer(String peerId,
ReplicationPeerConfig peerConfig,
boolean enabled)
Add a new replication peer for replicating data to slave cluster
|
private <PREQ,PRESP,RESP> |
adminCall(HBaseRpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.Interface stub,
PREQ preq,
RawAsyncHBaseAdmin.AdminRpcCall<PRESP,PREQ> rpcCall,
RawAsyncHBaseAdmin.Converter<RESP,PRESP> respConverter) |
CompletableFuture<Void> |
appendReplicationPeerTableCFs(String id,
Map<TableName,List<String>> tableCfs)
Append the replicable table-cf config of the specified peer
|
CompletableFuture<Void> |
assign(byte[] regionName) |
CompletableFuture<Boolean> |
balance(boolean forcible)
Invoke the balancer.
|
CompletableFuture<Boolean> |
balancerSwitch(boolean on)
Turn the load balancer on or off.
|
private <PREQ,PRESP,RESP> |
call(HBaseRpcController controller,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.Interface stub,
PREQ preq,
RawAsyncHBaseAdmin.MasterRpcCall<PRESP,PREQ> rpcCall,
RawAsyncHBaseAdmin.Converter<RESP,PRESP> respConverter) |
CompletableFuture<Boolean> |
catalogJanitorSwitch(boolean enabled)
Turn the catalog janitor on/off.
|
private void |
checkAndGetTableName(byte[] encodeRegionName,
AtomicReference<TableName> tableName,
CompletableFuture<TableName> result) |
private CompletableFuture<Void> |
checkAndSyncTableToPeerClusters(TableName tableName,
byte[][] splits)
Connect to peer and check the table descriptor on peer:
Create the same table on peer when not exist.
Throw an exception if the table already has replication enabled on any of the column
families.
Throw an exception if the table exists on peer cluster but descriptors are not same.
|
private CompletableFuture<TableName> |
checkRegionsAndGetTableName(byte[] encodeRegionNameA,
byte[] encodeRegionNameB) |
CompletableFuture<Boolean> |
cleanerChoreSwitch(boolean enabled)
Turn the cleaner chore on/off.
|
private CompletableFuture<CacheEvictionStats> |
clearBlockCache(ServerName serverName,
List<RegionInfo> hris) |
CompletableFuture<CacheEvictionStats> |
clearBlockCache(TableName tableName)
Clear all the blocks corresponding to this table from BlockCache.
|
CompletableFuture<Void> |
clearCompactionQueues(ServerName serverName,
Set<String> queues)
Clear compacting queues on a region server.
|
CompletableFuture<List<ServerName>> |
clearDeadServers(List<ServerName> servers)
Clear dead region servers from master.
|
CompletableFuture<Void> |
cloneSnapshot(String snapshotName,
TableName tableName)
Create a new table by cloning the snapshot content.
|
private CompletableFuture<Void> |
compact(ServerName sn,
RegionInfo hri,
boolean major,
byte[] columnFamily)
Compact the region at specific region server.
|
private CompletableFuture<Void> |
compact(TableName tableName,
byte[] columnFamily,
boolean major,
CompactType compactType)
Compact column family of a table, Asynchronous operation even if CompletableFuture.get()
|
CompletableFuture<Void> |
compact(TableName tableName,
byte[] columnFamily,
CompactType compactType)
Compact a column family within a table.
|
CompletableFuture<Void> |
compact(TableName tableName,
CompactType compactType)
Compact a table.
|
CompletableFuture<Void> |
compactRegion(byte[] regionName)
Compact an individual region.
|
CompletableFuture<Void> |
compactRegion(byte[] regionName,
byte[] columnFamily)
Compact a column family within a region.
|
private CompletableFuture<Void> |
compactRegion(byte[] regionName,
byte[] columnFamily,
boolean major) |
CompletableFuture<Void> |
compactRegionServer(ServerName sn)
Compact all regions on the region server.
|
private CompletableFuture<Void> |
compactRegionServer(ServerName sn,
boolean major) |
private boolean |
compareRegionsWithSplitKeys(List<HRegionLocation> locations,
byte[][] splitKeys) |
private CompletableFuture<Void> |
compareTableWithPeerCluster(TableName tableName,
TableDescriptor tableDesc,
ReplicationPeerDescription peer,
AsyncAdmin peerAdmin) |
private <T> void |
completeConditionalOnFuture(CompletableFuture<T> dependentFuture,
CompletableFuture<T> parentFuture) |
private <T> boolean |
completeExceptionally(CompletableFuture<T> future,
Throwable error) |
<S,R> CompletableFuture<R> |
coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker,
ServiceCaller<S,R> callable)
Execute the given coprocessor call on the master.
|
<S,R> CompletableFuture<R> |
coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker,
ServiceCaller<S,R> callable,
ServerName serverName)
Execute the given coprocessor call on the given region server.
|
CompletableFuture<Void> |
createNamespace(NamespaceDescriptor descriptor)
Create a new namespace.
|
CompletableFuture<Void> |
createTable(TableDescriptor desc)
Creates a new table.
|
CompletableFuture<Void> |
createTable(TableDescriptor desc,
byte[][] splitKeys)
Creates a new table with an initial set of empty regions defined by the specified split keys.
|
CompletableFuture<Void> |
createTable(TableDescriptor desc,
byte[] startKey,
byte[] endKey,
int numRegions)
Creates a new table with the specified number of regions.
|
private CompletableFuture<Void> |
createTable(TableName tableName,
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request) |
CompletableFuture<Void> |
decommissionRegionServers(List<ServerName> servers,
boolean offload)
Mark region server(s) as decommissioned to prevent additional regions from getting
assigned to them.
|
CompletableFuture<Void> |
deleteColumnFamily(TableName tableName,
byte[] columnFamily)
Delete a column family from a table.
|
CompletableFuture<Void> |
deleteNamespace(String name)
Delete an existing namespace.
|
CompletableFuture<Void> |
deleteSnapshot(String snapshotName)
Delete an existing snapshot.
|
CompletableFuture<Void> |
deleteSnapshots()
Delete all existing snapshots.
|
CompletableFuture<Void> |
deleteSnapshots(Pattern snapshotNamePattern)
Delete existing snapshots whose names match the pattern passed.
|
CompletableFuture<Void> |
deleteTable(TableName tableName)
Deletes a table.
|
CompletableFuture<Void> |
deleteTableSnapshots(Pattern tableNamePattern)
Delete all existing snapshots matching the given table name pattern.
|
CompletableFuture<Void> |
deleteTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern)
Delete all existing snapshots matching the given table name regular expression and snapshot
name regular expression.
|
CompletableFuture<Void> |
disableReplicationPeer(String peerId)
Stop the replication stream to the specified peer
|
CompletableFuture<Void> |
disableTable(TableName tableName)
Disable a table.
|
CompletableFuture<Void> |
disableTableReplication(TableName tableName)
Disable a table's replication switch.
|
CompletableFuture<Void> |
enableReplicationPeer(String peerId)
Restart the replication stream to the specified peer
|
CompletableFuture<Void> |
enableTable(TableName tableName)
Enable a table.
|
CompletableFuture<Void> |
enableTableReplication(TableName tableName)
Enable a table's replication switch.
|
CompletableFuture<Void> |
execProcedure(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster.
|
CompletableFuture<byte[]> |
execProcedureWithReturn(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster.
|
private <T> CompletableFuture<T> |
failedFuture(Throwable error) |
private CompletableFuture<Void> |
flush(ServerName serverName,
RegionInfo regionInfo) |
CompletableFuture<Void> |
flush(TableName tableName)
Flush a table.
|
CompletableFuture<Void> |
flushRegion(byte[] regionName)
Flush an individual region.
|
CompletableFuture<Void> |
flushRegionServer(ServerName sn)
Flush all region on the region server.
|
CompletableFuture<ClusterMetrics> |
getClusterMetrics() |
CompletableFuture<ClusterMetrics> |
getClusterMetrics(EnumSet<ClusterMetrics.Option> options) |
CompletableFuture<CompactionState> |
getCompactionState(TableName tableName,
CompactType compactType)
Get the current compaction state of a table.
|
CompletableFuture<CompactionState> |
getCompactionStateForRegion(byte[] regionName)
Get the current compaction state of region.
|
private CompletableFuture<List<SnapshotDescription>> |
getCompletedSnapshots(Pattern pattern) |
private CompletableFuture<List<SnapshotDescription>> |
getCompletedSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern) |
CompletableFuture<TableDescriptor> |
getDescriptor(TableName tableName)
Method for getting the tableDescriptor
|
CompletableFuture<Optional<Long>> |
getLastMajorCompactionTimestamp(TableName tableName)
Get the timestamp of the last major compaction for the passed table.
|
CompletableFuture<Optional<Long>> |
getLastMajorCompactionTimestampForRegion(byte[] regionName)
Get the timestamp of the last major compaction for the passed region.
|
CompletableFuture<String> |
getLocks()
List locks.
|
CompletableFuture<NamespaceDescriptor> |
getNamespaceDescriptor(String name)
Get a namespace descriptor by name
|
private void |
getProcedureResult(long procId,
CompletableFuture<Void> future,
int retries) |
CompletableFuture<String> |
getProcedures()
List procedures
|
CompletableFuture<List<QuotaSettings>> |
getQuota(QuotaFilter filter)
List the quotas based on the filter.
|
private CompletableFuture<RegionInfo> |
getRegionInfo(byte[] regionNameOrEncodedRegionName)
Get the region info for the passed region name.
|
(package private) CompletableFuture<HRegionLocation> |
getRegionLocation(byte[] regionNameOrEncodedRegionName)
Get the region location for the passed region name.
|
private CompletableFuture<List<RegionMetrics>> |
getRegionMetrics(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request,
ServerName serverName) |
CompletableFuture<List<RegionMetrics>> |
getRegionMetrics(ServerName serverName)
Get a list of
RegionMetrics of all regions hosted on a region seerver. |
CompletableFuture<List<RegionMetrics>> |
getRegionMetrics(ServerName serverName,
TableName tableName)
Get a list of
RegionMetrics of all regions hosted on a region seerver for a table. |
CompletableFuture<List<RegionInfo>> |
getRegions(ServerName serverName)
Get all the online regions on a region server.
|
CompletableFuture<List<RegionInfo>> |
getRegions(TableName tableName)
Get the regions of a given table.
|
CompletableFuture<ReplicationPeerConfig> |
getReplicationPeerConfig(String peerId)
Returns the configured ReplicationPeerConfig for the specified peer
|
CompletableFuture<List<SecurityCapability>> |
getSecurityCapabilities() |
private byte[][] |
getSplitKeys(byte[] startKey,
byte[] endKey,
int numRegions) |
private CompletableFuture<List<TableDescriptor>> |
getTableDescriptors(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request) |
private CompletableFuture<List<HRegionLocation>> |
getTableHRegionLocations(TableName tableName)
List all region locations for the specific table.
|
private CompletableFuture<List<TableName>> |
getTableNames(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request) |
private CompletableFuture<byte[][]> |
getTableSplits(TableName tableName) |
private CompletableFuture<Void> |
internalDeleteSnapshot(SnapshotDescription snapshot) |
private CompletableFuture<Void> |
internalDeleteSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern) |
private CompletableFuture<Void> |
internalRestoreSnapshot(String snapshotName,
TableName tableName) |
CompletableFuture<Boolean> |
isBalancerEnabled()
Query the current state of the balancer.
|
CompletableFuture<Boolean> |
isCatalogJanitorEnabled()
Query on the catalog janitor state.
|
CompletableFuture<Boolean> |
isCleanerChoreEnabled()
Query the current state of the cleaner chore.
|
CompletableFuture<Boolean> |
isMasterInMaintenanceMode()
Check whether master is in maintenance mode
|
CompletableFuture<Boolean> |
isMergeEnabled()
Query the current state of the Merge switch.
|
CompletableFuture<Boolean> |
isNormalizerEnabled()
Query the current state of the region normalizer
|
CompletableFuture<Boolean> |
isProcedureFinished(String signature,
String instance,
Map<String,String> props)
Check the current state of the specified procedure.
|
CompletableFuture<Boolean> |
isSnapshotFinished(SnapshotDescription snapshot)
Check the current state of the passed snapshot.
|
CompletableFuture<Boolean> |
isSplitEnabled()
Query the current state of the Split switch.
|
private CompletableFuture<Boolean> |
isSplitOrMergeOn(MasterSwitchType switchType) |
CompletableFuture<Boolean> |
isTableAvailable(TableName tableName) |
CompletableFuture<Boolean> |
isTableAvailable(TableName tableName,
byte[][] splitKeys)
Use this api to check if the table has been created with the specified number of splitkeys
which was used while creating the given table.
|
private CompletableFuture<Boolean> |
isTableAvailable(TableName tableName,
Optional<byte[][]> splitKeys) |
CompletableFuture<Boolean> |
isTableDisabled(TableName tableName) |
CompletableFuture<Boolean> |
isTableEnabled(TableName tableName) |
CompletableFuture<List<ServerName>> |
listDecommissionedRegionServers()
List region servers marked as decommissioned, which can not be assigned regions.
|
CompletableFuture<List<NamespaceDescriptor>> |
listNamespaceDescriptors()
List available namespace descriptors
|
CompletableFuture<List<TableCFs>> |
listReplicatedTableCFs()
Find all table and column families that are replicated from this cluster
|
CompletableFuture<List<ReplicationPeerDescription>> |
listReplicationPeers()
Return a list of replication peers.
|
CompletableFuture<List<ReplicationPeerDescription>> |
listReplicationPeers(Pattern pattern)
Return a list of replication peers.
|
private CompletableFuture<List<ReplicationPeerDescription>> |
listReplicationPeers(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request) |
CompletableFuture<List<SnapshotDescription>> |
listSnapshots()
List completed snapshots.
|
CompletableFuture<List<SnapshotDescription>> |
listSnapshots(Pattern pattern)
List all the completed snapshots matching the given pattern.
|
CompletableFuture<List<TableDescriptor>> |
listTableDescriptors(boolean includeSysTables)
List all the tables.
|
CompletableFuture<List<TableDescriptor>> |
listTableDescriptors(Pattern pattern,
boolean includeSysTables)
#listTables(boolean) |
CompletableFuture<List<TableDescriptor>> |
listTableDescriptorsByNamespace(String name)
Get list of table descriptors by namespace.
|
CompletableFuture<List<TableName>> |
listTableNames(boolean includeSysTables)
List all of the names of tables.
|
CompletableFuture<List<TableName>> |
listTableNames(Pattern pattern,
boolean includeSysTables)
List all of the names of userspace tables.
|
CompletableFuture<List<TableName>> |
listTableNamesByNamespace(String name)
Get list of table names by namespace.
|
CompletableFuture<List<SnapshotDescription>> |
listTableSnapshots(Pattern tableNamePattern)
List all the completed snapshots matching the given table name pattern.
|
CompletableFuture<List<SnapshotDescription>> |
listTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern)
List all the completed snapshots matching the given table name regular expression and snapshot
name regular expression.
|
CompletableFuture<Void> |
majorCompact(TableName tableName,
byte[] columnFamily,
CompactType compactType)
Major compact a column family within a table.
|
CompletableFuture<Void> |
majorCompact(TableName tableName,
CompactType compactType)
Major compact a table.
|
CompletableFuture<Void> |
majorCompactRegion(byte[] regionName)
Major compact a region.
|
CompletableFuture<Void> |
majorCompactRegion(byte[] regionName,
byte[] columnFamily)
Major compact a column family within region.
|
CompletableFuture<Void> |
majorCompactRegionServer(ServerName sn)
Compact all regions on the region server.
|
CompletableFuture<Void> |
mergeRegions(byte[] nameOfRegionA,
byte[] nameOfRegionB,
boolean forcible)
Merge two regions.
|
CompletableFuture<Boolean> |
mergeSwitch(boolean on)
Turn the Merge switch on or off.
|
CompletableFuture<Void> |
modifyColumnFamily(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Modify an existing column family on a table.
|
CompletableFuture<Void> |
modifyNamespace(NamespaceDescriptor descriptor)
Modify an existing namespace.
|
CompletableFuture<Void> |
modifyTable(TableDescriptor desc)
Modify an existing table, more IRB friendly version.
|
CompletableFuture<Void> |
move(byte[] regionName)
Move the region
r to a random server. |
CompletableFuture<Void> |
move(byte[] regionName,
ServerName destServerName)
Move the region
r to dest . |
private CompletableFuture<Void> |
moveRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request) |
private <T> AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder<T> |
newAdminCaller() |
private <T> AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder<T> |
newMasterCaller() |
private <T> AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder<T> |
newServerCaller() |
CompletableFuture<Boolean> |
normalize()
Invoke region normalizer.
|
CompletableFuture<Boolean> |
normalizerSwitch(boolean on)
Set region normalizer on/off.
|
CompletableFuture<Void> |
offline(byte[] regionName)
Offline specified region from master's in-memory state.
|
private <PREQ,PRESP> |
procedureCall(PREQ preq,
RawAsyncHBaseAdmin.MasterRpcCall<PRESP,PREQ> rpcCall,
RawAsyncHBaseAdmin.Converter<Long,PRESP> respConverter,
RawAsyncHBaseAdmin.ProcedureBiConsumer consumer) |
CompletableFuture<Void> |
recommissionRegionServer(ServerName server,
List<byte[]> encodedRegionNames)
Remove decommission marker from a region server to allow regions assignments.
|
CompletableFuture<Void> |
removeReplicationPeer(String peerId)
Remove a peer and stop the replication
|
CompletableFuture<Void> |
removeReplicationPeerTableCFs(String id,
Map<TableName,List<String>> tableCfs)
Remove some table-cfs from config of the specified peer
|
CompletableFuture<Void> |
restoreSnapshot(String snapshotName)
Restore the specified snapshot on the original table.
|
CompletableFuture<Void> |
restoreSnapshot(String snapshotName,
boolean takeFailSafeSnapshot)
Restore the specified snapshot on the original table.
|
private CompletableFuture<Void> |
restoreSnapshot(String snapshotName,
TableName tableName,
boolean takeFailSafeSnapshot) |
CompletableFuture<Void> |
rollWALWriter(ServerName serverName)
Roll the log writer.
|
CompletableFuture<Integer> |
runCatalogJanitor()
Ask for a scan of the catalog table.
|
CompletableFuture<Boolean> |
runCleanerChore()
Ask for cleaner chore to run.
|
CompletableFuture<Void> |
setQuota(QuotaSettings quota)
Apply the new quota settings.
|
private CompletableFuture<Boolean> |
setSplitOrMergeOn(boolean on,
MasterSwitchType switchType) |
private CompletableFuture<Void> |
setTableReplication(TableName tableName,
boolean enableRep)
Set the table's replication switch if the table's replication switch is already not set.
|
CompletableFuture<Void> |
shutdown()
Shuts down the HBase cluster.
|
CompletableFuture<Void> |
snapshot(SnapshotDescription snapshotDesc)
Take a snapshot and wait for the server to complete that snapshot asynchronously.
|
private CompletableFuture<Void> |
split(RegionInfo hri,
byte[] splitPoint) |
CompletableFuture<Void> |
split(TableName tableName)
Split a table.
|
CompletableFuture<Void> |
split(TableName tableName,
byte[] splitPoint)
Split a table.
|
CompletableFuture<Void> |
splitRegion(byte[] regionName)
Split an individual region.
|
CompletableFuture<Void> |
splitRegion(byte[] regionName,
byte[] splitPoint)
Split an individual region.
|
CompletableFuture<Boolean> |
splitSwitch(boolean on)
Turn the Split switch on or off.
|
CompletableFuture<Void> |
stopMaster()
Shuts down the current HBase master only.
|
CompletableFuture<Void> |
stopRegionServer(ServerName serverName)
Stop the designated regionserver.
|
CompletableFuture<Boolean> |
tableExists(TableName tableName) |
private byte[] |
toEncodeRegionName(byte[] regionName) |
CompletableFuture<Void> |
truncateTable(TableName tableName,
boolean preserveSplits)
Truncate a table.
|
private CompletableFuture<Void> |
trySyncTableToPeerCluster(TableName tableName,
byte[][] splits,
ReplicationPeerDescription peer) |
CompletableFuture<Void> |
unassign(byte[] regionName,
boolean forcible)
Unassign a region from current hosting regionserver.
|
CompletableFuture<Void> |
updateConfiguration()
Update the configuration and trigger an online config change on all the masters and
regionservers.
|
CompletableFuture<Void> |
updateConfiguration(ServerName serverName)
Update the configuration and trigger an online config change on the regionserver.
|
CompletableFuture<Void> |
updateReplicationPeerConfig(String peerId,
ReplicationPeerConfig peerConfig)
Update the peerConfig for the specified peer
|
private void |
verifySplitKeys(byte[][] splitKeys) |
private CompletableFuture<Void> |
waitProcedureResult(CompletableFuture<Long> procFuture) |
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
addReplicationPeer, balance, compact, compact, getBackupMasters, getCompactionState, getMaster, getMasterCoprocessorNames, getMasterInfoPort, getRegionServers, listDeadServers, listTableDescriptors, listTableNames, majorCompact, majorCompact, snapshot, snapshot
public static final String FLUSH_TABLE_PROCEDURE_SIGNATURE
private static final org.slf4j.Logger LOG
private final AsyncConnectionImpl connection
private final org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer retryTimer
private final AsyncTable<AdvancedScanResultConsumer> metaTable
private final long rpcTimeoutNs
private final long operationTimeoutNs
private final long pauseNs
private final int maxAttempts
private final int startLogErrorsCnt
private final NonceGenerator ng
RawAsyncHBaseAdmin(AsyncConnectionImpl connection, org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer retryTimer, AsyncAdminBuilderBase builder)
private <T> AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder<T> newMasterCaller()
private <T> AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder<T> newAdminCaller()
private <PREQ,PRESP,RESP> CompletableFuture<RESP> call(HBaseRpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.Interface stub, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP,PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<RESP,PRESP> respConverter)
private <PREQ,PRESP,RESP> CompletableFuture<RESP> adminCall(HBaseRpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.Interface stub, PREQ preq, RawAsyncHBaseAdmin.AdminRpcCall<PRESP,PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<RESP,PRESP> respConverter)
private <PREQ,PRESP> CompletableFuture<Void> procedureCall(PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP,PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long,PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer consumer)
public CompletableFuture<Boolean> tableExists(TableName tableName)
tableExists
in interface AsyncAdmin
tableName
- Table to check.CompletableFuture
.public CompletableFuture<List<TableDescriptor>> listTableDescriptors(boolean includeSysTables)
AsyncAdmin
listTableDescriptors
in interface AsyncAdmin
includeSysTables
- False to match only against userspace tablesCompletableFuture
.public CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern, boolean includeSysTables)
#listTables(boolean)
listTableDescriptors
in interface AsyncAdmin
pattern
- The compiled regular expression to match againstincludeSysTables
- False to match only against userspace tablesCompletableFuture
.private CompletableFuture<List<TableDescriptor>> getTableDescriptors(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request)
public CompletableFuture<List<TableName>> listTableNames(boolean includeSysTables)
AsyncAdmin
listTableNames
in interface AsyncAdmin
includeSysTables
- False to match only against userspace tablesCompletableFuture
.public CompletableFuture<List<TableName>> listTableNames(Pattern pattern, boolean includeSysTables)
AsyncAdmin
listTableNames
in interface AsyncAdmin
pattern
- The regular expression to match againstincludeSysTables
- False to match only against userspace tablesCompletableFuture
.private CompletableFuture<List<TableName>> getTableNames(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request)
public CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name)
AsyncAdmin
listTableDescriptorsByNamespace
in interface AsyncAdmin
name
- namespace nameCompletableFuture
.public CompletableFuture<List<TableName>> listTableNamesByNamespace(String name)
AsyncAdmin
listTableNamesByNamespace
in interface AsyncAdmin
name
- namespace nameCompletableFuture
.public CompletableFuture<TableDescriptor> getDescriptor(TableName tableName)
AsyncAdmin
getDescriptor
in interface AsyncAdmin
tableName
- as a TableName
CompletableFuture
.public CompletableFuture<Void> createTable(TableDescriptor desc)
AsyncAdmin
createTable
in interface AsyncAdmin
desc
- table descriptor for tablepublic CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
AsyncAdmin
createTable
in interface AsyncAdmin
desc
- table descriptor for tablestartKey
- beginning of key rangeendKey
- end of key rangenumRegions
- the total number of regions to createpublic CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys)
AsyncAdmin
createTable
in interface AsyncAdmin
desc
- table descriptor for tablesplitKeys
- array of split keys for the initial regions of the tableprivate CompletableFuture<Void> createTable(TableName tableName, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request)
public CompletableFuture<Void> modifyTable(TableDescriptor desc)
AsyncAdmin
modifyTable
in interface AsyncAdmin
desc
- modified description of the tablepublic CompletableFuture<Void> deleteTable(TableName tableName)
AsyncAdmin
deleteTable
in interface AsyncAdmin
tableName
- name of table to deletepublic CompletableFuture<Void> truncateTable(TableName tableName, boolean preserveSplits)
AsyncAdmin
truncateTable
in interface AsyncAdmin
tableName
- name of table to truncatepreserveSplits
- True if the splits should be preservedpublic CompletableFuture<Void> enableTable(TableName tableName)
AsyncAdmin
enableTable
in interface AsyncAdmin
tableName
- name of the tablepublic CompletableFuture<Void> disableTable(TableName tableName)
AsyncAdmin
disableTable
in interface AsyncAdmin
public CompletableFuture<Boolean> isTableEnabled(TableName tableName)
isTableEnabled
in interface AsyncAdmin
tableName
- name of table to checkCompletableFuture
.public CompletableFuture<Boolean> isTableDisabled(TableName tableName)
isTableDisabled
in interface AsyncAdmin
tableName
- name of table to checkCompletableFuture
.public CompletableFuture<Boolean> isTableAvailable(TableName tableName)
isTableAvailable
in interface AsyncAdmin
tableName
- name of table to checkCompletableFuture
.public CompletableFuture<Boolean> isTableAvailable(TableName tableName, byte[][] splitKeys)
AsyncAdmin
CompletableFuture
.isTableAvailable
in interface AsyncAdmin
tableName
- name of table to checksplitKeys
- keys to check if the table has been created with all split keysprivate CompletableFuture<Boolean> isTableAvailable(TableName tableName, Optional<byte[][]> splitKeys)
private boolean compareRegionsWithSplitKeys(List<HRegionLocation> locations, byte[][] splitKeys)
public CompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
AsyncAdmin
addColumnFamily
in interface AsyncAdmin
tableName
- name of the table to add column family tocolumnFamily
- column family descriptor of column family to be addedpublic CompletableFuture<Void> deleteColumnFamily(TableName tableName, byte[] columnFamily)
AsyncAdmin
deleteColumnFamily
in interface AsyncAdmin
tableName
- name of tablecolumnFamily
- name of column family to be deletedpublic CompletableFuture<Void> modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
AsyncAdmin
modifyColumnFamily
in interface AsyncAdmin
tableName
- name of tablecolumnFamily
- new column family descriptor to usepublic CompletableFuture<Void> createNamespace(NamespaceDescriptor descriptor)
AsyncAdmin
createNamespace
in interface AsyncAdmin
descriptor
- descriptor which describes the new namespacepublic CompletableFuture<Void> modifyNamespace(NamespaceDescriptor descriptor)
AsyncAdmin
modifyNamespace
in interface AsyncAdmin
descriptor
- descriptor which describes the new namespacepublic CompletableFuture<Void> deleteNamespace(String name)
AsyncAdmin
deleteNamespace
in interface AsyncAdmin
name
- namespace namepublic CompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(String name)
AsyncAdmin
getNamespaceDescriptor
in interface AsyncAdmin
name
- name of namespace descriptorCompletableFuture
.public CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors()
AsyncAdmin
listNamespaceDescriptors
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName)
AsyncAdmin
getRegions
in interface AsyncAdmin
public CompletableFuture<List<RegionInfo>> getRegions(TableName tableName)
AsyncAdmin
getRegions
in interface AsyncAdmin
public CompletableFuture<Void> flush(TableName tableName)
AsyncAdmin
flush
in interface AsyncAdmin
tableName
- table to flushpublic CompletableFuture<Void> flushRegion(byte[] regionName)
AsyncAdmin
flushRegion
in interface AsyncAdmin
regionName
- region to flushprivate CompletableFuture<Void> flush(ServerName serverName, RegionInfo regionInfo)
public CompletableFuture<Void> flushRegionServer(ServerName sn)
AsyncAdmin
flushRegionServer
in interface AsyncAdmin
sn
- server to flushpublic CompletableFuture<Void> compact(TableName tableName, CompactType compactType)
AsyncAdmin
compact
in interface AsyncAdmin
tableName
- table to compactcompactType
- CompactType
public CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, CompactType compactType)
AsyncAdmin
compact
in interface AsyncAdmin
tableName
- table to compactcolumnFamily
- column family within a tablecompactType
- CompactType
public CompletableFuture<Void> compactRegion(byte[] regionName)
AsyncAdmin
compactRegion
in interface AsyncAdmin
regionName
- region to compactpublic CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily)
AsyncAdmin
compactRegion
in interface AsyncAdmin
regionName
- region to compactcolumnFamily
- column family within a region. If not present, compact the region's all
column families.public CompletableFuture<Void> majorCompact(TableName tableName, CompactType compactType)
AsyncAdmin
majorCompact
in interface AsyncAdmin
tableName
- table to major compactcompactType
- CompactType
public CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType)
AsyncAdmin
majorCompact
in interface AsyncAdmin
tableName
- table to major compactcolumnFamily
- column family within a table. If not present, major compact the table's all
column families.compactType
- CompactType
public CompletableFuture<Void> majorCompactRegion(byte[] regionName)
AsyncAdmin
majorCompactRegion
in interface AsyncAdmin
regionName
- region to major compactpublic CompletableFuture<Void> majorCompactRegion(byte[] regionName, byte[] columnFamily)
AsyncAdmin
majorCompactRegion
in interface AsyncAdmin
regionName
- region to major compactcolumnFamily
- column family within a region. If not present, major compact the region's
all column families.public CompletableFuture<Void> compactRegionServer(ServerName sn)
AsyncAdmin
compactRegionServer
in interface AsyncAdmin
sn
- the region server namepublic CompletableFuture<Void> majorCompactRegionServer(ServerName sn)
AsyncAdmin
majorCompactRegionServer
in interface AsyncAdmin
sn
- the region server nameprivate CompletableFuture<Void> compactRegionServer(ServerName sn, boolean major)
private CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily, boolean major)
private CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(TableName tableName)
private CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, boolean major, CompactType compactType)
private CompletableFuture<Void> compact(ServerName sn, RegionInfo hri, boolean major, byte[] columnFamily)
private byte[] toEncodeRegionName(byte[] regionName)
private void checkAndGetTableName(byte[] encodeRegionName, AtomicReference<TableName> tableName, CompletableFuture<TableName> result)
private CompletableFuture<TableName> checkRegionsAndGetTableName(byte[] encodeRegionNameA, byte[] encodeRegionNameB)
public CompletableFuture<Boolean> mergeSwitch(boolean on)
AsyncAdmin
mergeSwitch
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> isMergeEnabled()
AsyncAdmin
isMergeEnabled
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> splitSwitch(boolean on)
AsyncAdmin
splitSwitch
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> isSplitEnabled()
AsyncAdmin
isSplitEnabled
in interface AsyncAdmin
CompletableFuture
private CompletableFuture<Boolean> setSplitOrMergeOn(boolean on, MasterSwitchType switchType)
private CompletableFuture<Boolean> isSplitOrMergeOn(MasterSwitchType switchType)
public CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible)
AsyncAdmin
mergeRegions
in interface AsyncAdmin
nameOfRegionA
- encoded or full name of region anameOfRegionB
- encoded or full name of region bforcible
- true if do a compulsory merge, otherwise we will only merge two adjacent
regionspublic CompletableFuture<Void> split(TableName tableName)
AsyncAdmin
split
in interface AsyncAdmin
tableName
- table to splitpublic CompletableFuture<Void> split(TableName tableName, byte[] splitPoint)
AsyncAdmin
split
in interface AsyncAdmin
tableName
- table to splitsplitPoint
- the explicit position to split onpublic CompletableFuture<Void> splitRegion(byte[] regionName)
AsyncAdmin
splitRegion
in interface AsyncAdmin
regionName
- region to splitpublic CompletableFuture<Void> splitRegion(byte[] regionName, byte[] splitPoint)
AsyncAdmin
splitRegion
in interface AsyncAdmin
regionName
- region to splitsplitPoint
- the explicit position to split on. If not present, it will decide by region
server.private CompletableFuture<Void> split(RegionInfo hri, byte[] splitPoint)
public CompletableFuture<Void> assign(byte[] regionName)
assign
in interface AsyncAdmin
regionName
- Encoded or full name of region to assign.public CompletableFuture<Void> unassign(byte[] regionName, boolean forcible)
AsyncAdmin
AsyncAdmin.move(byte[], ServerName)
if you want to control the region movement.unassign
in interface AsyncAdmin
regionName
- Encoded or full name of region to unassign. Will clear any existing
RegionPlan if one found.forcible
- If true, force unassign (Will remove region from regions-in-transition too if
present. If results in double assignment use hbck -fix to resolve. To be used by
experts).public CompletableFuture<Void> offline(byte[] regionName)
AsyncAdmin
offline
in interface AsyncAdmin
regionName
- Encoded or full name of region to offlinepublic CompletableFuture<Void> move(byte[] regionName)
AsyncAdmin
r
to a random server.move
in interface AsyncAdmin
regionName
- Encoded or full name of region to move.public CompletableFuture<Void> move(byte[] regionName, ServerName destServerName)
AsyncAdmin
r
to dest
.move
in interface AsyncAdmin
regionName
- Encoded or full name of region to move.destServerName
- The servername of the destination regionserver. If not present, we'll
assign to a random server. A server name is made of host, port and startcode. Here is
an example: host187.example.com,60020,1289493121758
private CompletableFuture<Void> moveRegion(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request)
public CompletableFuture<Void> setQuota(QuotaSettings quota)
AsyncAdmin
setQuota
in interface AsyncAdmin
quota
- the quota settingspublic CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter)
AsyncAdmin
getQuota
in interface AsyncAdmin
filter
- the quota settings filterpublic CompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
AsyncAdmin
addReplicationPeer
in interface AsyncAdmin
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication slave clusterenabled
- peer state, true if ENABLED and false if DISABLEDpublic CompletableFuture<Void> removeReplicationPeer(String peerId)
AsyncAdmin
removeReplicationPeer
in interface AsyncAdmin
peerId
- a short name that identifies the peerpublic CompletableFuture<Void> enableReplicationPeer(String peerId)
AsyncAdmin
enableReplicationPeer
in interface AsyncAdmin
peerId
- a short name that identifies the peerpublic CompletableFuture<Void> disableReplicationPeer(String peerId)
AsyncAdmin
disableReplicationPeer
in interface AsyncAdmin
peerId
- a short name that identifies the peerpublic CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId)
AsyncAdmin
getReplicationPeerConfig
in interface AsyncAdmin
peerId
- a short name that identifies the peerCompletableFuture
.public CompletableFuture<Void> updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
AsyncAdmin
updateReplicationPeerConfig
in interface AsyncAdmin
peerId
- a short name that identifies the peerpeerConfig
- new config for the peerpublic CompletableFuture<Void> appendReplicationPeerTableCFs(String id, Map<TableName,List<String>> tableCfs)
AsyncAdmin
appendReplicationPeerTableCFs
in interface AsyncAdmin
id
- a short that identifies the clustertableCfs
- A map from tableName to column family namespublic CompletableFuture<Void> removeReplicationPeerTableCFs(String id, Map<TableName,List<String>> tableCfs)
AsyncAdmin
removeReplicationPeerTableCFs
in interface AsyncAdmin
id
- a short name that identifies the clustertableCfs
- A map from tableName to column family namespublic CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers()
AsyncAdmin
listReplicationPeers
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(Pattern pattern)
AsyncAdmin
listReplicationPeers
in interface AsyncAdmin
pattern
- The compiled regular expression to match peer idCompletableFuture
.private CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request)
public CompletableFuture<List<TableCFs>> listReplicatedTableCFs()
AsyncAdmin
listReplicatedTableCFs
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<Void> snapshot(SnapshotDescription snapshotDesc)
AsyncAdmin
SnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
HBase. See TableName.isLegalFullyQualifiedTableName(byte[])
.
You should probably use AsyncAdmin.snapshot(String, org.apache.hadoop.hbase.TableName)
unless you
are sure about the type of snapshot that you want to take.snapshot
in interface AsyncAdmin
snapshotDesc
- snapshot to takepublic CompletableFuture<Boolean> isSnapshotFinished(SnapshotDescription snapshot)
AsyncAdmin
UnknownSnapshotException
.isSnapshotFinished
in interface AsyncAdmin
snapshot
- description of the snapshot to checkpublic CompletableFuture<Void> restoreSnapshot(String snapshotName)
AsyncAdmin
restoreSnapshot
in interface AsyncAdmin
snapshotName
- name of the snapshot to restorepublic CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot)
AsyncAdmin
restoreSnapshot
in interface AsyncAdmin
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true if the failsafe snapshot should be takenprivate CompletableFuture<Void> restoreSnapshot(String snapshotName, TableName tableName, boolean takeFailSafeSnapshot)
private <T> void completeConditionalOnFuture(CompletableFuture<T> dependentFuture, CompletableFuture<T> parentFuture)
public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName)
AsyncAdmin
cloneSnapshot
in interface AsyncAdmin
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredprivate CompletableFuture<Void> internalRestoreSnapshot(String snapshotName, TableName tableName)
public CompletableFuture<List<SnapshotDescription>> listSnapshots()
AsyncAdmin
listSnapshots
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<List<SnapshotDescription>> listSnapshots(Pattern pattern)
AsyncAdmin
listSnapshots
in interface AsyncAdmin
pattern
- The compiled regular expression to match againstCompletableFuture
private CompletableFuture<List<SnapshotDescription>> getCompletedSnapshots(Pattern pattern)
public CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern)
AsyncAdmin
listTableSnapshots
in interface AsyncAdmin
tableNamePattern
- The compiled table name regular expression to match againstCompletableFuture
public CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
AsyncAdmin
listTableSnapshots
in interface AsyncAdmin
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match againstCompletableFuture
private CompletableFuture<List<SnapshotDescription>> getCompletedSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
public CompletableFuture<Void> deleteSnapshot(String snapshotName)
AsyncAdmin
deleteSnapshot
in interface AsyncAdmin
snapshotName
- name of the snapshotpublic CompletableFuture<Void> deleteSnapshots()
AsyncAdmin
deleteSnapshots
in interface AsyncAdmin
public CompletableFuture<Void> deleteSnapshots(Pattern snapshotNamePattern)
AsyncAdmin
deleteSnapshots
in interface AsyncAdmin
snapshotNamePattern
- pattern for names of the snapshot to matchpublic CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern)
AsyncAdmin
deleteTableSnapshots
in interface AsyncAdmin
tableNamePattern
- The compiled table name regular expression to match againstpublic CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
AsyncAdmin
deleteTableSnapshots
in interface AsyncAdmin
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match againstprivate CompletableFuture<Void> internalDeleteSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
private CompletableFuture<Void> internalDeleteSnapshot(SnapshotDescription snapshot)
public CompletableFuture<Void> execProcedure(String signature, String instance, Map<String,String> props)
AsyncAdmin
execProcedure
in interface AsyncAdmin
signature
- A distributed procedure is uniquely identified by its signature (default the
root ZK node name of the procedure).instance
- The instance name of the procedure. For some procedures, this parameter is
optional.props
- Property/Value pairs of properties passing to the procedurepublic CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance, Map<String,String> props)
AsyncAdmin
execProcedureWithReturn
in interface AsyncAdmin
signature
- A distributed procedure is uniquely identified by its signature (default the
root ZK node name of the procedure).instance
- The instance name of the procedure. For some procedures, this parameter is
optional.props
- Property/Value pairs of properties passing to the procedurepublic CompletableFuture<Boolean> isProcedureFinished(String signature, String instance, Map<String,String> props)
AsyncAdmin
isProcedureFinished
in interface AsyncAdmin
signature
- The signature that uniquely identifies a procedureinstance
- The instance name of the procedureprops
- Property/Value pairs of properties passing to the procedureCompletableFuture
public CompletableFuture<Boolean> abortProcedure(long procId, boolean mayInterruptIfRunning)
AsyncAdmin
abortProcedure
in interface AsyncAdmin
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?CompletableFuture
public CompletableFuture<String> getProcedures()
AsyncAdmin
getProcedures
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<String> getLocks()
AsyncAdmin
getLocks
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Void> decommissionRegionServers(List<ServerName> servers, boolean offload)
AsyncAdmin
decommissionRegionServers
in interface AsyncAdmin
servers
- The list of servers to decommission.offload
- True to offload the regions from the decommissioned serverspublic CompletableFuture<List<ServerName>> listDecommissionedRegionServers()
AsyncAdmin
listDecommissionedRegionServers
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Void> recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
AsyncAdmin
recommissionRegionServer
in interface AsyncAdmin
server
- The server to recommission.encodedRegionNames
- Regions to load onto the server.CompletableFuture<HRegionLocation> getRegionLocation(byte[] regionNameOrEncodedRegionName)
CompletableFuture
regionNameOrEncodedRegionName
- CompletableFuture
private CompletableFuture<RegionInfo> getRegionInfo(byte[] regionNameOrEncodedRegionName)
CompletableFuture
regionNameOrEncodedRegionName
- CompletableFuture
private byte[][] getSplitKeys(byte[] startKey, byte[] endKey, int numRegions)
private void verifySplitKeys(byte[][] splitKeys)
private CompletableFuture<Void> waitProcedureResult(CompletableFuture<Long> procFuture)
private void getProcedureResult(long procId, CompletableFuture<Void> future, int retries)
private <T> CompletableFuture<T> failedFuture(Throwable error)
private <T> boolean completeExceptionally(CompletableFuture<T> future, Throwable error)
public CompletableFuture<ClusterMetrics> getClusterMetrics()
getClusterMetrics
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<ClusterMetrics.Option> options)
getClusterMetrics
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Void> shutdown()
AsyncAdmin
shutdown
in interface AsyncAdmin
public CompletableFuture<Void> stopMaster()
AsyncAdmin
stopMaster
in interface AsyncAdmin
public CompletableFuture<Void> stopRegionServer(ServerName serverName)
AsyncAdmin
stopRegionServer
in interface AsyncAdmin
public CompletableFuture<Void> updateConfiguration(ServerName serverName)
AsyncAdmin
updateConfiguration
in interface AsyncAdmin
serverName
- : The server whose config needs to be updated.public CompletableFuture<Void> updateConfiguration()
AsyncAdmin
updateConfiguration
in interface AsyncAdmin
public CompletableFuture<Void> rollWALWriter(ServerName serverName)
AsyncAdmin
When the returned CompletableFuture is done, it only means the rollWALWriter request was sent to the region server and may need some time to finish the rollWALWriter operation. As a side effect of this call, the named region server may schedule store flushes at the request of the wal.
rollWALWriter
in interface AsyncAdmin
serverName
- The servername of the region server.public CompletableFuture<Void> clearCompactionQueues(ServerName serverName, Set<String> queues)
AsyncAdmin
clearCompactionQueues
in interface AsyncAdmin
queues
- the set of queue namepublic CompletableFuture<List<SecurityCapability>> getSecurityCapabilities()
getSecurityCapabilities
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName)
AsyncAdmin
RegionMetrics
of all regions hosted on a region seerver.getRegionMetrics
in interface AsyncAdmin
RegionMetrics
wrapped by CompletableFuture
public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName, TableName tableName)
AsyncAdmin
RegionMetrics
of all regions hosted on a region seerver for a table.getRegionMetrics
in interface AsyncAdmin
RegionMetrics
wrapped by CompletableFuture
private CompletableFuture<List<RegionMetrics>> getRegionMetrics(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request, ServerName serverName)
public CompletableFuture<Boolean> isMasterInMaintenanceMode()
AsyncAdmin
isMasterInMaintenanceMode
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<CompactionState> getCompactionState(TableName tableName, CompactType compactType)
AsyncAdmin
getCompactionState
in interface AsyncAdmin
tableName
- table to examinecompactType
- CompactType
CompletableFuture
public CompletableFuture<CompactionState> getCompactionStateForRegion(byte[] regionName)
AsyncAdmin
getCompactionStateForRegion
in interface AsyncAdmin
regionName
- region to examineCompletableFuture
public CompletableFuture<Optional<Long>> getLastMajorCompactionTimestamp(TableName tableName)
AsyncAdmin
The timestamp of the oldest HFile resulting from a major compaction of that table, or not present if no such HFile could be found.
getLastMajorCompactionTimestamp
in interface AsyncAdmin
tableName
- table to examineCompletableFuture
public CompletableFuture<Optional<Long>> getLastMajorCompactionTimestampForRegion(byte[] regionName)
AsyncAdmin
The timestamp of the oldest HFile resulting from a major compaction of that region, or not present if no such HFile could be found.
getLastMajorCompactionTimestampForRegion
in interface AsyncAdmin
regionName
- region to examineCompletableFuture
public CompletableFuture<Boolean> balancerSwitch(boolean on)
AsyncAdmin
balancerSwitch
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<Boolean> balance(boolean forcible)
AsyncAdmin
balance
in interface AsyncAdmin
forcible
- whether we should force balance even if there is region in transition.CompletableFuture
.public CompletableFuture<Boolean> isBalancerEnabled()
AsyncAdmin
isBalancerEnabled
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<Boolean> normalizerSwitch(boolean on)
AsyncAdmin
normalizerSwitch
in interface AsyncAdmin
on
- whether normalizer should be on or offCompletableFuture
public CompletableFuture<Boolean> isNormalizerEnabled()
AsyncAdmin
isNormalizerEnabled
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> normalize()
AsyncAdmin
normalize
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> cleanerChoreSwitch(boolean enabled)
AsyncAdmin
cleanerChoreSwitch
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> isCleanerChoreEnabled()
AsyncAdmin
isCleanerChoreEnabled
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> runCleanerChore()
AsyncAdmin
runCleanerChore
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> catalogJanitorSwitch(boolean enabled)
AsyncAdmin
catalogJanitorSwitch
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> isCatalogJanitorEnabled()
AsyncAdmin
isCatalogJanitorEnabled
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Integer> runCatalogJanitor()
AsyncAdmin
runCatalogJanitor
in interface AsyncAdmin
CompletableFuture
public <S,R> CompletableFuture<R> coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker, ServiceCaller<S,R> callable)
AsyncAdmin
The stubMaker
is just a delegation to the newStub
call. Usually it is only a
one line lambda expression, like:
channel -> xxxService.newStub(channel)
coprocessorService
in interface AsyncAdmin
S
- the type of the asynchronous stubR
- the type of the return valuestubMaker
- a delegation to the actual newStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment of
ServiceCaller
for more details.CompletableFuture
.ServiceCaller
public <S,R> CompletableFuture<R> coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker, ServiceCaller<S,R> callable, ServerName serverName)
AsyncAdmin
The stubMaker
is just a delegation to the newStub
call. Usually it is only a
one line lambda expression, like:
channel -> xxxService.newStub(channel)
coprocessorService
in interface AsyncAdmin
S
- the type of the asynchronous stubR
- the type of the return valuestubMaker
- a delegation to the actual newStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment of
ServiceCaller
for more details.serverName
- the given region serverCompletableFuture
.ServiceCaller
public CompletableFuture<List<ServerName>> clearDeadServers(List<ServerName> servers)
AsyncAdmin
clearDeadServers
in interface AsyncAdmin
servers
- list of dead region servers.CompletableFuture
.private <T> AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder<T> newServerCaller()
public CompletableFuture<Void> enableTableReplication(TableName tableName)
AsyncAdmin
enableTableReplication
in interface AsyncAdmin
tableName
- name of the tablepublic CompletableFuture<Void> disableTableReplication(TableName tableName)
AsyncAdmin
disableTableReplication
in interface AsyncAdmin
tableName
- name of the tableprivate CompletableFuture<byte[][]> getTableSplits(TableName tableName)
private CompletableFuture<Void> checkAndSyncTableToPeerClusters(TableName tableName, byte[][] splits)
tableName
- name of the table to sync to the peersplits
- table split keysprivate CompletableFuture<Void> trySyncTableToPeerCluster(TableName tableName, byte[][] splits, ReplicationPeerDescription peer)
private CompletableFuture<Void> compareTableWithPeerCluster(TableName tableName, TableDescriptor tableDesc, ReplicationPeerDescription peer, AsyncAdmin peerAdmin)
private CompletableFuture<Void> setTableReplication(TableName tableName, boolean enableRep)
tableName
- name of the tableenableRep
- is replication switch enable or disablepublic CompletableFuture<CacheEvictionStats> clearBlockCache(TableName tableName)
AsyncAdmin
clearBlockCache
in interface AsyncAdmin
tableName
- table to clear block cacheCompletableFuture
.private CompletableFuture<CacheEvictionStats> clearBlockCache(ServerName serverName, List<RegionInfo> hris)
Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.