@InterfaceAudience.Private public class HBaseAdmin extends Object implements Admin
Connection.getAdmin()
to obtain an instance of Admin
instead of constructing
an HBaseAdmin directly.
Connection should be an unmanaged connection obtained via
ConnectionFactory.createConnection(Configuration)
ConnectionFactory
,
Connection
,
Admin
Modifier and Type | Class and Description |
---|---|
private static class |
HBaseAdmin.AbortProcedureFuture |
private static class |
HBaseAdmin.AddColumnFamilyFuture |
private static class |
HBaseAdmin.CreateTableFuture |
private static class |
HBaseAdmin.DeleteColumnFamilyFuture |
private static class |
HBaseAdmin.DeleteTableFuture |
private static class |
HBaseAdmin.DisableTableFuture |
private static class |
HBaseAdmin.EnableTableFuture |
private static class |
HBaseAdmin.MergeTableRegionsFuture |
private static class |
HBaseAdmin.ModifyColumnFamilyFuture |
private static class |
HBaseAdmin.ModifyTableFuture |
protected static class |
HBaseAdmin.NamespaceFuture |
protected static class |
HBaseAdmin.ProcedureFuture<V>
Future that waits on a procedure result.
|
private static class |
HBaseAdmin.ReplicationFuture |
private static class |
HBaseAdmin.RestoreSnapshotFuture |
private static class |
HBaseAdmin.SplitTableRegionFuture |
protected static class |
HBaseAdmin.TableFuture<V> |
private static class |
HBaseAdmin.ThrowableAbortable
Simple
Abortable , throwing RuntimeException on abort. |
private static class |
HBaseAdmin.TruncateTableFuture |
Modifier and Type | Field and Description |
---|---|
private boolean |
aborted |
private org.apache.hadoop.conf.Configuration |
conf |
private ClusterConnection |
connection |
private int |
getProcedureTimeout |
private static org.slf4j.Logger |
LOG |
private NonceGenerator |
ng |
private int |
numRetries |
private int |
operationTimeout |
private long |
pause |
private RpcRetryingCallerFactory |
rpcCallerFactory |
private RpcControllerFactory |
rpcControllerFactory |
private int |
rpcTimeout |
private int |
syncWaitTimeout |
Constructor and Description |
---|
HBaseAdmin(ClusterConnection connection) |
Modifier and Type | Method and Description |
---|---|
void |
abort(String why,
Throwable e)
Abort the server or client.
|
boolean |
abortProcedure(long procId,
boolean mayInterruptIfRunning)
Abort a procedure.
|
Future<Boolean> |
abortProcedureAsync(long procId,
boolean mayInterruptIfRunning)
Abort a procedure but does not block and wait for completion.
|
Future<Void> |
addColumnFamilyAsync(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Add a column family to an existing table.
|
Future<Void> |
addReplicationPeerAsync(String peerId,
ReplicationPeerConfig peerConfig,
boolean enabled)
Add a new replication peer but does not block and wait for it.
|
void |
assign(byte[] regionName)
Assign a Region.
|
private org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse |
asyncSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) |
static void |
available(org.apache.hadoop.conf.Configuration conf)
Is HBase available? Throw an exception if not.
|
boolean |
balance()
Invoke the balancer.
|
boolean |
balance(boolean force)
Invoke the balancer.
|
boolean |
balancerSwitch(boolean on,
boolean synchronous)
Turn the load balancer on or off.
|
boolean |
catalogJanitorSwitch(boolean enable)
Enable/Disable the catalog janitor/
|
private void |
checkAndSyncTableDescToPeers(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 TableName |
checkTableExists(TableName tableName)
Check if table exists or not
|
boolean |
cleanerChoreSwitch(boolean on)
Enable/Disable the cleaner chore.
|
private CacheEvictionStats |
clearBlockCache(ServerName sn,
List<RegionInfo> hris) |
CacheEvictionStats |
clearBlockCache(TableName tableName)
Clear all the blocks corresponding to this table from BlockCache.
|
void |
clearCompactionQueues(ServerName sn,
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
|
private Boolean |
clearSlowLogsResponses(ServerName serverName) |
Future<Void> |
cloneSnapshotAsync(String snapshotName,
TableName tableName,
boolean restoreAcl)
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 |
closeRegion(byte[] regionName,
String unused)
Deprecated.
|
void |
closeRegion(ServerName unused,
HRegionInfo hri)
Deprecated.
|
void |
closeRegion(String regionName,
String unused)
Deprecated.
|
boolean |
closeRegionWithEncodedRegionName(String encodedRegionName,
String unused)
Deprecated.
|
private void |
compact(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin,
RegionInfo hri,
boolean major,
byte[] family) |
void |
compact(TableName tableName)
Compact a table.
|
void |
compact(TableName tableName,
byte[] columnFamily)
Compact a column family within a table.
|
private void |
compact(TableName tableName,
byte[] columnFamily,
boolean major,
CompactType compactType)
Compact 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.
|
private void |
compactRegion(byte[] regionName,
byte[] columnFamily,
boolean major)
Compact an individual region.
|
void |
compactRegionServer(ServerName serverName)
Compact all regions on the region server.
|
CoprocessorRpcChannel |
coprocessorService()
Creates and returns a
RpcChannel instance connected to the active
master. |
CoprocessorRpcChannel |
coprocessorService(ServerName serverName)
Creates and returns a
RpcChannel instance
connected to the passed region server. |
Future<Void> |
createNamespaceAsync(NamespaceDescriptor descriptor)
Create a new namespace.
|
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.
|
void |
deleteColumn(TableName tableName,
byte[] columnFamily)
Deprecated.
Since 2.0. Will be removed in 3.0. Use
Admin.deleteColumnFamily(TableName, byte[]) instead. |
Future<Void> |
deleteColumnFamilyAsync(TableName tableName,
byte[] columnFamily)
Delete a column family from a table.
|
Future<Void> |
deleteNamespaceAsync(String name)
Delete an existing namespace.
|
void |
deleteSnapshot(byte[] snapshotName)
Delete an existing snapshot.
|
void |
deleteSnapshot(String snapshotName)
Delete an existing snapshot.
|
void |
deleteSnapshots(Pattern pattern)
Delete existing snapshots whose names match the pattern passed.
|
void |
deleteSnapshots(String regex)
Delete existing snapshots whose names match the pattern passed.
|
Future<Void> |
deleteTableAsync(TableName tableName)
Deletes the table but does not block and wait for it to be completely removed.
|
HTableDescriptor[] |
deleteTables(Pattern pattern)
Delete tables matching the passed in pattern and wait on completion.
|
HTableDescriptor[] |
deleteTables(String regex)
Deletes tables matching the passed in pattern and wait on completion.
|
void |
deleteTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern)
Delete all existing snapshots matching the given table name regular expression and snapshot
name regular expression.
|
void |
deleteTableSnapshots(String tableNameRegex,
String snapshotNameRegex)
Delete all existing snapshots matching the given table name regular expression and snapshot
name regular expression.
|
Future<Void> |
disableReplicationPeerAsync(String peerId)
Disable a replication peer but does not block and wait for it.
|
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.
|
HTableDescriptor[] |
disableTables(Pattern pattern)
Disable tables matching the passed in pattern and wait on completion.
|
HTableDescriptor[] |
disableTables(String regex)
Disable tables matching the passed in pattern and wait on completion.
|
Future<Void> |
enableReplicationPeerAsync(String peerId)
Enable a replication peer but does not block and wait for it.
|
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.
|
HTableDescriptor[] |
enableTables(Pattern pattern)
Enable tables matching the passed in pattern and wait on completion.
|
HTableDescriptor[] |
enableTables(String regex)
Enable tables matching the passed in pattern and wait on completion.
|
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.
|
private <C extends RetryingCallable<V> & Closeable,V> |
executeCallable(C callable) |
private static <C extends RetryingCallable<V> & Closeable,V> |
executeCallable(C callable,
RpcRetryingCallerFactory rpcCallerFactory,
int operationTimeout,
int rpcTimeout) |
private void |
flush(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin,
RegionInfo info) |
void |
flush(TableName tableName)
Flush a table.
|
void |
flushRegion(byte[] regionName)
Flush an individual region.
|
void |
flushRegionServer(ServerName serverName)
Flush all regions on the region server.
|
private static <T> T |
get(Future<T> future,
long timeout,
TimeUnit units)
Do a get with a timeout against the passed in
future . |
Pair<Integer,Integer> |
getAlterStatus(byte[] tableName)
Get the status of
alter (a.k.a modify ) command - indicates how many
regions have received the updated schema Asynchronous operation. |
Pair<Integer,Integer> |
getAlterStatus(TableName tableName)
Get the status of an
alter (a.k.a modify ) command - indicates how
many regions have received the updated schema Asynchronous operation. |
ClusterMetrics |
getClusterMetrics(EnumSet<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() |
Connection |
getConnection() |
SpaceQuotaSnapshot |
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. |
SpaceQuotaSnapshot |
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.
|
(package private) static HTableDescriptor |
getHTableDescriptor(TableName tableName,
Connection connection,
RpcRetryingCallerFactory rpcCallerFactory,
RpcControllerFactory rpcControllerFactory,
int operationTimeout,
int rpcTimeout)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
getTableDescriptor(TableName,
Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int) |
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.
|
NamespaceDescriptor |
getNamespaceDescriptor(String name)
Get a namespace descriptor by name.
|
List<HRegionInfo> |
getOnlineRegions(ServerName sn)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0
Use
getRegions(ServerName) . |
int |
getOperationTimeout()
Return the operation timeout for a rpc call.
|
private long |
getPauseTime(int tries) |
String |
getProcedures()
Get procedures.
|
List<QuotaSettings> |
getQuota(QuotaFilter filter)
List the quotas based on the filter.
|
QuotaRetriever |
getQuotaRetriever(QuotaFilter filter)
Return a QuotaRetriever to list the quotas based on the filter.
|
(package private) Pair<RegionInfo,ServerName> |
getRegion(byte[] regionName) |
List<RegionMetrics> |
getRegionMetrics(ServerName serverName,
TableName tableName)
Get
RegionMetrics of all regions hosted on a regionserver for a table. |
private byte[] |
getRegionName(byte[] regionNameOrEncodedRegionName)
If the input is a region name, it is returned as is.
|
List<RegionInfo> |
getRegions(ServerName sn)
Get all the online regions on a region server.
|
List<RegionInfo> |
getRegions(TableName tableName)
Get the regions of a given table.
|
Map<TableName,SpaceQuotaSnapshot> |
getRegionServerSpaceQuotaSnapshots(ServerName serverName)
Fetches the observed
SpaceQuotaSnapshotView s observed by a RegionServer. |
ReplicationPeerConfig |
getReplicationPeerConfig(String peerId)
Returns the configured ReplicationPeerConfig for the specified peer.
|
private RpcControllerFactory |
getRpcControllerFactory() |
List<SecurityCapability> |
getSecurityCapabilities()
Return the set of supported security capabilities.
|
private List<OnlineLogRecord> |
getSlowLogResponseFromServer(ServerName serverName,
LogQueryFilter logQueryFilter) |
List<OnlineLogRecord> |
getSlowLogResponses(Set<ServerName> serverNames,
LogQueryFilter logQueryFilter)
Retrieves online slow/large RPC logs from the provided list of
RegionServers
|
private List<OnlineLogRecord> |
getSlowLogResponsesFromServer(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin,
LogQueryFilter logQueryFilter) |
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.
|
HTableDescriptor |
getTableDescriptor(TableName tableName)
Get a table descriptor.
|
(package private) static TableDescriptor |
getTableDescriptor(TableName tableName,
Connection connection,
RpcRetryingCallerFactory rpcCallerFactory,
RpcControllerFactory rpcControllerFactory,
int operationTimeout,
int rpcTimeout) |
HTableDescriptor[] |
getTableDescriptors(List<String> names)
Get tableDescriptors.
|
HTableDescriptor[] |
getTableDescriptorsByTableName(List<TableName> tableNames)
Get tableDescriptors.
|
private TableName |
getTableNameBeforeRestoreSnapshot(String snapshotName)
Check whether the snapshot exists and contains disabled table
|
List<HRegionInfo> |
getTableRegions(TableName tableName)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0
Use
getRegions(TableName) . |
private byte[][] |
getTableSplits(TableName tableName) |
List<UserPermission> |
getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest)
Get the global/namespace/table permissions for user
|
void |
grant(UserPermission userPermission,
boolean mergeExistingPermissions)
Grants user specific permissions
|
List<Boolean> |
hasUserPermissions(String userName,
List<Permission> permissions)
Check if the user has specific permissions
|
private void |
internalDeleteSnapshot(SnapshotDescription snapshot) |
private Future<Void> |
internalRestoreSnapshotAsync(String snapshotName,
TableName tableName,
boolean restoreAcl)
Execute Restore/Clone snapshot and wait for the server to complete (blocking).
|
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 |
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 snapshotDesc)
Check the current state of the passed snapshot.
|
boolean |
isSplitEnabled()
Query the current state of the split switch.
|
boolean |
isTableAvailable(TableName tableName) |
boolean |
isTableAvailable(TableName tableName,
byte[][] splitKeys)
Use this api to check if the table has been created with the specified number of splitkeys
which was used while creating the given table.
|
boolean |
isTableDisabled(TableName tableName) |
boolean |
isTableEnabled(TableName tableName) |
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<SnapshotDescription> |
listSnapshots()
List completed snapshots.
|
List<SnapshotDescription> |
listSnapshots(Pattern pattern)
List all the completed snapshots matching the given pattern.
|
List<SnapshotDescription> |
listSnapshots(String regex)
List all the completed snapshots matching the given regular expression.
|
List<TableDescriptor> |
listTableDescriptors()
List all the userspace tables.
|
List<TableDescriptor> |
listTableDescriptors(List<TableName> tableNames)
Get tableDescriptors.
|
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.
|
HTableDescriptor[] |
listTableDescriptorsByNamespace(String name)
Get list of table descriptors by namespace.
|
TableName[] |
listTableNames()
List all of the names of userspace tables.
|
TableName[] |
listTableNames(Pattern pattern,
boolean includeSysTables)
List all of the names of userspace tables.
|
TableName[] |
listTableNames(String regex)
List all of the names of userspace tables.
|
TableName[] |
listTableNames(String regex,
boolean includeSysTables)
List all of the names of userspace tables.
|
TableName[] |
listTableNamesByNamespace(String name)
Get list of table names by namespace.
|
HTableDescriptor[] |
listTables()
List all the userspace tables.
|
HTableDescriptor[] |
listTables(Pattern pattern)
List all the userspace tables that match the given pattern.
|
HTableDescriptor[] |
listTables(Pattern pattern,
boolean includeSysTables)
List all the tables matching the given pattern.
|
HTableDescriptor[] |
listTables(String regex)
List all the userspace tables matching the given regular expression.
|
HTableDescriptor[] |
listTables(String regex,
boolean includeSysTables)
List all the tables matching the given pattern.
|
List<SnapshotDescription> |
listTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern)
List all the completed snapshots matching the given table name regular expression and snapshot
name regular expression.
|
List<SnapshotDescription> |
listTableSnapshots(String tableNameRegex,
String snapshotNameRegex)
List all the completed snapshots matching the given table name regular expression and snapshot
name regular expression.
|
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.
|
void |
mergeRegions(byte[] nameOfRegionA,
byte[] nameOfRegionB,
boolean forcible)
Deprecated.
Since 2.0. Will be removed in 3.0. Use
Admin.mergeRegionsAsync(byte[], byte[], boolean) instead. |
Future<Void> |
mergeRegionsAsync(byte[][] nameofRegionsToMerge,
boolean forcible)
Merge two regions.
|
void |
mergeRegionsSync(byte[] nameOfRegionA,
byte[] nameOfRegionB,
boolean forcible)
Merge two regions.
|
boolean |
mergeSwitch(boolean enabled,
boolean synchronous)
Turn the merge switch on or off.
|
Future<Void> |
modifyColumnFamilyAsync(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Modify an existing column family on a table.
|
Future<Void> |
modifyNamespaceAsync(NamespaceDescriptor descriptor)
Modify an existing namespace.
|
Future<Void> |
modifyTableAsync(TableDescriptor td)
Modify an existing table, more IRB (ruby) friendly version.
|
void |
move(byte[] encodedRegionName)
Move the region
encodedRegionName to a random server. |
void |
move(byte[] encodedRegionName,
ServerName destServerName)
Move the region
rencodedRegionName to destServerName . |
boolean |
normalize()
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.
|
Future<Void> |
removeReplicationPeerAsync(String peerId)
Remove a replication peer but does not block and wait for it.
|
void |
restoreSnapshot(byte[] snapshotName)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(byte[] snapshotName,
boolean takeFailSafeSnapshot)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(String snapshotName)
Restore the specified snapshot on the original table.
|
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.
|
Future<Void> |
restoreSnapshotAsync(String snapshotName)
Restore the specified snapshot on the original table.
|
void |
revoke(UserPermission userPermission)
Revokes user specific permissions
|
void |
rollWALWriter(ServerName serverName)
Roll the log writer.
|
private org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse |
rollWALWriterImpl(ServerName sn) |
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.
|
private void |
setTableRep(TableName tableName,
boolean enableRep)
Set the table's replication switch if the table's replication switch is already not set.
|
void |
shutdown()
Shuts down the HBase cluster.
|
void |
snapshot(SnapshotDescription snapshotDesc)
Take a snapshot and wait for the server to complete that snapshot (blocking).
|
Future<Void> |
snapshotAsync(SnapshotDescription snapshotDesc)
Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a
single snapshot should be taken at a time, or results may be undefined.
|
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.
|
private boolean |
splitOrMergeSwitch(boolean enabled,
boolean synchronous,
MasterSwitchType switchType) |
void |
splitRegion(byte[] regionName)
Split an individual region.
|
void |
splitRegion(byte[] regionName,
byte[] splitPoint)
Split an individual region.
|
Future<Void> |
splitRegionAsync(byte[] regionName)
Split an individual region.
|
Future<Void> |
splitRegionAsync(byte[] regionName,
byte[] splitPoint)
Split an individual region.
|
(package private) Future<Void> |
splitRegionAsync(RegionInfo hri,
byte[] splitPoint) |
void |
splitRegionSync(byte[] regionName,
byte[] splitPoint)
Split one region.
|
void |
splitRegionSync(byte[] regionName,
byte[] splitPoint,
long timeout,
TimeUnit units)
Split one 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.
|
private Boolean |
switchCompact(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin,
boolean onOrOff) |
boolean |
switchRpcThrottle(boolean enable)
Switch the rpc throttle enable state.
|
boolean |
tableExists(TableName tableName) |
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,
boolean force)
Unassign a region from current hosting regionserver.
|
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.
|
Future<Void> |
updateReplicationPeerConfigAsync(String peerId,
ReplicationPeerConfig peerConfig)
Update the peerConfig for the specified peer but does not block and wait for it.
|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
addColumn, addColumnFamily, addReplicationPeer, addReplicationPeer, addReplicationPeerAsync, appendReplicationPeerTableCFs, balancer, balancer, cloneSnapshot, cloneSnapshot, cloneSnapshot, cloneSnapshotAsync, compactRegionServer, createNamespace, createTable, createTable, deleteColumnFamily, deleteNamespace, deleteTable, disableReplicationPeer, disableTable, enableCatalogJanitor, enableReplicationPeer, enableTable, execProcedureWithRet, getBackupMasters, getClusterMetrics, getClusterStatus, getMaster, getMasterCoprocessorNames, getMasterCoprocessors, getMasterInfoPort, getRegionMetrics, getRegionServers, getRegionServers, hasUserPermissions, isSplitOrMergeEnabled, listDeadServers, listTableDescriptors, listTableNames, mergeRegionsAsync, modifyColumn, modifyColumnFamily, modifyNamespace, modifyTable, modifyTable, modifyTableAsync, move, removeReplicationPeer, removeReplicationPeerTableCFs, runCatalogScan, setBalancerRunning, setCleanerChoreRunning, setNormalizerRunning, setSplitOrMergeEnabled, snapshot, snapshot, snapshot, snapshot, snapshot, takeSnapshotAsync, truncateTable, updateReplicationPeerConfig
private static final org.slf4j.Logger LOG
private ClusterConnection connection
private final org.apache.hadoop.conf.Configuration conf
private final long pause
private final int numRetries
private final int syncWaitTimeout
private boolean aborted
private int operationTimeout
private int rpcTimeout
private int getProcedureTimeout
private RpcRetryingCallerFactory rpcCallerFactory
private RpcControllerFactory rpcControllerFactory
private NonceGenerator ng
HBaseAdmin(ClusterConnection connection) throws IOException
IOException
public int getOperationTimeout()
Admin
getOperationTimeout
in interface Admin
Admin.getSyncWaitTimeout()
public int getSyncWaitTimeout()
Admin
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 Admin.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).getSyncWaitTimeout
in interface Admin
Admin.getOperationTimeout()
public void abort(String why, Throwable e)
Abortable
public boolean isAborted()
Abortable
public boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException
Admin
abortProcedure
in interface Admin
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 occurspublic Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning) throws IOException
Admin
abortProcedureAsync
in interface Admin
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 occurspublic List<TableDescriptor> listTableDescriptors() throws IOException
Admin
listTableDescriptors
in interface Admin
IOException
- if a remote or network exception occurspublic List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) throws IOException
Admin
listTableDescriptors
in interface Admin
pattern
- The compiled regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occursAdmin.listTables()
public TableDescriptor getDescriptor(TableName tableName) throws TableNotFoundException, IOException
Admin
getDescriptor
in interface Admin
tableName
- as a TableName
TableNotFoundException
IOException
- if a remote or network exception occurspublic Future<Void> modifyTableAsync(TableDescriptor td) throws IOException
Admin
modifyTableAsync
in interface Admin
td
- description of the tableIOException
- if a remote or network exception occurspublic List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException
Admin
listTableDescriptorsByNamespace
in interface Admin
name
- namespace nameIOException
- if a remote or network exception occurspublic List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException
Admin
listTableDescriptors
in interface Admin
tableNames
- List of table namesIOException
- if a remote or network exception occurspublic List<RegionInfo> getRegions(ServerName sn) throws IOException
Admin
getRegions
in interface Admin
RegionInfo
IOException
- if a remote or network exception occurspublic List<RegionInfo> getRegions(TableName tableName) throws IOException
Admin
getRegions
in interface Admin
tableName
- the name of the tableRegionInfo
.IOException
- if a remote or network exception occurspublic Connection getConnection()
getConnection
in interface Admin
public boolean tableExists(TableName tableName) throws IOException
tableExists
in interface Admin
tableName
- Table to check.true
if table exists already.IOException
- if a remote or network exception occurspublic HTableDescriptor[] listTables() throws IOException
Admin
listTables
in interface Admin
IOException
- if a remote or network exception occursAdmin.listTableDescriptors()
public HTableDescriptor[] listTables(Pattern pattern) throws IOException
Admin
listTables
in interface Admin
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occursAdmin.listTables()
,
Admin.listTableDescriptors(Pattern)
public HTableDescriptor[] listTables(String regex) throws IOException
Admin
listTables
in interface Admin
regex
- The regular expression to match againstIOException
- if a remote or network exception occursAdmin.listTableDescriptors(Pattern)
public HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables) throws IOException
Admin
listTables
in interface Admin
pattern
- The compiled regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occursAdmin.listTables()
,
Admin.listTableDescriptors(java.util.regex.Pattern, boolean)
public HTableDescriptor[] listTables(String regex, boolean includeSysTables) throws IOException
Admin
listTables
in interface Admin
regex
- The regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occursAdmin.listTables(java.util.regex.Pattern, boolean)
public TableName[] listTableNames() throws IOException
Admin
listTableNames
in interface Admin
IOException
- if a remote or network exception occurspublic TableName[] listTableNames(String regex) throws IOException
Admin
listTableNames
in interface Admin
regex
- The regular expression to match againstIOException
- if a remote or network exception occurspublic TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException
Admin
listTableNames
in interface Admin
pattern
- The regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occurspublic TableName[] listTableNames(String regex, boolean includeSysTables) throws IOException
Admin
listTableNames
in interface Admin
regex
- The regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occurspublic HTableDescriptor getTableDescriptor(TableName tableName) throws IOException
Admin
getTableDescriptor
in interface Admin
tableName
- as a TableName
TableNotFoundException
IOException
- if a remote or network exception occursstatic TableDescriptor getTableDescriptor(TableName tableName, Connection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout) throws IOException
IOException
@Deprecated static HTableDescriptor getHTableDescriptor(TableName tableName, Connection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout) throws IOException
getTableDescriptor(TableName,
Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int)
IOException
private long getPauseTime(int tries)
public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) throws IOException
Admin
createTable
in interface Admin
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 occursMasterNotRunningException
- 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).public Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys) throws IOException
Admin
createTableAsync
in interface Admin
desc
- table descriptor for tablesplitKeys
- keys to check if the table has been created with all split keysIOException
- if a remote or network exception occurspublic Future<Void> deleteTableAsync(TableName tableName) throws IOException
Admin
deleteTableAsync
in interface Admin
tableName
- name of table to deleteIOException
- if a remote or network exception occurspublic HTableDescriptor[] deleteTables(String regex) throws IOException
Admin
Admin.listTableDescriptors(Pattern)
and Admin.deleteTable(org.apache.hadoop.hbase.TableName)
deleteTables
in interface Admin
regex
- The regular expression to match table names againstIOException
- if a remote or network exception occursAdmin.deleteTables(java.util.regex.Pattern)
,
Admin.deleteTable(org.apache.hadoop.hbase.TableName)
public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException
listTables(java.util.regex.Pattern)
and
Admin.deleteTable(TableName)
deleteTables
in interface Admin
pattern
- The pattern to match table names againstIOException
- if a remote or network exception occurspublic Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits) throws IOException
Admin
truncateTableAsync
in interface Admin
tableName
- name of table to deletepreserveSplits
- true
if the splits should be preservedIOException
- if a remote or network exception occursprivate byte[][] getTableSplits(TableName tableName) throws IOException
IOException
public Future<Void> enableTableAsync(TableName tableName) throws IOException
Admin
enableTableAsync
in interface Admin
tableName
- name of table to deleteIOException
- if a remote or network exception occurspublic HTableDescriptor[] enableTables(String regex) throws IOException
Admin
Admin.listTableDescriptors(Pattern)
and Admin.enableTable(org.apache.hadoop.hbase.TableName)
enableTables
in interface Admin
regex
- The regular expression to match table names againstIOException
- if a remote or network exception occursAdmin.enableTables(java.util.regex.Pattern)
,
Admin.enableTable(org.apache.hadoop.hbase.TableName)
public HTableDescriptor[] enableTables(Pattern pattern) throws IOException
Admin
Admin.listTableDescriptors(java.util.regex.Pattern)
and
Admin.enableTable(org.apache.hadoop.hbase.TableName)
enableTables
in interface Admin
pattern
- The pattern to match table names againstIOException
- if a remote or network exception occurspublic Future<Void> disableTableAsync(TableName tableName) throws IOException
Admin
disableTableAsync
in interface Admin
tableName
- name of table to deleteIOException
- if a remote or network exception occurspublic HTableDescriptor[] disableTables(String regex) throws IOException
Admin
Admin.listTableDescriptors(Pattern)
and Admin.disableTable(org.apache.hadoop.hbase.TableName)
disableTables
in interface Admin
regex
- The regular expression to match table names againstIOException
- if a remote or network exception occursAdmin.disableTables(java.util.regex.Pattern)
,
Admin.disableTable(org.apache.hadoop.hbase.TableName)
public HTableDescriptor[] disableTables(Pattern pattern) throws IOException
Admin
Admin.listTableDescriptors(java.util.regex.Pattern)
and
Admin.disableTable(org.apache.hadoop.hbase.TableName)
disableTables
in interface Admin
pattern
- The pattern to match table names againstIOException
- if a remote or network exception occurspublic boolean isTableEnabled(TableName tableName) throws IOException
isTableEnabled
in interface Admin
tableName
- name of table to checktrue
if table is on-lineIOException
- if a remote or network exception occurspublic boolean isTableDisabled(TableName tableName) throws IOException
isTableDisabled
in interface Admin
tableName
- name of table to checktrue
if table is off-lineIOException
- if a remote or network exception occurspublic boolean isTableAvailable(TableName tableName) throws IOException
isTableAvailable
in interface Admin
tableName
- name of table to checktrue
if all regions of the table are availableIOException
- if a remote or network exception occurspublic boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException
Admin
false
.isTableAvailable
in interface Admin
tableName
- name of table to checksplitKeys
- keys to check if the table has been created with all split keysIOException
- if a remote or network excpetion occurspublic Pair<Integer,Integer> getAlterStatus(TableName tableName) throws IOException
Admin
alter
(a.k.a modify
) command - indicates how
many regions have received the updated schema Asynchronous operation.getAlterStatus
in interface Admin
tableName
- TableName instanceIOException
- if a remote or network exception occurspublic Pair<Integer,Integer> getAlterStatus(byte[] tableName) throws IOException
Admin
alter
(a.k.a modify
) command - indicates how many
regions have received the updated schema Asynchronous operation.getAlterStatus
in interface Admin
tableName
- name of the table to get the status ofIOException
- if a remote or network exception occurspublic Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
Admin
addColumnFamilyAsync
in interface Admin
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 occurs@Deprecated public void deleteColumn(TableName tableName, byte[] columnFamily) throws IOException
Admin.deleteColumnFamily(TableName, byte[])
instead.Admin.deleteColumnFamily(TableName, byte[])
instead because it
returns a Future
from which you can learn whether success or failure.deleteColumn
in interface Admin
tableName
- name of tablecolumnFamily
- name of column family to be deletedIOException
- if a remote or network exception occurspublic Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily) throws IOException
Admin
deleteColumnFamilyAsync
in interface Admin
tableName
- name of tablecolumnFamily
- name of column family to be deletedIOException
- if a remote or network exception occurspublic Future<Void> modifyColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
Admin
modifyColumnFamilyAsync
in interface Admin
tableName
- name of tablecolumnFamily
- new column family descriptor to useIOException
- if a remote or network exception occurs@Deprecated public void closeRegion(String regionName, String unused) throws IOException
Admin
Admin.unassign(byte[], boolean)
to unassign the region. For expert-admins.closeRegion
in interface Admin
regionName
- region name to closeunused
- Deprecated. Not used.IOException
- if a remote or network exception occurs@Deprecated public void closeRegion(byte[] regionName, String unused) throws IOException
Admin
Admin.unassign(byte[], boolean)
to unassign the region. For expert-admins.closeRegion
in interface Admin
regionName
- region name to closeunused
- Deprecated. Not used.IOException
- if a remote or network exception occurs@Deprecated public boolean closeRegionWithEncodedRegionName(String encodedRegionName, String unused) throws IOException
Admin
Admin.unassign(byte[], boolean)
to unassign the region. For expert-admins.closeRegionWithEncodedRegionName
in interface Admin
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
.unused
- Deprecated. Not used.true
always.IOException
- if a remote or network exception occurs@Deprecated public void closeRegion(ServerName unused, HRegionInfo hri) throws IOException
Admin
Admin.unassign(byte[], boolean)
to unassign the region. For expert-admins.closeRegion
in interface Admin
unused
- Deprecated. Not used.IOException
- if a remote or network exception occurs@Deprecated public List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException
getRegions(ServerName)
.Admin
getOnlineRegions
in interface Admin
sn
- HRegionInfo
.IOException
- if a remote or network exception occurspublic void flush(TableName tableName) throws IOException
Admin
flush
in interface Admin
tableName
- table to flushIOException
- if a remote or network exception occurspublic void flushRegion(byte[] regionName) throws IOException
Admin
flushRegion
in interface Admin
regionName
- region to flushIOException
- if a remote or network exception occursprivate void flush(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, RegionInfo info) throws IOException
IOException
public void flushRegionServer(ServerName serverName) throws IOException
Admin
flushRegionServer
in interface Admin
serverName
- the region server name to flushIOException
- if a remote or network exception occurspublic void compact(TableName tableName) throws IOException
compact
in interface Admin
tableName
- table to compactIOException
- if a remote or network exception occurspublic void compactRegion(byte[] regionName) throws IOException
Admin
compactRegion
in interface Admin
regionName
- region to compactIOException
- if a remote or network exception occurspublic void compact(TableName tableName, byte[] columnFamily) throws IOException
compact
in interface Admin
tableName
- table to compactcolumnFamily
- column family within a tableIOException
- if a remote or network exception occurspublic void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException
compactRegion
in interface Admin
regionName
- region to compactcolumnFamily
- column family within a regionIOException
- if a remote or network exception occurspublic Map<ServerName,Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList) throws IOException
Admin
compactionSwitch
in interface Admin
switchState
- Set to true
to enable, false
to disable.serverNamesList
- list of region servers.IOException
private Boolean switchCompact(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, boolean onOrOff) throws IOException
IOException
public void compactRegionServer(ServerName serverName) throws IOException
Admin
compactRegionServer
in interface Admin
serverName
- the region server nameIOException
- if a remote or network exception occurspublic void majorCompactRegionServer(ServerName serverName) throws IOException
Admin
majorCompactRegionServer
in interface Admin
serverName
- the region server nameIOException
- if a remote or network exception occurspublic void majorCompact(TableName tableName) throws IOException
Admin
majorCompact
in interface Admin
tableName
- table to major compactIOException
- if a remote or network exception occurspublic void majorCompactRegion(byte[] regionName) throws IOException
Admin
majorCompactRegion
in interface Admin
regionName
- region to major compactIOException
- if a remote or network exception occurspublic void majorCompact(TableName tableName, byte[] columnFamily) throws IOException
majorCompact
in interface Admin
tableName
- table to major compactcolumnFamily
- column family within a tableIOException
- if a remote or network exception occurspublic void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException
Admin
majorCompactRegion
in interface Admin
regionName
- egion to major compactcolumnFamily
- column family within a regionIOException
- if a remote or network exception occursprivate void compact(TableName tableName, byte[] columnFamily, boolean major, CompactType compactType) throws IOException
tableName
- table or region to compactcolumnFamily
- column family within a table or regionmajor
- True if we are to do a major compaction.compactType
- CompactType
IOException
- if a remote or network exception occursprivate void compactRegion(byte[] regionName, byte[] columnFamily, boolean major) throws IOException
regionName
- region to compactcolumnFamily
- column family within a table or regionmajor
- True if we are to do a major compaction.IOException
- if a remote or network exception occursInterruptedException
private void compact(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, RegionInfo hri, boolean major, byte[] family) throws IOException
IOException
public void move(byte[] encodedRegionName) throws IOException
Admin
encodedRegionName
to a random server.move
in interface Admin
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
public void move(byte[] encodedRegionName, ServerName destServerName) throws IOException
Admin
rencodedRegionName
to destServerName
.move
in interface Admin
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
public void assign(byte[] regionName) throws MasterNotRunningException, ZooKeeperConnectionException, IOException
Admin
assign
in interface Admin
regionName
- Region name to assign.IOException
- if a remote or network exception occursMasterNotRunningException
ZooKeeperConnectionException
public void unassign(byte[] regionName, boolean force) throws IOException
Admin
Admin.move(byte[], ServerName)
if you want to control the region movement.unassign
in interface Admin
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 occurspublic void offline(byte[] regionName) throws IOException
Admin
offline
in interface Admin
regionName
- Region to offline.IOException
- if a remote or network exception occurspublic boolean balancerSwitch(boolean on, boolean synchronous) throws IOException
Admin
balancerSwitch
in interface Admin
on
- 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 occurspublic boolean balance() throws IOException
Admin
balance
in interface Admin
true
if balancer ran, false
otherwise.IOException
- if a remote or network exception occurspublic boolean balance(boolean force) throws IOException
Admin
balance
in interface Admin
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 occurspublic boolean isBalancerEnabled() throws IOException
Admin
isBalancerEnabled
in interface Admin
true
if the balancer is enabled, false
otherwise.IOException
- if a remote or network exception occurspublic CacheEvictionStats clearBlockCache(TableName tableName) throws IOException
clearBlockCache
in interface Admin
tableName
- table to clear block cacheIOException
- if a remote or network exception occursprivate CacheEvictionStats clearBlockCache(ServerName sn, List<RegionInfo> hris) throws IOException
IOException
public boolean normalize() throws IOException
normalize
in interface Admin
IOException
- if a remote or network exception occurspublic boolean isNormalizerEnabled() throws IOException
Admin
isNormalizerEnabled
in interface Admin
true
if region normalizer is enabled, false
otherwise.IOException
- if a remote or network exception occurspublic boolean normalizerSwitch(boolean on) throws IOException
Admin
normalizerSwitch
in interface Admin
IOException
- if a remote or network exception occurspublic boolean catalogJanitorSwitch(boolean enable) throws IOException
Admin
catalogJanitorSwitch
in interface Admin
enable
- if true
enables the catalog janitorIOException
- if a remote or network exception occurspublic int runCatalogJanitor() throws IOException
Admin
runCatalogJanitor
in interface Admin
IOException
- if a remote or network exception occurspublic boolean isCatalogJanitorEnabled() throws IOException
Admin
isCatalogJanitorEnabled
in interface Admin
IOException
- if a remote or network exception occurspublic boolean cleanerChoreSwitch(boolean on) throws IOException
Admin
cleanerChoreSwitch
in interface Admin
on
- if true
enables the cleaner choreIOException
- if a remote or network exception occurspublic boolean runCleanerChore() throws IOException
Admin
runCleanerChore
in interface Admin
true
if cleaner chore ran, false
otherwiseIOException
- if a remote or network exception occurspublic boolean isCleanerChoreEnabled() throws IOException
Admin
isCleanerChoreEnabled
in interface Admin
IOException
- if a remote or network exception occurspublic void mergeRegionsSync(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
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 occurs@Deprecated public void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
Admin.mergeRegionsAsync(byte[], byte[], boolean)
instead.mergeRegions
in interface Admin
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 occurspublic Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible) throws IOException
mergeRegionsAsync
in interface Admin
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 occurspublic void splitRegionSync(byte[] regionName, byte[] splitPoint) throws IOException
regionName
- encoded or full name of regionsplitPoint
- key where region splitsIOException
- if a remote or network exception occurspublic void splitRegionSync(byte[] regionName, byte[] splitPoint, long timeout, TimeUnit units) throws IOException
regionName
- region to be splitsplitPoint
- split pointtimeout
- how long to wait on splitunits
- time unitsIOException
- if a remote or network exception occurspublic Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException
Admin
splitRegionAsync
in interface Admin
regionName
- region to splitsplitPoint
- the explicit position to split onIOException
- if a remote or network exception occursFuture<Void> splitRegionAsync(RegionInfo hri, byte[] splitPoint) throws IOException
IOException
public void split(TableName tableName) throws IOException
Admin
split
in interface Admin
tableName
- table to splitIOException
- if a remote or network exception occurspublic void splitRegion(byte[] regionName) throws IOException
Admin
splitRegion
in interface Admin
regionName
- region to splitIOException
- if a remote or network exception occurspublic void split(TableName tableName, byte[] splitPoint) throws IOException
Admin
split
in interface Admin
tableName
- table to splitsplitPoint
- the explicit position to split onIOException
- if a remote or network exception occurspublic void splitRegion(byte[] regionName, byte[] splitPoint) throws IOException
Admin
splitRegion
in interface Admin
regionName
- region to splitsplitPoint
- the explicit position to split onIOException
- if a remote or network exception occursPair<RegionInfo,ServerName> getRegion(byte[] regionName) throws IOException
regionName
- Name of a region.regionName
is
a verified region name (we call MetaTableAccessor.getRegionLocation(Connection, byte[])
else null.
Throw IllegalArgumentException if regionName
is null.IOException
- if a remote or network exception occursprivate byte[] getRegionName(byte[] regionNameOrEncodedRegionName) throws IOException
IOException
private TableName checkTableExists(TableName tableName) throws IOException
tableName
- Name of a table.IOException
- if a remote or network exception occurs.TableNotFoundException
- if table does not exist.public void shutdown() throws IOException
Admin
shutdown
in interface Admin
IOException
- if a remote or network exception occurspublic void stopMaster() throws IOException
Admin
stopMaster
in interface Admin
IOException
- if a remote or network exception occursAdmin.shutdown()
public void stopRegionServer(String hostnamePort) throws IOException
Admin
stopRegionServer
in interface Admin
hostnamePort
- Hostname and port delimited by a :
as in
example.org:1234
IOException
- if a remote or network exception occurspublic boolean isMasterInMaintenanceMode() throws IOException
Admin
isMasterInMaintenanceMode
in interface Admin
IOException
- if a remote or network exception occurspublic ClusterMetrics getClusterMetrics(EnumSet<ClusterMetrics.Option> options) throws IOException
Admin
ClusterMetrics.Option
to get desired status.getClusterMetrics
in interface Admin
IOException
- if a remote or network exception occurspublic List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName) throws IOException
Admin
RegionMetrics
of all regions hosted on a regionserver for a table.getRegionMetrics
in interface Admin
serverName
- region server from which RegionMetrics
is required.tableName
- get RegionMetrics
of regions belonging to the tableIOException
- if a remote or network exception occurspublic org.apache.hadoop.conf.Configuration getConfiguration()
getConfiguration
in interface Admin
private static <T> T get(Future<T> future, long timeout, TimeUnit units) throws IOException
future
.IOException
public Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException
Admin
createNamespaceAsync
in interface Admin
descriptor
- descriptor which describes the new namespaceIOException
- if a remote or network exception occurspublic Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException
Admin
modifyNamespaceAsync
in interface Admin
descriptor
- descriptor which describes the new namespaceIOException
- if a remote or network exception occurspublic Future<Void> deleteNamespaceAsync(String name) throws IOException
Admin
deleteNamespaceAsync
in interface Admin
name
- namespace nameIOException
- if a remote or network exception occurspublic NamespaceDescriptor getNamespaceDescriptor(String name) throws NamespaceNotFoundException, IOException
Admin
getNamespaceDescriptor
in interface Admin
name
- name of namespace descriptorNamespaceNotFoundException
IOException
- if a remote or network exception occurspublic String[] listNamespaces() throws IOException
listNamespaces
in interface Admin
IOException
- if a remote or network exception occurspublic NamespaceDescriptor[] listNamespaceDescriptors() throws IOException
listNamespaceDescriptors
in interface Admin
IOException
- if a remote or network exception occurspublic String getProcedures() throws IOException
Admin
getProcedures
in interface Admin
IOException
- if a remote or network exception occurspublic String getLocks() throws IOException
Admin
getLocks
in interface Admin
IOException
- if a remote or network exception occurspublic HTableDescriptor[] listTableDescriptorsByNamespace(String name) throws IOException
Admin
listTableDescriptorsByNamespace
in interface Admin
name
- namespace nameIOException
- if a remote or network exception occurspublic TableName[] listTableNamesByNamespace(String name) throws IOException
Admin
listTableNamesByNamespace
in interface Admin
name
- namespace nameIOException
- if a remote or network exception occurspublic static void available(org.apache.hadoop.conf.Configuration conf) throws MasterNotRunningException, ZooKeeperConnectionException, IOException
conf
- system configurationMasterNotRunningException
- if the master is not running.ZooKeeperConnectionException
- if unable to connect to zookeeper. // TODO do not expose
ZKConnectionException.IOException
@Deprecated public List<HRegionInfo> getTableRegions(TableName tableName) throws IOException
getRegions(TableName)
.Admin
getTableRegions
in interface Admin
tableName
- HRegionInfo
.IOException
- if a remote or network exception occurspublic void close() throws IOException
close
in interface Closeable
close
in interface AutoCloseable
close
in interface Admin
IOException
public HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames) throws IOException
Admin
getTableDescriptorsByTableName
in interface Admin
tableNames
- List of table namesIOException
- if a remote or network exception occurspublic HTableDescriptor[] getTableDescriptors(List<String> names) throws IOException
Admin
getTableDescriptors
in interface Admin
names
- List of table namesIOException
- if a remote or network exception occursprivate org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriterImpl(ServerName sn) throws IOException, FailedLogCloseException
IOException
FailedLogCloseException
public void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException
Admin
rollWALWriter
in interface Admin
serverName
- The servername of the regionserver.IOException
- if a remote or network exception occursFailedLogCloseException
public CompactionState getCompactionState(TableName tableName) throws IOException
Admin
getCompactionState
in interface Admin
tableName
- table to examineIOException
- if a remote or network exception occurspublic CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException
Admin
getCompactionStateForRegion
in interface Admin
regionName
- region to examineIOException
- if a remote or network exception occurspublic void snapshot(SnapshotDescription snapshotDesc) throws IOException, SnapshotCreationException, IllegalArgumentException
Admin
SnapshotCreationException
indicating the duplicate naming. Snapshot names follow the
same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
. You should
probably use Admin.snapshot(String, org.apache.hadoop.hbase.TableName)
or
Admin.snapshot(byte[], org.apache.hadoop.hbase.TableName)
unless you are sure about the type
of snapshot that you want to take.snapshot
in interface Admin
snapshotDesc
- snapshot to takeIOException
- or we lose contact with the master.SnapshotCreationException
- if snapshot failed to be takenIllegalArgumentException
- if the snapshot request is formatted incorrectlypublic Future<Void> snapshotAsync(SnapshotDescription snapshotDesc) throws IOException, SnapshotCreationException
Admin
snapshotAsync
in interface Admin
snapshotDesc
- snapshot to takeIOException
- if the snapshot did not succeed or we lose contact with the master.SnapshotCreationException
- if snapshot creation failedprivate org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse asyncSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws IOException
IOException
public boolean isSnapshotFinished(SnapshotDescription snapshotDesc) throws IOException, HBaseSnapshotException, UnknownSnapshotException
Admin
UnknownSnapshotException
.isSnapshotFinished
in interface Admin
snapshotDesc
- description of the snapshot to checkIOException
- if we have a network issueHBaseSnapshotException
- if the snapshot failedUnknownSnapshotException
- if the requested snapshot is
unknownpublic void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException
Admin
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.restoreSnapshot
in interface Admin
snapshotName
- name of the snapshot to restoreIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be
restoredpublic void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException
Admin
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.restoreSnapshot
in interface Admin
snapshotName
- name of the snapshot to restoreIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredpublic void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
Admin
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".restoreSnapshot
in interface Admin
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 restoredprivate TableName getTableNameBeforeRestoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException
snapshotName
- name of the snapshot to restoreIOException
- if a remote or network exception occursRestoreSnapshotException
- if no valid snapshot is foundpublic void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
Admin
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".restoreSnapshot
in interface Admin
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 restoredpublic void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) throws IOException, RestoreSnapshotException
Admin
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".restoreSnapshot
in interface Admin
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 restoredpublic Future<Void> restoreSnapshotAsync(String snapshotName) throws IOException, RestoreSnapshotException
Admin
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.restoreSnapshotAsync
in interface Admin
snapshotName
- name of the snapshot to restoreIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredpublic Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException
Admin
cloneSnapshotAsync
in interface Admin
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 clonedpublic byte[] execProcedureWithReturn(String signature, String instance, Map<String,String> props) throws IOException
Admin
execProcedureWithReturn
in interface Admin
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 occurspublic void execProcedure(String signature, String instance, Map<String,String> props) throws IOException
Admin
execProcedure
in interface Admin
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 occurspublic boolean isProcedureFinished(String signature, String instance, Map<String,String> props) throws IOException
Admin
isProcedureFinished
in interface Admin
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 errorprivate Future<Void> internalRestoreSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, RestoreSnapshotException
isTableAvailable(org.apache.hadoop.hbase.TableName)
-- it is not safe to
create an HTable instance to this table before it is available.snapshotName
- snapshot to restoretableName
- table name to restore the snapshot onIOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectlypublic List<SnapshotDescription> listSnapshots() throws IOException
Admin
listSnapshots
in interface Admin
IOException
- if a network error occurspublic List<SnapshotDescription> listSnapshots(String regex) throws IOException
Admin
listSnapshots
in interface Admin
regex
- The regular expression to match againstIOException
- if a remote or network exception occurspublic List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException
Admin
listSnapshots
in interface Admin
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occurspublic List<SnapshotDescription> listTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException
Admin
listTableSnapshots
in interface Admin
tableNameRegex
- The table name regular expression to match againstsnapshotNameRegex
- The snapshot name regular expression to match againstIOException
- if a remote or network exception occurspublic List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException
Admin
listTableSnapshots
in interface Admin
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 occurspublic void deleteSnapshot(byte[] snapshotName) throws IOException
Admin
deleteSnapshot
in interface Admin
snapshotName
- name of the snapshotIOException
- if a remote or network exception occurspublic void deleteSnapshot(String snapshotName) throws IOException
Admin
deleteSnapshot
in interface Admin
snapshotName
- name of the snapshotIOException
- if a remote or network exception occurspublic void deleteSnapshots(String regex) throws IOException
Admin
deleteSnapshots
in interface Admin
regex
- The regular expression to match againstIOException
- if a remote or network exception occurspublic void deleteSnapshots(Pattern pattern) throws IOException
Admin
deleteSnapshots
in interface Admin
pattern
- pattern for names of the snapshot to matchIOException
- if a remote or network exception occursprivate void internalDeleteSnapshot(SnapshotDescription snapshot) throws IOException
IOException
public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException
Admin
deleteTableSnapshots
in interface Admin
tableNameRegex
- The table name regular expression to match againstsnapshotNameRegex
- The snapshot name regular expression to match againstIOException
- if a remote or network exception occurspublic void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException
Admin
deleteTableSnapshots
in interface Admin
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 occurspublic void setQuota(QuotaSettings quota) throws IOException
Admin
setQuota
in interface Admin
quota
- the quota settingsIOException
- if a remote or network exception occurspublic QuotaRetriever getQuotaRetriever(QuotaFilter filter) throws IOException
Admin
getQuotaRetriever
in interface Admin
filter
- the quota settings filterIOException
- if a remote or network exception occurspublic List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException
Admin
getQuota
in interface Admin
filter
- the quota settings filterIOException
- if a remote or network exception occursprivate <C extends RetryingCallable<V> & Closeable,V> V executeCallable(C callable) throws IOException
IOException
private static <C extends RetryingCallable<V> & Closeable,V> V executeCallable(C callable, RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout) throws IOException
IOException
public CoprocessorRpcChannel coprocessorService()
Admin
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);
coprocessorService
in interface Admin
public CoprocessorRpcChannel coprocessorService(ServerName serverName)
Admin
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);
coprocessorService
in interface Admin
serverName
- the server name to which the endpoint call is madepublic void updateConfiguration(ServerName server) throws IOException
Admin
updateConfiguration
in interface Admin
server
- : The server whose config needs to be updated.IOException
- if a remote or network exception occurspublic void updateConfiguration() throws IOException
Admin
updateConfiguration
in interface Admin
IOException
- if a remote or network exception occurspublic long getLastMajorCompactionTimestamp(TableName tableName) throws IOException
Admin
getLastMajorCompactionTimestamp
in interface Admin
tableName
- table to examineIOException
- if a remote or network exception occurspublic long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException
Admin
getLastMajorCompactionTimestampForRegion
in interface Admin
regionName
- region to examineIOException
- if a remote or network exception occurspublic void compact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException
compact
in interface Admin
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
public void compact(TableName tableName, CompactType compactType) throws IOException, InterruptedException
compact
in interface Admin
tableName
- table to compactcompactType
- CompactType
IOException
- if a remote or network exception occursInterruptedException
public void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException
majorCompact
in interface Admin
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
public void majorCompact(TableName tableName, CompactType compactType) throws IOException, InterruptedException
majorCompact
in interface Admin
tableName
- table to compactcompactType
- CompactType
IOException
- if a remote or network exception occursInterruptedException
public CompactionState getCompactionState(TableName tableName, CompactType compactType) throws IOException
getCompactionState
in interface Admin
tableName
- table to examinecompactType
- CompactType
IOException
- if a remote or network exception occurspublic List<SecurityCapability> getSecurityCapabilities() throws IOException
Admin
getSecurityCapabilities
in interface Admin
IOException
- if a remote or network exception occurspublic boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException
Admin
splitSwitch
in interface Admin
enabled
- enabled or notsynchronous
- If true
, it waits until current split() call, if outstanding,
to return.IOException
- if a remote or network exception occurspublic boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException
Admin
mergeSwitch
in interface Admin
enabled
- enabled or notsynchronous
- If true
, it waits until current merge() call, if outstanding,
to return.IOException
- if a remote or network exception occursprivate boolean splitOrMergeSwitch(boolean enabled, boolean synchronous, MasterSwitchType switchType) throws IOException
IOException
public boolean isSplitEnabled() throws IOException
Admin
isSplitEnabled
in interface Admin
true
if the switch is enabled, false
otherwise.IOException
- if a remote or network exception occurspublic boolean isMergeEnabled() throws IOException
Admin
isMergeEnabled
in interface Admin
true
if the switch is enabled, false
otherwise.IOException
- if a remote or network exception occursprivate RpcControllerFactory getRpcControllerFactory()
public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException
Admin
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.
addReplicationPeerAsync
in interface Admin
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 occurspublic Future<Void> removeReplicationPeerAsync(String peerId) throws IOException
Admin
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.
removeReplicationPeerAsync
in interface Admin
peerId
- a short name that identifies the peerIOException
- IOException if a remote or network exception occurspublic Future<Void> enableReplicationPeerAsync(String peerId) throws IOException
Admin
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.
enableReplicationPeerAsync
in interface Admin
peerId
- a short name that identifies the peerIOException
- IOException if a remote or network exception occurspublic Future<Void> disableReplicationPeerAsync(String peerId) throws IOException
Admin
disableReplicationPeerAsync
in interface Admin
peerId
- a short name that identifies the peerIOException
- IOException if a remote or network exception occurspublic ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException
Admin
getReplicationPeerConfig
in interface Admin
peerId
- a short name that identifies the peerIOException
- if a remote or network exception occurspublic Future<Void> updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig) throws IOException
Admin
updateReplicationPeerConfigAsync
in interface Admin
peerId
- a short name that identifies the peerpeerConfig
- new config for the replication peerIOException
- IOException if a remote or network exception occurspublic List<ReplicationPeerDescription> listReplicationPeers() throws IOException
Admin
listReplicationPeers
in interface Admin
IOException
- if a remote or network exception occurspublic List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException
Admin
listReplicationPeers
in interface Admin
pattern
- The compiled regular expression to match peer idIOException
- if a remote or network exception occurspublic void decommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException
Admin
decommissionRegionServers
in interface Admin
servers
- The list of servers to decommission.offload
- True to offload the regions from the decommissioned serversIOException
- if a remote or network exception occurspublic List<ServerName> listDecommissionedRegionServers() throws IOException
Admin
listDecommissionedRegionServers
in interface Admin
IOException
- if a remote or network exception occurspublic void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) throws IOException
Admin
recommissionRegionServer
in interface Admin
server
- The server to recommission.encodedRegionNames
- Regions to load onto the server.IOException
- if a remote or network exception occurspublic List<TableCFs> listReplicatedTableCFs() throws IOException
Admin
listReplicatedTableCFs
in interface Admin
IOException
- if a remote or network exception occurspublic void enableTableReplication(TableName tableName) throws IOException
Admin
enableTableReplication
in interface Admin
tableName
- name of the tableIOException
- if a remote or network exception occurspublic void disableTableReplication(TableName tableName) throws IOException
Admin
disableTableReplication
in interface Admin
tableName
- name of the tableIOException
- if a remote or network exception occursprivate void checkAndSyncTableDescToPeers(TableName tableName, byte[][] splits) throws IOException
tableName
- name of the table to sync to the peersplits
- table split keysIOException
- if a remote or network exception occursprivate void setTableRep(TableName tableName, boolean enableRep) throws IOException
tableName
- name of the tableenableRep
- is replication switch enable or disableIOException
- if a remote or network exception occurspublic void clearCompactionQueues(ServerName sn, Set<String> queues) throws IOException, InterruptedException
Admin
clearCompactionQueues
in interface Admin
sn
- the region server namequeues
- the set of queue nameIOException
- if a remote or network exception occursInterruptedException
public List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException
Admin
clearDeadServers
in interface Admin
servers
- list of dead region servers.IOException
- if a remote or network exception occurspublic void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) throws IOException
Admin
cloneTableSchema
in interface Admin
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 occurspublic boolean switchRpcThrottle(boolean enable) throws IOException
Admin
switchRpcThrottle
in interface Admin
enable
- Set to true
to enable, false
to disable.IOException
- if a remote or network exception occurspublic boolean isRpcThrottleEnabled() throws IOException
Admin
isRpcThrottleEnabled
in interface Admin
IOException
- if a remote or network exception occurspublic boolean exceedThrottleQuotaSwitch(boolean enable) throws IOException
Admin
exceedThrottleQuotaSwitch
in interface Admin
enable
- Set to true
to enable, false
to disable.IOException
- if a remote or network exception occurspublic Map<TableName,Long> getSpaceQuotaTableSizes() throws IOException
Admin
getSpaceQuotaTableSizes
in interface Admin
IOException
- if a remote or network exception occurspublic Map<TableName,SpaceQuotaSnapshot> getRegionServerSpaceQuotaSnapshots(ServerName serverName) throws IOException
Admin
SpaceQuotaSnapshotView
s observed by a RegionServer.getRegionServerSpaceQuotaSnapshots
in interface Admin
IOException
- if a remote or network exception occurspublic SpaceQuotaSnapshot getCurrentSpaceQuotaSnapshot(String namespace) throws IOException
Admin
namespace
or null if the Master has
no quota information on that namespace.getCurrentSpaceQuotaSnapshot
in interface Admin
IOException
- if a remote or network exception occurspublic SpaceQuotaSnapshot getCurrentSpaceQuotaSnapshot(TableName tableName) throws IOException
Admin
tableName
or null if the Master has
no quota information on that table.getCurrentSpaceQuotaSnapshot
in interface Admin
IOException
- if a remote or network exception occurspublic void grant(UserPermission userPermission, boolean mergeExistingPermissions) throws IOException
Admin
grant
in interface Admin
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 occurspublic void revoke(UserPermission userPermission) throws IOException
Admin
revoke
in interface Admin
userPermission
- user name and the specific permissionIOException
- if a remote or network exception occurspublic List<UserPermission> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) throws IOException
Admin
getUserPermissions
in interface Admin
getUserPermissionsRequest
- A request contains which user, global, namespace or table
permissions neededIOException
- if a remote or network exception occurspublic Future<Void> splitRegionAsync(byte[] regionName) throws IOException
Admin
splitRegionAsync
in interface Admin
regionName
- region to splitIOException
- if a remote or network exception occurspublic Future<Void> createTableAsync(TableDescriptor desc) throws IOException
Admin
createTableAsync
in interface Admin
desc
- table descriptor for tableIOException
- if a remote or network exception occurspublic List<Boolean> hasUserPermissions(String userName, List<Permission> permissions) throws IOException
Admin
hasUserPermissions
in interface Admin
userName
- the user namepermissions
- the specific permission listIOException
- if a remote or network exception occurspublic boolean snapshotCleanupSwitch(boolean on, boolean synchronous) throws IOException
Admin
snapshotCleanupSwitch
in interface Admin
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 occurspublic boolean isSnapshotCleanupEnabled() throws IOException
Admin
isSnapshotCleanupEnabled
in interface Admin
true
if the auto snapshot cleanup is enabled,
false
otherwise.IOException
- if a remote or network exception occurspublic List<OnlineLogRecord> getSlowLogResponses(@Nullable Set<ServerName> serverNames, LogQueryFilter logQueryFilter) throws IOException
Admin
getSlowLogResponses
in interface Admin
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 occursprivate List<OnlineLogRecord> getSlowLogResponseFromServer(ServerName serverName, LogQueryFilter logQueryFilter) throws IOException
IOException
private List<OnlineLogRecord> getSlowLogResponsesFromServer(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, LogQueryFilter logQueryFilter) throws IOException
IOException
public List<Boolean> clearSlowLogResponses(@Nullable Set<ServerName> serverNames) throws IOException
Admin
clearSlowLogResponses
in interface Admin
serverNames
- Set of Server names to clean slowlog responses fromIOException
- if a remote or network exception occursprivate Boolean clearSlowLogsResponses(ServerName serverName) throws IOException
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.