@InterfaceAudience.Public public interface Admin extends 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 |
addColumn(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Deprecated.
As of release 2.0.0.
This will be removed in HBase 3.0.0.
Use
addColumnFamily(TableName, ColumnFamilyDescriptor) . |
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.
|
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.
|
boolean |
balance()
Invoke the balancer.
|
boolean |
balance(boolean force)
Invoke the balancer.
|
default boolean |
balancer()
Deprecated.
Since 2.0.0. Will be removed in 3.0.0.
Use
balance() instead. |
default boolean |
balancer(boolean force)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0.
Use
balance(boolean) instead. |
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(byte[] snapshotName,
TableName tableName)
Deprecated.
since 2.3.0, will be removed in 3.0.0. Use
cloneSnapshot(String, TableName) instead. |
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 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.
|
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 serverName)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0.
Use
unassign(byte[], boolean) . |
void |
closeRegion(ServerName sn,
HRegionInfo hri)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0
(HBASE-18231).
Use
unassign(byte[], boolean) . |
void |
closeRegion(String regionname,
String serverName)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0.
Use
unassign(byte[], boolean) . |
boolean |
closeRegionWithEncodedRegionName(String encodedRegionName,
String serverName)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0.
Use
unassign(byte[], boolean) . |
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.
|
default void |
compactRegionServer(ServerName sn,
boolean major)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0. Use
compactRegionServer(ServerName) or
majorCompactRegionServer(ServerName) . |
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. |
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.
|
void |
deleteColumn(TableName tableName,
byte[] columnFamily)
Deprecated.
As of release 2.0.0.
This will be removed in HBase 3.0.0.
Use
deleteColumnFamily(TableName, byte[]) }. |
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(byte[] snapshotName)
Deprecated.
Since 2.2.0. Will be removed in 3.0.0. Use
deleteSnapshot(String) instead. |
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)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
Use
deleteSnapshots(Pattern) instead. |
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.
|
HTableDescriptor[] |
deleteTables(Pattern pattern)
Deprecated.
since 2.0 version and will be removed in 3.0 version
This is just a trivial helper method without any magic.
Consider using
listTableDescriptors(java.util.regex.Pattern)
and deleteTable(TableName) |
HTableDescriptor[] |
deleteTables(String regex)
Deprecated.
since 2.0 version and will be removed in 3.0 version
This is just a trivial helper method without any magic.
Consider using
listTableDescriptors(Pattern)
and deleteTable(TableName) |
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)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
Use
deleteTableSnapshots(Pattern, Pattern) instead. |
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.
|
HTableDescriptor[] |
disableTables(Pattern pattern)
Deprecated.
since 2.0 version and will be removed in 3.0 version
This is just a trivial helper method without any magic.
Consider using
listTableDescriptors(java.util.regex.Pattern)
and disableTable(org.apache.hadoop.hbase.TableName) |
HTableDescriptor[] |
disableTables(String regex)
Deprecated.
since 2.0 version and will be removed in 3.0 version
This is just a trivial helper method without any magic.
Consider using
listTableDescriptors(Pattern)
and disableTable(org.apache.hadoop.hbase.TableName) |
default boolean |
enableCatalogJanitor(boolean enable)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
catalogJanitorSwitch(boolean) }
instead. |
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.
|
HTableDescriptor[] |
enableTables(Pattern pattern)
Deprecated.
since 2.0 version and will be removed in 3.0 version
This is just a trivial helper method without any magic.
Consider using
listTableDescriptors(java.util.regex.Pattern)
and enableTable(org.apache.hadoop.hbase.TableName) |
HTableDescriptor[] |
enableTables(String regex)
Deprecated.
since 2.0 version and will be removed in 3.0 version
This is just a trivial helper method without any magic.
Consider using
listTableDescriptors(Pattern)
and enableTable(org.apache.hadoop.hbase.TableName) |
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.
|
default byte[] |
execProcedureWithRet(String signature,
String instance,
Map<String,String> props)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
execProcedureWithReturn(String, String, Map) } instead. |
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 |
flushRegion(byte[] regionName)
Flush an individual region.
|
void |
flushRegionServer(ServerName serverName)
Flush all regions on the region server.
|
Pair<Integer,Integer> |
getAlterStatus(byte[] tableName)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future
on an operation.
|
Pair<Integer,Integer> |
getAlterStatus(TableName tableName)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future
on an operation.
|
default Collection<ServerName> |
getBackupMasters() |
default ClusterMetrics |
getClusterMetrics()
Get whole cluster metrics, containing status about:
|
ClusterMetrics |
getClusterMetrics(EnumSet<ClusterMetrics.Option> options)
Get cluster status with a set of
ClusterMetrics.Option to get desired status. |
default ClusterStatus |
getClusterStatus()
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
getClusterMetrics() |
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() |
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.
|
default ServerName |
getMaster() |
default List<String> |
getMasterCoprocessorNames()
Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().
|
default String[] |
getMasterCoprocessors()
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
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.
|
List<HRegionInfo> |
getOnlineRegions(ServerName sn)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0
(HBASE-17980).
Use
getRegions(ServerName sn) . |
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.
|
QuotaRetriever |
getQuotaRetriever(QuotaFilter filter)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
getQuota(QuotaFilter) . |
default 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() |
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.
|
List<SecurityCapability> |
getSecurityCapabilities()
Return the set of supported security capabilities.
|
List<OnlineLogRecord> |
getSlowLogResponses(Set<ServerName> serverNames,
LogQueryFilter logQueryFilter)
Retrieves online slow/large RPC logs from the provided list of
RegionServers
|
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)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
Use
getDescriptor(TableName) . |
HTableDescriptor[] |
getTableDescriptors(List<String> names)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
listTableDescriptors(List) |
HTableDescriptor[] |
getTableDescriptorsByTableName(List<TableName> tableNames)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
listTableDescriptors(List) |
List<HRegionInfo> |
getTableRegions(TableName tableName)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0
(HBASE-17980).
Use
getRegions(TableName) . |
List<UserPermission> |
getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest)
Get the global/namespace/table permissions for user
|
void |
grant(UserPermission userPermission,
boolean mergeExistingPermissions)
Grants user specific permissions
|
default List<Boolean> |
hasUserPermissions(List<Permission> permissions)
Check if call user has specific permissions
|
List<Boolean> |
hasUserPermissions(String userName,
List<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 |
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.
|
default boolean |
isSplitOrMergeEnabled(MasterSwitchType switchType)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
isSplitEnabled() or isMergeEnabled() instead. |
boolean |
isTableAvailable(TableName tableName) |
boolean |
isTableAvailable(TableName tableName,
byte[][] splitKeys)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
isTableAvailable(TableName) |
boolean |
isTableDisabled(TableName tableName) |
boolean |
isTableEnabled(TableName tableName) |
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<SnapshotDescription> |
listSnapshots()
List completed snapshots.
|
List<SnapshotDescription> |
listSnapshots(Pattern pattern)
List all the completed snapshots matching the given pattern.
|
List<SnapshotDescription> |
listSnapshots(String regex)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
Use
listSnapshots(Pattern) instead. |
List<TableDescriptor> |
listTableDescriptors()
List all the userspace 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.
|
HTableDescriptor[] |
listTableDescriptorsByNamespace(String name)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
listTableDescriptorsByNamespace(byte[]) |
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[] |
listTableNames(String regex)
Deprecated.
since 2.0 version and will be removed in 3.0 version. Use
listTableNames(Pattern) instead. |
TableName[] |
listTableNames(String regex,
boolean includeSysTables)
Deprecated.
since 2.0 version and will be removed in 3.0 version. Use
listTableNames(Pattern, boolean) instead. |
TableName[] |
listTableNamesByNamespace(String name)
Get list of table names by namespace.
|
HTableDescriptor[] |
listTables()
Deprecated.
since 2.0 version and will be removed in 3.0 version.
Use
listTableDescriptors() . |
HTableDescriptor[] |
listTables(Pattern pattern)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
Use
listTableDescriptors(java.util.regex.Pattern) . |
HTableDescriptor[] |
listTables(Pattern pattern,
boolean includeSysTables)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
Use
listTableDescriptors(java.util.regex.Pattern, boolean) . |
HTableDescriptor[] |
listTables(String regex)
Deprecated.
since 2.0 version and will be removed in 3.0 version. Use
listTableDescriptors(Pattern) instead. |
HTableDescriptor[] |
listTables(String regex,
boolean includeSysTables)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
Use
listTableDescriptors(Pattern, boolean) . |
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)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
Use
listTableSnapshots(Pattern, Pattern) instead. |
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
mergeRegionsAsync(byte[], byte[], boolean) instead. |
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 |
modifyColumn(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Deprecated.
As of release 2.0.0.
This will be removed in HBase 3.0.0.
Use
modifyColumnFamily(TableName, ColumnFamilyDescriptor) . |
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 |
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.
|
default void |
modifyTable(TableName tableName,
TableDescriptor td)
Deprecated.
since 2.0 version and will be removed in 3.0 version. use
modifyTable(TableDescriptor) |
Future<Void> |
modifyTableAsync(TableDescriptor td)
Modify an existing table, more IRB (ruby) friendly version.
|
default Future<Void> |
modifyTableAsync(TableName tableName,
TableDescriptor td)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
modifyTableAsync(TableDescriptor) |
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
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.
|
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.
|
default void |
restoreSnapshot(byte[] snapshotName)
Deprecated.
since 2.3.0, will be removed in 3.0.0. Use
restoreSnapshot(String)
instead. |
default void |
restoreSnapshot(byte[] snapshotName,
boolean takeFailSafeSnapshot)
Deprecated.
since 2.3.0, will be removed in 3.0.0. Use
restoreSnapshot(String, boolean) instead. |
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.
|
Future<Void> |
restoreSnapshotAsync(String snapshotName)
Deprecated.
since 2.3.0, will be removed in 3.0.0. The implementation does not take care of the
failsafe property, so do not use it any more.
|
void |
revoke(UserPermission userPermission)
Revokes user specific permissions
|
void |
rollWALWriter(ServerName serverName)
Roll the log writer.
|
int |
runCatalogJanitor()
Ask for a scan of the catalog table.
|
default int |
runCatalogScan()
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
runCatalogJanitor() }
instead. |
boolean |
runCleanerChore()
Ask for cleaner chore to run.
|
default boolean |
setBalancerRunning(boolean on,
boolean synchronous)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0.
Use
balancerSwitch(boolean, boolean) instead. |
default boolean |
setCleanerChoreRunning(boolean on)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
cleanerChoreSwitch(boolean) }
instead. |
default boolean |
setNormalizerRunning(boolean on)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
normalizerSwitch(boolean) }
instead. |
void |
setQuota(QuotaSettings quota)
Apply the new quota settings.
|
default boolean[] |
setSplitOrMergeEnabled(boolean enabled,
boolean synchronous,
MasterSwitchType... switchTypes)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
splitSwitch(boolean, boolean)
or mergeSwitch(boolean, boolean) instead. |
void |
shutdown()
Shuts down the HBase cluster.
|
default void |
snapshot(byte[] snapshotName,
TableName tableName)
Deprecated.
since 2.3.0, will be removed in 3.0.0. Use
snapshot(String, TableName)
instead. |
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) 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.
|
void |
splitRegion(byte[] regionName)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0.
Use
splitRegionAsync(byte[], byte[]) . |
void |
splitRegion(byte[] regionName,
byte[] splitPoint)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0.
Use
splitRegionAsync(byte[], byte[]) . |
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) |
default void |
takeSnapshotAsync(SnapshotDescription snapshot)
Deprecated.
Since 2.0.0. Will be removed in 3.0.0. Use
snapshotAsync(SnapshotDescription) instead. |
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,
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.
|
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.
|
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)
Abortable
boolean isAborted()
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 occurs@Deprecated HTableDescriptor[] listTables() throws IOException
listTableDescriptors()
.IOException
- if a remote or network exception occurslistTableDescriptors()
List<TableDescriptor> listTableDescriptors() throws IOException
IOException
- if a remote or network exception occurs@Deprecated HTableDescriptor[] listTables(Pattern pattern) throws IOException
listTableDescriptors(java.util.regex.Pattern)
.pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occurslistTables()
,
listTableDescriptors(Pattern)
default List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occurslistTables()
@Deprecated HTableDescriptor[] listTables(String regex) throws IOException
listTableDescriptors(Pattern)
instead.regex
- The regular expression to match againstIOException
- if a remote or network exception occurslistTableDescriptors(Pattern)
@Deprecated HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables) throws IOException
listTableDescriptors(java.util.regex.Pattern, boolean)
.pattern
- The compiled regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occurslistTables()
,
listTableDescriptors(java.util.regex.Pattern, boolean)
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 occurslistTables()
@Deprecated HTableDescriptor[] listTables(String regex, boolean includeSysTables) throws IOException
listTableDescriptors(Pattern, boolean)
.regex
- The regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occurslistTables(java.util.regex.Pattern, boolean)
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 occurs@Deprecated TableName[] listTableNames(String regex) throws IOException
listTableNames(Pattern)
instead.regex
- 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 occurs@Deprecated TableName[] listTableNames(String regex, boolean includeSysTables) throws IOException
listTableNames(Pattern, boolean)
instead.regex
- The regular expression to match againstincludeSysTables
- false
to match only against userspace tablesIOException
- if a remote or network exception occurs@Deprecated HTableDescriptor getTableDescriptor(TableName tableName) throws TableNotFoundException, IOException
getDescriptor(TableName)
.tableName
- as a TableName
TableNotFoundException
IOException
- if a remote or network exception occursTableDescriptor getDescriptor(TableName tableName) throws TableNotFoundException, IOException
tableName
- as a TableName
TableNotFoundException
IOException
- 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 createIllegalArgumentException
- if the table name is reservedIOException
- 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).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 occurs@Deprecated HTableDescriptor[] deleteTables(String regex) throws IOException
listTableDescriptors(Pattern)
and deleteTable(TableName)
listTableDescriptors(Pattern)
and deleteTable(org.apache.hadoop.hbase.TableName)
regex
- The regular expression to match table names againstIOException
- if a remote or network exception occursdeleteTables(java.util.regex.Pattern)
,
deleteTable(org.apache.hadoop.hbase.TableName)
@Deprecated HTableDescriptor[] deleteTables(Pattern pattern) throws IOException
listTableDescriptors(java.util.regex.Pattern)
and deleteTable(TableName)
listTableDescriptors(java.util.regex.Pattern)
and
deleteTable(org.apache.hadoop.hbase.TableName)
pattern
- The pattern to match table names againstIOException
- 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
- if a remote or network exception occurs 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 occurs@Deprecated HTableDescriptor[] enableTables(String regex) throws IOException
listTableDescriptors(Pattern)
and enableTable(org.apache.hadoop.hbase.TableName)
listTableDescriptors(Pattern)
and enableTable(org.apache.hadoop.hbase.TableName)
regex
- The regular expression to match table names againstIOException
- if a remote or network exception occursenableTables(java.util.regex.Pattern)
,
enableTable(org.apache.hadoop.hbase.TableName)
@Deprecated HTableDescriptor[] enableTables(Pattern pattern) throws IOException
listTableDescriptors(java.util.regex.Pattern)
and enableTable(org.apache.hadoop.hbase.TableName)
listTableDescriptors(java.util.regex.Pattern)
and
enableTable(org.apache.hadoop.hbase.TableName)
pattern
- The pattern to match table names againstIOException
- 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.tableName
- IOException
- There could be couple types of IOException TableNotFoundException means the
table doesn't exist. TableNotEnabledException means the table isn't in enabled state.@Deprecated HTableDescriptor[] disableTables(String regex) throws IOException
listTableDescriptors(Pattern)
and disableTable(org.apache.hadoop.hbase.TableName)
listTableDescriptors(Pattern)
and disableTable(org.apache.hadoop.hbase.TableName)
regex
- The regular expression to match table names againstIOException
- if a remote or network exception occursdisableTables(java.util.regex.Pattern)
,
disableTable(org.apache.hadoop.hbase.TableName)
@Deprecated HTableDescriptor[] disableTables(Pattern pattern) throws IOException
listTableDescriptors(java.util.regex.Pattern)
and disableTable(org.apache.hadoop.hbase.TableName)
listTableDescriptors(java.util.regex.Pattern)
and
disableTable(org.apache.hadoop.hbase.TableName)
pattern
- The pattern to match table names againstIOException
- if a remote or network exception occursboolean 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 occurs@Deprecated boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException
isTableAvailable(TableName)
false
.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 occurs@Deprecated Pair<Integer,Integer> getAlterStatus(TableName tableName) throws IOException
alter
(a.k.a modify
) command - indicates how
many regions have received the updated schema Asynchronous operation.tableName
- TableName instanceIOException
- if a remote or network exception occurs@Deprecated Pair<Integer,Integer> getAlterStatus(byte[] tableName) throws IOException
alter
(a.k.a modify
) command - indicates how many
regions have received the updated schema Asynchronous operation.tableName
- name of the table to get the status ofIOException
- if a remote or network exception occurs@Deprecated default void addColumn(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
addColumnFamily(TableName, ColumnFamilyDescriptor)
.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 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 occurs@Deprecated void deleteColumn(TableName tableName, byte[] columnFamily) throws IOException
deleteColumnFamily(TableName, byte[])
}.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 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 occurs@Deprecated default void modifyColumn(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
modifyColumnFamily(TableName, ColumnFamilyDescriptor)
.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 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 occurs@Deprecated void closeRegion(String regionname, String serverName) throws IOException
unassign(byte[], boolean)
.unassign(byte[], boolean)
to unassign the region. For expert-admins.regionname
- region name to closeserverName
- Deprecated. Not used.IOException
- if a remote or network exception occurs@Deprecated void closeRegion(byte[] regionname, String serverName) throws IOException
unassign(byte[], boolean)
.unassign(byte[], boolean)
to unassign the region. For expert-admins.regionname
- region name to closeserverName
- Deprecated. Not used.IOException
- if a remote or network exception occurs@Deprecated boolean closeRegionWithEncodedRegionName(String encodedRegionName, String serverName) throws IOException
unassign(byte[], boolean)
.unassign(byte[], boolean)
to unassign the region. For expert-admins.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
.serverName
- Deprecated. Not used.true
always.IOException
- if a remote or network exception occurs@Deprecated void closeRegion(ServerName sn, HRegionInfo hri) throws IOException
unassign(byte[], boolean)
.unassign(byte[], boolean)
to unassign the region. For expert-admins.sn
- Deprecated. Not used.IOException
- if a remote or network exception occurs@Deprecated List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException
getRegions(ServerName sn)
.IOException
- 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 flushRegion(byte[] regionName) throws IOException
regionName
- region to flushIOException
- 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
@Deprecated default void compactRegionServer(ServerName sn, boolean major) throws IOException, InterruptedException
compactRegionServer(ServerName)
or
majorCompactRegionServer(ServerName)
.sn
- the region server namemajor
- if it's major compactionIOException
- 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
void 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
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. 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, boolean force) throws IOException
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 occurs@Deprecated default boolean setBalancerRunning(boolean on, boolean synchronous) throws IOException
balancerSwitch(boolean, boolean)
instead.synchronous
- If true
, it waits until current balance() call, if
outstanding, to return.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 occurs@Deprecated default boolean balancer() throws IOException
balance()
instead.true
if balancer ran, false
otherwise.IOException
- if a remote or network exception occursboolean balance() throws IOException
true
if balancer ran, false
otherwise.IOException
- if a remote or network exception occurs@Deprecated default boolean balancer(boolean force) throws IOException
balance(boolean)
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 balance(boolean force) throws IOException
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 occursboolean normalize() throws IOException
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 occurs@Deprecated default boolean setNormalizerRunning(boolean on) throws IOException
normalizerSwitch(boolean)
}
instead.IOException
- if a remote or network exception occursboolean normalizerSwitch(boolean on) throws IOException
IOException
- if a remote or network exception occurs@Deprecated default boolean enableCatalogJanitor(boolean enable) throws IOException
catalogJanitorSwitch(boolean)
}
instead.enable
- if true
enables the catalog janitorIOException
- 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 occurs@Deprecated default int runCatalogScan() throws IOException
runCatalogJanitor()
}
instead.IOException
- 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 occurs@Deprecated default boolean setCleanerChoreRunning(boolean on) throws IOException
cleanerChoreSwitch(boolean)
}
instead.on
- if true
enables the cleaner choreIOException
- 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 void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
mergeRegionsAsync(byte[], 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 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 occurs@Deprecated void splitRegion(byte[] regionName) throws IOException
splitRegionAsync(byte[], byte[])
.regionName
- region 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 occurs@Deprecated void splitRegion(byte[] regionName, byte[] splitPoint) throws IOException
splitRegionAsync(byte[], byte[])
.regionName
- region 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 occurs@Deprecated default void modifyTable(TableName tableName, TableDescriptor td) throws IOException
modifyTable(TableDescriptor)
tableName
- name of table.td
- modified description of the tableIOException
- 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 occurs@Deprecated default Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td) throws IOException
modifyTableAsync(TableDescriptor)
tableName
- name of table.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 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 occurs@Deprecated default ClusterStatus getClusterStatus() throws IOException
getClusterMetrics()
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 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<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 occursdefault List<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
IOException
- 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 occurs@Deprecated HTableDescriptor[] listTableDescriptorsByNamespace(String name) throws IOException
listTableDescriptorsByNamespace(byte[])
name
- namespace nameIOException
- 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 occurs@Deprecated List<HRegionInfo> getTableRegions(TableName tableName) throws IOException
getRegions(TableName)
.tableName
- the name of the tableHRegionInfo
.IOException
- 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() throws IOException
close
in interface AutoCloseable
close
in interface Closeable
IOException
@Deprecated HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames) throws IOException
listTableDescriptors(List)
tableNames
- List of table namesIOException
- if a remote or network exception occursList<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException
tableNames
- List of table namesIOException
- if a remote or network exception occurs@Deprecated HTableDescriptor[] getTableDescriptors(List<String> names) throws IOException
listTableDescriptors(List)
names
- 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
@Deprecated default String[] getMasterCoprocessors() throws IOException
getMasterCoprocessorNames()
IOException
- if a remote or network exception occursClusterMetrics.getMasterCoprocessorNames()
default List<String> getMasterCoprocessorNames() throws IOException
IOException
- if a remote or network exception occursClusterMetrics.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 incorrectly@Deprecated default void snapshot(byte[] snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
snapshot(String, TableName)
instead.SnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
HBase.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)
or
snapshot(byte[], 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 incorrectly@Deprecated default void takeSnapshotAsync(SnapshotDescription snapshot) throws IOException, SnapshotCreationException
snapshotAsync(SnapshotDescription)
instead.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 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
unknown@Deprecated default void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException
restoreSnapshot(String)
instead.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 incorrectlyvoid 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 incorrectly@Deprecated Future<Void> restoreSnapshotAsync(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 restored@Deprecated default void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
restoreSnapshot(String, boolean)
instead.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 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 incorrectly@Deprecated default void cloneSnapshot(byte[] snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException
cloneSnapshot(String, TableName)
instead.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) 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) 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 existsFuture<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 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 occurs@Deprecated default byte[] execProcedureWithRet(String signature, String instance, Map<String,String> props) throws IOException
execProcedureWithReturn(String, String, Map)
} instead.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 occurs@Deprecated List<SnapshotDescription> listSnapshots(String regex) throws IOException
listSnapshots(Pattern)
instead.regex
- The regular expression to match againstIOException
- if a remote or network exception occursList<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException
pattern
- The compiled regular expression to match againstIOException
- if a remote or network exception occurs@Deprecated List<SnapshotDescription> listTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException
listTableSnapshots(Pattern, Pattern)
instead.tableNameRegex
- The table name regular expression to match againstsnapshotNameRegex
- The snapshot name 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 occurs@Deprecated void deleteSnapshot(byte[] snapshotName) throws IOException
deleteSnapshot(String)
instead.snapshotName
- name of the snapshotIOException
- if a remote or network exception occursvoid deleteSnapshot(String snapshotName) throws IOException
snapshotName
- name of the snapshotIOException
- if a remote or network exception occurs@Deprecated void deleteSnapshots(String regex) throws IOException
deleteSnapshots(Pattern)
instead.regex
- The regular expression to match againstIOException
- 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 occurs@Deprecated void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException
deleteTableSnapshots(Pattern, Pattern)
instead.tableNameRegex
- The table name regular expression to match againstsnapshotNameRegex
- The snapshot name regular expression to match againstIOException
- 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 occurs@Deprecated QuotaRetriever getQuotaRetriever(QuotaFilter filter) throws IOException
getQuota(QuotaFilter)
.filter
- the quota settings filterIOException
- if a remote or network exception occursList<QuotaSettings> getQuota(QuotaFilter filter) throws IOException
filter
- the quota settings filterIOException
- if a remote or network exception occursCoprocessorRpcChannel coprocessorService()
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);
CoprocessorRpcChannel coprocessorService(ServerName serverName)
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 occursdefault int getMasterInfoPort() throws IOException
IOException
- if a remote or network exception occursList<SecurityCapability> getSecurityCapabilities() throws IOException
IOException
- if a remote or network exception occursUnsupportedOperationException
@Deprecated default boolean[] setSplitOrMergeEnabled(boolean enabled, boolean synchronous, MasterSwitchType... switchTypes) throws IOException
splitSwitch(boolean, boolean)
or mergeSwitch(boolean, boolean)
instead.enabled
- enabled or notsynchronous
- If true
, it waits until current split() call, if outstanding,
to return.switchTypes
- switchType list MasterSwitchType
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 occurs@Deprecated default boolean isSplitOrMergeEnabled(MasterSwitchType switchType) throws IOException
isSplitEnabled()
or isMergeEnabled()
instead.true
if the switch is enabled, false
otherwise.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 occursvoid 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
- if a remote or network exception occursList<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(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(UserPermission userPermission) throws IOException
userPermission
- user name and the specific permissionIOException
- if a remote or network exception occursList<UserPermission> getUserPermissions(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<Permission> permissions) throws IOException
userName
- the user namepermissions
- the specific permission listIOException
- if a remote or network exception occursdefault List<Boolean> hasUserPermissions(List<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 occursList<OnlineLogRecord> getSlowLogResponses(Set<ServerName> serverNames, LogQueryFilter logQueryFilter) throws IOException
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 occursCopyright © 2007–2020 The Apache Software Foundation. All rights reserved.