Class RawAsyncHBaseAdmin
- All Implemented Interfaces:
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.
- Since:
- 2.0.0
- See Also:
-
Nested Class Summary
Modifier and TypeClassDescriptionprivate static class
private static interface
private static interface
private static class
private static class
private static class
private static class
private class
private static class
private static class
private static class
private static interface
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
-
Field Summary
Modifier and TypeFieldDescriptionprivate final AsyncConnectionImpl
static final String
private static final org.slf4j.Logger
private final int
private final AsyncTable<AdvancedScanResultConsumer>
private final NonceGenerator
private final long
private final long
private final long
private final org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer
private final long
private final int
-
Constructor Summary
ConstructorDescriptionRawAsyncHBaseAdmin
(AsyncConnectionImpl connection, org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer retryTimer, AsyncAdminBuilderBase builder) -
Method Summary
Modifier and TypeMethodDescriptionabortProcedure
(long procId, boolean mayInterruptIfRunning) Abort a procedure Do not use.addColumnFamily
(TableName tableName, ColumnFamilyDescriptor columnFamily) Add a column family to an existing table.addReplicationPeer
(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) Add a new replication peer for replicating data to slave clusteraddRSGroup
(String groupName) Creates a new RegionServer group with the given nameprivate <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) Append the replicable table-cf config of the specified peerassign
(byte[] regionName) Assign an individual region.balance
(BalanceRequest request) Invoke the balancer with the given balance request.balanceRSGroup
(String groupName, BalanceRequest request) Balance regions in the given RegionServer groupbalancerSwitch
(boolean on, boolean drainRITs) Turn the load balancer on or off.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) 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[][] encodedRegionNames) cleanerChoreSwitch
(boolean enabled) Turn the cleaner chore on/off.private CompletableFuture<CacheEvictionStats>
clearBlockCache
(ServerName serverName, List<RegionInfo> hris) clearBlockCache
(TableName tableName) Clear all the blocks corresponding to this table from BlockCache.clearCompactionQueues
(ServerName serverName, Set<String> queues) Clear compacting queues on a region server.clearDeadServers
(List<ServerName> servers) Clear dead region servers from master.clearSlowLogResponses
(Set<ServerName> serverNames) Clears online slow RPC logs from the provided list of RegionServersprivate CompletableFuture<Boolean>
clearSlowLogsResponses
(ServerName serverName) cloneSnapshot
(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Create a new table by cloning the snapshot content.cloneTableSchema
(TableName tableName, TableName newTableName, boolean preserveSplits) Create a new table by cloning the existent table schema.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()compact
(TableName tableName, byte[] columnFamily, CompactType compactType) Compact a column family within a table.compact
(TableName tableName, CompactType compactType) Compact a table.compactionSwitch
(boolean switchState, List<String> serverNamesList) Turn the compaction on or off.compactRegion
(byte[] regionName) Compact an individual region.compactRegion
(byte[] regionName, byte[] columnFamily) Compact a column family within a region.private CompletableFuture<Void>
compactRegion
(byte[] regionName, byte[] columnFamily, boolean major) Compact all regions on the region server.private CompletableFuture<Void>
compactRegionServer
(ServerName sn, boolean major) private CompletableFuture<Void>
compareTableWithPeerCluster
(TableName tableName, TableDescriptor tableDesc, ReplicationPeerDescription peer, AsyncAdmin peerAdmin) private static CompletableFuture<Boolean>
completeCheckTableState
(CompletableFuture<Boolean> future, TableState tableState, Throwable error, TableState.State targetState, TableName tableName) Utility for completing passed TableStateCompletableFuture
future
using passed parameters.private <T> void
completeConditionalOnFuture
(CompletableFuture<T> dependentFuture, CompletableFuture<T> parentFuture) private <T> boolean
completeExceptionally
(CompletableFuture<T> future, Throwable error) private static <T> CompletableFuture<List<T>>
convertToFutureOfList
(List<CompletableFuture<T>> futures) <S,
R> CompletableFuture<R> coprocessorService
(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable) Execute the given coprocessor call on the master.<S,
R> CompletableFuture<R> coprocessorService
(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, ServerName serverName) Execute the given coprocessor call on the given region server.createNamespace
(NamespaceDescriptor descriptor) Create a new namespace.createTable
(TableDescriptor desc) Creates a new table.createTable
(TableDescriptor desc, byte[][] splitKeys) Creates a new table with an initial set of empty regions defined by the specified split keys.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) decommissionRegionServers
(List<ServerName> servers, boolean offload) Mark region server(s) as decommissioned to prevent additional regions from getting assigned to them.deleteColumnFamily
(TableName tableName, byte[] columnFamily) Delete a column family from a table.deleteNamespace
(String name) Delete an existing namespace.deleteSnapshot
(String snapshotName) Delete an existing snapshot.Delete all existing snapshots.deleteSnapshots
(Pattern snapshotNamePattern) Delete existing snapshots whose names match the pattern passed.deleteTable
(TableName tableName) Deletes a table.deleteTableSnapshots
(Pattern tableNamePattern) Delete all existing snapshots matching the given table name pattern.deleteTableSnapshots
(Pattern tableNamePattern, Pattern snapshotNamePattern) Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.disableReplicationPeer
(String peerId) Stop the replication stream to the specified peerdisableTable
(TableName tableName) Disable a table.disableTableReplication
(TableName tableName) Disable a table's replication switch.enableReplicationPeer
(String peerId) Restart the replication stream to the specified peerenableTable
(TableName tableName) Enable a table.enableTableReplication
(TableName tableName) Enable a table's replication switch.exceedThrottleQuotaSwitch
(boolean enable) Switch the exceed throttle quota.Execute a distributed procedure on a cluster.CompletableFuture<byte[]>
Execute a distributed procedure on a cluster.private <T> CompletableFuture<T>
failedFuture
(Throwable error) private CompletableFuture<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse>
flush
(ServerName serverName, RegionInfo regionInfo, byte[] columnFamily, boolean writeFlushWALMarker) Flush a table.Flush the specified column family stores on all regions of the passed table.Flush the specified column family stores on all regions of the passed table.Flush master local regionflushRegion
(byte[] regionName) Flush an individual region.flushRegion
(byte[] regionName, byte[] columnFamily) Flush a column family within a region.(package private) CompletableFuture<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse>
flushRegionInternal
(byte[] regionName, byte[] columnFamily, boolean writeFlushWALMarker) This method is for internal use only, where we need the response of the flush.Flush all region on the region server.private CompletableFuture<List<LogEntry>>
getBalancerDecisions
(int limit) private CompletableFuture<List<LogEntry>>
getBalancerRejections
(int limit) getCachedFilesList
(ServerName serverName) Get the list of cached filesReturns cluster status wrapped byCompletableFuture
getClusterMetrics
(EnumSet<ClusterMetrics.Option> options) Returns cluster status wrapped byCompletableFuture
getCompactionState
(TableName tableName, CompactType compactType) Get the current compaction state of a table.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) getConfiguredNamespacesAndTablesInRSGroup
(String groupName) Get the namespaces and tables which have this RegionServer group in descriptor.getCurrentSpaceQuotaSnapshot
(String namespace) Returns the Master's view of a quota on the givennamespace
or null if the Master has no quota information on that namespace.private CompletableFuture<SpaceQuotaSnapshot>
getCurrentSpaceQuotaSnapshot
(RawAsyncHBaseAdmin.Converter<SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse> converter) getCurrentSpaceQuotaSnapshot
(TableName tableName) Returns the Master's view of a quota on the giventableName
or null if the Master has no quota information on that table.getDescriptor
(TableName tableName) Method for getting the tableDescriptorgetLastMajorCompactionTimestamp
(TableName tableName) Get the timestamp of the last major compaction for the passed table.getLastMajorCompactionTimestampForRegion
(byte[] regionName) Get the timestamp of the last major compaction for the passed region.getLocks()
List locks.getLogEntries
(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String, Object> filterParams) Retrieve recent online records from HMaster / RegionServers.getNamespaceDescriptor
(String name) Get a namespace descriptor by nameprivate void
getProcedureResult
(long procId, CompletableFuture<Void> future, int retries) List proceduresgetQuota
(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.getRegionMetrics
(ServerName serverName) Get a list ofRegionMetrics
of all regions hosted on a region server.getRegionMetrics
(ServerName serverName, TableName tableName) Get a list ofRegionMetrics
of all regions hosted on a region server for a table.private CompletableFuture<List<RegionMetrics>>
getRegionMetrics
(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request, ServerName serverName) getRegions
(ServerName serverName) Get all the online regions on a region server.getRegions
(TableName tableName) Get the regions of a given table.private CompletableFuture<List<ServerName>>
getRegionServerList
(List<String> serverNamesList) getRegionServerSpaceQuotaSnapshots
(ServerName serverName) Fetches the observedSpaceQuotaSnapshotView
s observed by a RegionServer.getReplicationPeerConfig
(String peerId) Returns the configured ReplicationPeerConfig for the specified peergetRSGroup
(String groupName) Get group info for the given group namegetRSGroup
(Address hostPort) Get group info for the given hostPortgetRSGroup
(TableName table) Get group info for the given tableReturns the list of supported security capabilities.private CompletableFuture<List<LogEntry>>
getSlowLogResponseFromServer
(ServerName serverName, Map<String, Object> filterParams, int limit, String logType) private CompletableFuture<List<LogEntry>>
getSlowLogResponses
(Map<String, Object> filterParams, Set<ServerName> serverNames, int limit, String logType) Fetches the table sizes on the filesystem as tracked by the HBase Master.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) getUserPermissions
(GetUserPermissionsRequest getUserPermissionsRequest) Get the global/namespace/table permissions for usergrant
(UserPermission userPermission, boolean mergeExistingPermissions) Grants user specific permissionshasUserPermissions
(String userName, List<Permission> permissions) Check if the user has specific permissionsprivate CompletableFuture<Void>
internalDeleteSnapshot
(SnapshotDescription snapshot) private CompletableFuture<Void>
internalDeleteSnapshots
(Pattern tableNamePattern, Pattern snapshotNamePattern) private CompletableFuture<Void>
internalRestoreSnapshot
(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Query the current state of the balancer.Query on the catalog janitor state.Query the current state of the cleaner chore.Check whether master is in maintenance modeQuery the current state of the Merge switch.Query the current state of the region normalizerCheck the current state of the specified procedure.isReplicationPeerEnabled
(String peerId) Check if a replication peer is enabled.Check whether replication peer modification is enabled.Get if the rpc throttle is enabled.Query the current state of the auto snapshot cleanup based on TTL.isSnapshotFinished
(SnapshotDescription snapshot) Check the current state of the passed snapshot.Query the current state of the Split switch.private CompletableFuture<Boolean>
isSplitOrMergeOn
(MasterSwitchType switchType) isTableAvailable
(TableName tableName) Check if a table is available.isTableDisabled
(TableName tableName) Check if a table is disabled.isTableEnabled
(TableName tableName) Check if a table is enabled.private void
legacyFlush
(CompletableFuture<Void> future, TableName tableName, List<byte[]> columnFamilies) List region servers marked as decommissioned, which can not be assigned regions.List available namespace descriptorsList available namespacesFind all table and column families that are replicated from this clusterReturn a list of replication peers.listReplicationPeers
(Pattern pattern) Return a list of replication peers.private CompletableFuture<List<ReplicationPeerDescription>>
listReplicationPeers
(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request) Lists current set of RegionServer groupsList completed snapshots.listSnapshots
(Pattern pattern) List all the completed snapshots matching the given pattern.listTableDescriptors
(boolean includeSysTables) List all the tables.listTableDescriptors
(List<TableName> tableNames) List specific tables including system tables.listTableDescriptors
(Pattern pattern, boolean includeSysTables) Get list of table descriptors by namespace.listTableDescriptorsByState
(boolean isEnabled) List all enabled or disabled table descriptorslistTableNames
(boolean includeSysTables) List all of the names of tables.listTableNames
(Pattern pattern, boolean includeSysTables) List all of the names of userspace tables.Get list of table names by namespace.listTableNamesByState
(boolean isEnabled) List all enabled or disabled table nameslistTablesInRSGroup
(String groupName) Get all tables in this RegionServer group.listTableSnapshots
(Pattern tableNamePattern) List all the completed snapshots matching the given table name pattern.listTableSnapshots
(Pattern tableNamePattern, Pattern snapshotNamePattern) List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.majorCompact
(TableName tableName, byte[] columnFamily, CompactType compactType) Major compact a column family within a table.majorCompact
(TableName tableName, CompactType compactType) Major compact a table.majorCompactRegion
(byte[] regionName) Major compact a region.majorCompactRegion
(byte[] regionName, byte[] columnFamily) Major compact a column family within region.Compact all regions on the region server.mergeRegions
(List<byte[]> nameOfRegionsToMerge, boolean forcible) Merge multiple regions (>=2).mergeSwitch
(boolean enabled, boolean drainMerges) Turn the Merge switch on or off.modifyColumnFamily
(TableName tableName, ColumnFamilyDescriptor columnFamily) Modify an existing column family on a table.modifyColumnFamilyStoreFileTracker
(TableName tableName, byte[] family, String dstSFT) Change the store file tracker of the given table's given family.modifyNamespace
(NamespaceDescriptor descriptor) Modify an existing namespace.modifyTable
(TableDescriptor desc) Modify an existing table, more IRB friendly version.modifyTable
(TableDescriptor desc, boolean reopenRegions) Modify an existing table, more IRB friendly version.modifyTableStoreFileTracker
(TableName tableName, String dstSFT) Change the store file tracker of the given table.move
(byte[] regionName) Move the regionr
to a random server.move
(byte[] regionName, ServerName destServerName) Move the regionr
todest
.private CompletableFuture<Void>
moveRegion
(RegionInfo regionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request) moveServersToRSGroup
(Set<Address> servers, String groupName) Move given set of servers to the specified target RegionServer groupprivate <T> AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder<T>
(package private) <T> AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder<T>
(package private) <T> AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder<T>
Invoke region normalizer.private CompletableFuture<Boolean>
normalize
(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request) normalizerSwitch
(boolean on) Set region normalizer on/off.offline
(byte[] regionName) Offline specified region from master's in-memory state.private <PREQ,
PRESP>
CompletableFuture<Void>procedureCall
(Consumer<AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder<?>> prioritySetter, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer consumer) private <PREQ,
PRESP>
CompletableFuture<Void>procedureCall
(TableName tableName, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer consumer) private <PREQ,
PRESP>
CompletableFuture<Void>procedureCall
(PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer consumer) recommissionRegionServer
(ServerName server, List<byte[]> encodedRegionNames) Remove decommission marker from a region server to allow regions assignments.removeReplicationPeer
(String peerId) Remove a peer and stop the replicationRemove some table-cfs from config of the specified peerremoveRSGroup
(String groupName) Remove RegionServer group associated with the given nameremoveServersFromRSGroup
(Set<Address> servers) Remove decommissioned servers from group 1.renameRSGroup
(String oldName, String newName) Rename rsgroupreplicationPeerModificationSwitch
(boolean on, boolean drainProcedures) Enable or disable replication peer modification.restoreSnapshot
(String snapshotName) Restore the specified snapshot on the original table.restoreSnapshot
(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) Restore the specified snapshot on the original table.private CompletableFuture<Void>
restoreSnapshot
(String snapshotName, TableName tableName, boolean takeFailSafeSnapshot, boolean restoreAcl) revoke
(UserPermission userPermission) Revokes user specific permissionsrollWALWriter
(ServerName serverName) Roll the log writer.Ask for a scan of the catalog table.Ask for cleaner chore to run.setQuota
(QuotaSettings quota) Apply the new quota settings.setRSGroup
(Set<TableName> tables, String groupName) Set the RegionServer group for tablesprivate CompletableFuture<Boolean>
setSplitOrMergeOn
(boolean enabled, boolean synchronous, 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.shutdown()
Shuts down the HBase cluster.snapshot
(SnapshotDescription snapshotDesc) Take a snapshot and wait for the server to complete that snapshot asynchronously.snapshotCleanupSwitch
(boolean on, boolean sync) Turn on or off the auto snapshot cleanup based on TTL.private CompletableFuture<Void>
split
(RegionInfo hri, byte[] splitPoint) Split a table.Split a table.splitRegion
(byte[] regionName) Split an individual region.splitRegion
(byte[] regionName, byte[] splitPoint) Split an individual region.splitSwitch
(boolean enabled, boolean drainSplits) Turn the Split switch on or off.Shuts down the current HBase master only.stopRegionServer
(ServerName serverName) Stop the designated regionserver.private CompletableFuture<Boolean>
switchCompact
(ServerName serverName, boolean onOrOff) switchRpcThrottle
(boolean enable) Switch the rpc throttle enabled state.tableExists
(TableName tableName) Check if a table exists.private byte[]
toEncodeRegionName
(byte[] regionName) transitReplicationPeerSyncReplicationState
(String peerId, SyncReplicationState clusterState) Transit current cluster to a new state in a synchronous replication peer.truncateRegion
(byte[] regionName) Truncate an individual region.private CompletableFuture<Void>
truncateRegion
(RegionInfo hri) truncateTable
(TableName tableName, boolean preserveSplits) Truncate a table.private CompletableFuture<Void>
trySyncTableToPeerCluster
(TableName tableName, byte[][] splits, ReplicationPeerDescription peer) unassign
(byte[] regionName) Unassign a region from current hosting regionserver.Update the configuration and trigger an online config change on all the masters and regionservers.updateConfiguration
(String groupName) Update the configuration and trigger an online config change on all the regionservers in the RSGroup.updateConfiguration
(ServerName serverName) Update the configuration and trigger an online config change on the regionserver.updateReplicationPeerConfig
(String peerId, ReplicationPeerConfig peerConfig) Update the peerConfig for the specified peerupdateRSGroupConfig
(String groupName, Map<String, String> configuration) Update RSGroup configurationprivate void
verifySplitKeys
(byte[][] splitKeys) private CompletableFuture<Void>
waitProcedureResult
(CompletableFuture<Long> procFuture) private void
waitSnapshotFinish
(SnapshotDescription snapshot, CompletableFuture<Void> future, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse resp) private void
waitUntilAllReplicationPeerModificationProceduresDone
(CompletableFuture<Boolean> future, boolean prevOn, int retries) Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
Methods inherited from interface org.apache.hadoop.hbase.client.AsyncAdmin
addReplicationPeer, balance, balance, balanceRSGroup, balancerSwitch, cloneSnapshot, cloneSnapshot, compact, compact, getBackupMasters, getCompactionState, getMaster, getMasterCoprocessorNames, getMasterInfoPort, getRegionServers, getRegionServers, getReplicationPeerSyncReplicationState, getSlowLogResponses, hasUserPermissions, listDeadServers, listTableDescriptors, listTableNames, listUnknownServers, majorCompact, majorCompact, mergeRegions, mergeSwitch, normalize, replicationPeerModificationSwitch, restoreSnapshot, snapshot, snapshot, splitSwitch, unassign
-
Field Details
-
FLUSH_TABLE_PROCEDURE_SIGNATURE
- See Also:
-
LOG
-
connection
-
retryTimer
-
metaTable
-
rpcTimeoutNs
-
operationTimeoutNs
-
pauseNs
-
pauseNsForServerOverloaded
-
maxAttempts
-
startLogErrorsCnt
-
ng
-
-
Constructor Details
-
RawAsyncHBaseAdmin
RawAsyncHBaseAdmin(AsyncConnectionImpl connection, org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer retryTimer, AsyncAdminBuilderBase builder)
-
-
Method Details
-
newMasterCaller
-
newAdminCaller
-
call
private <PREQ,PRESP, CompletableFuture<RESP> callRESP> (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) -
adminCall
private <PREQ,PRESP, CompletableFuture<RESP> adminCallRESP> (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) -
procedureCall
private <PREQ,PRESP> CompletableFuture<Void> procedureCall(PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer consumer) -
procedureCall
private <PREQ,PRESP> CompletableFuture<Void> procedureCall(TableName tableName, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer consumer) -
procedureCall
private <PREQ,PRESP> CompletableFuture<Void> procedureCall(Consumer<AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder<?>> prioritySetter, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer consumer) -
tableExists
Description copied from interface:AsyncAdmin
Check if a table exists.- Specified by:
tableExists
in interfaceAsyncAdmin
- Parameters:
tableName
- Table to check.- Returns:
- True if table exists already. The return value will be wrapped by a
CompletableFuture
.
-
listTableDescriptors
Description copied from interface:AsyncAdmin
List all the tables.- Specified by:
listTableDescriptors
in interfaceAsyncAdmin
- Parameters:
includeSysTables
- False to match only against userspace tables- Returns:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture
.
-
listTableDescriptors
public CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern, boolean includeSysTables) - Specified by:
listTableDescriptors
in interfaceAsyncAdmin
- Parameters:
pattern
- The compiled regular expression to match againstincludeSysTables
- False to match only against userspace tables- Returns:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture
.
-
listTableDescriptors
Description copied from interface:AsyncAdmin
List specific tables including system tables.- Specified by:
listTableDescriptors
in interfaceAsyncAdmin
- Parameters:
tableNames
- the table list to match against- Returns:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture
.
-
getTableDescriptors
private CompletableFuture<List<TableDescriptor>> getTableDescriptors(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request) -
listTableNames
Description copied from interface:AsyncAdmin
List all of the names of tables.- Specified by:
listTableNames
in interfaceAsyncAdmin
- Parameters:
includeSysTables
- False to match only against userspace tables- Returns:
- a list of table names wrapped by a
CompletableFuture
.
-
listTableNames
Description copied from interface:AsyncAdmin
List all of the names of userspace tables.- Specified by:
listTableNames
in interfaceAsyncAdmin
- Parameters:
pattern
- The regular expression to match againstincludeSysTables
- False to match only against userspace tables- Returns:
- a list of table names wrapped by a
CompletableFuture
.
-
listTableNamesByState
Description copied from interface:AsyncAdmin
List all enabled or disabled table names- Specified by:
listTableNamesByState
in interfaceAsyncAdmin
- Parameters:
isEnabled
- is true means return enabled table names, false means return disabled table names- Returns:
- a list of table names wrapped by a
CompletableFuture
.
-
getTableNames
private CompletableFuture<List<TableName>> getTableNames(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request) -
listTableDescriptorsByNamespace
Description copied from interface:AsyncAdmin
Get list of table descriptors by namespace.- Specified by:
listTableDescriptorsByNamespace
in interfaceAsyncAdmin
- Parameters:
name
- namespace name- Returns:
- returns a list of TableDescriptors wrapped by a
CompletableFuture
.
-
listTableDescriptorsByState
Description copied from interface:AsyncAdmin
List all enabled or disabled table descriptors- Specified by:
listTableDescriptorsByState
in interfaceAsyncAdmin
- Parameters:
isEnabled
- is true means return enabled table descriptors, false means return disabled table descriptors- Returns:
- a list of table names wrapped by a
CompletableFuture
.
-
listTableNamesByNamespace
Description copied from interface:AsyncAdmin
Get list of table names by namespace.- Specified by:
listTableNamesByNamespace
in interfaceAsyncAdmin
- Parameters:
name
- namespace name- Returns:
- The list of table names in the namespace wrapped by a
CompletableFuture
.
-
getDescriptor
Description copied from interface:AsyncAdmin
Method for getting the tableDescriptor- Specified by:
getDescriptor
in interfaceAsyncAdmin
- Parameters:
tableName
- as aTableName
- Returns:
- the read-only tableDescriptor wrapped by a
CompletableFuture
.
-
createTable
Description copied from interface:AsyncAdmin
Creates a new table.- Specified by:
createTable
in interfaceAsyncAdmin
- Parameters:
desc
- table descriptor for table
-
createTable
public CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) Description copied from interface:AsyncAdmin
Creates a new table with the specified number of regions. The start key specified will become the end key of the first region of the table, and the end key specified will become the start key of the last region of the table (the first region has a null start key and the last region has a null end key). BigInteger math will be used to divide the key range specified into enough segments to make the required number of total regions.- Specified by:
createTable
in interfaceAsyncAdmin
- Parameters:
desc
- table descriptor for tablestartKey
- beginning of key rangeendKey
- end of key rangenumRegions
- the total number of regions to create
-
createTable
Description copied from interface:AsyncAdmin
Creates a new table with an initial set of empty regions defined by the specified split keys. The total number of regions created will be the number of split keys plus one. Note : Avoid passing empty split key.- Specified by:
createTable
in interfaceAsyncAdmin
- Parameters:
desc
- table descriptor for tablesplitKeys
- array of split keys for the initial regions of the table
-
createTable
private CompletableFuture<Void> createTable(TableName tableName, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request) -
modifyTable
Description copied from interface:AsyncAdmin
Modify an existing table, more IRB friendly version.- Specified by:
modifyTable
in interfaceAsyncAdmin
- Parameters:
desc
- modified description of the table
-
modifyTable
Description copied from interface:AsyncAdmin
Modify an existing table, more IRB friendly version.- Specified by:
modifyTable
in interfaceAsyncAdmin
- Parameters:
desc
- description of the tablereopenRegions
- By default, 'modifyTable' reopens all regions, potentially causing a RIT (Region In Transition) storm in large tables. If set to 'false', regions will remain unaware of the modification until they are individually reopened. Please note that this may temporarily result in configuration inconsistencies among regions.
-
modifyTableStoreFileTracker
Description copied from interface:AsyncAdmin
Change the store file tracker of the given table.- Specified by:
modifyTableStoreFileTracker
in interfaceAsyncAdmin
- Parameters:
tableName
- the table you want to changedstSFT
- the destination store file tracker
-
deleteTable
Description copied from interface:AsyncAdmin
Deletes a table.- Specified by:
deleteTable
in interfaceAsyncAdmin
- Parameters:
tableName
- name of table to delete
-
truncateTable
Description copied from interface:AsyncAdmin
Truncate a table.- Specified by:
truncateTable
in interfaceAsyncAdmin
- Parameters:
tableName
- name of table to truncatepreserveSplits
- True if the splits should be preserved
-
enableTable
Description copied from interface:AsyncAdmin
Enable a table. The table has to be in disabled state for it to be enabled.- Specified by:
enableTable
in interfaceAsyncAdmin
- Parameters:
tableName
- name of the table
-
disableTable
Description copied from interface:AsyncAdmin
Disable a table. The table has to be in enabled state for it to be disabled.- Specified by:
disableTable
in interfaceAsyncAdmin
-
completeCheckTableState
private static CompletableFuture<Boolean> completeCheckTableState(CompletableFuture<Boolean> future, TableState tableState, Throwable error, TableState.State targetState, TableName tableName) Utility for completing passed TableStateCompletableFuture
future
using passed parameters. Sets error or boolean result ('true' if table matches the passed-in targetState). -
isTableEnabled
Description copied from interface:AsyncAdmin
Check if a table is enabled.- Specified by:
isTableEnabled
in interfaceAsyncAdmin
- Parameters:
tableName
- name of table to check- Returns:
- true if table is on-line. The return value will be wrapped by a
CompletableFuture
.
-
isTableDisabled
Description copied from interface:AsyncAdmin
Check if a table is disabled.- Specified by:
isTableDisabled
in interfaceAsyncAdmin
- Parameters:
tableName
- name of table to check- Returns:
- true if table is off-line. The return value will be wrapped by a
CompletableFuture
.
-
isTableAvailable
Description copied from interface:AsyncAdmin
Check if a table is available.- Specified by:
isTableAvailable
in interfaceAsyncAdmin
- Parameters:
tableName
- name of table to check- Returns:
- true if all regions of the table are available. The return value will be wrapped by a
CompletableFuture
.
-
addColumnFamily
public CompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) Description copied from interface:AsyncAdmin
Add a column family to an existing table.- Specified by:
addColumnFamily
in interfaceAsyncAdmin
- Parameters:
tableName
- name of the table to add column family tocolumnFamily
- column family descriptor of column family to be added
-
deleteColumnFamily
Description copied from interface:AsyncAdmin
Delete a column family from a table.- Specified by:
deleteColumnFamily
in interfaceAsyncAdmin
- Parameters:
tableName
- name of tablecolumnFamily
- name of column family to be deleted
-
modifyColumnFamily
public CompletableFuture<Void> modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) Description copied from interface:AsyncAdmin
Modify an existing column family on a table.- Specified by:
modifyColumnFamily
in interfaceAsyncAdmin
- Parameters:
tableName
- name of tablecolumnFamily
- new column family descriptor to use
-
modifyColumnFamilyStoreFileTracker
public CompletableFuture<Void> modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT) Description copied from interface:AsyncAdmin
Change the store file tracker of the given table's given family.- Specified by:
modifyColumnFamilyStoreFileTracker
in interfaceAsyncAdmin
- Parameters:
tableName
- the table you want to changefamily
- the family you want to changedstSFT
- the destination store file tracker
-
createNamespace
Description copied from interface:AsyncAdmin
Create a new namespace.- Specified by:
createNamespace
in interfaceAsyncAdmin
- Parameters:
descriptor
- descriptor which describes the new namespace
-
modifyNamespace
Description copied from interface:AsyncAdmin
Modify an existing namespace.- Specified by:
modifyNamespace
in interfaceAsyncAdmin
- Parameters:
descriptor
- descriptor which describes the new namespace
-
deleteNamespace
Description copied from interface:AsyncAdmin
Delete an existing namespace. Only empty namespaces (no tables) can be removed.- Specified by:
deleteNamespace
in interfaceAsyncAdmin
- Parameters:
name
- namespace name
-
getNamespaceDescriptor
Description copied from interface:AsyncAdmin
Get a namespace descriptor by name- Specified by:
getNamespaceDescriptor
in interfaceAsyncAdmin
- Parameters:
name
- name of namespace descriptor- Returns:
- A descriptor wrapped by a
CompletableFuture
.
-
listNamespaces
Description copied from interface:AsyncAdmin
List available namespaces- Specified by:
listNamespaces
in interfaceAsyncAdmin
- Returns:
- List of namespaces wrapped by a
CompletableFuture
.
-
listNamespaceDescriptors
Description copied from interface:AsyncAdmin
List available namespace descriptors- Specified by:
listNamespaceDescriptors
in interfaceAsyncAdmin
- Returns:
- List of descriptors wrapped by a
CompletableFuture
.
-
getRegions
Description copied from interface:AsyncAdmin
Get all the online regions on a region server.- Specified by:
getRegions
in interfaceAsyncAdmin
-
getRegions
Description copied from interface:AsyncAdmin
Get the regions of a given table.- Specified by:
getRegions
in interfaceAsyncAdmin
-
flush
Description copied from interface:AsyncAdmin
Flush a table.- Specified by:
flush
in interfaceAsyncAdmin
- Parameters:
tableName
- table to flush
-
flush
Description copied from interface:AsyncAdmin
Flush the specified column family stores on all regions of the passed table. This runs as a synchronous operation.- Specified by:
flush
in interfaceAsyncAdmin
- Parameters:
tableName
- table to flushcolumnFamily
- column family within a table
-
flush
Description copied from interface:AsyncAdmin
Flush the specified column family stores on all regions of the passed table. This runs as a synchronous operation.- Specified by:
flush
in interfaceAsyncAdmin
- Parameters:
tableName
- table to flushcolumnFamilyList
- column families within a table
-
legacyFlush
private void legacyFlush(CompletableFuture<Void> future, TableName tableName, List<byte[]> columnFamilies) -
flushRegion
Description copied from interface:AsyncAdmin
Flush an individual region.- Specified by:
flushRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to flush
-
flushRegion
Description copied from interface:AsyncAdmin
Flush a column family within a region.- Specified by:
flushRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to flushcolumnFamily
- column family within a region. If not present, flush the region's all column families.
-
flushRegionInternal
CompletableFuture<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse> flushRegionInternal(byte[] regionName, byte[] columnFamily, boolean writeFlushWALMarker) This method is for internal use only, where we need the response of the flush. As it exposes the protobuf message, please do NOT try to expose it as a public API. -
flush
private CompletableFuture<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse> flush(ServerName serverName, RegionInfo regionInfo, byte[] columnFamily, boolean writeFlushWALMarker) -
flushRegionServer
Description copied from interface:AsyncAdmin
Flush all region on the region server.- Specified by:
flushRegionServer
in interfaceAsyncAdmin
- Parameters:
sn
- server to flush
-
compact
Description copied from interface:AsyncAdmin
Compact a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found for normal compaction type.- Specified by:
compact
in interfaceAsyncAdmin
- Parameters:
tableName
- table to compactcompactType
-CompactType
-
compact
public CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, CompactType compactType) Description copied from interface:AsyncAdmin
Compact a column family within a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found for normal compaction type.- Specified by:
compact
in interfaceAsyncAdmin
- Parameters:
tableName
- table to compactcolumnFamily
- column family within a tablecompactType
-CompactType
-
compactRegion
Description copied from interface:AsyncAdmin
Compact an individual region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Specified by:
compactRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to compact
-
compactRegion
Description copied from interface:AsyncAdmin
Compact a column family within a region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Specified by:
compactRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to compactcolumnFamily
- column family within a region. If not present, compact the region's all column families.
-
majorCompact
Description copied from interface:AsyncAdmin
Major compact a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found for normal compaction type.- Specified by:
majorCompact
in interfaceAsyncAdmin
- Parameters:
tableName
- table to major compactcompactType
-CompactType
-
majorCompact
public CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) Description copied from interface:AsyncAdmin
Major compact a column family within a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found.- Specified by:
majorCompact
in interfaceAsyncAdmin
- Parameters:
tableName
- table to major compactcolumnFamily
- column family within a table. If not present, major compact the table's all column families.compactType
-CompactType
-
majorCompactRegion
Description copied from interface:AsyncAdmin
Major compact a region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Specified by:
majorCompactRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to major compact
-
majorCompactRegion
Description copied from interface:AsyncAdmin
Major compact a column family within region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Specified by:
majorCompactRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to major compactcolumnFamily
- column family within a region. If not present, major compact the region's all column families.
-
compactRegionServer
Description copied from interface:AsyncAdmin
Compact all regions on the region server.- Specified by:
compactRegionServer
in interfaceAsyncAdmin
- Parameters:
sn
- the region server name
-
majorCompactRegionServer
Description copied from interface:AsyncAdmin
Compact all regions on the region server.- Specified by:
majorCompactRegionServer
in interfaceAsyncAdmin
- Parameters:
sn
- the region server name
-
compactRegionServer
-
compactRegion
private CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily, boolean major) -
getTableHRegionLocations
List all region locations for the specific table. -
compact
private CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, boolean major, CompactType compactType) Compact column family of a table, Asynchronous operation even if CompletableFuture.get() -
compact
private CompletableFuture<Void> compact(ServerName sn, RegionInfo hri, boolean major, byte[] columnFamily) Compact the region at specific region server. -
toEncodeRegionName
-
checkAndGetTableName
private void checkAndGetTableName(byte[] encodeRegionName, AtomicReference<TableName> tableName, CompletableFuture<TableName> result) -
checkRegionsAndGetTableName
-
mergeSwitch
Description copied from interface:AsyncAdmin
Turn the Merge switch on or off. Notice that, the method itself is always non-blocking, which means it will always return immediately. ThedrainMerges
parameter only effects when will we complete the returnedCompletableFuture
.- Specified by:
mergeSwitch
in interfaceAsyncAdmin
- Parameters:
enabled
- enabled or notdrainMerges
- Iftrue
, it waits until current merge() call, if outstanding, to return.- Returns:
- Previous switch value wrapped by a
CompletableFuture
-
isMergeEnabled
Description copied from interface:AsyncAdmin
Query the current state of the Merge switch.- Specified by:
isMergeEnabled
in interfaceAsyncAdmin
- Returns:
- true if the switch is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
splitSwitch
Description copied from interface:AsyncAdmin
Turn the Split switch on or off. Notice that, the method itself is always non-blocking, which means it will always return immediately. ThedrainSplits
parameter only effects when will we complete the returnedCompletableFuture
.- Specified by:
splitSwitch
in interfaceAsyncAdmin
- Parameters:
enabled
- enabled or notdrainSplits
- Iftrue
, it waits until current split() call, if outstanding, to return.- Returns:
- Previous switch value wrapped by a
CompletableFuture
-
isSplitEnabled
Description copied from interface:AsyncAdmin
Query the current state of the Split switch.- Specified by:
isSplitEnabled
in interfaceAsyncAdmin
- Returns:
- true if the switch is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
setSplitOrMergeOn
private CompletableFuture<Boolean> setSplitOrMergeOn(boolean enabled, boolean synchronous, MasterSwitchType switchType) -
isSplitOrMergeOn
-
mergeRegions
Description copied from interface:AsyncAdmin
Merge multiple regions (>=2).- Specified by:
mergeRegions
in interfaceAsyncAdmin
- Parameters:
nameOfRegionsToMerge
- encoded or full name of daughter regionsforcible
- true if do a compulsory merge, otherwise we will only merge two adjacent regions
-
split
Description copied from interface:AsyncAdmin
Split a table. The method will execute split action for each region in table.- Specified by:
split
in interfaceAsyncAdmin
- Parameters:
tableName
- table to split
-
split
Description copied from interface:AsyncAdmin
Split a table.- Specified by:
split
in interfaceAsyncAdmin
- Parameters:
tableName
- table to splitsplitPoint
- the explicit position to split on
-
splitRegion
Description copied from interface:AsyncAdmin
Split an individual region.- Specified by:
splitRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to split
-
splitRegion
Description copied from interface:AsyncAdmin
Split an individual region.- Specified by:
splitRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to splitsplitPoint
- the explicit position to split on. If not present, it will decide by region server.
-
split
-
truncateRegion
Description copied from interface:AsyncAdmin
Truncate an individual region.- Specified by:
truncateRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to truncate
-
truncateRegion
-
assign
Description copied from interface:AsyncAdmin
Assign an individual region.- Specified by:
assign
in interfaceAsyncAdmin
- Parameters:
regionName
- Encoded or full name of region to assign.
-
unassign
Description copied from interface:AsyncAdmin
Unassign a region from current hosting regionserver. Region will then be assigned to a regionserver chosen at random. Region could be reassigned back to the same server. UseAsyncAdmin.move(byte[], ServerName)
if you want to control the region movement.- Specified by:
unassign
in interfaceAsyncAdmin
- Parameters:
regionName
- Encoded or full name of region to unassign.
-
offline
Description copied from interface:AsyncAdmin
Offline specified region from master's in-memory state. It will not attempt to reassign the region as in unassign. This API can be used when a region not served by any region server and still online as per Master's in memory state. If this API is incorrectly used on active region then master will loose track of that region. This is a special method that should be used by experts or hbck.- Specified by:
offline
in interfaceAsyncAdmin
- Parameters:
regionName
- Encoded or full name of region to offline
-
move
Description copied from interface:AsyncAdmin
Move the regionr
to a random server.- Specified by:
move
in interfaceAsyncAdmin
- Parameters:
regionName
- Encoded or full name of region to move.
-
move
Description copied from interface:AsyncAdmin
Move the regionr
todest
.- Specified by:
move
in interfaceAsyncAdmin
- Parameters:
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
-
moveRegion
private CompletableFuture<Void> moveRegion(RegionInfo regionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request) -
setQuota
Description copied from interface:AsyncAdmin
Apply the new quota settings.- Specified by:
setQuota
in interfaceAsyncAdmin
- Parameters:
quota
- the quota settings
-
getQuota
Description copied from interface:AsyncAdmin
List the quotas based on the filter.- Specified by:
getQuota
in interfaceAsyncAdmin
- Parameters:
filter
- the quota settings filter- Returns:
- the QuotaSetting list, which wrapped by a CompletableFuture.
-
addReplicationPeer
public CompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) Description copied from interface:AsyncAdmin
Add a new replication peer for replicating data to slave cluster- Specified by:
addReplicationPeer
in interfaceAsyncAdmin
- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication slave clusterenabled
- peer state, true if ENABLED and false if DISABLED
-
removeReplicationPeer
Description copied from interface:AsyncAdmin
Remove a peer and stop the replication- Specified by:
removeReplicationPeer
in interfaceAsyncAdmin
- Parameters:
peerId
- a short name that identifies the peer
-
enableReplicationPeer
Description copied from interface:AsyncAdmin
Restart the replication stream to the specified peer- Specified by:
enableReplicationPeer
in interfaceAsyncAdmin
- Parameters:
peerId
- a short name that identifies the peer
-
disableReplicationPeer
Description copied from interface:AsyncAdmin
Stop the replication stream to the specified peer- Specified by:
disableReplicationPeer
in interfaceAsyncAdmin
- Parameters:
peerId
- a short name that identifies the peer
-
getReplicationPeerConfig
Description copied from interface:AsyncAdmin
Returns the configured ReplicationPeerConfig for the specified peer- Specified by:
getReplicationPeerConfig
in interfaceAsyncAdmin
- Parameters:
peerId
- a short name that identifies the peer- Returns:
- ReplicationPeerConfig for the peer wrapped by a
CompletableFuture
.
-
updateReplicationPeerConfig
public CompletableFuture<Void> updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) Description copied from interface:AsyncAdmin
Update the peerConfig for the specified peer- Specified by:
updateReplicationPeerConfig
in interfaceAsyncAdmin
- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- new config for the peer
-
transitReplicationPeerSyncReplicationState
public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState clusterState) Description copied from interface:AsyncAdmin
Transit current cluster to a new state in a synchronous replication peer.- Specified by:
transitReplicationPeerSyncReplicationState
in interfaceAsyncAdmin
- Parameters:
peerId
- a short name that identifies the peerclusterState
- a new state of current cluster
-
appendReplicationPeerTableCFs
public CompletableFuture<Void> appendReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs) Description copied from interface:AsyncAdmin
Append the replicable table-cf config of the specified peer- Specified by:
appendReplicationPeerTableCFs
in interfaceAsyncAdmin
- Parameters:
id
- a short that identifies the clustertableCfs
- A map from tableName to column family names
-
removeReplicationPeerTableCFs
public CompletableFuture<Void> removeReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs) Description copied from interface:AsyncAdmin
Remove some table-cfs from config of the specified peer- Specified by:
removeReplicationPeerTableCFs
in interfaceAsyncAdmin
- Parameters:
id
- a short name that identifies the clustertableCfs
- A map from tableName to column family names
-
listReplicationPeers
Description copied from interface:AsyncAdmin
Return a list of replication peers.- Specified by:
listReplicationPeers
in interfaceAsyncAdmin
- Returns:
- a list of replication peers description. The return value will be wrapped by a
CompletableFuture
.
-
listReplicationPeers
Description copied from interface:AsyncAdmin
Return a list of replication peers.- Specified by:
listReplicationPeers
in interfaceAsyncAdmin
- Parameters:
pattern
- The compiled regular expression to match peer id- Returns:
- a list of replication peers description. The return value will be wrapped by a
CompletableFuture
.
-
listReplicationPeers
private CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request) -
listReplicatedTableCFs
Description copied from interface:AsyncAdmin
Find all table and column families that are replicated from this cluster- Specified by:
listReplicatedTableCFs
in interfaceAsyncAdmin
- Returns:
- the replicated table-cfs list of this cluster. The return value will be wrapped by a
CompletableFuture
.
-
snapshot
Description copied from interface:AsyncAdmin
Take a snapshot and wait for the server to complete that snapshot asynchronously. Snapshots are taken sequentially even when requested concurrently, across all tables. Snapshots are considered unique based on the name of the snapshot. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with aSnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. SeeTableName.isLegalFullyQualifiedTableName(byte[])
. You should probably useAsyncAdmin.snapshot(String, org.apache.hadoop.hbase.TableName)
unless you are sure about the type of snapshot that you want to take.- Specified by:
snapshot
in interfaceAsyncAdmin
- Parameters:
snapshotDesc
- snapshot to take
-
waitSnapshotFinish
private void waitSnapshotFinish(SnapshotDescription snapshot, CompletableFuture<Void> future, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse resp) -
isSnapshotFinished
Description copied from interface:AsyncAdmin
Check the current state of the passed snapshot. There are three possible states:- running - returns false
- finished - returns true
- finished with error - throws the exception that caused the snapshot to fail
UnknownSnapshotException
.- Specified by:
isSnapshotFinished
in interfaceAsyncAdmin
- Parameters:
snapshot
- description of the snapshot to check- Returns:
- true if the snapshot is completed, false if the snapshot is still running
-
restoreSnapshot
Description copied from interface:AsyncAdmin
Restore the specified snapshot on the original table. (The table must be disabled) If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted.- Specified by:
restoreSnapshot
in interfaceAsyncAdmin
- Parameters:
snapshotName
- name of the snapshot to restore
-
restoreSnapshot
public CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) Description copied from interface:AsyncAdmin
Restore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".- Specified by:
restoreSnapshot
in interfaceAsyncAdmin
- Parameters:
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true if the failsafe snapshot should be takenrestoreAcl
-true
to restore acl of snapshot
-
restoreSnapshot
private CompletableFuture<Void> restoreSnapshot(String snapshotName, TableName tableName, boolean takeFailSafeSnapshot, boolean restoreAcl) -
completeConditionalOnFuture
private <T> void completeConditionalOnFuture(CompletableFuture<T> dependentFuture, CompletableFuture<T> parentFuture) -
cloneSnapshot
public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Description copied from interface:AsyncAdmin
Create a new table by cloning the snapshot content.- Specified by:
cloneSnapshot
in interfaceAsyncAdmin
- Parameters:
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredrestoreAcl
-true
to restore acl of snapshotcustomSFT
- specify the StroreFileTracker used for the table
-
internalRestoreSnapshot
private CompletableFuture<Void> internalRestoreSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) -
listSnapshots
Description copied from interface:AsyncAdmin
List completed snapshots.- Specified by:
listSnapshots
in interfaceAsyncAdmin
- Returns:
- a list of snapshot descriptors for completed snapshots wrapped by a
CompletableFuture
-
listSnapshots
Description copied from interface:AsyncAdmin
List all the completed snapshots matching the given pattern.- Specified by:
listSnapshots
in interfaceAsyncAdmin
- Parameters:
pattern
- The compiled regular expression to match against- Returns:
- - returns a List of SnapshotDescription wrapped by a
CompletableFuture
-
getCompletedSnapshots
-
listTableSnapshots
Description copied from interface:AsyncAdmin
List all the completed snapshots matching the given table name pattern.- Specified by:
listTableSnapshots
in interfaceAsyncAdmin
- Parameters:
tableNamePattern
- The compiled table name regular expression to match against- Returns:
- - returns a List of completed SnapshotDescription wrapped by a
CompletableFuture
-
listTableSnapshots
public CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) Description copied from interface:AsyncAdmin
List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.- Specified by:
listTableSnapshots
in interfaceAsyncAdmin
- Parameters:
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match against- Returns:
- - returns a List of completed SnapshotDescription wrapped by a
CompletableFuture
-
getCompletedSnapshots
private CompletableFuture<List<SnapshotDescription>> getCompletedSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) -
deleteSnapshot
Description copied from interface:AsyncAdmin
Delete an existing snapshot.- Specified by:
deleteSnapshot
in interfaceAsyncAdmin
- Parameters:
snapshotName
- name of the snapshot
-
deleteSnapshots
Description copied from interface:AsyncAdmin
Delete all existing snapshots.- Specified by:
deleteSnapshots
in interfaceAsyncAdmin
-
deleteSnapshots
Description copied from interface:AsyncAdmin
Delete existing snapshots whose names match the pattern passed.- Specified by:
deleteSnapshots
in interfaceAsyncAdmin
- Parameters:
snapshotNamePattern
- pattern for names of the snapshot to match
-
deleteTableSnapshots
Description copied from interface:AsyncAdmin
Delete all existing snapshots matching the given table name pattern.- Specified by:
deleteTableSnapshots
in interfaceAsyncAdmin
- Parameters:
tableNamePattern
- The compiled table name regular expression to match against
-
deleteTableSnapshots
public CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) Description copied from interface:AsyncAdmin
Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.- Specified by:
deleteTableSnapshots
in interfaceAsyncAdmin
- Parameters:
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match against
-
internalDeleteSnapshots
private CompletableFuture<Void> internalDeleteSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) -
internalDeleteSnapshot
-
execProcedure
public CompletableFuture<Void> execProcedure(String signature, String instance, Map<String, String> props) Description copied from interface:AsyncAdmin
Execute a distributed procedure on a cluster.- Specified by:
execProcedure
in interfaceAsyncAdmin
- Parameters:
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 procedure
-
execProcedureWithReturn
public CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance, Map<String, String> props) Description copied from interface:AsyncAdmin
Execute a distributed procedure on a cluster.- Specified by:
execProcedureWithReturn
in interfaceAsyncAdmin
- Parameters:
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 procedure- Returns:
- data returned after procedure execution. null if no return data.
-
isProcedureFinished
public CompletableFuture<Boolean> isProcedureFinished(String signature, String instance, Map<String, String> props) Description copied from interface:AsyncAdmin
Check the current state of the specified procedure. There are three possible states:- running - returns false
- finished - returns true
- finished with error - throws the exception that caused the procedure to fail
- Specified by:
isProcedureFinished
in interfaceAsyncAdmin
- Parameters:
signature
- The signature that uniquely identifies a procedureinstance
- The instance name of the procedureprops
- Property/Value pairs of properties passing to the procedure- Returns:
- true if the specified procedure is finished successfully, false if it is still running.
The value is wrapped by
CompletableFuture
-
abortProcedure
Description copied from interface:AsyncAdmin
Abort a procedure Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.- Specified by:
abortProcedure
in interfaceAsyncAdmin
- Parameters:
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?- Returns:
- true if aborted, false if procedure already completed or does not exist. the value is
wrapped by
CompletableFuture
- See Also:
-
getProcedures
Description copied from interface:AsyncAdmin
List procedures- Specified by:
getProcedures
in interfaceAsyncAdmin
- Returns:
- procedure list JSON wrapped by
CompletableFuture
-
getLocks
Description copied from interface:AsyncAdmin
List locks.- Specified by:
getLocks
in interfaceAsyncAdmin
- Returns:
- lock list JSON wrapped by
CompletableFuture
-
decommissionRegionServers
Description copied from interface:AsyncAdmin
Mark region server(s) as decommissioned to prevent additional regions from getting assigned to them. Optionally unload the regions on the servers. If there are multiple servers to be decommissioned, decommissioning them at the same time can prevent wasteful region movements. Region unloading is asynchronous.- Specified by:
decommissionRegionServers
in interfaceAsyncAdmin
- Parameters:
servers
- The list of servers to decommission.offload
- True to offload the regions from the decommissioned servers
-
listDecommissionedRegionServers
Description copied from interface:AsyncAdmin
List region servers marked as decommissioned, which can not be assigned regions.- Specified by:
listDecommissionedRegionServers
in interfaceAsyncAdmin
- Returns:
- List of decommissioned region servers wrapped by
CompletableFuture
-
recommissionRegionServer
public CompletableFuture<Void> recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) Description copied from interface:AsyncAdmin
Remove decommission marker from a region server to allow regions assignments. Load regions onto the server if a list of regions is given. Region loading is asynchronous.- Specified by:
recommissionRegionServer
in interfaceAsyncAdmin
- Parameters:
server
- The server to recommission.encodedRegionNames
- Regions to load onto the server.
-
getRegionLocation
Get the region location for the passed region name. The region name may be a full region name or encoded region name. If the region does not found, then it'll throw an UnknownRegionException wrapped by aCompletableFuture
- Parameters:
regionNameOrEncodedRegionName
- region name or encoded region name- Returns:
- region location, wrapped by a
CompletableFuture
-
getRegionInfo
Get the region info for the passed region name. The region name may be a full region name or encoded region name. If the region does not found, then it'll throw an UnknownRegionException wrapped by aCompletableFuture
- Returns:
- region info, wrapped by a
CompletableFuture
-
getSplitKeys
-
verifySplitKeys
-
waitProcedureResult
-
getProcedureResult
-
failedFuture
-
completeExceptionally
-
getClusterMetrics
Description copied from interface:AsyncAdmin
Returns cluster status wrapped byCompletableFuture
- Specified by:
getClusterMetrics
in interfaceAsyncAdmin
-
getClusterMetrics
Description copied from interface:AsyncAdmin
Returns cluster status wrapped byCompletableFuture
- Specified by:
getClusterMetrics
in interfaceAsyncAdmin
-
shutdown
Description copied from interface:AsyncAdmin
Shuts down the HBase cluster.- Specified by:
shutdown
in interfaceAsyncAdmin
-
stopMaster
Description copied from interface:AsyncAdmin
Shuts down the current HBase master only.- Specified by:
stopMaster
in interfaceAsyncAdmin
-
stopRegionServer
Description copied from interface:AsyncAdmin
Stop the designated regionserver.- Specified by:
stopRegionServer
in interfaceAsyncAdmin
-
updateConfiguration
Description copied from interface:AsyncAdmin
Update the configuration and trigger an online config change on the regionserver.- Specified by:
updateConfiguration
in interfaceAsyncAdmin
- Parameters:
serverName
- : The server whose config needs to be updated.
-
updateConfiguration
Description copied from interface:AsyncAdmin
Update the configuration and trigger an online config change on all the masters and regionservers.- Specified by:
updateConfiguration
in interfaceAsyncAdmin
-
updateConfiguration
Description copied from interface:AsyncAdmin
Update the configuration and trigger an online config change on all the regionservers in the RSGroup.- Specified by:
updateConfiguration
in interfaceAsyncAdmin
- Parameters:
groupName
- the group name
-
rollWALWriter
Description copied from interface:AsyncAdmin
Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.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.
- Specified by:
rollWALWriter
in interfaceAsyncAdmin
- Parameters:
serverName
- The servername of the region server.
-
clearCompactionQueues
Description copied from interface:AsyncAdmin
Clear compacting queues on a region server.- Specified by:
clearCompactionQueues
in interfaceAsyncAdmin
- Parameters:
serverName
- The servername of the region server.queues
- the set of queue name
-
getSecurityCapabilities
Description copied from interface:AsyncAdmin
Returns the list of supported security capabilities. The return value will be wrapped by aCompletableFuture
.- Specified by:
getSecurityCapabilities
in interfaceAsyncAdmin
-
getRegionMetrics
Description copied from interface:AsyncAdmin
Get a list ofRegionMetrics
of all regions hosted on a region server.- Specified by:
getRegionMetrics
in interfaceAsyncAdmin
- Returns:
- list of
RegionMetrics
wrapped byCompletableFuture
-
getRegionMetrics
public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName, TableName tableName) Description copied from interface:AsyncAdmin
Get a list ofRegionMetrics
of all regions hosted on a region server for a table.- Specified by:
getRegionMetrics
in interfaceAsyncAdmin
- Returns:
- a list of
RegionMetrics
wrapped byCompletableFuture
-
getRegionMetrics
private CompletableFuture<List<RegionMetrics>> getRegionMetrics(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request, ServerName serverName) -
isMasterInMaintenanceMode
Description copied from interface:AsyncAdmin
Check whether master is in maintenance mode- Specified by:
isMasterInMaintenanceMode
in interfaceAsyncAdmin
- Returns:
- true if master is in maintenance mode, false otherwise. The return value will be
wrapped by a
CompletableFuture
-
getCompactionState
public CompletableFuture<CompactionState> getCompactionState(TableName tableName, CompactType compactType) Description copied from interface:AsyncAdmin
Get the current compaction state of a table. It could be in a major compaction, a minor compaction, both, or none.- Specified by:
getCompactionState
in interfaceAsyncAdmin
- Parameters:
tableName
- table to examinecompactType
-CompactType
- Returns:
- the current compaction state wrapped by a
CompletableFuture
-
getCompactionStateForRegion
Description copied from interface:AsyncAdmin
Get the current compaction state of region. It could be in a major compaction, a minor compaction, both, or none.- Specified by:
getCompactionStateForRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to examine- Returns:
- the current compaction state wrapped by a
CompletableFuture
-
getLastMajorCompactionTimestamp
Description copied from interface:AsyncAdmin
Get the timestamp of the last major compaction for the passed table.The timestamp of the oldest HFile resulting from a major compaction of that table, or not present if no such HFile could be found.
- Specified by:
getLastMajorCompactionTimestamp
in interfaceAsyncAdmin
- Parameters:
tableName
- table to examine- Returns:
- the last major compaction timestamp wrapped by a
CompletableFuture
-
getLastMajorCompactionTimestampForRegion
public CompletableFuture<Optional<Long>> getLastMajorCompactionTimestampForRegion(byte[] regionName) Description copied from interface:AsyncAdmin
Get the timestamp of the last major compaction for the passed region.The timestamp of the oldest HFile resulting from a major compaction of that region, or not present if no such HFile could be found.
- Specified by:
getLastMajorCompactionTimestampForRegion
in interfaceAsyncAdmin
- Parameters:
regionName
- region to examine- Returns:
- the last major compaction timestamp wrapped by a
CompletableFuture
-
compactionSwitch
public CompletableFuture<Map<ServerName,Boolean>> compactionSwitch(boolean switchState, List<String> serverNamesList) Description copied from interface:AsyncAdmin
Turn the compaction on or off. Disabling compactions will also interrupt any currently ongoing compactions. This state is ephemeral. The setting will be lost on restart. Compaction can also be enabled/disabled by modifying configuration hbase.regionserver.compaction.enabled in hbase-site.xml.- Specified by:
compactionSwitch
in interfaceAsyncAdmin
- Parameters:
switchState
- Set totrue
to enable,false
to disable.serverNamesList
- list of region servers.- Returns:
- Previous compaction states for region servers
-
getRegionServerList
-
switchCompact
-
balancerSwitch
Description copied from interface:AsyncAdmin
Turn the load balancer on or off. Notice that, the method itself is always non-blocking, which means it will always return immediately. ThedrainRITs
parameter only effects when will we complete the returnedCompletableFuture
.- Specified by:
balancerSwitch
in interfaceAsyncAdmin
- Parameters:
on
- Set totrue
to enable,false
to disable.drainRITs
- Iftrue
, it waits until current balance() call, if outstanding, to return.- Returns:
- Previous balancer value wrapped by a
CompletableFuture
.
-
balance
Description copied from interface:AsyncAdmin
Invoke the balancer with the given balance request. The BalanceRequest defines how the balancer will run. SeeBalanceRequest
for more details.- Specified by:
balance
in interfaceAsyncAdmin
- Parameters:
request
- defines how the balancer should run- Returns:
BalanceResponse
with details about the results of the invocation.
-
isBalancerEnabled
Description copied from interface:AsyncAdmin
Query the current state of the balancer.- Specified by:
isBalancerEnabled
in interfaceAsyncAdmin
- Returns:
- true if the balance switch is on, false otherwise. The return value will be wrapped by
a
CompletableFuture
.
-
normalizerSwitch
Description copied from interface:AsyncAdmin
Set region normalizer on/off.- Specified by:
normalizerSwitch
in interfaceAsyncAdmin
- Parameters:
on
- whether normalizer should be on or off- Returns:
- Previous normalizer value wrapped by a
CompletableFuture
-
isNormalizerEnabled
Description copied from interface:AsyncAdmin
Query the current state of the region normalizer- Specified by:
isNormalizerEnabled
in interfaceAsyncAdmin
- Returns:
- true if region normalizer is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
normalize
Description copied from interface:AsyncAdmin
Invoke region normalizer. Can NOT run for various reasons. Check logs.- Specified by:
normalize
in interfaceAsyncAdmin
- Parameters:
ntfp
- limit to tables matching the specified filter.- Returns:
- true if region normalizer ran, false otherwise. The return value will be wrapped by a
CompletableFuture
-
normalize
private CompletableFuture<Boolean> normalize(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request) -
cleanerChoreSwitch
Description copied from interface:AsyncAdmin
Turn the cleaner chore on/off.- Specified by:
cleanerChoreSwitch
in interfaceAsyncAdmin
- Returns:
- Previous cleaner state wrapped by a
CompletableFuture
-
isCleanerChoreEnabled
Description copied from interface:AsyncAdmin
Query the current state of the cleaner chore.- Specified by:
isCleanerChoreEnabled
in interfaceAsyncAdmin
- Returns:
- true if cleaner chore is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
runCleanerChore
Description copied from interface:AsyncAdmin
Ask for cleaner chore to run.- Specified by:
runCleanerChore
in interfaceAsyncAdmin
- Returns:
- true if cleaner chore ran, false otherwise. The return value will be wrapped by a
CompletableFuture
-
catalogJanitorSwitch
Description copied from interface:AsyncAdmin
Turn the catalog janitor on/off.- Specified by:
catalogJanitorSwitch
in interfaceAsyncAdmin
- Returns:
- the previous state wrapped by a
CompletableFuture
-
isCatalogJanitorEnabled
Description copied from interface:AsyncAdmin
Query on the catalog janitor state.- Specified by:
isCatalogJanitorEnabled
in interfaceAsyncAdmin
- Returns:
- true if the catalog janitor is on, false otherwise. The return value will be wrapped by
a
CompletableFuture
-
runCatalogJanitor
Description copied from interface:AsyncAdmin
Ask for a scan of the catalog table.- Specified by:
runCatalogJanitor
in interfaceAsyncAdmin
- Returns:
- the number of entries cleaned. The return value will be wrapped by a
CompletableFuture
-
coprocessorService
public <S,R> CompletableFuture<R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable) Description copied from interface:AsyncAdmin
Execute the given coprocessor call on the master.The
stubMaker
is just a delegation to thenewStub
call. Usually it is only a one line lambda expression, like:channel -> xxxService.newStub(channel)
- Specified by:
coprocessorService
in interfaceAsyncAdmin
- Type Parameters:
S
- the type of the asynchronous stubR
- the type of the return value- Parameters:
stubMaker
- a delegation to the actualnewStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment ofServiceCaller
for more details.- Returns:
- the return value of the protobuf rpc call, wrapped by a
CompletableFuture
. - See Also:
-
coprocessorService
public <S,R> CompletableFuture<R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, ServerName serverName) Description copied from interface:AsyncAdmin
Execute the given coprocessor call on the given region server.The
stubMaker
is just a delegation to thenewStub
call. Usually it is only a one line lambda expression, like:channel -> xxxService.newStub(channel)
- Specified by:
coprocessorService
in interfaceAsyncAdmin
- Type Parameters:
S
- the type of the asynchronous stubR
- the type of the return value- Parameters:
stubMaker
- a delegation to the actualnewStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment ofServiceCaller
for more details.serverName
- the given region server- Returns:
- the return value of the protobuf rpc call, wrapped by a
CompletableFuture
. - See Also:
-
clearDeadServers
Description copied from interface:AsyncAdmin
Clear dead region servers from master.- Specified by:
clearDeadServers
in interfaceAsyncAdmin
- Parameters:
servers
- list of dead region servers.- Returns:
- - returns a list of servers that not cleared wrapped by a
CompletableFuture
.
-
newServerCaller
-
enableTableReplication
Description copied from interface:AsyncAdmin
Enable a table's replication switch.- Specified by:
enableTableReplication
in interfaceAsyncAdmin
- Parameters:
tableName
- name of the table
-
disableTableReplication
Description copied from interface:AsyncAdmin
Disable a table's replication switch.- Specified by:
disableTableReplication
in interfaceAsyncAdmin
- Parameters:
tableName
- name of the table
-
getTableSplits
-
checkAndSyncTableToPeerClusters
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.
- Parameters:
tableName
- name of the table to sync to the peersplits
- table split keys
-
trySyncTableToPeerCluster
private CompletableFuture<Void> trySyncTableToPeerCluster(TableName tableName, byte[][] splits, ReplicationPeerDescription peer) -
compareTableWithPeerCluster
private CompletableFuture<Void> compareTableWithPeerCluster(TableName tableName, TableDescriptor tableDesc, ReplicationPeerDescription peer, AsyncAdmin peerAdmin) -
setTableReplication
Set the table's replication switch if the table's replication switch is already not set.- Parameters:
tableName
- name of the tableenableRep
- is replication switch enable or disable
-
isReplicationPeerEnabled
Description copied from interface:AsyncAdmin
Check if a replication peer is enabled.- Specified by:
isReplicationPeerEnabled
in interfaceAsyncAdmin
- Parameters:
peerId
- id of replication peer to check- Returns:
- true if replication peer is enabled. The return value will be wrapped by a
CompletableFuture
-
waitUntilAllReplicationPeerModificationProceduresDone
private void waitUntilAllReplicationPeerModificationProceduresDone(CompletableFuture<Boolean> future, boolean prevOn, int retries) -
replicationPeerModificationSwitch
public CompletableFuture<Boolean> replicationPeerModificationSwitch(boolean on, boolean drainProcedures) Description copied from interface:AsyncAdmin
Enable or disable replication peer modification. This is especially useful when you want to change the replication peer storage.- Specified by:
replicationPeerModificationSwitch
in interfaceAsyncAdmin
- Parameters:
on
-true
means enable, otherwise disabledrainProcedures
- iftrue
, will wait until all the running replication peer modification procedures finish- Returns:
- the previous enable/disable state wrapped by a
CompletableFuture
-
isReplicationPeerModificationEnabled
Description copied from interface:AsyncAdmin
Check whether replication peer modification is enabled.- Specified by:
isReplicationPeerModificationEnabled
in interfaceAsyncAdmin
- Returns:
true
if modification is enabled, otherwisefalse
, wrapped by aCompletableFuture
-
clearBlockCache
Description copied from interface:AsyncAdmin
Clear all the blocks corresponding to this table from BlockCache. For expert-admins. Calling this API will drop all the cached blocks specific to a table from BlockCache. This can significantly impact the query performance as the subsequent queries will have to retrieve the blocks from underlying filesystem.- Specified by:
clearBlockCache
in interfaceAsyncAdmin
- Parameters:
tableName
- table to clear block cache- Returns:
- CacheEvictionStats related to the eviction wrapped by a
CompletableFuture
.
-
cloneTableSchema
public CompletableFuture<Void> cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) Description copied from interface:AsyncAdmin
Create a new table by cloning the existent table schema.- Specified by:
cloneTableSchema
in interfaceAsyncAdmin
- Parameters:
tableName
- name of the table to be clonednewTableName
- name of the new table where the table will be createdpreserveSplits
- True if the splits should be preserved
-
clearBlockCache
private CompletableFuture<CacheEvictionStats> clearBlockCache(ServerName serverName, List<RegionInfo> hris) -
switchRpcThrottle
Description copied from interface:AsyncAdmin
Switch the rpc throttle enabled state.- Specified by:
switchRpcThrottle
in interfaceAsyncAdmin
- Parameters:
enable
- Set totrue
to enable,false
to disable.- Returns:
- Previous rpc throttle enabled value
-
isRpcThrottleEnabled
Description copied from interface:AsyncAdmin
Get if the rpc throttle is enabled.- Specified by:
isRpcThrottleEnabled
in interfaceAsyncAdmin
- Returns:
- True if rpc throttle is enabled
-
exceedThrottleQuotaSwitch
Description copied from interface:AsyncAdmin
Switch the exceed throttle quota. If enabled, user/table/namespace throttle quota can be exceeded if region server has availble quota.- Specified by:
exceedThrottleQuotaSwitch
in interfaceAsyncAdmin
- Parameters:
enable
- Set totrue
to enable,false
to disable.- Returns:
- Previous exceed throttle enabled value
-
getSpaceQuotaTableSizes
Description copied from interface:AsyncAdmin
Fetches the table sizes on the filesystem as tracked by the HBase Master.- Specified by:
getSpaceQuotaTableSizes
in interfaceAsyncAdmin
-
getRegionServerSpaceQuotaSnapshots
public CompletableFuture<Map<TableName,SpaceQuotaSnapshot>> getRegionServerSpaceQuotaSnapshots(ServerName serverName) Description copied from interface:AsyncAdmin
Fetches the observedSpaceQuotaSnapshotView
s observed by a RegionServer.- Specified by:
getRegionServerSpaceQuotaSnapshots
in interfaceAsyncAdmin
-
getCurrentSpaceQuotaSnapshot
private CompletableFuture<SpaceQuotaSnapshot> getCurrentSpaceQuotaSnapshot(RawAsyncHBaseAdmin.Converter<SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse> converter) -
getCurrentSpaceQuotaSnapshot
Description copied from interface:AsyncAdmin
Returns the Master's view of a quota on the givennamespace
or null if the Master has no quota information on that namespace.- Specified by:
getCurrentSpaceQuotaSnapshot
in interfaceAsyncAdmin
-
getCurrentSpaceQuotaSnapshot
Description copied from interface:AsyncAdmin
Returns the Master's view of a quota on the giventableName
or null if the Master has no quota information on that table.- Specified by:
getCurrentSpaceQuotaSnapshot
in interfaceAsyncAdmin
-
grant
public CompletableFuture<Void> grant(UserPermission userPermission, boolean mergeExistingPermissions) Description copied from interface:AsyncAdmin
Grants user specific permissions- Specified by:
grant
in interfaceAsyncAdmin
- Parameters:
userPermission
- user name and the specific permissionmergeExistingPermissions
- If set to false, later granted permissions will override previous granted permissions. otherwise, it'll merge with previous granted permissions.
-
revoke
Description copied from interface:AsyncAdmin
Revokes user specific permissions- Specified by:
revoke
in interfaceAsyncAdmin
- Parameters:
userPermission
- user name and the specific permission
-
getUserPermissions
public CompletableFuture<List<UserPermission>> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) Description copied from interface:AsyncAdmin
Get the global/namespace/table permissions for user- Specified by:
getUserPermissions
in interfaceAsyncAdmin
- Parameters:
getUserPermissionsRequest
- A request contains which user, global, namespace or table permissions needed- Returns:
- The user and permission list
-
hasUserPermissions
public CompletableFuture<List<Boolean>> hasUserPermissions(String userName, List<Permission> permissions) Description copied from interface:AsyncAdmin
Check if the user has specific permissions- Specified by:
hasUserPermissions
in interfaceAsyncAdmin
- Parameters:
userName
- the user namepermissions
- the specific permission list- Returns:
- True if user has the specific permissions
-
snapshotCleanupSwitch
Description copied from interface:AsyncAdmin
Turn on or off the auto snapshot cleanup based on TTL. Notice that, the method itself is always non-blocking, which means it will always return immediately. Thesync
parameter only effects when will we complete the returnedCompletableFuture
.- Specified by:
snapshotCleanupSwitch
in interfaceAsyncAdmin
- Parameters:
on
- Set totrue
to enable,false
to disable.sync
- Iftrue
, it waits until current snapshot cleanup is completed, if outstanding.- Returns:
- Previous auto snapshot cleanup value wrapped by a
CompletableFuture
.
-
isSnapshotCleanupEnabled
Description copied from interface:AsyncAdmin
Query the current state of the auto snapshot cleanup based on TTL.- Specified by:
isSnapshotCleanupEnabled
in interfaceAsyncAdmin
- Returns:
- true if the auto snapshot cleanup is enabled, false otherwise. The return value will be
wrapped by a
CompletableFuture
.
-
moveServersToRSGroup
Description copied from interface:AsyncAdmin
Move given set of servers to the specified target RegionServer group- Specified by:
moveServersToRSGroup
in interfaceAsyncAdmin
- Parameters:
servers
- set of servers to movegroupName
- the group to move servers to
-
addRSGroup
Description copied from interface:AsyncAdmin
Creates a new RegionServer group with the given name- Specified by:
addRSGroup
in interfaceAsyncAdmin
- Parameters:
groupName
- the name of the group
-
removeRSGroup
Description copied from interface:AsyncAdmin
Remove RegionServer group associated with the given name- Specified by:
removeRSGroup
in interfaceAsyncAdmin
- Parameters:
groupName
- the group name
-
balanceRSGroup
Description copied from interface:AsyncAdmin
Balance regions in the given RegionServer group- Specified by:
balanceRSGroup
in interfaceAsyncAdmin
- Parameters:
groupName
- the group namerequest
- options to define how the balancer should run- Returns:
- BalanceResponse details about the balancer run
-
listRSGroups
Description copied from interface:AsyncAdmin
Lists current set of RegionServer groups- Specified by:
listRSGroups
in interfaceAsyncAdmin
-
getSlowLogResponses
private CompletableFuture<List<LogEntry>> getSlowLogResponses(Map<String, Object> filterParams, Set<ServerName> serverNames, int limit, String logType) -
getSlowLogResponseFromServer
private CompletableFuture<List<LogEntry>> getSlowLogResponseFromServer(ServerName serverName, Map<String, Object> filterParams, int limit, String logType) -
clearSlowLogResponses
public CompletableFuture<List<Boolean>> clearSlowLogResponses(@Nullable Set<ServerName> serverNames) Description copied from interface:AsyncAdmin
Clears online slow RPC logs from the provided list of RegionServers- Specified by:
clearSlowLogResponses
in interfaceAsyncAdmin
- Parameters:
serverNames
- Set of Server names to clean slowlog responses from- Returns:
- List of booleans representing if online slowlog response buffer is cleaned from each
RegionServer. The return value wrapped by a
CompletableFuture
-
clearSlowLogsResponses
-
convertToFutureOfList
private static <T> CompletableFuture<List<T>> convertToFutureOfList(List<CompletableFuture<T>> futures) -
listTablesInRSGroup
Description copied from interface:AsyncAdmin
Get all tables in this RegionServer group.- Specified by:
listTablesInRSGroup
in interfaceAsyncAdmin
- Parameters:
groupName
- the group name- See Also:
-
getConfiguredNamespacesAndTablesInRSGroup
public CompletableFuture<Pair<List<String>,List<TableName>>> getConfiguredNamespacesAndTablesInRSGroup(String groupName) Description copied from interface:AsyncAdmin
Get the namespaces and tables which have this RegionServer group in descriptor. The difference between this method andAsyncAdmin.listTablesInRSGroup(String)
is that, this method will not include the table which is actually in this RegionServr group but without the RegionServer group configuration in itsTableDescriptor
. For example, we have a group 'A', and we make namespace 'nsA' in this group, then all the tables under this namespace will in the group 'A', but this method will not return these tables but only the namespace 'nsA', while theAsyncAdmin.listTablesInRSGroup(String)
will return all these tables.- Specified by:
getConfiguredNamespacesAndTablesInRSGroup
in interfaceAsyncAdmin
- Parameters:
groupName
- the group name- See Also:
-
getRSGroup
Description copied from interface:AsyncAdmin
Get group info for the given hostPort- Specified by:
getRSGroup
in interfaceAsyncAdmin
- Parameters:
hostPort
- HostPort to get RSGroupInfo for
-
removeServersFromRSGroup
Description copied from interface:AsyncAdmin
Remove decommissioned servers from group 1. Sometimes we may find the server aborted due to some hardware failure and we must offline the server for repairing. Or we need to move some servers to join other clusters. So we need to remove these servers from the group. 2. Dead/recovering/live servers will be disallowed.- Specified by:
removeServersFromRSGroup
in interfaceAsyncAdmin
- Parameters:
servers
- set of servers to remove
-
setRSGroup
Description copied from interface:AsyncAdmin
Set the RegionServer group for tables- Specified by:
setRSGroup
in interfaceAsyncAdmin
- Parameters:
tables
- tables to set group forgroupName
- group name for tables
-
getRSGroup
Description copied from interface:AsyncAdmin
Get group info for the given table- Specified by:
getRSGroup
in interfaceAsyncAdmin
- Parameters:
table
- table name to get RSGroupInfo for
-
getRSGroup
Description copied from interface:AsyncAdmin
Get group info for the given group name- Specified by:
getRSGroup
in interfaceAsyncAdmin
- Parameters:
groupName
- the group name- Returns:
- group info
-
renameRSGroup
Description copied from interface:AsyncAdmin
Rename rsgroup- Specified by:
renameRSGroup
in interfaceAsyncAdmin
- Parameters:
oldName
- old rsgroup namenewName
- new rsgroup name
-
updateRSGroupConfig
public CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration) Description copied from interface:AsyncAdmin
Update RSGroup configuration- Specified by:
updateRSGroupConfig
in interfaceAsyncAdmin
- Parameters:
groupName
- the group nameconfiguration
- new configuration of the group name to be set
-
getBalancerDecisions
-
getBalancerRejections
-
getLogEntries
public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String, Object> filterParams) Description copied from interface:AsyncAdmin
Retrieve recent online records from HMaster / RegionServers. Examples include slow/large RPC logs, balancer decisions by master.- Specified by:
getLogEntries
in interfaceAsyncAdmin
- Parameters:
serverNames
- servers to retrieve records from, useful in case of records maintained by RegionServer as we can select specific server. In case of servertype=MASTER, logs will only come from the currently active master.logType
- string representing type of log recordsserverType
- enum for server type: HMaster or RegionServerlimit
- put a limit to list of records that server should send in responsefilterParams
- additional filter params
-
flushMasterStore
Description copied from interface:AsyncAdmin
Flush master local region- Specified by:
flushMasterStore
in interfaceAsyncAdmin
-
getCachedFilesList
Description copied from interface:AsyncAdmin
Get the list of cached files- Specified by:
getCachedFilesList
in interfaceAsyncAdmin
-