@InterfaceAudience.Public public interface Admin extends org.apache.hadoop.hbase.Abortable, Closeable
Connection.getAdmin()
and call
close()
when done.
Admin can be used to create, drop, list, enable and disable and otherwise modify tables, as well as perform other administrative operations.
ConnectionFactory
,
Connection
,
Table
Modifier and Type | Method and Description |
---|---|
void |
abort(String why,
Throwable e)
Abort the server or client.
|
default boolean |
abortProcedure(long procId,
boolean mayInterruptIfRunning)
Deprecated.
since 2.1.1 and will be removed in 4.0.0.
|
Future<Boolean> |
abortProcedureAsync(long procId,
boolean mayInterruptIfRunning)
Deprecated.
since 2.1.1 and will be removed in 4.0.0.
|
default void |
addColumnFamily(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Add a column family to an existing table.
|
Future<Void> |
addColumnFamilyAsync(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Add a column family to an existing table.
|
default void |
addReplicationPeer(String peerId,
ReplicationPeerConfig peerConfig)
Add a new replication peer for replicating data to slave cluster.
|
default void |
addReplicationPeer(String peerId,
ReplicationPeerConfig peerConfig,
boolean enabled)
Add a new replication peer for replicating data to slave cluster.
|
default Future<Void> |
addReplicationPeerAsync(String peerId,
ReplicationPeerConfig peerConfig)
Add a new replication peer but does not block and wait for it.
|
Future<Void> |
addReplicationPeerAsync(String peerId,
ReplicationPeerConfig peerConfig,
boolean enabled)
Add a new replication peer but does not block and wait for it.
|
void |
addRSGroup(String groupName)
Creates a new RegionServer group with the given name
|
default void |
appendReplicationPeerTableCFs(String id,
Map<TableName,List<String>> tableCfs)
Append the replicable table column family config from the specified peer.
|
void |
assign(byte[] regionName)
Assign a Region.
|
default boolean |
balance()
Invoke the balancer.
|
BalanceResponse |
balance(BalanceRequest request)
Invoke the balancer with the given balance request.
|
default boolean |
balance(boolean force)
Deprecated.
Since 2.5.0. Will be removed in 4.0.0. Use
balance(BalanceRequest)
instead. |
default BalanceResponse |
balanceRSGroup(String groupName)
Balance regions in the given RegionServer group
|
BalanceResponse |
balanceRSGroup(String groupName,
BalanceRequest request)
Balance regions in the given RegionServer group, running based on the given
BalanceRequest . |
boolean |
balancerSwitch(boolean onOrOff,
boolean synchronous)
Turn the load balancer on or off.
|
boolean |
catalogJanitorSwitch(boolean onOrOff)
Enable/Disable the catalog janitor/
|
boolean |
cleanerChoreSwitch(boolean onOrOff)
Enable/Disable the cleaner chore.
|
CacheEvictionStats |
clearBlockCache(TableName tableName)
Clear all the blocks corresponding to this table from BlockCache.
|
void |
clearCompactionQueues(ServerName serverName,
Set<String> queues)
Clear compacting queues on a regionserver.
|
List<ServerName> |
clearDeadServers(List<ServerName> servers)
Clear dead region servers from master.
|
List<Boolean> |
clearSlowLogResponses(Set<ServerName> serverNames)
Clears online slow/large RPC logs from the provided list of RegionServers
|
default void |
cloneSnapshot(String snapshotName,
TableName tableName)
Create a new table by cloning the snapshot content.
|
default void |
cloneSnapshot(String snapshotName,
TableName tableName,
boolean restoreAcl)
Create a new table by cloning the snapshot content.
|
default void |
cloneSnapshot(String snapshotName,
TableName tableName,
boolean restoreAcl,
String customSFT)
Create a new table by cloning the snapshot content.
|
default Future<Void> |
cloneSnapshotAsync(String snapshotName,
TableName tableName)
Create a new table by cloning the snapshot content, but does not block and wait for it to be
completely cloned.
|
default Future<Void> |
cloneSnapshotAsync(String snapshotName,
TableName tableName,
boolean restoreAcl)
Create a new table by cloning the snapshot content.
|
Future<Void> |
cloneSnapshotAsync(String snapshotName,
TableName tableName,
boolean restoreAcl,
String customSFT)
Create a new table by cloning the snapshot content.
|
void |
cloneTableSchema(TableName tableName,
TableName newTableName,
boolean preserveSplits)
Create a new table by cloning the existent table schema.
|
void |
close() |
void |
compact(TableName tableName)
Compact a table.
|
void |
compact(TableName tableName,
byte[] columnFamily)
Compact a column family within a table.
|
void |
compact(TableName tableName,
byte[] columnFamily,
CompactType compactType)
Compact a column family within a table.
|
void |
compact(TableName tableName,
CompactType compactType)
Compact a table.
|
Map<ServerName,Boolean> |
compactionSwitch(boolean switchState,
List<String> serverNamesList)
Turn the compaction on or off.
|
void |
compactRegion(byte[] regionName)
Compact an individual region.
|
void |
compactRegion(byte[] regionName,
byte[] columnFamily)
Compact a column family within a region.
|
void |
compactRegionServer(ServerName serverName)
Compact all regions on the region server.
|
CoprocessorRpcChannel |
coprocessorService()
Deprecated.
since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any
more. Use the coprocessorService methods in
AsyncAdmin instead. |
CoprocessorRpcChannel |
coprocessorService(ServerName serverName)
Deprecated.
since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any
more. Use the coprocessorService methods in
AsyncAdmin instead. |
default void |
createNamespace(NamespaceDescriptor descriptor)
Create a new namespace.
|
Future<Void> |
createNamespaceAsync(NamespaceDescriptor descriptor)
Create a new namespace.
|
default void |
createTable(TableDescriptor desc)
Creates a new table.
|
default void |
createTable(TableDescriptor desc,
byte[][] splitKeys)
Creates a new table with an initial set of empty regions defined by the specified split keys.
|
void |
createTable(TableDescriptor desc,
byte[] startKey,
byte[] endKey,
int numRegions)
Creates a new table with the specified number of regions.
|
Future<Void> |
createTableAsync(TableDescriptor desc)
Creates a new table but does not block and wait for it to come online.
|
Future<Void> |
createTableAsync(TableDescriptor desc,
byte[][] splitKeys)
Creates a new table but does not block and wait for it to come online.
|
void |
decommissionRegionServers(List<ServerName> servers,
boolean offload)
Mark region server(s) as decommissioned to prevent additional regions from getting assigned to
them.
|
default void |
deleteColumnFamily(TableName tableName,
byte[] columnFamily)
Delete a column family from a table.
|
Future<Void> |
deleteColumnFamilyAsync(TableName tableName,
byte[] columnFamily)
Delete a column family from a table.
|
default void |
deleteNamespace(String name)
Delete an existing namespace.
|
Future<Void> |
deleteNamespaceAsync(String name)
Delete an existing namespace.
|
void |
deleteSnapshot(String snapshotName)
Delete an existing snapshot.
|
void |
deleteSnapshots(Pattern pattern)
Delete existing snapshots whose names match the pattern passed.
|
default void |
deleteTable(TableName tableName)
Deletes a table.
|
Future<Void> |
deleteTableAsync(TableName tableName)
Deletes the table but does not block and wait for it to be completely removed.
|
void |
deleteTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern)
Delete all existing snapshots matching the given table name regular expression and snapshot
name regular expression.
|
default void |
disableReplicationPeer(String peerId)
Stop the replication stream to the specified peer.
|
Future<Void> |
disableReplicationPeerAsync(String peerId)
Disable a replication peer but does not block and wait for it.
|
default void |
disableTable(TableName tableName)
Disable table and wait on completion.
|
Future<Void> |
disableTableAsync(TableName tableName)
Disable the table but does not block and wait for it to be completely disabled.
|
void |
disableTableReplication(TableName tableName)
Disable a table's replication switch.
|
default void |
enableReplicationPeer(String peerId)
Restart the replication stream to the specified peer.
|
Future<Void> |
enableReplicationPeerAsync(String peerId)
Enable a replication peer but does not block and wait for it.
|
default void |
enableTable(TableName tableName)
Enable a table.
|
Future<Void> |
enableTableAsync(TableName tableName)
Enable the table but does not block and wait for it to be completely enabled.
|
void |
enableTableReplication(TableName tableName)
Enable a table's replication switch.
|
boolean |
exceedThrottleQuotaSwitch(boolean enable)
Switch the exceed throttle quota.
|
void |
execProcedure(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster.
|
byte[] |
execProcedureWithReturn(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster.
|
void |
flush(TableName tableName)
Flush a table.
|
void |
flush(TableName tableName,
byte[] columnFamily)
Flush the specified column family stores on all regions of the passed table.
|
void |
flushMasterStore()
Flush master local region
|
void |
flushRegion(byte[] regionName)
Flush an individual region.
|
void |
flushRegion(byte[] regionName,
byte[] columnFamily)
Flush a column family within a region.
|
void |
flushRegionServer(ServerName serverName)
Flush all regions on the region server.
|
default Collection<ServerName> |
getBackupMasters()
Get a list of current backup masters.
|
default ClusterMetrics |
getClusterMetrics()
Get whole cluster metrics, containing status about:
|
ClusterMetrics |
getClusterMetrics(EnumSet<org.apache.hadoop.hbase.ClusterMetrics.Option> options)
Get cluster status with a set of
ClusterMetrics.Option to get desired status. |
CompactionState |
getCompactionState(TableName tableName)
Get the current compaction state of a table.
|
CompactionState |
getCompactionState(TableName tableName,
CompactType compactType)
Get the current compaction state of a table.
|
CompactionState |
getCompactionStateForRegion(byte[] regionName)
Get the current compaction state of region.
|
org.apache.hadoop.conf.Configuration |
getConfiguration()
Returns Configuration used by the instance.
|
Pair<List<String>,List<TableName>> |
getConfiguredNamespacesAndTablesInRSGroup(String groupName)
Get the namespaces and tables which have this RegionServer group in descriptor.
|
Connection |
getConnection()
Returns Connection used by this object.
|
SpaceQuotaSnapshotView |
getCurrentSpaceQuotaSnapshot(String namespace)
Returns the Master's view of a quota on the given
namespace or null if the Master has
no quota information on that namespace. |
SpaceQuotaSnapshotView |
getCurrentSpaceQuotaSnapshot(TableName tableName)
Returns the Master's view of a quota on the given
tableName or null if the Master has
no quota information on that table. |
TableDescriptor |
getDescriptor(TableName tableName)
Get a table descriptor.
|
long |
getLastMajorCompactionTimestamp(TableName tableName)
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 0 if no such HFile could be found.
|
long |
getLastMajorCompactionTimestampForRegion(byte[] regionName)
Get the timestamp of the last major compaction for the passed region.
|
String |
getLocks()
Get locks.
|
List<LogEntry> |
getLogEntries(Set<ServerName> serverNames,
String logType,
ServerType serverType,
int limit,
Map<String,Object> filterParams)
Retrieve recent online records from HMaster / RegionServers.
|
default ServerName |
getMaster()
Get the current active master.
|
default List<String> |
getMasterCoprocessorNames()
Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
|
default int |
getMasterInfoPort()
Get the info port of the current master if one is available.
|
NamespaceDescriptor |
getNamespaceDescriptor(String name)
Get a namespace descriptor by name.
|
int |
getOperationTimeout()
Return the operation timeout for a rpc call.
|
String |
getProcedures()
Get procedures.
|
List<QuotaSettings> |
getQuota(QuotaFilter filter)
List the quotas based on the filter.
|
List<RegionMetrics> |
getRegionMetrics(ServerName serverName)
Get
RegionMetrics of all regions hosted on a regionserver. |
List<RegionMetrics> |
getRegionMetrics(ServerName serverName,
TableName tableName)
Get
RegionMetrics of all regions hosted on a regionserver for a table. |
List<RegionInfo> |
getRegions(ServerName serverName)
Get all the online regions on a region server.
|
List<RegionInfo> |
getRegions(TableName tableName)
Get the regions of a given table.
|
default Collection<ServerName> |
getRegionServers()
Get the live server list.
|
default Collection<ServerName> |
getRegionServers(boolean excludeDecommissionedRS)
Retrieve all current live region servers including decommissioned if excludeDecommissionedRS is
false, else non-decommissioned ones only
|
Map<TableName,? extends SpaceQuotaSnapshotView> |
getRegionServerSpaceQuotaSnapshots(ServerName serverName)
Fetches the observed
SpaceQuotaSnapshotView s observed by a RegionServer. |
ReplicationPeerConfig |
getReplicationPeerConfig(String peerId)
Returns the configured ReplicationPeerConfig for the specified peer.
|
default SyncReplicationState |
getReplicationPeerSyncReplicationState(String peerId)
Get the current cluster state in a synchronous replication peer.
|
RSGroupInfo |
getRSGroup(Address hostPort)
Get group info for the given hostPort
|
RSGroupInfo |
getRSGroup(String groupName)
Get group info for the given group name
|
RSGroupInfo |
getRSGroup(TableName tableName)
Get group info for the given table
|
List<SecurityCapability> |
getSecurityCapabilities()
Return the set of supported security capabilities.
|
default List<OnlineLogRecord> |
getSlowLogResponses(Set<ServerName> serverNames,
LogQueryFilter logQueryFilter)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
getLogEntries(Set, String, ServerType, int, Map) instead. |
Map<TableName,Long> |
getSpaceQuotaTableSizes()
Fetches the table sizes on the filesystem as tracked by the HBase Master.
|
int |
getSyncWaitTimeout()
Return the blocking wait time for an asynchronous operation.
|
List<org.apache.hadoop.hbase.security.access.UserPermission> |
getUserPermissions(org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest getUserPermissionsRequest)
Get the global/namespace/table permissions for user
|
void |
grant(org.apache.hadoop.hbase.security.access.UserPermission userPermission,
boolean mergeExistingPermissions)
Grants user specific permissions
|
default List<Boolean> |
hasUserPermissions(List<org.apache.hadoop.hbase.security.access.Permission> permissions)
Check if call user has specific permissions
|
List<Boolean> |
hasUserPermissions(String userName,
List<org.apache.hadoop.hbase.security.access.Permission> permissions)
Check if the user has specific permissions
|
boolean |
isAborted()
Check if the server or client was aborted.
|
boolean |
isBalancerEnabled()
Query the current state of the balancer.
|
boolean |
isCatalogJanitorEnabled()
Query on the catalog janitor state (Enabled/Disabled?).
|
boolean |
isCleanerChoreEnabled()
Query on the cleaner chore state (Enabled/Disabled?).
|
boolean |
isMasterInMaintenanceMode()
Check whether Master is in maintenance mode.
|
boolean |
isMergeEnabled()
Query the current state of the merge switch.
|
boolean |
isNormalizerEnabled()
Query the current state of the region normalizer.
|
boolean |
isProcedureFinished(String signature,
String instance,
Map<String,String> props)
Check the current state of the specified procedure.
|
boolean |
isReplicationPeerEnabled(String peerId)
Check if a replication peer is enabled.
|
boolean |
isReplicationPeerModificationEnabled()
Check whether replication peer modification is enabled.
|
boolean |
isRpcThrottleEnabled()
Get if the rpc throttle is enabled.
|
boolean |
isSnapshotCleanupEnabled()
Query the current state of the auto snapshot cleanup based on TTL.
|
boolean |
isSnapshotFinished(SnapshotDescription snapshot)
Check the current state of the passed snapshot.
|
boolean |
isSplitEnabled()
Query the current state of the split switch.
|
boolean |
isTableAvailable(TableName tableName)
Check if a table is available.
|
boolean |
isTableDisabled(TableName tableName)
Check if a table is disabled.
|
boolean |
isTableEnabled(TableName tableName)
Check if a table is enabled.
|
default List<ServerName> |
listDeadServers()
List dead region servers.
|
List<ServerName> |
listDecommissionedRegionServers()
List region servers marked as decommissioned, which can not be assigned regions.
|
NamespaceDescriptor[] |
listNamespaceDescriptors()
List available namespace descriptors
|
String[] |
listNamespaces()
List available namespaces
|
List<TableCFs> |
listReplicatedTableCFs()
Find all table and column families that are replicated from this cluster
|
List<ReplicationPeerDescription> |
listReplicationPeers()
Return a list of replication peers.
|
List<ReplicationPeerDescription> |
listReplicationPeers(Pattern pattern)
Return a list of replication peers.
|
List<RSGroupInfo> |
listRSGroups()
Lists current set of RegionServer groups
|
List<SnapshotDescription> |
listSnapshots()
List completed snapshots.
|
List<SnapshotDescription> |
listSnapshots(Pattern pattern)
List all the completed snapshots matching the given pattern.
|
List<TableDescriptor> |
listTableDescriptors()
List all the userspace tables.
|
List<TableDescriptor> |
listTableDescriptors(boolean includeSysTables)
List all userspace tables and whether or not include system tables.
|
List<TableDescriptor> |
listTableDescriptors(List<TableName> tableNames)
Get tableDescriptors.
|
default List<TableDescriptor> |
listTableDescriptors(Pattern pattern)
List all the userspace tables that match the given pattern.
|
List<TableDescriptor> |
listTableDescriptors(Pattern pattern,
boolean includeSysTables)
List all the tables matching the given pattern.
|
List<TableDescriptor> |
listTableDescriptorsByNamespace(byte[] name)
Get list of table descriptors by namespace.
|
List<TableDescriptor> |
listTableDescriptorsByState(boolean isEnabled)
List all enabled or disabled tables
|
TableName[] |
listTableNames()
List all of the names of userspace tables.
|
default TableName[] |
listTableNames(Pattern pattern)
List all of the names of userspace tables.
|
TableName[] |
listTableNames(Pattern pattern,
boolean includeSysTables)
List all of the names of userspace tables.
|
TableName[] |
listTableNamesByNamespace(String name)
Get list of table names by namespace.
|
List<TableName> |
listTableNamesByState(boolean isEnabled)
List all enabled or disabled table names
|
List<TableName> |
listTablesInRSGroup(String groupName)
Get all tables in this RegionServer group.
|
List<SnapshotDescription> |
listTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern)
List all the completed snapshots matching the given table name regular expression and snapshot
name regular expression.
|
default List<ServerName> |
listUnknownServers()
List unknown region servers.
|
void |
majorCompact(TableName tableName)
Major compact a table.
|
void |
majorCompact(TableName tableName,
byte[] columnFamily)
Major compact a column family within a table.
|
void |
majorCompact(TableName tableName,
byte[] columnFamily,
CompactType compactType)
Major compact a column family within a table.
|
void |
majorCompact(TableName tableName,
CompactType compactType)
Major compact a table.
|
void |
majorCompactRegion(byte[] regionName)
Major compact a table or an individual region.
|
void |
majorCompactRegion(byte[] regionName,
byte[] columnFamily)
Major compact a column family within region.
|
void |
majorCompactRegionServer(ServerName serverName)
Major compact all regions on the region server.
|
Future<Void> |
mergeRegionsAsync(byte[][] nameofRegionsToMerge,
boolean forcible)
Merge multiple regions (>=2).
|
default Future<Void> |
mergeRegionsAsync(byte[] nameOfRegionA,
byte[] nameOfRegionB,
boolean forcible)
Deprecated.
since 2.3.0 and will be removed in 4.0.0. Multi-region merge feature is now
supported. Use
mergeRegionsAsync(byte[][], boolean) instead. |
boolean |
mergeSwitch(boolean enabled,
boolean synchronous)
Turn the merge switch on or off.
|
default void |
modifyColumnFamily(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Modify an existing column family on a table.
|
Future<Void> |
modifyColumnFamilyAsync(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Modify an existing column family on a table.
|
default void |
modifyColumnFamilyStoreFileTracker(TableName tableName,
byte[] family,
String dstSFT)
Change the store file tracker of the given table's given family.
|
Future<Void> |
modifyColumnFamilyStoreFileTrackerAsync(TableName tableName,
byte[] family,
String dstSFT)
Change the store file tracker of the given table's given family.
|
default void |
modifyNamespace(NamespaceDescriptor descriptor)
Modify an existing namespace.
|
Future<Void> |
modifyNamespaceAsync(NamespaceDescriptor descriptor)
Modify an existing namespace.
|
default void |
modifyTable(TableDescriptor td)
Modify an existing table, more IRB friendly version.
|
Future<Void> |
modifyTableAsync(TableDescriptor td)
Modify an existing table, more IRB (ruby) friendly version.
|
default void |
modifyTableStoreFileTracker(TableName tableName,
String dstSFT)
Change the store file tracker of the given table.
|
Future<Void> |
modifyTableStoreFileTrackerAsync(TableName tableName,
String dstSFT)
Change the store file tracker of the given table.
|
void |
move(byte[] encodedRegionName)
Move the region
encodedRegionName to a random server. |
default void |
move(byte[] encodedRegionName,
byte[] destServerName)
Deprecated.
since 2.2.0 and will be removed in 4.0.0. Use
move(byte[], ServerName)
instead. And if you want to move the region to a random server, please use
move(byte[]) . |
void |
move(byte[] encodedRegionName,
ServerName destServerName)
Move the region
encodedRegionName to destServerName . |
void |
moveServersToRSGroup(Set<Address> servers,
String targetGroup)
Move given set of servers to the specified target RegionServer group
|
default boolean |
normalize()
Invoke region normalizer.
|
boolean |
normalize(NormalizeTableFilterParams ntfp)
Invoke region normalizer.
|
boolean |
normalizerSwitch(boolean on)
Turn region normalizer on or off.
|
void |
offline(byte[] regionName)
Offline specified region from master's in-memory state.
|
void |
recommissionRegionServer(ServerName server,
List<byte[]> encodedRegionNames)
Remove decommission marker from a region server to allow regions assignments.
|
default void |
removeReplicationPeer(String peerId)
Remove a peer and stop the replication.
|
Future<Void> |
removeReplicationPeerAsync(String peerId)
Remove a replication peer but does not block and wait for it.
|
default void |
removeReplicationPeerTableCFs(String id,
Map<TableName,List<String>> tableCfs)
Remove some table-cfs from config of the specified peer.
|
void |
removeRSGroup(String groupName)
Remove RegionServer group associated with the given name
|
void |
removeServersFromRSGroup(Set<Address> servers)
Remove decommissioned servers from group 1.
|
void |
renameRSGroup(String oldName,
String newName)
Rename rsgroup
|
default boolean |
replicationPeerModificationSwitch(boolean on)
Enable or disable replication peer modification.
|
boolean |
replicationPeerModificationSwitch(boolean on,
boolean drainProcedures)
Enable or disable replication peer modification.
|
void |
restoreSnapshot(String snapshotName)
Restore the specified snapshot on the original table.
|
default void |
restoreSnapshot(String snapshotName,
boolean takeFailSafeSnapshot)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(String snapshotName,
boolean takeFailSafeSnapshot,
boolean restoreAcl)
Restore the specified snapshot on the original table.
|
void |
revoke(org.apache.hadoop.hbase.security.access.UserPermission userPermission)
Revokes user specific permissions
|
void |
rollWALWriter(ServerName serverName)
Roll the log writer.
|
int |
runCatalogJanitor()
Ask for a scan of the catalog table.
|
boolean |
runCleanerChore()
Ask for cleaner chore to run.
|
void |
setQuota(QuotaSettings quota)
Apply the new quota settings.
|
void |
setRSGroup(Set<TableName> tables,
String groupName)
Set the RegionServer group for tables
|
void |
shutdown()
Shuts down the HBase cluster.
|
void |
snapshot(SnapshotDescription snapshot)
Take a snapshot and wait for the server to complete that snapshot (blocking).
|
default void |
snapshot(String snapshotName,
TableName tableName)
Take a snapshot for the given table.
|
default void |
snapshot(String snapshotName,
TableName tableName,
Map<String,Object> snapshotProps)
Create typed snapshot of the table.
|
default void |
snapshot(String snapshotName,
TableName tableName,
SnapshotType type)
Create typed snapshot of the table.
|
default void |
snapshot(String snapshotName,
TableName tableName,
SnapshotType type,
Map<String,Object> snapshotProps)
Create typed snapshot of the table.
|
Future<Void> |
snapshotAsync(SnapshotDescription snapshot)
Take a snapshot without waiting for the server to complete that snapshot (asynchronous).
|
boolean |
snapshotCleanupSwitch(boolean on,
boolean synchronous)
Turn on or off the auto snapshot cleanup based on TTL.
|
void |
split(TableName tableName)
Split a table.
|
void |
split(TableName tableName,
byte[] splitPoint)
Split a table.
|
Future<Void> |
splitRegionAsync(byte[] regionName)
Split an individual region.
|
Future<Void> |
splitRegionAsync(byte[] regionName,
byte[] splitPoint)
Split an individual region.
|
boolean |
splitSwitch(boolean enabled,
boolean synchronous)
Turn the split switch on or off.
|
void |
stopMaster()
Shuts down the current HBase master only.
|
void |
stopRegionServer(String hostnamePort)
Stop the designated regionserver.
|
boolean |
switchRpcThrottle(boolean enable)
Switch the rpc throttle enable state.
|
boolean |
tableExists(TableName tableName)
Check if a table exists.
|
default void |
transitReplicationPeerSyncReplicationState(String peerId,
SyncReplicationState state)
Transit current cluster to a new state in a synchronous replication peer.
|
Future<Void> |
transitReplicationPeerSyncReplicationStateAsync(String peerId,
SyncReplicationState state)
Transit current cluster to a new state in a synchronous replication peer.
|
default void |
truncateTable(TableName tableName,
boolean preserveSplits)
Truncate a table.
|
Future<Void> |
truncateTableAsync(TableName tableName,
boolean preserveSplits)
Truncate the table but does not block and wait for it to be completely enabled.
|
void |
unassign(byte[] regionName)
Unassign a Region.
|
default void |
unassign(byte[] regionName,
boolean force)
Deprecated.
since 2.4.0 and will be removed in 4.0.0. Use
unassign(byte[]) instead. |
void |
updateConfiguration()
Update the configuration and trigger an online config change on all the regionservers.
|
void |
updateConfiguration(ServerName server)
Update the configuration and trigger an online config change on the regionserver.
|
void |
updateConfiguration(String groupName)
Update the configuration and trigger an online config change on all the regionservers in the
RSGroup.
|
default void |
updateReplicationPeerConfig(String peerId,
ReplicationPeerConfig peerConfig)
Update the peerConfig for the specified peer.
|
Future<Void> |
updateReplicationPeerConfigAsync(String peerId,
ReplicationPeerConfig peerConfig)
Update the peerConfig for the specified peer but does not block and wait for it.
|
void |
updateRSGroupConfig(String groupName,
Map<String,String> configuration)
Update RSGroup configuration
|
int getOperationTimeout()
getSyncWaitTimeout()
int getSyncWaitTimeout()
hbase.client.sync.wait.timeout.msec
.
For several operations, such as createTable, deleteTable, etc, the rpc call will finish right
after we schedule a procedure at master side, so the timeout will not be controlled by the
above getOperationTimeout()
. And timeout value here tells you how much time we will
wait until the procedure at master side is finished.
In general, you can consider that the implementation for XXXX method is just a
XXXXAsync().get(getSyncWaitTimeout(), TimeUnit.MILLISECONDS).getOperationTimeout()
void abort(String why, Throwable e)
org.apache.hadoop.hbase.Abortable
abort
in interface org.apache.hadoop.hbase.Abortable
why
- Why we're aborting.e
- Throwable that caused abort. Can be null.boolean isAborted()
org.apache.hadoop.hbase.Abortable
isAborted
in interface org.apache.hadoop.hbase.Abortable
Connection getConnection()
boolean tableExists(TableName tableName) throws IOException
tableName
- Table to check.true
if table exists already.IOException
- if a remote or network exception occursList<TableDescriptor> listTableDescriptors() throws IOException
IOException
- if a remote or network exception occursList<TableDescriptor> listTableDescriptors(boolean includeSysTables) throws IOException
IOException
- if a remote or network exception occursdefault List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occurslistTableDescriptors()
List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) throws IOException
pattern
- The compiled regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occurslistTableDescriptors()
List<TableDescriptor> listTableDescriptorsByState(boolean isEnabled) throws IOException
isEnabled
- is true means return enabled tables, false means return disabled tablesIOException
TableName[] listTableNames() throws IOException
IOException
- if a remote or network exception occursdefault TableName[] listTableNames(Pattern pattern) throws IOException
pattern
- The regular expression to match againstIOException
- if a remote or network exception occursTableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException
pattern
- The regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occursList<TableName> listTableNamesByState(boolean isEnabled) throws IOException
isEnabled
- is true means return enabled table names, false means return disabled table
namesIOException
TableDescriptor getDescriptor(TableName tableName) throws TableNotFoundException, IOException
tableName
- as a TableName
TableNotFoundException
- if the table was not foundIOException
- if a remote or network exception occursdefault void createTable(TableDescriptor desc) throws IOException
desc
- table descriptor for tableIllegalArgumentException
- if the table name is reservedMasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If
concurrent threads, the table may
have been created between
test-for-existence and
attempt-at-creation).IOException
- if a remote or network exception
occursvoid createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) throws IOException
desc
- table descriptor for tablestartKey
- beginning of key rangeendKey
- end of key rangenumRegions
- the total number of regions to createIOException
- if a remote or network exception
occursIllegalArgumentException
- if the table name is reservedMasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If
concurrent threads, the table may
have been created between
test-for-existence and
attempt-at-creation).default void createTable(TableDescriptor desc, byte[][] splitKeys) throws IOException
desc
- table descriptor for tablesplitKeys
- array of split keys for the initial regions of the tableIllegalArgumentException
- if the table name is reserved, if the
split keys are repeated and if the
split key has empty byte array.MasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If
concurrent threads, the table may
have been created between
test-for-existence and
attempt-at-creation).IOException
- if a remote or network exception
occursFuture<Void> createTableAsync(TableDescriptor desc) throws IOException
desc
- table descriptor for tableIOException
- if a remote or network exception occursFuture<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys) throws IOException
desc
- table descriptor for tablesplitKeys
- keys to check if the table has been created with all split keysIOException
- if a remote or network exception occursdefault void deleteTable(TableName tableName) throws IOException
tableName
- name of table to deleteIOException
- if a remote or network exception occursFuture<Void> deleteTableAsync(TableName tableName) throws IOException
tableName
- name of table to deleteIOException
- if a remote or network exception occursdefault void truncateTable(TableName tableName, boolean preserveSplits) throws IOException
tableName
- name of table to truncatepreserveSplits
- true
if the splits should be preservedIOException
- if a remote or network exception occursFuture<Void> truncateTableAsync(TableName tableName, boolean preserveSplits) throws IOException
tableName
- name of table to deletepreserveSplits
- true
if the splits should be preservedIOException
- if a remote or network exception occursdefault void enableTable(TableName tableName) throws IOException
enableTableAsync(org.apache.hadoop.hbase.TableName)
and isTableEnabled(org.apache.hadoop.hbase.TableName)
instead. The table has to be in
disabled state for it to be enabled.tableName
- name of the tableIOException
- There could be couple types of IOException TableNotFoundException means the
table doesn't exist. TableNotDisabledException means the table isn't in
disabled state.isTableEnabled(org.apache.hadoop.hbase.TableName)
,
disableTable(org.apache.hadoop.hbase.TableName)
,
enableTableAsync(org.apache.hadoop.hbase.TableName)
Future<Void> enableTableAsync(TableName tableName) throws IOException
tableName
- name of table to deleteIOException
- if a remote or network exception occursFuture<Void> disableTableAsync(TableName tableName) throws IOException
tableName
- name of table to deleteIOException
- if a remote or network exception occursdefault void disableTable(TableName tableName) throws IOException
disableTableAsync(org.apache.hadoop.hbase.TableName)
and
isTableDisabled(org.apache.hadoop.hbase.TableName)
instead. The table has to be in
enabled state for it to be disabled.IOException
- There could be couple types of IOException TableNotFoundException means the
table doesn't exist. TableNotEnabledException means the table isn't in
enabled state.boolean isTableEnabled(TableName tableName) throws IOException
tableName
- name of table to checktrue
if table is on-lineIOException
- if a remote or network exception occursboolean isTableDisabled(TableName tableName) throws IOException
tableName
- name of table to checktrue
if table is off-lineIOException
- if a remote or network exception occursboolean isTableAvailable(TableName tableName) throws IOException
tableName
- name of table to checktrue
if all regions of the table are availableIOException
- if a remote or network exception occursdefault void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)
instead because it returns a
Future
from which you can learn whether success or failure.tableName
- name of the table to add column family tocolumnFamily
- column family descriptor of column family to be addedIOException
- if a remote or network exception occursFuture<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
tableName
- name of the table to add column family tocolumnFamily
- column family descriptor of column family to be addedIOException
- if a remote or network exception occursdefault void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException
deleteColumnFamily(TableName, byte[])
instead because it returns a Future
from
which you can learn whether success or failure.tableName
- name of tablecolumnFamily
- name of column family to be deletedIOException
- if a remote or network exception occursFuture<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily) throws IOException
tableName
- name of tablecolumnFamily
- name of column family to be deletedIOException
- if a remote or network exception occursdefault void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor)
instead because it returns
a Future
from which you can learn whether success or failure.tableName
- name of tablecolumnFamily
- new column family descriptor to useIOException
- if a remote or network exception occursFuture<Void> modifyColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
tableName
- name of tablecolumnFamily
- new column family descriptor to useIOException
- if a remote or network exception occursdefault void modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT) throws IOException
tableName
- the table you want to changefamily
- the family you want to changedstSFT
- the destination store file trackerIOException
- if a remote or network exception occursFuture<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family, String dstSFT) throws IOException
tableName
- the table you want to changefamily
- the family you want to changedstSFT
- the destination store file trackerIOException
- if a remote or network exception occursList<RegionInfo> getRegions(ServerName serverName) throws IOException
RegionInfo
IOException
- if a remote or network exception occursvoid flush(TableName tableName) throws IOException
tableName
- table to flushIOException
- if a remote or network exception occursvoid flush(TableName tableName, byte[] columnFamily) throws IOException
tableName
- table to flushcolumnFamily
- column family within a tableIOException
- if a remote or network exception occursvoid flushRegion(byte[] regionName) throws IOException
regionName
- region to flushIOException
- if a remote or network exception occursvoid flushRegion(byte[] regionName, byte[] columnFamily) throws IOException
regionName
- region to flushcolumnFamily
- column family within a regionIOException
- if a remote or network exception occursvoid flushRegionServer(ServerName serverName) throws IOException
serverName
- the region server name to flushIOException
- if a remote or network exception occursvoid compact(TableName tableName) throws IOException
tableName
- table to compactIOException
- if a remote or network exception occursvoid compactRegion(byte[] regionName) throws IOException
regionName
- region to compactIOException
- if a remote or network exception occursvoid compact(TableName tableName, byte[] columnFamily) throws IOException
tableName
- table to compactcolumnFamily
- column family within a tableIOException
- if a remote or network exception occursvoid compactRegion(byte[] regionName, byte[] columnFamily) throws IOException
regionName
- region to compactcolumnFamily
- column family within a regionIOException
- if a remote or network exception occursvoid compact(TableName tableName, CompactType compactType) throws IOException, InterruptedException
tableName
- table to compactcompactType
- CompactType
IOException
- if a remote or network exception occursInterruptedException
void compact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException
tableName
- table to compactcolumnFamily
- column family within a tablecompactType
- CompactType
IOException
- if not a mob column family or if a remote or network exception occursInterruptedException
void majorCompact(TableName tableName) throws IOException
tableName
- table to major compactIOException
- if a remote or network exception occursvoid majorCompactRegion(byte[] regionName) throws IOException
regionName
- region to major compactIOException
- if a remote or network exception occursvoid majorCompact(TableName tableName, byte[] columnFamily) throws IOException
tableName
- table to major compactcolumnFamily
- column family within a tableIOException
- if a remote or network exception occursvoid majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException
regionName
- egion to major compactcolumnFamily
- column family within a regionIOException
- if a remote or network exception occursvoid majorCompact(TableName tableName, CompactType compactType) throws IOException, InterruptedException
tableName
- table to compactcompactType
- CompactType
IOException
- if a remote or network exception occursInterruptedException
void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException
tableName
- table to compactcolumnFamily
- column family within a tablecompactType
- CompactType
IOException
- if not a mob column family or if a remote or network exception occursInterruptedException
Map<ServerName,Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList) throws IOException
switchState
- Set to true
to enable, false
to disable.serverNamesList
- list of region servers.IOException
- if a remote or network exception occursvoid compactRegionServer(ServerName serverName) throws IOException
serverName
- the region server nameIOException
- if a remote or network exception occursvoid majorCompactRegionServer(ServerName serverName) throws IOException
serverName
- the region server nameIOException
- if a remote or network exception occursvoid move(byte[] encodedRegionName) throws IOException
encodedRegionName
to a random server.encodedRegionName
- The encoded region name; i.e. the hash that makes up the region name
suffix: e.g. if regionname is
TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.
,
then the encoded region name is:
527db22f95c8a9e0116f0cc13c680396
.IOException
- if we can't find a region named encodedRegionName
@Deprecated default void move(byte[] encodedRegionName, byte[] destServerName) throws IOException
move(byte[], ServerName)
instead. And if you want to move the region to a random server, please use
move(byte[])
.rencodedRegionName
to destServerName
.encodedRegionName
- The encoded region name; i.e. the hash that makes up the region name
suffix: e.g. if regionname is
TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.
,
then the encoded region name is:
527db22f95c8a9e0116f0cc13c680396
.destServerName
- The servername of the destination regionserver. If passed the empty
byte array 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
IOException
- if we can't find a region named encodedRegionName
void move(byte[] encodedRegionName, ServerName destServerName) throws IOException
encodedRegionName
to destServerName
.encodedRegionName
- The encoded region name; i.e. the hash that makes up the region name
suffix: e.g. if regionname is
TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.
,
then the encoded region name is:
527db22f95c8a9e0116f0cc13c680396
.destServerName
- The servername of the destination regionserver. A server name is made
of host, port and startcode. Here is an example:
host187.example.com,60020,1289493121758
IOException
- if we can't find a region named encodedRegionName
void assign(byte[] regionName) throws IOException
regionName
- Region name to assign.IOException
- if a remote or network exception occursvoid unassign(byte[] regionName) throws IOException
regionName
- Region name to assign.IOException
- if a remote or network exception occurs@Deprecated default void unassign(byte[] regionName, boolean force) throws IOException
unassign(byte[])
instead.move(byte[], ServerName)
if you want to control the region movement.regionName
- Region to unassign. Will clear any existing RegionPlan if one found.force
- If true
, force unassign (Will remove region from
regions-in-transition too if present. If results in double assignment use
hbck -fix to resolve. To be used by experts).IOException
- if a remote or network exception occursvoid offline(byte[] regionName) throws IOException
regionName
- Region to offline.IOException
- if a remote or network exception occursboolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException
onOrOff
- Set to true
to enable, false
to disable.synchronous
- If true
, it waits until current balance() call, if outstanding,
to return.IOException
- if a remote or network exception occursdefault boolean balance() throws IOException
true
if balancer ran, false
otherwise.IOException
- if a remote or network exception occursBalanceResponse balance(BalanceRequest request) throws IOException
BalanceRequest
for more details.request
- defines how the balancer should runBalanceResponse
with details about the results of the invocation.IOException
- if a remote or network exception occurs@Deprecated default boolean balance(boolean force) throws IOException
balance(BalanceRequest)
instead.force
- whether we should force balance even if there is region in transitiontrue
if balancer ran, false
otherwise.IOException
- if a remote or network exception occursboolean isBalancerEnabled() throws IOException
true
if the balancer is enabled, false
otherwise.IOException
- if a remote or network exception occursCacheEvictionStats clearBlockCache(TableName tableName) throws IOException
tableName
- table to clear block cacheIOException
- if a remote or network exception occursdefault boolean normalize() throws IOException
true
if region normalizer ran, false
otherwise.IOException
- if a remote or network exception occursboolean normalize(NormalizeTableFilterParams ntfp) throws IOException
ntfp
- limit to tables matching the specified filter.true
if region normalizer ran, false
otherwise.IOException
- if a remote or network exception occursboolean isNormalizerEnabled() throws IOException
true
if region normalizer is enabled, false
otherwise.IOException
- if a remote or network exception occursboolean normalizerSwitch(boolean on) throws IOException
IOException
- if a remote or network exception occursboolean catalogJanitorSwitch(boolean onOrOff) throws IOException
onOrOff
- if true
enables the catalog janitorIOException
- if a remote or network exception occursint runCatalogJanitor() throws IOException
IOException
- if a remote or network exception occursboolean isCatalogJanitorEnabled() throws IOException
IOException
- if a remote or network exception occursboolean cleanerChoreSwitch(boolean onOrOff) throws IOException
onOrOff
- if true
enables the cleaner choreIOException
- if a remote or network exception occursboolean runCleanerChore() throws IOException
true
if cleaner chore ran, false
otherwiseIOException
- if a remote or network exception occursboolean isCleanerChoreEnabled() throws IOException
IOException
- if a remote or network exception occurs@Deprecated default Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
mergeRegionsAsync(byte[][], boolean)
instead.nameOfRegionA
- encoded or full name of region anameOfRegionB
- encoded or full name of region bforcible
- true
if do a compulsory merge, otherwise we will only merge
two adjacent regionsIOException
- if a remote or network exception occursFuture<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible) throws IOException
nameofRegionsToMerge
- encoded or full name of daughter regionsforcible
- true
if do a compulsory merge, otherwise we will only
merge adjacent regionsIOException
- if a remote or network exception occursvoid split(TableName tableName) throws IOException
tableName
- table to splitIOException
- if a remote or network exception occursvoid split(TableName tableName, byte[] splitPoint) throws IOException
tableName
- table to splitsplitPoint
- the explicit position to split onIOException
- if a remote or network exception occursFuture<Void> splitRegionAsync(byte[] regionName) throws IOException
regionName
- region to splitIOException
- if a remote or network exception occursFuture<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException
regionName
- region to splitsplitPoint
- the explicit position to split onIOException
- if a remote or network exception occursdefault void modifyTable(TableDescriptor td) throws IOException
td
- modified description of the tableIOException
- if a remote or network exception occursFuture<Void> modifyTableAsync(TableDescriptor td) throws IOException
td
- description of the tableIOException
- if a remote or network exception occursdefault void modifyTableStoreFileTracker(TableName tableName, String dstSFT) throws IOException
tableName
- the table you want to changedstSFT
- the destination store file trackerIOException
- if a remote or network exception occursFuture<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT) throws IOException
tableName
- the table you want to changedstSFT
- the destination store file trackerIOException
- if a remote or network exception occursvoid shutdown() throws IOException
IOException
- if a remote or network exception occursvoid stopMaster() throws IOException
IOException
- if a remote or network exception occursshutdown()
boolean isMasterInMaintenanceMode() throws IOException
IOException
- if a remote or network exception occursvoid stopRegionServer(String hostnamePort) throws IOException
hostnamePort
- Hostname and port delimited by a :
as in
example.org:1234
IOException
- if a remote or network exception occursdefault ClusterMetrics getClusterMetrics() throws IOException
hbase version cluster id primary/backup master(s) master's coprocessors live/dead regionservers balancer regions in transition
IOException
- if a remote or network exception occursClusterMetrics getClusterMetrics(EnumSet<org.apache.hadoop.hbase.ClusterMetrics.Option> options) throws IOException
ClusterMetrics.Option
to get desired status.IOException
- if a remote or network exception occursdefault ServerName getMaster() throws IOException
IOException
- if a remote or network exception occursdefault Collection<ServerName> getBackupMasters() throws IOException
IOException
- if a remote or network exception occursdefault Collection<ServerName> getRegionServers() throws IOException
IOException
- if a remote or network exception occursdefault Collection<ServerName> getRegionServers(boolean excludeDecommissionedRS) throws IOException
excludeDecommissionedRS
- should we exclude decommissioned RS nodesIOException
- if a remote or network exception occursList<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException
RegionMetrics
of all regions hosted on a regionserver.serverName
- region server from which RegionMetrics
is required.RegionMetrics
list of all regions hosted on a region serverIOException
- if a remote or network exception occursList<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName) throws IOException
RegionMetrics
of all regions hosted on a regionserver for a table.serverName
- region server from which RegionMetrics
is required.tableName
- get RegionMetrics
of regions belonging to the tableIOException
- if a remote or network exception occursorg.apache.hadoop.conf.Configuration getConfiguration()
default void createNamespace(NamespaceDescriptor descriptor) throws IOException
descriptor
- descriptor which describes the new namespace.IOException
- if a remote or network exception occursFuture<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException
descriptor
- descriptor which describes the new namespaceIOException
- if a remote or network exception occursdefault void modifyNamespace(NamespaceDescriptor descriptor) throws IOException
descriptor
- descriptor which describes the new namespaceIOException
- if a remote or network exception occursFuture<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException
descriptor
- descriptor which describes the new namespaceIOException
- if a remote or network exception occursdefault void deleteNamespace(String name) throws IOException
name
- namespace nameIOException
- if a remote or network exception occursFuture<Void> deleteNamespaceAsync(String name) throws IOException
name
- namespace nameIOException
- if a remote or network exception occursNamespaceDescriptor getNamespaceDescriptor(String name) throws NamespaceNotFoundException, IOException
name
- name of namespace descriptorNamespaceNotFoundException
- if the namespace was not foundIOException
- if a remote or network exception
occursString[] listNamespaces() throws IOException
IOException
- if a remote or network exception occursNamespaceDescriptor[] listNamespaceDescriptors() throws IOException
IOException
- if a remote or network exception occursList<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException
name
- namespace nameIOException
- if a remote or network exception occursTableName[] listTableNamesByNamespace(String name) throws IOException
name
- namespace nameIOException
- if a remote or network exception occursList<RegionInfo> getRegions(TableName tableName) throws IOException
tableName
- the name of the tableRegionInfo
.IOException
- if a remote or network exception occursvoid close()
close
in interface AutoCloseable
close
in interface Closeable
List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException
tableNames
- List of table namesIOException
- if a remote or network exception occurs@Deprecated default boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?true
if aborted, false
if procedure already completed or does
not existIOException
- if a remote or network exception occurs@Deprecated Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning) throws IOException
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?true
if aborted, false
if procedure already completed or does
not existIOException
- if a remote or network exception occursString getProcedures() throws IOException
IOException
- if a remote or network exception occursString getLocks() throws IOException
IOException
- if a remote or network exception occursvoid rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException
serverName
- The servername of the regionserver.IOException
- if a remote or network exception occursFailedLogCloseException
- if we failed to close the WALdefault List<String> getMasterCoprocessorNames() throws IOException
IOException
ClusterMetrics.getMasterCoprocessorNames()
CompactionState getCompactionState(TableName tableName) throws IOException
tableName
- table to examineIOException
- if a remote or network exception occursCompactionState getCompactionState(TableName tableName, CompactType compactType) throws IOException
tableName
- table to examinecompactType
- CompactType
IOException
- if a remote or network exception occursCompactionState getCompactionStateForRegion(byte[] regionName) throws IOException
regionName
- region to examineIOException
- if a remote or network exception occurslong getLastMajorCompactionTimestamp(TableName tableName) throws IOException
tableName
- table to examineIOException
- if a remote or network exception occurslong getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException
regionName
- region to examineIOException
- if a remote or network exception occursdefault void snapshot(String snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
SnapshotCreationException
indicating the duplicate
naming. Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
.snapshotName
- name of the snapshot to be createdtableName
- name of the table for which snapshot is createdIOException
- if a remote or network
exception occursSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is
formatted incorrectlydefault void snapshot(String snapshotName, TableName tableName, SnapshotType type) throws IOException, SnapshotCreationException, IllegalArgumentException
SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
.snapshotName
- name to give the snapshot on the filesystem. Must be unique from all other
snapshots stored on the clustertableName
- name of the table to snapshottype
- type of snapshot to takeIOException
- we fail to reach the masterSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlydefault void snapshot(String snapshotName, TableName tableName, SnapshotType type, Map<String,Object> snapshotProps) throws IOException, SnapshotCreationException, IllegalArgumentException
SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
. Snapshot can
live with ttl seconds.snapshotName
- name to give the snapshot on the filesystem. Must be unique from all other
snapshots stored on the clustertableName
- name of the table to snapshottype
- type of snapshot to takesnapshotProps
- snapshot additional properties e.g. TTLIOException
- we fail to reach the masterSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlydefault void snapshot(String snapshotName, TableName tableName, Map<String,Object> snapshotProps) throws IOException, SnapshotCreationException, IllegalArgumentException
SnapshotCreationException
indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
. Snapshot can
live with ttl seconds.snapshotName
- name to give the snapshot on the filesystem. Must be unique from all other
snapshots stored on the clustertableName
- name of the table to snapshotsnapshotProps
- snapshot additional properties e.g. TTLIOException
- we fail to reach the masterSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlyvoid snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException, IllegalArgumentException
SnapshotCreationException
indicating the duplicate naming. Snapshot names follow the
same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
. You should
probably use snapshot(String, org.apache.hadoop.hbase.TableName)
unless you are sure
about the type of snapshot that you want to take.snapshot
- snapshot to takeIOException
- or we lose contact with the master.SnapshotCreationException
- if snapshot failed to be takenIllegalArgumentException
- if the snapshot request is formatted incorrectlyFuture<Void> snapshotAsync(SnapshotDescription snapshot) throws IOException, SnapshotCreationException
snapshot
- snapshot to takeIOException
- if the snapshot did not succeed or we lose contact with the
master.SnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectlyboolean isSnapshotFinished(SnapshotDescription snapshot) throws IOException, HBaseSnapshotException, UnknownSnapshotException
UnknownSnapshotException
.snapshot
- description of the snapshot to checkIOException
- if we have a network issueHBaseSnapshotException
- if the snapshot failedUnknownSnapshotException
- if the requested snapshot is
unknownvoid restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException
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.snapshotName
- name of the snapshot to restoreIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlydefault void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
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".snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true
if the failsafe snapshot should be takenIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlyvoid restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) throws IOException, RestoreSnapshotException
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".snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true
if the failsafe snapshot should be takenrestoreAcl
- true
to restore acl of snapshotIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlydefault void cloneSnapshot(String snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredIOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid namedefault void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredrestoreAcl
- true
to clone acl into newly created tablecustomSFT
- specify the StoreFileTracker used for the tableIOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid namedefault void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredrestoreAcl
- true
to clone acl into newly created tableIOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid namedefault Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName) throws IOException, TableExistsException
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredIOException
- if a remote or network exception occursTableExistsException
- if table to be cloned already existsdefault Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredrestoreAcl
- true
to clone acl into newly created tableIOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid nameFuture<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredrestoreAcl
- true
to clone acl into newly created tablecustomSFT
- specify the StroreFileTracker used for the tableIOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid namevoid execProcedure(String signature, String instance, Map<String,String> props) throws IOException
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 procedureIOException
- if a remote or network exception occursbyte[] execProcedureWithReturn(String signature, String instance, Map<String,String> props) throws IOException
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 procedureIOException
- if a remote or network exception occursboolean isProcedureFinished(String signature, String instance, Map<String,String> props) throws IOException
signature
- The signature that uniquely identifies a procedureinstance
- The instance name of the procedureprops
- Property/Value pairs of properties passing to the proceduretrue
if the specified procedure is finished successfully,
false
if it is still runningIOException
- if the specified procedure finished with errorList<SnapshotDescription> listSnapshots() throws IOException
IOException
- if a network error occursList<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occursList<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match againstIOException
- if a remote or network exception occursvoid deleteSnapshot(String snapshotName) throws IOException
snapshotName
- name of the snapshotIOException
- if a remote or network exception occursvoid deleteSnapshots(Pattern pattern) throws IOException
pattern
- pattern for names of the snapshot to matchIOException
- if a remote or network exception occursvoid deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match againstIOException
- if a remote or network exception occursvoid setQuota(QuotaSettings quota) throws IOException
quota
- the quota settingsIOException
- if a remote or network exception occursList<QuotaSettings> getQuota(QuotaFilter filter) throws IOException
filter
- the quota settings filterIOException
- if a remote or network exception occurs@Deprecated CoprocessorRpcChannel coprocessorService()
AsyncAdmin
instead.RpcChannel
instance connected to the active master.
The obtained RpcChannel
instance can be
used to access a published coprocessor
Service
using standard protobuf service
invocations:
CoprocessorRpcChannel channel = myAdmin.coprocessorService(); MyService.BlockingInterface service = MyService.newBlockingStub(channel); MyCallRequest request = MyCallRequest.newBuilder() ... .build(); MyCallResponse response = service.myCall(null, request);
@Deprecated CoprocessorRpcChannel coprocessorService(ServerName serverName)
AsyncAdmin
instead.RpcChannel
instance connected to the passed region server.
The obtained RpcChannel
instance can be
used to access a published coprocessor
Service
using standard protobuf service
invocations:
CoprocessorRpcChannel channel = myAdmin.coprocessorService(serverName); MyService.BlockingInterface service = MyService.newBlockingStub(channel); MyCallRequest request = MyCallRequest.newBuilder() ... .build(); MyCallResponse response = service.myCall(null, request);
serverName
- the server name to which the endpoint call is madevoid updateConfiguration(ServerName server) throws IOException
server
- : The server whose config needs to be updated.IOException
- if a remote or network exception occursvoid updateConfiguration() throws IOException
IOException
- if a remote or network exception occursvoid updateConfiguration(String groupName) throws IOException
groupName
- the group nameIOException
- if a remote or network exception occursdefault int getMasterInfoPort() throws IOException
IOException
- if a remote or network exception occursList<SecurityCapability> getSecurityCapabilities() throws IOException
IOException
- if a remote or network exception occursboolean splitSwitch(boolean enabled, boolean synchronous) throws IOException
enabled
- enabled or notsynchronous
- If true
, it waits until current split() call, if outstanding,
to return.IOException
- if a remote or network exception occursboolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException
enabled
- enabled or notsynchronous
- If true
, it waits until current merge() call, if outstanding,
to return.IOException
- if a remote or network exception occursboolean isSplitEnabled() throws IOException
true
if the switch is enabled, false
otherwise.IOException
- if a remote or network exception occursboolean isMergeEnabled() throws IOException
true
if the switch is enabled, false
otherwise.IOException
- if a remote or network exception occursdefault void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig) throws IOException
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peerIOException
- if a remote or network exception occursdefault void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peerenabled
- peer state, true if ENABLED and false if DISABLEDIOException
- if a remote or network exception occursdefault Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig) throws IOException
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peerIOException
- IOException if a remote or network exception occursFuture<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException
You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peerenabled
- peer state, true if ENABLED and false if DISABLEDIOException
- IOException if a remote or network exception occursdefault void removeReplicationPeer(String peerId) throws IOException
peerId
- a short name that identifies the peerIOException
- if a remote or network exception occursFuture<Void> removeReplicationPeerAsync(String peerId) throws IOException
You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
peerId
- a short name that identifies the peerIOException
- IOException if a remote or network exception occursdefault void enableReplicationPeer(String peerId) throws IOException
peerId
- a short name that identifies the peerIOException
- if a remote or network exception occursFuture<Void> enableReplicationPeerAsync(String peerId) throws IOException
You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
peerId
- a short name that identifies the peerIOException
- IOException if a remote or network exception occursdefault void disableReplicationPeer(String peerId) throws IOException
peerId
- a short name that identifies the peerIOException
- if a remote or network exception occursFuture<Void> disableReplicationPeerAsync(String peerId) throws IOException
peerId
- a short name that identifies the peerIOException
- IOException if a remote or network exception occursReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException
peerId
- a short name that identifies the peerIOException
- if a remote or network exception occursdefault void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) throws IOException
peerId
- a short name that identifies the peerpeerConfig
- new config for the replication peerIOException
- if a remote or network exception occursFuture<Void> updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig) throws IOException
peerId
- a short name that identifies the peerpeerConfig
- new config for the replication peerIOException
- IOException if a remote or network exception occursdefault void appendReplicationPeerTableCFs(String id, Map<TableName,List<String>> tableCfs) throws ReplicationException, IOException
id
- a short that identifies the clustertableCfs
- A map from tableName to column family namesReplicationException
- if tableCfs has conflict with existing configIOException
- if a remote or network exception occursdefault void removeReplicationPeerTableCFs(String id, Map<TableName,List<String>> tableCfs) throws ReplicationException, IOException
id
- a short name that identifies the clustertableCfs
- A map from tableName to column family namesReplicationException
- if tableCfs has conflict with existing configIOException
- if a remote or network exception occursList<ReplicationPeerDescription> listReplicationPeers() throws IOException
IOException
- if a remote or network exception occursList<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException
pattern
- The compiled regular expression to match peer idIOException
- if a remote or network exception occursdefault void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state) throws IOException
peerId
- a short name that identifies the peerstate
- a new state of current clusterIOException
- if a remote or network exception occursFuture<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId, SyncReplicationState state) throws IOException
You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw ExecutionException if there was an error while executing the operation or TimeoutException in case the wait timeout was not long enough to allow the operation to complete.
peerId
- a short name that identifies the peerstate
- a new state of current clusterIOException
- if a remote or network exception occursdefault SyncReplicationState getReplicationPeerSyncReplicationState(String peerId) throws IOException
peerId
- a short name that identifies the peerIOException
- if a remote or network exception occursboolean isReplicationPeerEnabled(String peerId) throws IOException
peerId
- id of replication peer to checktrue
if replication peer is enabledIOException
- if a remote or network exception occursdefault boolean replicationPeerModificationSwitch(boolean on) throws IOException
on
- true
means enable, otherwise disableIOException
boolean replicationPeerModificationSwitch(boolean on, boolean drainProcedures) throws IOException
on
- true
means enable, otherwise disabledrainProcedures
- if true
, will wait until all the running replication peer
modification procedures finishIOException
boolean isReplicationPeerModificationEnabled() throws IOException
true
if modification is enabled, otherwise false
IOException
void decommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException
servers
- The list of servers to decommission.offload
- True to offload the regions from the decommissioned serversIOException
- if a remote or network exception occursList<ServerName> listDecommissionedRegionServers() throws IOException
IOException
- if a remote or network exception occursvoid recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) throws IOException
server
- The server to recommission.encodedRegionNames
- Regions to load onto the server.IOException
- if a remote or network exception occursList<TableCFs> listReplicatedTableCFs() throws IOException
IOException
- if a remote or network exception occursvoid enableTableReplication(TableName tableName) throws IOException
tableName
- name of the tableIOException
- if a remote or network exception occursvoid disableTableReplication(TableName tableName) throws IOException
tableName
- name of the tableIOException
- if a remote or network exception occursvoid clearCompactionQueues(ServerName serverName, Set<String> queues) throws IOException, InterruptedException
serverName
- the region server namequeues
- the set of queue nameIOException
- if a remote or network exception occursInterruptedException
default List<ServerName> listDeadServers() throws IOException
IOException
default List<ServerName> listUnknownServers() throws IOException
IOException
List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException
servers
- list of dead region servers.IOException
- if a remote or network exception occursvoid cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) throws IOException
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 preservedIOException
- if a remote or network exception occursboolean switchRpcThrottle(boolean enable) throws IOException
enable
- Set to true
to enable, false
to disable.IOException
- if a remote or network exception occursboolean isRpcThrottleEnabled() throws IOException
IOException
- if a remote or network exception occursboolean exceedThrottleQuotaSwitch(boolean enable) throws IOException
enable
- Set to true
to enable, false
to disable.IOException
- if a remote or network exception occursMap<TableName,Long> getSpaceQuotaTableSizes() throws IOException
IOException
- if a remote or network exception occursMap<TableName,? extends SpaceQuotaSnapshotView> getRegionServerSpaceQuotaSnapshots(ServerName serverName) throws IOException
SpaceQuotaSnapshotView
s observed by a RegionServer.IOException
- if a remote or network exception occursSpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(String namespace) throws IOException
namespace
or null if the Master has
no quota information on that namespace.IOException
- if a remote or network exception occursSpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(TableName tableName) throws IOException
tableName
or null if the Master has
no quota information on that table.IOException
- if a remote or network exception occursvoid grant(org.apache.hadoop.hbase.security.access.UserPermission userPermission, boolean mergeExistingPermissions) throws IOException
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.IOException
- if a remote or network exception occursvoid revoke(org.apache.hadoop.hbase.security.access.UserPermission userPermission) throws IOException
userPermission
- user name and the specific permissionIOException
- if a remote or network exception occursList<org.apache.hadoop.hbase.security.access.UserPermission> getUserPermissions(org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest getUserPermissionsRequest) throws IOException
getUserPermissionsRequest
- A request contains which user, global, namespace or table
permissions neededIOException
- if a remote or network exception occursList<Boolean> hasUserPermissions(String userName, List<org.apache.hadoop.hbase.security.access.Permission> permissions) throws IOException
userName
- the user namepermissions
- the specific permission listIOException
- if a remote or network exception occursdefault List<Boolean> hasUserPermissions(List<org.apache.hadoop.hbase.security.access.Permission> permissions) throws IOException
permissions
- the specific permission listIOException
- if a remote or network exception occursboolean snapshotCleanupSwitch(boolean on, boolean synchronous) throws IOException
on
- Set to true
to enable, false
to disable.synchronous
- If true
, it waits until current snapshot cleanup is completed,
if outstanding.IOException
- if a remote or network exception occursboolean isSnapshotCleanupEnabled() throws IOException
true
if the auto snapshot cleanup is enabled, false
otherwise.IOException
- if a remote or network exception occurs@Deprecated default List<OnlineLogRecord> getSlowLogResponses(Set<ServerName> serverNames, LogQueryFilter logQueryFilter) throws IOException
getLogEntries(Set, String, ServerType, int, Map)
instead.serverNames
- Server names to get slowlog responses fromlogQueryFilter
- filter to be used if provided (determines slow / large RPC logs)IOException
- if a remote or network exception occursList<Boolean> clearSlowLogResponses(Set<ServerName> serverNames) throws IOException
serverNames
- Set of Server names to clean slowlog responses fromIOException
- if a remote or network exception occursvoid addRSGroup(String groupName) throws IOException
groupName
- the name of the groupIOException
- if a remote or network exception occursRSGroupInfo getRSGroup(String groupName) throws IOException
groupName
- the group nameIOException
- if a remote or network exception occursRSGroupInfo getRSGroup(Address hostPort) throws IOException
hostPort
- HostPort to get RSGroupInfo forIOException
- if a remote or network exception occursRSGroupInfo getRSGroup(TableName tableName) throws IOException
tableName
- table name to get RSGroupInfo forIOException
- if a remote or network exception occursList<RSGroupInfo> listRSGroups() throws IOException
IOException
- if a remote or network exception occursList<TableName> listTablesInRSGroup(String groupName) throws IOException
groupName
- the group nameIOException
- if a remote or network exception occursgetConfiguredNamespacesAndTablesInRSGroup(String)
Pair<List<String>,List<TableName>> getConfiguredNamespacesAndTablesInRSGroup(String groupName) throws IOException
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 its TableDescriptor
. 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 the listTablesInRSGroup(String)
will return all these tables.groupName
- the group nameIOException
- if a remote or network exception occurslistTablesInRSGroup(String)
void removeRSGroup(String groupName) throws IOException
groupName
- the group nameIOException
- if a remote or network exception occursvoid removeServersFromRSGroup(Set<Address> servers) throws IOException
servers
- set of servers to removeIOException
- if a remote or network exception occursvoid moveServersToRSGroup(Set<Address> servers, String targetGroup) throws IOException
servers
- set of servers to movetargetGroup
- the group to move servers toIOException
- if a remote or network exception occursvoid setRSGroup(Set<TableName> tables, String groupName) throws IOException
tables
- tables to set group forgroupName
- group name for tablesIOException
- if a remote or network exception occursdefault BalanceResponse balanceRSGroup(String groupName) throws IOException
groupName
- the group nameIOException
- if a remote or network exception occursBalanceResponse balanceRSGroup(String groupName, BalanceRequest request) throws IOException
BalanceRequest
.IOException
void renameRSGroup(String oldName, String newName) throws IOException
oldName
- old rsgroup namenewName
- new rsgroup nameIOException
- if a remote or network exception occursvoid updateRSGroupConfig(String groupName, Map<String,String> configuration) throws IOException
groupName
- the group nameconfiguration
- new configuration of the group name to be setIOException
- if a remote or network exception occursList<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String,Object> filterParams) throws IOException
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 paramsIOException
- if a remote or network exception occursvoid flushMasterStore() throws IOException
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.