@InterfaceAudience.Private @InterfaceStability.Evolving public class HBaseAdmin extends Object implements Admin
Connection.getAdmin() to obtain an instance of Admin instead of constructing
an HBaseAdmin directly.
Connection should be an unmanaged connection obtained via
ConnectionFactory.createConnection(Configuration)
ConnectionFactory,
Connection,
Admin| Modifier and Type | Class and Description |
|---|---|
private static class |
HBaseAdmin.AbortProcedureFuture |
private static class |
HBaseAdmin.AddColumnFamilyFuture |
private static class |
HBaseAdmin.CreateTableFuture |
private static class |
HBaseAdmin.DeleteColumnFamilyFuture |
private static class |
HBaseAdmin.DeleteTableFuture |
private static class |
HBaseAdmin.DisableTableFuture |
private static class |
HBaseAdmin.EnableTableFuture |
private static class |
HBaseAdmin.MergeTableRegionsFuture |
private static class |
HBaseAdmin.ModifyColumnFamilyFuture |
private static class |
HBaseAdmin.ModifyTableFuture |
protected static class |
HBaseAdmin.NamespaceFuture |
protected static class |
HBaseAdmin.ProcedureFuture<V>
Future that waits on a procedure result.
|
private static class |
HBaseAdmin.RestoreSnapshotFuture |
private static class |
HBaseAdmin.SplitTableRegionFuture |
protected static class |
HBaseAdmin.TableFuture<V> |
private static class |
HBaseAdmin.ThrowableAbortable
Simple
Abortable, throwing RuntimeException on abort. |
private static class |
HBaseAdmin.TruncateTableFuture |
| Modifier and Type | Field and Description |
|---|---|
private boolean |
aborted |
private org.apache.hadoop.conf.Configuration |
conf |
private ClusterConnection |
connection |
private int |
getProcedureTimeout |
private static org.slf4j.Logger |
LOG |
private NonceGenerator |
ng |
private int |
numRetries |
private int |
operationTimeout |
private long |
pause |
private RpcRetryingCallerFactory |
rpcCallerFactory |
private RpcControllerFactory |
rpcControllerFactory |
private int |
rpcTimeout |
private int |
syncWaitTimeout |
| Constructor and Description |
|---|
HBaseAdmin(ClusterConnection connection) |
| Modifier and Type | Method and Description |
|---|---|
void |
abort(String why,
Throwable e)
Abort the server or client.
|
boolean |
abortProcedure(long procId,
boolean mayInterruptIfRunning)
Abort a procedure.
|
Future<Boolean> |
abortProcedureAsync(long procId,
boolean mayInterruptIfRunning)
Abort a procedure but does not block and wait for completion.
|
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.
|
void |
addReplicationPeer(String peerId,
ReplicationPeerConfig peerConfig,
boolean enabled)
Add a new replication peer for replicating data to slave cluster.
|
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.
|
private org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse |
asyncSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) |
static void |
available(org.apache.hadoop.conf.Configuration conf)
Is HBase available? Throw an exception if not.
|
boolean |
balance()
Invoke the balancer.
|
boolean |
balance(boolean force)
Invoke the balancer.
|
boolean |
balancerSwitch(boolean on,
boolean synchronous)
Turn the load balancer on or off.
|
boolean |
catalogJanitorSwitch(boolean enable)
Enable/Disable the catalog janitor/
|
private void |
checkAndSyncTableDescToPeers(TableName tableName,
byte[][] splits)
Connect to peer and check the table descriptor on peer:
Create the same table on peer when not exist.
Throw an exception if the table already has replication enabled on any of the column
families.
Throw an exception if the table exists on peer cluster but descriptors are not same.
|
private TableName |
checkTableExists(TableName tableName)
Check if table exists or not
|
boolean |
cleanerChoreSwitch(boolean on)
Enable/Disable the cleaner chore.
|
private CacheEvictionStats |
clearBlockCache(ServerName sn,
List<RegionInfo> hris) |
CacheEvictionStats |
clearBlockCache(TableName tableName)
Clear all the blocks corresponding to this table from BlockCache.
|
void |
clearCompactionQueues(ServerName sn,
Set<String> queues)
Clear compacting queues on a regionserver.
|
List<ServerName> |
clearDeadServers(List<ServerName> servers)
Clear dead region servers from master.
|
void |
cloneSnapshot(byte[] snapshotName,
TableName tableName)
Create a new table by cloning the snapshot content.
|
void |
cloneSnapshot(String snapshotName,
TableName tableName)
Create a new table by cloning the snapshot content.
|
void |
cloneSnapshot(String snapshotName,
TableName tableName,
boolean restoreAcl)
Create a new table by cloning the snapshot content.
|
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.
|
void |
close() |
void |
closeRegion(byte[] regionName,
String unused)
Deprecated.
|
void |
closeRegion(ServerName unused,
HRegionInfo hri)
Deprecated.
|
void |
closeRegion(String regionName,
String unused)
Deprecated.
|
boolean |
closeRegionWithEncodedRegionName(String encodedRegionName,
String unused)
Deprecated.
|
private void |
compact(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin,
RegionInfo hri,
boolean major,
byte[] family) |
void |
compact(TableName tableName)
Compact a table.
|
void |
compact(TableName tableName,
byte[] columnFamily)
Compact a column family within a table.
|
private void |
compact(TableName tableName,
byte[] columnFamily,
boolean major,
CompactType compactType)
Compact a table.
|
void |
compact(TableName tableName,
byte[] columnFamily,
CompactType compactType)
Compact a column family within a table.
|
void |
compact(TableName tableName,
CompactType compactType)
Compact a table.
|
void |
compactRegion(byte[] regionName)
Compact an individual region.
|
void |
compactRegion(byte[] regionName,
byte[] columnFamily)
Compact a column family within a region.
|
private void |
compactRegion(byte[] regionName,
byte[] columnFamily,
boolean major)
Compact an individual region.
|
void |
compactRegionServer(ServerName serverName)
Compact all regions on the region server.
|
CoprocessorRpcChannel |
coprocessorService()
Creates and returns a
RpcChannel instance connected to the active
master. |
CoprocessorRpcChannel |
coprocessorService(ServerName serverName)
Creates and returns a
RpcChannel instance
connected to the passed region server. |
void |
createNamespace(NamespaceDescriptor descriptor)
Create a new namespace.
|
Future<Void> |
createNamespaceAsync(NamespaceDescriptor descriptor)
Create a new namespace.
|
void |
createTable(TableDescriptor desc)
Creates a new table.
|
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,
byte[][] splitKeys)
Creates a new table but does not block and wait for it to come online.
|
void |
decommissionRegionServers(List<ServerName> servers,
boolean offload)
Mark region server(s) as decommissioned to prevent additional regions from getting
assigned to them.
|
void |
deleteColumn(TableName tableName,
byte[] columnFamily)
Deprecated.
Since 2.0. Will be removed in 3.0. Use
deleteColumnFamily(TableName, byte[]) instead. |
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.
|
void |
deleteNamespace(String name)
Delete an existing namespace.
|
Future<Void> |
deleteNamespaceAsync(String name)
Delete an existing namespace.
|
void |
deleteSnapshot(byte[] snapshotName)
Delete an existing snapshot.
|
void |
deleteSnapshot(String snapshotName)
Delete an existing snapshot.
|
void |
deleteSnapshots(Pattern pattern)
Delete existing snapshots whose names match the pattern passed.
|
void |
deleteSnapshots(String regex)
Delete existing snapshots whose names match the pattern passed.
|
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)
Delete tables matching the passed in pattern and wait on completion.
|
HTableDescriptor[] |
deleteTables(String regex)
Deletes tables matching the passed in pattern and wait on completion.
|
void |
deleteTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern)
Delete all existing snapshots matching the given table name regular expression and snapshot
name regular expression.
|
void |
deleteTableSnapshots(String tableNameRegex,
String snapshotNameRegex)
Delete all existing snapshots matching the given table name regular expression and snapshot
name regular expression.
|
void |
disableReplicationPeer(String peerId)
Stop the replication stream to the specified peer.
|
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)
Disable tables matching the passed in pattern and wait on completion.
|
HTableDescriptor[] |
disableTables(String regex)
Disable tables matching the passed in pattern and wait on completion.
|
void |
enableReplicationPeer(String peerId)
Restart the replication stream to the specified peer.
|
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)
Enable tables matching the passed in pattern and wait on completion.
|
HTableDescriptor[] |
enableTables(String regex)
Enable tables matching the passed in pattern and wait on completion.
|
void |
execProcedure(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster.
|
byte[] |
execProcedureWithReturn(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster.
|
private <C extends RetryingCallable<V> & Closeable,V> |
executeCallable(C callable) |
private static <C extends RetryingCallable<V> & Closeable,V> |
executeCallable(C callable,
RpcRetryingCallerFactory rpcCallerFactory,
int operationTimeout,
int rpcTimeout) |
private void |
flush(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin,
RegionInfo info) |
void |
flush(TableName tableName)
Flush a table.
|
void |
flushRegion(byte[] regionName)
Flush an individual region.
|
void |
flushRegionServer(ServerName serverName)
Flush all regions on the region server.
|
private static <T> T |
get(Future<T> future,
long timeout,
TimeUnit units)
Do a get with a timeout against the passed in
future. |
Pair<Integer,Integer> |
getAlterStatus(byte[] tableName)
Get the status of
alter (a.k.a modify) command - indicates how many
regions have received the updated schema Asynchronous operation. |
Pair<Integer,Integer> |
getAlterStatus(TableName tableName)
Get the status of an
alter (a.k.a modify) command - indicates how
many regions have received the updated schema Asynchronous operation. |
ClusterMetrics |
getClusterMetrics(EnumSet<ClusterMetrics.Option> options)
Get cluster status with a set of
ClusterMetrics.Option to get desired status. |
CompactionState |
getCompactionState(TableName tableName)
Get the current compaction state of a table.
|
CompactionState |
getCompactionState(TableName tableName,
CompactType compactType)
Get the current compaction state of a table.
|
CompactionState |
getCompactionStateForRegion(byte[] regionName)
Get the current compaction state of region.
|
org.apache.hadoop.conf.Configuration |
getConfiguration() |
Connection |
getConnection() |
TableDescriptor |
getDescriptor(TableName tableName)
Get a table descriptor.
|
(package private) static HTableDescriptor |
getHTableDescriptor(TableName tableName,
Connection connection,
RpcRetryingCallerFactory rpcCallerFactory,
RpcControllerFactory rpcControllerFactory,
int operationTimeout,
int rpcTimeout)
Deprecated.
since 2.0 version and will be removed in 3.0 version.
use
getTableDescriptor(TableName,
Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int) |
long |
getLastMajorCompactionTimestamp(TableName tableName)
Get the timestamp of the last major compaction for the passed table
The timestamp of the oldest HFile resulting from a major compaction of that table,
or 0 if no such HFile could be found.
|
long |
getLastMajorCompactionTimestampForRegion(byte[] regionName)
Get the timestamp of the last major compaction for the passed region.
|
String |
getLocks()
Get locks.
|
NamespaceDescriptor |
getNamespaceDescriptor(String name)
Get a namespace descriptor by name.
|
List<HRegionInfo> |
getOnlineRegions(ServerName sn)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0
Use
getRegions(ServerName). |
int |
getOperationTimeout() |
private long |
getPauseTime(int tries) |
String |
getProcedures()
Get procedures.
|
List<QuotaSettings> |
getQuota(QuotaFilter filter)
List the quotas based on the filter.
|
QuotaRetriever |
getQuotaRetriever(QuotaFilter filter)
Return a QuotaRetriever to list the quotas based on the filter.
|
(package private) Pair<RegionInfo,ServerName> |
getRegion(byte[] regionName) |
List<RegionMetrics> |
getRegionMetrics(ServerName serverName,
TableName tableName)
Get
RegionMetrics of all regions hosted on a regionserver for a table. |
private byte[] |
getRegionName(byte[] regionNameOrEncodedRegionName)
If the input is a region name, it is returned as is.
|
List<RegionInfo> |
getRegions(ServerName sn)
Get all the online regions on a region server.
|
List<RegionInfo> |
getRegions(TableName tableName)
Get the regions of a given table.
|
ReplicationPeerConfig |
getReplicationPeerConfig(String peerId)
Returns the configured ReplicationPeerConfig for the specified peer.
|
private RpcControllerFactory |
getRpcControllerFactory() |
List<SecurityCapability> |
getSecurityCapabilities()
Return the set of supported security capabilities.
|
HTableDescriptor |
getTableDescriptor(TableName tableName)
Get a table descriptor.
|
(package private) static TableDescriptor |
getTableDescriptor(TableName tableName,
Connection connection,
RpcRetryingCallerFactory rpcCallerFactory,
RpcControllerFactory rpcControllerFactory,
int operationTimeout,
int rpcTimeout) |
HTableDescriptor[] |
getTableDescriptors(List<String> names)
Get tableDescriptors.
|
HTableDescriptor[] |
getTableDescriptorsByTableName(List<TableName> tableNames)
Get tableDescriptors.
|
private TableName |
getTableNameBeforeRestoreSnapshot(String snapshotName) |
List<HRegionInfo> |
getTableRegions(TableName tableName)
Deprecated.
As of release 2.0.0, this will be removed in HBase 3.0.0
Use
getRegions(TableName). |
private byte[][] |
getTableSplits(TableName tableName) |
private void |
internalDeleteSnapshot(SnapshotDescription snapshot) |
private Future<Void> |
internalRestoreSnapshotAsync(String snapshotName,
TableName tableName,
boolean restoreAcl)
Execute Restore/Clone snapshot and wait for the server to complete (blocking).
|
boolean |
isAborted()
Check if the server or client was aborted.
|
boolean |
isBalancerEnabled()
Query the current state of the balancer.
|
boolean |
isCatalogJanitorEnabled()
Query on the catalog janitor state (Enabled/Disabled?).
|
boolean |
isCleanerChoreEnabled()
Query on the cleaner chore state (Enabled/Disabled?).
|
boolean |
isMasterInMaintenanceMode()
Check whether Master is in maintenance mode.
|
boolean |
isMergeEnabled()
Query the current state of the merge switch.
|
boolean |
isNormalizerEnabled()
Query the current state of the region normalizer.
|
boolean |
isProcedureFinished(String signature,
String instance,
Map<String,String> props)
Check the current state of the specified procedure.
|
boolean |
isSnapshotFinished(SnapshotDescription snapshotDesc)
Check the current state of the passed snapshot.
|
boolean |
isSplitEnabled()
Query the current state of the split switch.
|
boolean |
isTableAvailable(TableName tableName) |
boolean |
isTableAvailable(TableName tableName,
byte[][] splitKeys)
Use this api to check if the table has been created with the specified number of splitkeys
which was used while creating the given table.
|
boolean |
isTableDisabled(TableName tableName) |
boolean |
isTableEnabled(TableName tableName) |
List<ServerName> |
listDecommissionedRegionServers()
List region servers marked as decommissioned, which can not be assigned regions.
|
NamespaceDescriptor[] |
listNamespaceDescriptors()
List available namespace descriptors.
|
List<TableCFs> |
listReplicatedTableCFs()
Find all table and column families that are replicated from this cluster
|
List<ReplicationPeerDescription> |
listReplicationPeers()
Return a list of replication peers.
|
List<ReplicationPeerDescription> |
listReplicationPeers(Pattern pattern)
Return a list of replication peers.
|
List<SnapshotDescription> |
listSnapshots()
List completed snapshots.
|
List<SnapshotDescription> |
listSnapshots(Pattern pattern)
List all the completed snapshots matching the given pattern.
|
List<SnapshotDescription> |
listSnapshots(String regex)
List all the completed snapshots matching the given regular expression.
|
List<TableDescriptor> |
listTableDescriptors()
List all the userspace tables.
|
List<TableDescriptor> |
listTableDescriptors(List<TableName> tableNames)
Get tableDescriptors.
|
List<TableDescriptor> |
listTableDescriptors(Pattern pattern)
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)
Get list of table descriptors by namespace.
|
TableName[] |
listTableNames()
List all of the names of userspace tables.
|
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)
List all of the names of userspace tables.
|
TableName[] |
listTableNames(String regex,
boolean includeSysTables)
List all of the names of userspace tables.
|
TableName[] |
listTableNamesByNamespace(String name)
Get list of table names by namespace.
|
HTableDescriptor[] |
listTables()
List all the userspace tables.
|
HTableDescriptor[] |
listTables(Pattern pattern)
List all the userspace tables that match the given pattern.
|
HTableDescriptor[] |
listTables(Pattern pattern,
boolean includeSysTables)
List all the tables matching the given pattern.
|
HTableDescriptor[] |
listTables(String regex)
List all the userspace tables matching the given regular expression.
|
HTableDescriptor[] |
listTables(String regex,
boolean includeSysTables)
List all the tables matching the given pattern.
|
List<SnapshotDescription> |
listTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern)
List all the completed snapshots matching the given table name regular expression and snapshot
name regular expression.
|
List<SnapshotDescription> |
listTableSnapshots(String tableNameRegex,
String snapshotNameRegex)
List all the completed snapshots matching the given table name regular expression and snapshot
name regular expression.
|
void |
majorCompact(TableName tableName)
Major compact a table.
|
void |
majorCompact(TableName tableName,
byte[] columnFamily)
Major compact a column family within a table.
|
void |
majorCompact(TableName tableName,
byte[] columnFamily,
CompactType compactType)
Major compact a column family within a table.
|
void |
majorCompact(TableName tableName,
CompactType compactType)
Major compact a table.
|
void |
majorCompactRegion(byte[] regionName)
Major compact a table or an individual region.
|
void |
majorCompactRegion(byte[] regionName,
byte[] columnFamily)
Major compact a column family within region.
|
void |
majorCompactRegionServer(ServerName serverName)
Major compact all regions on the region server.
|
void |
mergeRegions(byte[] nameOfRegionA,
byte[] nameOfRegionB,
boolean forcible)
Deprecated.
Since 2.0. Will be removed in 3.0. Use
mergeRegionsAsync(byte[], byte[], boolean) instead. |
Future<Void> |
mergeRegionsAsync(byte[][] nameofRegionsToMerge,
boolean forcible)
Merge two regions.
|
Future<Void> |
mergeRegionsAsync(byte[] nameOfRegionA,
byte[] nameOfRegionB,
boolean forcible)
Merge two regions.
|
void |
mergeRegionsSync(byte[] nameOfRegionA,
byte[] nameOfRegionB,
boolean forcible)
Merge two regions.
|
boolean |
mergeSwitch(boolean enabled,
boolean synchronous)
Turn the merge switch on or off.
|
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.
|
void |
modifyNamespace(NamespaceDescriptor descriptor)
Modify an existing namespace.
|
Future<Void> |
modifyNamespaceAsync(NamespaceDescriptor descriptor)
Modify an existing namespace.
|
void |
modifyTable(TableDescriptor td)
Modify an existing table, more IRB friendly version.
|
void |
modifyTable(TableName tableName,
TableDescriptor td)
Modify an existing table, more IRB friendly version.
|
Future<Void> |
modifyTableAsync(TableDescriptor td)
Modify an existing table, more IRB (ruby) friendly version.
|
Future<Void> |
modifyTableAsync(TableName tableName,
TableDescriptor td)
Modify an existing table, more IRB friendly version.
|
void |
move(byte[] encodedRegionName,
byte[] destServerName)
Move the region
r to dest. |
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.
|
void |
removeReplicationPeer(String peerId)
Remove a peer and stop the replication.
|
void |
removeReplicationPeerTableCFs(String id,
Map<TableName,List<String>> tableCfs)
Remove some table-cfs from config of the specified peer.
|
void |
restoreSnapshot(byte[] snapshotName)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(byte[] snapshotName,
boolean takeFailSafeSnapshot)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(String snapshotName)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(String snapshotName,
boolean takeFailSafeSnapshot)
Restore the specified snapshot on the original table.
|
void |
restoreSnapshot(String snapshotName,
boolean takeFailSafeSnapshot,
boolean restoreAcl)
Restore the specified snapshot on the original table.
|
Future<Void> |
restoreSnapshotAsync(String snapshotName)
Restore the specified snapshot on the original table.
|
byte[][] |
rollHLogWriter(String serverName)
Deprecated.
|
void |
rollWALWriter(ServerName serverName)
Roll the log writer.
|
private org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse |
rollWALWriterImpl(ServerName sn) |
int |
runCatalogJanitor()
Ask for a scan of the catalog table.
|
boolean |
runCleanerChore()
Ask for cleaner chore to run.
|
void |
setQuota(QuotaSettings quota)
Apply the new quota settings.
|
private void |
setTableRep(TableName tableName,
boolean enableRep)
Set the table's replication switch if the table's replication switch is already not set.
|
void |
shutdown()
Shuts down the HBase cluster.
|
void |
snapshot(byte[] snapshotName,
TableName tableName)
Create a timestamp consistent snapshot for the given table.
|
void |
snapshot(SnapshotDescription snapshotDesc)
Take a snapshot and wait for the server to complete that snapshot (blocking).
|
void |
snapshot(String snapshotName,
TableName tableName)
Take a snapshot for the given table.
|
void |
snapshot(String snapshotName,
TableName tableName,
SnapshotType type)
Create typed snapshot of the table.
|
void |
snapshotAsync(SnapshotDescription snapshotDesc)
Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a
single snapshot should be taken at a time, or results may be undefined.
|
void |
split(TableName tableName)
+ Split a table.
|
void |
split(TableName tableName,
byte[] splitPoint)
Split a table.
|
private boolean |
splitOrMergeSwitch(boolean enabled,
boolean synchronous,
MasterSwitchType switchType) |
void |
splitRegion(byte[] regionName)
Split an individual region.
|
void |
splitRegion(byte[] regionName,
byte[] splitPoint)
Split an individual region.
|
Future<Void> |
splitRegionAsync(byte[] regionName,
byte[] splitPoint)
Split an individual region.
|
(package private) Future<Void> |
splitRegionAsync(RegionInfo hri,
byte[] splitPoint) |
void |
splitRegionSync(byte[] regionName,
byte[] splitPoint)
Split one region.
|
void |
splitRegionSync(byte[] regionName,
byte[] splitPoint,
long timeout,
TimeUnit units)
Split one region.
|
boolean |
splitSwitch(boolean enabled,
boolean synchronous)
Turn the split switch on or off.
|
void |
stopMaster()
Shuts down the current HBase master only.
|
void |
stopRegionServer(String hostnamePort)
Stop the designated regionserver.
|
boolean |
tableExists(TableName tableName) |
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.
|
void |
updateReplicationPeerConfig(String peerId,
ReplicationPeerConfig peerConfig)
Update the peerConfig for the specified peer.
|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, waitaddColumn, addReplicationPeer, balancer, balancer, compactRegionServer, enableCatalogJanitor, execProcedureWithRet, getBackupMasters, getClusterMetrics, getClusterStatus, getMaster, getMasterCoprocessorNames, getMasterCoprocessors, getMasterInfoPort, getRegionMetrics, getRegionServers, isSplitOrMergeEnabled, listDeadServers, modifyColumn, runCatalogScan, setBalancerRunning, setCleanerChoreRunning, setNormalizerRunning, setSplitOrMergeEnabled, takeSnapshotAsyncprivate static final org.slf4j.Logger LOG
private ClusterConnection connection
private final org.apache.hadoop.conf.Configuration conf
private final long pause
private final int numRetries
private final int syncWaitTimeout
private boolean aborted
private int operationTimeout
private int rpcTimeout
private int getProcedureTimeout
private RpcRetryingCallerFactory rpcCallerFactory
private RpcControllerFactory rpcControllerFactory
private NonceGenerator ng
HBaseAdmin(ClusterConnection connection) throws IOException
IOExceptionpublic int getOperationTimeout()
getOperationTimeout in interface Adminpublic void abort(String why, Throwable e)
Abortablepublic boolean isAborted()
Abortablepublic boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException
AdminabortProcedure in interface AdminprocId - 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 existIOExceptionpublic Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning) throws IOException
AdminabortProcedureAsync in interface AdminprocId - 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 existIOExceptionpublic List<TableDescriptor> listTableDescriptors() throws IOException
AdminlistTableDescriptors in interface AdminIOException - if a remote or network exception occurspublic List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException
AdminlistTableDescriptors in interface Adminpattern - The compiled regular expression to match againstIOException - if a remote or network exception occursAdmin.listTables()public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) throws IOException
AdminlistTableDescriptors in interface Adminpattern - The compiled regular expression to match againstincludeSysTables - false to match only against userspace tablesIOException - if a remote or network exception occursAdmin.listTables()public TableDescriptor getDescriptor(TableName tableName) throws TableNotFoundException, IOException
AdmingetDescriptor in interface AdmintableName - as a TableNameTableNotFoundExceptionIOException - if a remote or network exception occurspublic void modifyTable(TableDescriptor td) throws IOException
AdminmodifyTable in interface Admintd - modified description of the tableIOException - if a remote or network exception occurspublic Future<Void> modifyTableAsync(TableDescriptor td) throws IOException
AdminmodifyTableAsync in interface Admintd - description of the tableIOException - if a remote or network exception occurspublic List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException
AdminlistTableDescriptorsByNamespace in interface Adminname - namespace nameIOExceptionpublic List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException
AdminlistTableDescriptors in interface AdmintableNames - List of table namesIOException - if a remote or network exception occurspublic List<RegionInfo> getRegions(ServerName sn) throws IOException
AdmingetRegions in interface AdminRegionInfoIOExceptionpublic List<RegionInfo> getRegions(TableName tableName) throws IOException
AdmingetRegions in interface AdmintableName - the name of the tableRegionInfo.IOExceptionpublic Connection getConnection()
getConnection in interface Adminpublic boolean tableExists(TableName tableName) throws IOException
tableExists in interface AdmintableName - Table to check.true if table exists already.IOExceptionpublic HTableDescriptor[] listTables() throws IOException
AdminlistTables in interface AdminIOException - if a remote or network exception occursAdmin.listTableDescriptors()public HTableDescriptor[] listTables(Pattern pattern) throws IOException
AdminlistTables in interface Adminpattern - The compiled regular expression to match againstIOException - if a remote or network exception occursAdmin.listTables(),
Admin.listTableDescriptors(Pattern)public HTableDescriptor[] listTables(String regex) throws IOException
AdminlistTables in interface Adminregex - The regular expression to match againstIOException - if a remote or network exception occursAdmin.listTableDescriptors(Pattern)public HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables) throws IOException
AdminlistTables in interface Adminpattern - The compiled regular expression to match againstincludeSysTables - false to match only against userspace tablesIOException - if a remote or network exception occursAdmin.listTables(),
Admin.listTableDescriptors(java.util.regex.Pattern, boolean)public HTableDescriptor[] listTables(String regex, boolean includeSysTables) throws IOException
AdminlistTables in interface Adminregex - The regular expression to match againstincludeSysTables - false to match only against userspace tablesIOException - if a remote or network exception occursAdmin.listTables(java.util.regex.Pattern, boolean)public TableName[] listTableNames() throws IOException
AdminlistTableNames in interface AdminIOException - if a remote or network exception occurspublic TableName[] listTableNames(Pattern pattern) throws IOException
AdminlistTableNames in interface Adminpattern - The regular expression to match againstIOException - if a remote or network exception occurspublic TableName[] listTableNames(String regex) throws IOException
AdminlistTableNames in interface Adminregex - The regular expression to match againstIOException - if a remote or network exception occurspublic TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException
AdminlistTableNames in interface Adminpattern - The regular expression to match againstincludeSysTables - false to match only against userspace tablesIOException - if a remote or network exception occurspublic TableName[] listTableNames(String regex, boolean includeSysTables) throws IOException
AdminlistTableNames in interface Adminregex - The regular expression to match againstincludeSysTables - false to match only against userspace tablesIOException - if a remote or network exception occurspublic HTableDescriptor getTableDescriptor(TableName tableName) throws IOException
AdmingetTableDescriptor in interface AdmintableName - as a TableNameTableNotFoundExceptionIOException - if a remote or network exception occursstatic TableDescriptor getTableDescriptor(TableName tableName, Connection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout) throws IOException
IOException@Deprecated static HTableDescriptor getHTableDescriptor(TableName tableName, Connection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout) throws IOException
getTableDescriptor(TableName,
Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int)IOExceptionprivate long getPauseTime(int tries)
public void createTable(TableDescriptor desc) throws IOException
AdmincreateTable in interface Admindesc - table descriptor for tableMasterNotRunningException - 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 occurspublic void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) throws IOException
AdmincreateTable in interface Admindesc - table descriptor for tablestartKey - beginning of key rangeendKey - end of key rangenumRegions - the total number of regions to createMasterNotRunningException - 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).IOExceptionpublic void createTable(TableDescriptor desc, byte[][] splitKeys) throws IOException
AdmincreateTable in interface Admindesc - table descriptor for tablesplitKeys - array of split keys for the initial regions of the tableMasterNotRunningException - 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).IOExceptionpublic Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys) throws IOException
AdmincreateTableAsync in interface Admindesc - table descriptor for tablesplitKeys - keys to check if the table has been created with all split keysIOException - if a remote or network exception occurspublic void deleteTable(TableName tableName) throws IOException
AdmindeleteTable in interface AdmintableName - name of table to deleteIOException - if a remote or network exception occurspublic Future<Void> deleteTableAsync(TableName tableName) throws IOException
AdmindeleteTableAsync in interface AdmintableName - name of table to deleteIOException - if a remote or network exception occurspublic HTableDescriptor[] deleteTables(String regex) throws IOException
AdminAdmin.listTableDescriptors(Pattern)
and Admin.deleteTable(org.apache.hadoop.hbase.TableName)deleteTables in interface Adminregex - The regular expression to match table names againstIOExceptionAdmin.deleteTables(java.util.regex.Pattern),
Admin.deleteTable(org.apache.hadoop.hbase.TableName)public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException
listTables(java.util.regex.Pattern) and
deleteTable(TableName)deleteTables in interface Adminpattern - The pattern to match table names againstIOExceptionpublic void truncateTable(TableName tableName, boolean preserveSplits) throws IOException
AdmintruncateTable in interface AdmintableName - name of table to truncatepreserveSplits - true if the splits should be preservedIOException - if a remote or network exception occurspublic Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits) throws IOException
AdmintruncateTableAsync in interface AdmintableName - name of table to deletepreserveSplits - true if the splits should be preservedIOException - if a remote or network exception occursprivate byte[][] getTableSplits(TableName tableName) throws IOException
IOExceptionpublic void enableTable(TableName tableName) throws IOException
AdminAdmin.enableTableAsync(org.apache.hadoop.hbase.TableName)
and Admin.isTableEnabled(org.apache.hadoop.hbase.TableName) instead. The table has to be in
disabled state for it to be enabled.enableTable in interface AdmintableName - 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.Admin.isTableEnabled(org.apache.hadoop.hbase.TableName),
Admin.disableTable(org.apache.hadoop.hbase.TableName),
Admin.enableTableAsync(org.apache.hadoop.hbase.TableName)public Future<Void> enableTableAsync(TableName tableName) throws IOException
AdminenableTableAsync in interface AdmintableName - name of table to deleteIOException - if a remote or network exception occurspublic HTableDescriptor[] enableTables(String regex) throws IOException
AdminAdmin.listTableDescriptors(Pattern) and Admin.enableTable(org.apache.hadoop.hbase.TableName)enableTables in interface Adminregex - The regular expression to match table names againstIOExceptionAdmin.enableTables(java.util.regex.Pattern),
Admin.enableTable(org.apache.hadoop.hbase.TableName)public HTableDescriptor[] enableTables(Pattern pattern) throws IOException
AdminAdmin.listTableDescriptors(java.util.regex.Pattern) and
Admin.enableTable(org.apache.hadoop.hbase.TableName)enableTables in interface Adminpattern - The pattern to match table names againstIOExceptionpublic void disableTable(TableName tableName) throws IOException
AdminAdmin.disableTableAsync(org.apache.hadoop.hbase.TableName) and
Admin.isTableDisabled(org.apache.hadoop.hbase.TableName) instead. The table has to be in
enabled state for it to be disabled.disableTable in interface AdminIOException - There could be couple types of IOException TableNotFoundException means the
table doesn't exist. TableNotEnabledException means the table isn't in enabled state.public Future<Void> disableTableAsync(TableName tableName) throws IOException
AdmindisableTableAsync in interface AdmintableName - name of table to deleteIOException - if a remote or network exception occurspublic HTableDescriptor[] disableTables(String regex) throws IOException
AdminAdmin.listTableDescriptors(Pattern) and Admin.disableTable(org.apache.hadoop.hbase.TableName)disableTables in interface Adminregex - The regular expression to match table names againstIOExceptionAdmin.disableTables(java.util.regex.Pattern),
Admin.disableTable(org.apache.hadoop.hbase.TableName)public HTableDescriptor[] disableTables(Pattern pattern) throws IOException
AdminAdmin.listTableDescriptors(java.util.regex.Pattern) and
Admin.disableTable(org.apache.hadoop.hbase.TableName)disableTables in interface Adminpattern - The pattern to match table names againstIOExceptionpublic boolean isTableEnabled(TableName tableName) throws IOException
isTableEnabled in interface AdmintableName - name of table to checktrue if table is on-lineIOException - if a remote or network exception occurspublic boolean isTableDisabled(TableName tableName) throws IOException
isTableDisabled in interface AdmintableName - name of table to checktrue if table is off-lineIOException - if a remote or network exception occurspublic boolean isTableAvailable(TableName tableName) throws IOException
isTableAvailable in interface AdmintableName - name of table to checktrue if all regions of the table are availableIOException - if a remote or network exception occurspublic boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException
Adminfalse.isTableAvailable in interface AdmintableName - name of table to checksplitKeys - keys to check if the table has been created with all split keysIOException - if a remote or network excpetion occurspublic Pair<Integer,Integer> getAlterStatus(TableName tableName) throws IOException
Adminalter (a.k.a modify) command - indicates how
many regions have received the updated schema Asynchronous operation.getAlterStatus in interface AdmintableName - TableName instanceIOException - if a remote or network exception occurspublic Pair<Integer,Integer> getAlterStatus(byte[] tableName) throws IOException
Adminalter (a.k.a modify) command - indicates how many
regions have received the updated schema Asynchronous operation.getAlterStatus in interface AdmintableName - name of the table to get the status ofIOException - if a remote or network exception occurspublic void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
AdminAdmin.addColumnFamilyAsync(TableName, ColumnFamilyDescriptor) instead because it
returns a Future from which you can learn whether success or failure.addColumnFamily in interface AdmintableName - name of the table to add column family tocolumnFamily - column family descriptor of column family to be addedIOException - if a remote or network exception occurspublic Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
AdminaddColumnFamilyAsync in interface AdmintableName - name of the table to add column family tocolumnFamily - column family descriptor of column family to be addedIOException - if a remote or network exception occurs@Deprecated public void deleteColumn(TableName tableName, byte[] columnFamily) throws IOException
deleteColumnFamily(TableName, byte[]) instead.Admin.deleteColumnFamily(TableName, byte[]) instead because it
returns a Future from which you can learn whether success or failure.deleteColumn in interface AdmintableName - name of tablecolumnFamily - name of column family to be deletedIOException - if a remote or network exception occurspublic void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException
AdminAdmin.deleteColumnFamily(TableName, byte[]) instead because it
returns a Future from which you can learn whether success or failure.deleteColumnFamily in interface AdmintableName - name of tablecolumnFamily - name of column family to be deletedIOException - if a remote or network exception occurspublic Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily) throws IOException
AdmindeleteColumnFamilyAsync in interface AdmintableName - name of tablecolumnFamily - name of column family to be deletedIOException - if a remote or network exception occurspublic void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
AdminAdmin.modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor) instead because it
returns a Future from which you can learn whether success or failure.modifyColumnFamily in interface AdmintableName - name of tablecolumnFamily - new column family descriptor to useIOException - if a remote or network exception occurspublic Future<Void> modifyColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException
AdminmodifyColumnFamilyAsync in interface AdmintableName - name of tablecolumnFamily - new column family descriptor to useIOException - if a remote or network exception occurs@Deprecated public void closeRegion(String regionName, String unused) throws IOException
AdminAdmin.unassign(byte[], boolean) to unassign the region. For expert-admins.closeRegion in interface AdminregionName - region name to closeunused - Deprecated. Not used.IOException - if a remote or network exception occurs@Deprecated public void closeRegion(byte[] regionName, String unused) throws IOException
AdminAdmin.unassign(byte[], boolean) to unassign the region. For expert-admins.closeRegion in interface AdminregionName - region name to closeunused - Deprecated. Not used.IOException - if a remote or network exception occurs@Deprecated public boolean closeRegionWithEncodedRegionName(String encodedRegionName, String unused) throws IOException
AdminAdmin.unassign(byte[], boolean) to unassign the region. For expert-admins.closeRegionWithEncodedRegionName in interface AdminencodedRegionName - The encoded region name; i.e. the hash that makes up the region name
suffix: e.g. if regionname is
TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.,
then the encoded region name is: 527db22f95c8a9e0116f0cc13c680396.unused - Deprecated. Not used.true always.IOException - if a remote or network exception occurs@Deprecated public void closeRegion(ServerName unused, HRegionInfo hri) throws IOException
AdminAdmin.unassign(byte[], boolean) to unassign the region. For expert-admins.closeRegion in interface Adminunused - Deprecated. Not used.IOException@Deprecated public List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException
getRegions(ServerName).AdmingetOnlineRegions in interface Adminsn - HRegionInfo.IOExceptionpublic void flush(TableName tableName) throws IOException
Adminflush in interface AdmintableName - table to flushIOException - if a remote or network exception occurspublic void flushRegion(byte[] regionName) throws IOException
AdminflushRegion in interface AdminregionName - region to flushIOException - if a remote or network exception occursprivate void flush(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, RegionInfo info) throws IOException
IOExceptionpublic void flushRegionServer(ServerName serverName) throws IOException
AdminflushRegionServer in interface AdminserverName - the region server name to flushIOException - if a remote or network exception occurspublic void compact(TableName tableName) throws IOException
compact in interface AdmintableName - table to compactIOException - if a remote or network exception occurspublic void compactRegion(byte[] regionName) throws IOException
AdmincompactRegion in interface AdminregionName - region to compactIOException - if a remote or network exception occurspublic void compact(TableName tableName, byte[] columnFamily) throws IOException
compact in interface AdmintableName - table to compactcolumnFamily - column family within a tableIOException - if a remote or network exception occurspublic void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException
compactRegion in interface AdminregionName - region to compactcolumnFamily - column family within a regionIOException - if a remote or network exception occurspublic void compactRegionServer(ServerName serverName) throws IOException
AdmincompactRegionServer in interface AdminserverName - the region server nameIOException - if a remote or network exception occurspublic void majorCompactRegionServer(ServerName serverName) throws IOException
AdminmajorCompactRegionServer in interface AdminserverName - the region server nameIOException - if a remote or network exception occurspublic void majorCompact(TableName tableName) throws IOException
AdminmajorCompact in interface AdmintableName - table to major compactIOException - if a remote or network exception occurspublic void majorCompactRegion(byte[] regionName) throws IOException
AdminmajorCompactRegion in interface AdminregionName - region to major compactIOException - if a remote or network exception occurspublic void majorCompact(TableName tableName, byte[] columnFamily) throws IOException
majorCompact in interface AdmintableName - table to major compactcolumnFamily - column family within a tableIOException - if a remote or network exception occurspublic void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException
AdminmajorCompactRegion in interface AdminregionName - egion to major compactcolumnFamily - column family within a regionIOException - if a remote or network exception occursprivate void compact(TableName tableName, byte[] columnFamily, boolean major, CompactType compactType) throws IOException
tableName - table or region to compactcolumnFamily - column family within a table or regionmajor - True if we are to do a major compaction.compactType - CompactTypeIOException - if a remote or network exception occursprivate void compactRegion(byte[] regionName, byte[] columnFamily, boolean major) throws IOException
regionName - region to compactcolumnFamily - column family within a table or regionmajor - True if we are to do a major compaction.IOException - if a remote or network exception occursInterruptedExceptionprivate void compact(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, RegionInfo hri, boolean major, byte[] family) throws IOException
IOExceptionpublic void move(byte[] encodedRegionName, byte[] destServerName) throws IOException
Adminr to dest.move in interface AdminencodedRegionName - 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,1289493121758IOException - if we can't find a region named
encodedRegionNamepublic void assign(byte[] regionName) throws MasterNotRunningException, ZooKeeperConnectionException, IOException
Adminassign in interface AdminregionName - Region name to assign.MasterNotRunningExceptionZooKeeperConnectionExceptionIOExceptionpublic void unassign(byte[] regionName, boolean force) throws IOException
AdminAdmin.move(byte[], byte[]) if you want to control the region movement.unassign in interface AdminregionName - 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).IOExceptionpublic void offline(byte[] regionName) throws IOException
Adminoffline in interface AdminregionName - Region to offline.IOExceptionpublic boolean balancerSwitch(boolean on, boolean synchronous) throws IOException
AdminbalancerSwitch in interface Adminon - Set to true to enable, false to disable.synchronous - If true, it waits until current balance() call, if
outstanding, to return.IOExceptionpublic boolean balance() throws IOException
Adminbalance in interface Admintrue if balancer ran, false otherwise.IOExceptionpublic boolean balance(boolean force) throws IOException
Adminbalance in interface Adminforce - whether we should force balance even if there is region in transitiontrue if balancer ran, false otherwise.IOExceptionpublic boolean isBalancerEnabled() throws IOException
AdminisBalancerEnabled in interface Admintrue if the balancer is enabled, false otherwise.IOExceptionpublic CacheEvictionStats clearBlockCache(TableName tableName) throws IOException
clearBlockCache in interface AdmintableName - table to clear block cacheIOException - if a remote or network exception occursprivate CacheEvictionStats clearBlockCache(ServerName sn, List<RegionInfo> hris) throws IOException
IOExceptionpublic boolean normalize() throws IOException
normalize in interface AdminIOExceptionpublic boolean isNormalizerEnabled() throws IOException
AdminisNormalizerEnabled in interface Admintrue if region normalizer is enabled, false otherwise.IOExceptionpublic boolean normalizerSwitch(boolean on) throws IOException
AdminnormalizerSwitch in interface AdminIOExceptionpublic boolean catalogJanitorSwitch(boolean enable) throws IOException
AdmincatalogJanitorSwitch in interface Adminenable - if true enables the catalog janitorIOExceptionpublic int runCatalogJanitor() throws IOException
AdminrunCatalogJanitor in interface AdminIOExceptionpublic boolean isCatalogJanitorEnabled() throws IOException
AdminisCatalogJanitorEnabled in interface AdminIOExceptionpublic boolean cleanerChoreSwitch(boolean on) throws IOException
AdmincleanerChoreSwitch in interface Adminon - if true enables the cleaner choreIOExceptionpublic boolean runCleanerChore() throws IOException
AdminrunCleanerChore in interface Admintrue if cleaner chore ran, false otherwiseIOExceptionpublic boolean isCleanerChoreEnabled() throws IOException
AdminisCleanerChoreEnabled in interface AdminIOExceptionpublic void mergeRegionsSync(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
nameOfRegionA - encoded or full name of region anameOfRegionB - encoded or full name of region bforcible - true if do a compulsory merge, otherwise we will only merge
two adjacent regionsIOException@Deprecated public void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
mergeRegionsAsync(byte[], byte[], boolean) instead.mergeRegions in interface AdminnameOfRegionA - 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 regionsIOExceptionpublic Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException
mergeRegionsAsync in interface AdminnameOfRegionA - 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 regionsIOExceptionpublic Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible) throws IOException
mergeRegionsAsync in interface AdminnameofRegionsToMerge - encoded or full name of daughter regionsforcible - true if do a compulsory merge, otherwise we will only merge
adjacent regionsIOExceptionpublic void splitRegionSync(byte[] regionName, byte[] splitPoint) throws IOException
regionName - encoded or full name of regionsplitPoint - key where region splitsIOExceptionpublic void splitRegionSync(byte[] regionName, byte[] splitPoint, long timeout, TimeUnit units) throws IOException
regionName - region to be splitsplitPoint - split pointtimeout - how long to wait on splitunits - time unitsIOExceptionpublic Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException
AdminsplitRegionAsync in interface AdminregionName - region to splitsplitPoint - the explicit position to split onIOException - if a remote or network exception occursFuture<Void> splitRegionAsync(RegionInfo hri, byte[] splitPoint) throws IOException
IOExceptionpublic void split(TableName tableName) throws IOException
Adminsplit in interface AdmintableName - table to splitIOException - if a remote or network exception occurspublic void splitRegion(byte[] regionName) throws IOException
AdminsplitRegion in interface AdminregionName - region to splitIOException - if a remote or network exception occurspublic void split(TableName tableName, byte[] splitPoint) throws IOException
Adminsplit in interface AdmintableName - table to splitsplitPoint - the explicit position to split onIOException - if a remote or network exception occurspublic void splitRegion(byte[] regionName, byte[] splitPoint) throws IOException
AdminsplitRegion in interface AdminregionName - region to splitsplitPoint - the explicit position to split onIOException - if a remote or network exception occurspublic void modifyTable(TableName tableName, TableDescriptor td) throws IOException
AdminmodifyTable in interface AdmintableName - name of table.td - modified description of the tableIOException - if a remote or network exception occurspublic Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td) throws IOException
AdminmodifyTableAsync in interface AdmintableName - name of table.td - modified description of the tableIOException - if a remote or network exception occursPair<RegionInfo,ServerName> getRegion(byte[] regionName) throws IOException
regionName - Name of a region.regionName is
a verified region name (we call MetaTableAccessor.getRegionLocation(Connection, byte[])
else null.
Throw IllegalArgumentException if regionName is null.IOExceptionprivate byte[] getRegionName(byte[] regionNameOrEncodedRegionName) throws IOException
IOExceptionprivate TableName checkTableExists(TableName tableName) throws IOException
tableName - Name of a table.IOException - if a remote or network exception occurs.TableNotFoundException - if table does not exist.public void shutdown() throws IOException
AdminShuts down the HBase cluster.
Notice that, a success shutdown call may ends with an error since the remote server has already been shutdown.
shutdown in interface AdminIOException - if a remote or network exception occurspublic void stopMaster() throws IOException
AdminShuts down the current HBase master only. Does not shutdown the cluster.
Notice that, a success stopMaster call may ends with an error since the remote server has already been shutdown.
stopMaster in interface AdminIOException - if a remote or network exception occursAdmin.shutdown()public void stopRegionServer(String hostnamePort) throws IOException
AdminstopRegionServer in interface AdminhostnamePort - Hostname and port delimited by a : as in
example.org:1234IOException - if a remote or network exception occurspublic boolean isMasterInMaintenanceMode() throws IOException
AdminisMasterInMaintenanceMode in interface AdminIOException - if a remote or network exception occurspublic ClusterMetrics getClusterMetrics(EnumSet<ClusterMetrics.Option> options) throws IOException
AdminClusterMetrics.Option to get desired status.getClusterMetrics in interface AdminIOException - if a remote or network exception occurspublic List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName) throws IOException
AdminRegionMetrics of all regions hosted on a regionserver for a table.getRegionMetrics in interface AdminserverName - region server from which RegionMetrics is required.tableName - get RegionMetrics of regions belonging to the tableIOException - if a remote or network exception occurspublic org.apache.hadoop.conf.Configuration getConfiguration()
getConfiguration in interface Adminprivate static <T> T get(Future<T> future, long timeout, TimeUnit units) throws IOException
future.IOExceptionpublic void createNamespace(NamespaceDescriptor descriptor) throws IOException
AdmincreateNamespace in interface Admindescriptor - descriptor which describes the new namespace.IOExceptionpublic Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException
AdmincreateNamespaceAsync in interface Admindescriptor - descriptor which describes the new namespaceIOExceptionpublic void modifyNamespace(NamespaceDescriptor descriptor) throws IOException
AdminmodifyNamespace in interface Admindescriptor - descriptor which describes the new namespaceIOExceptionpublic Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException
AdminmodifyNamespaceAsync in interface Admindescriptor - descriptor which describes the new namespaceIOExceptionpublic void deleteNamespace(String name) throws IOException
AdmindeleteNamespace in interface Adminname - namespace nameIOExceptionpublic Future<Void> deleteNamespaceAsync(String name) throws IOException
AdmindeleteNamespaceAsync in interface Adminname - namespace nameIOExceptionpublic NamespaceDescriptor getNamespaceDescriptor(String name) throws NamespaceNotFoundException, IOException
AdmingetNamespaceDescriptor in interface Adminname - name of namespace descriptorNamespaceNotFoundExceptionIOException - if a remote or network exception occurspublic NamespaceDescriptor[] listNamespaceDescriptors() throws IOException
AdminlistNamespaceDescriptors in interface AdminIOExceptionpublic String getProcedures() throws IOException
AdmingetProcedures in interface AdminIOExceptionpublic String getLocks() throws IOException
AdmingetLocks in interface AdminIOException - if a remote or network exception occurspublic HTableDescriptor[] listTableDescriptorsByNamespace(String name) throws IOException
AdminlistTableDescriptorsByNamespace in interface Adminname - namespace nameIOExceptionpublic TableName[] listTableNamesByNamespace(String name) throws IOException
AdminlistTableNamesByNamespace in interface Adminname - namespace nameIOExceptionpublic static void available(org.apache.hadoop.conf.Configuration conf) throws MasterNotRunningException, ZooKeeperConnectionException, IOException
conf - system configurationMasterNotRunningException - if the master is not running.ZooKeeperConnectionException - if unable to connect to zookeeper. // TODO do not expose
ZKConnectionException.IOException@Deprecated public List<HRegionInfo> getTableRegions(TableName tableName) throws IOException
getRegions(TableName).AdmingetTableRegions in interface AdmintableName - HRegionInfo.IOExceptionpublic void close() throws IOException
close in interface Closeableclose in interface AutoCloseableclose in interface AdminIOExceptionpublic HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames) throws IOException
AdmingetTableDescriptorsByTableName in interface AdmintableNames - List of table namesIOException - if a remote or network exception occurspublic HTableDescriptor[] getTableDescriptors(List<String> names) throws IOException
AdmingetTableDescriptors in interface Adminnames - List of table namesIOException - if a remote or network exception occursprivate org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriterImpl(ServerName sn) throws IOException, FailedLogCloseException
IOExceptionFailedLogCloseException@Deprecated public byte[][] rollHLogWriter(String serverName) throws IOException, FailedLogCloseException
rollWALWriter(ServerName)serverName - The servername of the regionserver. A server name is made of host,
port and startcode. This is mandatory. Here is an example:
host187.example.com,60020,1289493121758HRegionInfo.getEncodedName() that would allow the wal to
clean up some underlying files. null if there's nothing to flush.IOException - if a remote or network exception occursFailedLogCloseExceptionpublic void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException
AdminrollWALWriter in interface AdminserverName - The servername of the regionserver.IOException - if a remote or network exception occursFailedLogCloseExceptionpublic CompactionState getCompactionState(TableName tableName) throws IOException
AdmingetCompactionState in interface AdmintableName - table to examineIOException - if a remote or network exception occurspublic CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException
AdmingetCompactionStateForRegion in interface AdminregionName - region to examineIOException - if a remote or network exception occurspublic void snapshot(String snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
AdminSnapshotCreationException indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See TableName.isLegalFullyQualifiedTableName(byte[]).snapshot in interface AdminsnapshotName - 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 incorrectlypublic void snapshot(byte[] snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException
AdminSnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
HBase.snapshot in interface AdminsnapshotName - 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 incorrectlypublic void snapshot(String snapshotName, TableName tableName, SnapshotType type) throws IOException, SnapshotCreationException, IllegalArgumentException
AdminSnapshotCreationException indicating the
duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[]).snapshot in interface AdminsnapshotName - 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 incorrectlypublic void snapshot(SnapshotDescription snapshotDesc) throws IOException, SnapshotCreationException, IllegalArgumentException
AdminSnapshotCreationException indicating the duplicate naming.
Snapshot names follow the same naming constraints as tables in HBase. See TableName.isLegalFullyQualifiedTableName(byte[]). You should probably
use Admin.snapshot(String, org.apache.hadoop.hbase.TableName) or
Admin.snapshot(byte[], org.apache.hadoop.hbase.TableName) unless you are sure about the type
of snapshot that you want to take.snapshot in interface AdminsnapshotDesc - snapshot to takeIOException - or we lose contact with the master.SnapshotCreationException - if snapshot failed to be takenIllegalArgumentException - if the snapshot request is formatted incorrectlypublic void snapshotAsync(SnapshotDescription snapshotDesc) throws IOException, SnapshotCreationException
AdminsnapshotAsync in interface AdminsnapshotDesc - snapshot to takeIOException - if the snapshot did not succeed or we lose contact with the master.SnapshotCreationException - if snapshot creation failedprivate org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse asyncSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws IOException
IOExceptionpublic boolean isSnapshotFinished(SnapshotDescription snapshotDesc) throws IOException, HBaseSnapshotException, UnknownSnapshotException
AdminUnknownSnapshotException.isSnapshotFinished in interface AdminsnapshotDesc - description of the snapshot to checkIOException - if we have a network issueHBaseSnapshotException - if the snapshot failedUnknownSnapshotException - if the requested snapshot is
unknownpublic void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException
Admintrue, a
snapshot of the current table is taken before executing the restore operation. In case of
restore failure, the failsafe snapshot will be restored. If the restore completes without
problem the failsafe snapshot is deleted.restoreSnapshot in interface AdminsnapshotName - name of the snapshot to restoreIOException - if a remote or network exception occursRestoreSnapshotException - if snapshot failed to be
restoredpublic void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException
Admintrue, a
snapshot of the current table is taken before executing the restore operation. In case of
restore failure, the failsafe snapshot will be restored. If the restore completes without
problem the failsafe snapshot is deleted.restoreSnapshot in interface AdminsnapshotName - name of the snapshot to restoreIOException - if a remote or network exception occursRestoreSnapshotException - if snapshot failed to be restoredpublic void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
Admintrue, a snapshot of the current table is taken before
executing the restore operation. In case of restore failure, the failsafe snapshot will be
restored. If the restore completes without problem the failsafe snapshot is deleted. The
failsafe snapshot name is configurable by using the property
"hbase.snapshot.restore.failsafe.name".restoreSnapshot in interface AdminsnapshotName - name of the snapshot to restoretakeFailSafeSnapshot - true if the failsafe snapshot should be takenIOException - if a remote or network exception occursRestoreSnapshotException - if snapshot failed to be restoredprivate TableName getTableNameBeforeRestoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException
IOExceptionRestoreSnapshotExceptionpublic void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException
Admintrue, a snapshot of the current table is taken before
executing the restore operation. In case of restore failure, the failsafe snapshot will be
restored. If the restore completes without problem the failsafe snapshot is deleted. The
failsafe snapshot name is configurable by using the property
"hbase.snapshot.restore.failsafe.name".restoreSnapshot in interface AdminsnapshotName - name of the snapshot to restoretakeFailSafeSnapshot - true if the failsafe snapshot should be takenIOException - if a remote or network exception occursRestoreSnapshotException - if snapshot failed to be restoredpublic void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) throws IOException, RestoreSnapshotException
Admintrue, a snapshot of the current table is taken before
executing the restore operation. In case of restore failure, the failsafe snapshot will be
restored. If the restore completes without problem the failsafe snapshot is deleted. The
failsafe snapshot name is configurable by using the property
"hbase.snapshot.restore.failsafe.name".restoreSnapshot in interface AdminsnapshotName - name of the snapshot to restoretakeFailSafeSnapshot - true if the failsafe snapshot should be takenrestoreAcl - true to restore acl of snapshotIOException - if a remote or network exception occursRestoreSnapshotException - if snapshot failed to be restoredpublic Future<Void> restoreSnapshotAsync(String snapshotName) throws IOException, RestoreSnapshotException
Admintrue, a
snapshot of the current table is taken before executing the restore operation. In case of
restore failure, the failsafe snapshot will be restored. If the restore completes without
problem the failsafe snapshot is deleted.restoreSnapshotAsync in interface AdminsnapshotName - name of the snapshot to restoreIOException - if a remote or network exception occursRestoreSnapshotException - if snapshot failed to be restoredpublic void cloneSnapshot(byte[] snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException
AdmincloneSnapshot in interface AdminsnapshotName - 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 clonedpublic void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException
AdmincloneSnapshot in interface AdminsnapshotName - name of the snapshot to be clonedtableName - name of the table where the snapshot will be restoredrestoreAcl - true to clone acl into newly created tableIOException - if a remote or network exception occursTableExistsException - if table to be created already existsRestoreSnapshotException - if snapshot failed to be clonedpublic void cloneSnapshot(String snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException
AdmincloneSnapshot in interface AdminsnapshotName - 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 clonedpublic Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName) throws IOException, TableExistsException
AdmincloneSnapshotAsync in interface AdminsnapshotName - 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 existspublic byte[] execProcedureWithReturn(String signature, String instance, Map<String,String> props) throws IOException
AdminexecProcedureWithReturn in interface Adminsignature - 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 procedureIOExceptionpublic void execProcedure(String signature, String instance, Map<String,String> props) throws IOException
AdminexecProcedure in interface Adminsignature - 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 procedureIOExceptionpublic boolean isProcedureFinished(String signature, String instance, Map<String,String> props) throws IOException
AdminisProcedureFinished in interface Adminsignature - The signature that uniquely identifies a procedureinstance - The instance name of the procedureprops - Property/Value pairs of properties passing to the proceduretrue if the specified procedure is finished successfully, false if it is still runningIOException - if the specified procedure finished with errorprivate Future<Void> internalRestoreSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, RestoreSnapshotException
isTableAvailable(org.apache.hadoop.hbase.TableName) -- it is not safe to
create an HTable instance to this table before it is available.snapshotName - snapshot to restoretableName - table name to restore the snapshot onIOException - if a remote or network exception occursRestoreSnapshotException - if snapshot failed to be restoredIllegalArgumentException - if the restore request is formatted incorrectlypublic List<SnapshotDescription> listSnapshots() throws IOException
AdminlistSnapshots in interface AdminIOException - if a network error occurspublic List<SnapshotDescription> listSnapshots(String regex) throws IOException
AdminlistSnapshots in interface Adminregex - The regular expression to match againstIOException - if a remote or network exception occurspublic List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException
AdminlistSnapshots in interface Adminpattern - The compiled regular expression to match againstIOException - if a remote or network exception occurspublic List<SnapshotDescription> listTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException
AdminlistTableSnapshots in interface AdmintableNameRegex - The table name regular expression to match againstsnapshotNameRegex - The snapshot name regular expression to match againstIOException - if a remote or network exception occurspublic List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException
AdminlistTableSnapshots in interface AdmintableNamePattern - The compiled table name regular expression to match againstsnapshotNamePattern - The compiled snapshot name regular expression to match againstIOException - if a remote or network exception occurspublic void deleteSnapshot(byte[] snapshotName) throws IOException
AdmindeleteSnapshot in interface AdminsnapshotName - name of the snapshotIOException - if a remote or network exception occurspublic void deleteSnapshot(String snapshotName) throws IOException
AdmindeleteSnapshot in interface AdminsnapshotName - name of the snapshotIOException - if a remote or network exception occurspublic void deleteSnapshots(String regex) throws IOException
AdmindeleteSnapshots in interface Adminregex - The regular expression to match againstIOException - if a remote or network exception occurspublic void deleteSnapshots(Pattern pattern) throws IOException
AdmindeleteSnapshots in interface Adminpattern - pattern for names of the snapshot to matchIOException - if a remote or network exception occursprivate void internalDeleteSnapshot(SnapshotDescription snapshot) throws IOException
IOExceptionpublic void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException
AdmindeleteTableSnapshots in interface AdmintableNameRegex - The table name regular expression to match againstsnapshotNameRegex - The snapshot name regular expression to match againstIOException - if a remote or network exception occurspublic void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException
AdmindeleteTableSnapshots in interface AdmintableNamePattern - The compiled table name regular expression to match againstsnapshotNamePattern - The compiled snapshot name regular expression to match againstIOException - if a remote or network exception occurspublic void setQuota(QuotaSettings quota) throws IOException
AdminsetQuota in interface Adminquota - the quota settingsIOException - if a remote or network exception occurspublic QuotaRetriever getQuotaRetriever(QuotaFilter filter) throws IOException
AdmingetQuotaRetriever in interface Adminfilter - the quota settings filterIOException - if a remote or network exception occurspublic List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException
AdmingetQuota in interface Adminfilter - the quota settings filterIOException - if a remote or network exception occursprivate <C extends RetryingCallable<V> & Closeable,V> V executeCallable(C callable) throws IOException
IOExceptionprivate static <C extends RetryingCallable<V> & Closeable,V> V executeCallable(C callable, RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout) throws IOException
IOExceptionpublic CoprocessorRpcChannel coprocessorService()
AdminRpcChannel instance connected to the active
master. The obtained RpcChannel instance can be used to access
a published coprocessor Service using standard protobuf service
invocations:
CoprocessorRpcChannel channel = myAdmin.coprocessorService();
MyService.BlockingInterface service = MyService.newBlockingStub(channel);
MyCallRequest request = MyCallRequest.newBuilder()
...
.build();
MyCallResponse response = service.myCall(null, request);
coprocessorService in interface Adminpublic CoprocessorRpcChannel coprocessorService(ServerName serverName)
AdminRpcChannel instance
connected to the passed region server.
The obtained RpcChannel instance can be used to access a published
coprocessor Service using standard protobuf service invocations:
CoprocessorRpcChannel channel = myAdmin.coprocessorService(serverName);
MyService.BlockingInterface service = MyService.newBlockingStub(channel);
MyCallRequest request = MyCallRequest.newBuilder()
...
.build();
MyCallResponse response = service.myCall(null, request);
coprocessorService in interface AdminserverName - the server name to which the endpoint call is madepublic void updateConfiguration(ServerName server) throws IOException
AdminupdateConfiguration in interface Adminserver - : The server whose config needs to be updated.IOExceptionpublic void updateConfiguration() throws IOException
AdminupdateConfiguration in interface AdminIOExceptionpublic long getLastMajorCompactionTimestamp(TableName tableName) throws IOException
AdmingetLastMajorCompactionTimestamp in interface AdmintableName - table to examineIOException - if a remote or network exception occurspublic long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException
AdmingetLastMajorCompactionTimestampForRegion in interface AdminregionName - region to examineIOException - if a remote or network exception occurspublic void compact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException
compact in interface AdmintableName - table to compactcolumnFamily - column family within a tablecompactType - CompactTypeIOException - if not a mob column family or if a remote or network exception occursInterruptedExceptionpublic void compact(TableName tableName, CompactType compactType) throws IOException, InterruptedException
compact in interface AdmintableName - table to compactcompactType - CompactTypeIOException - if a remote or network exception occursInterruptedExceptionpublic void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException
majorCompact in interface AdmintableName - table to compactcolumnFamily - column family within a tablecompactType - CompactTypeIOException - if not a mob column family or if a remote or network exception occursInterruptedExceptionpublic void majorCompact(TableName tableName, CompactType compactType) throws IOException, InterruptedException
majorCompact in interface AdmintableName - table to compactcompactType - CompactTypeIOException - if a remote or network exception occursInterruptedExceptionpublic CompactionState getCompactionState(TableName tableName, CompactType compactType) throws IOException
getCompactionState in interface AdmintableName - table to examinecompactType - CompactTypeIOException - if a remote or network exception occurspublic List<SecurityCapability> getSecurityCapabilities() throws IOException
AdmingetSecurityCapabilities in interface AdminIOExceptionpublic boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException
AdminsplitSwitch in interface Adminenabled - enabled or notsynchronous - If true, it waits until current split() call, if outstanding,
to return.IOExceptionpublic boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException
AdminmergeSwitch in interface Adminenabled - enabled or notsynchronous - If true, it waits until current merge() call, if outstanding,
to return.IOExceptionprivate boolean splitOrMergeSwitch(boolean enabled, boolean synchronous, MasterSwitchType switchType) throws IOException
IOExceptionpublic boolean isSplitEnabled() throws IOException
AdminisSplitEnabled in interface Admintrue if the switch is enabled, false otherwise.IOExceptionpublic boolean isMergeEnabled() throws IOException
AdminisMergeEnabled in interface Admintrue if the switch is enabled, false otherwise.IOExceptionprivate RpcControllerFactory getRpcControllerFactory()
public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException
AdminaddReplicationPeer in interface AdminpeerId - a short name that identifies the peerpeerConfig - configuration for the replication slave clusterenabled - peer state, true if ENABLED and false if DISABLEDIOException - if a remote or network exception occurspublic void removeReplicationPeer(String peerId) throws IOException
AdminremoveReplicationPeer in interface AdminpeerId - a short name that identifies the peerIOException - if a remote or network exception occurspublic void enableReplicationPeer(String peerId) throws IOException
AdminenableReplicationPeer in interface AdminpeerId - a short name that identifies the peerIOException - if a remote or network exception occurspublic void disableReplicationPeer(String peerId) throws IOException
AdmindisableReplicationPeer in interface AdminpeerId - a short name that identifies the peerIOException - if a remote or network exception occurspublic ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException
AdmingetReplicationPeerConfig in interface AdminpeerId - a short name that identifies the peerIOException - if a remote or network exception occurspublic void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) throws IOException
AdminupdateReplicationPeerConfig in interface AdminpeerId - a short name that identifies the peerpeerConfig - new config for the peerIOException - if a remote or network exception occurspublic void appendReplicationPeerTableCFs(String id, Map<TableName,List<String>> tableCfs) throws ReplicationException, IOException
AdminappendReplicationPeerTableCFs in interface Adminid - 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 occurspublic void removeReplicationPeerTableCFs(String id, Map<TableName,List<String>> tableCfs) throws ReplicationException, IOException
AdminremoveReplicationPeerTableCFs in interface Adminid - 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 occurspublic List<ReplicationPeerDescription> listReplicationPeers() throws IOException
AdminlistReplicationPeers in interface AdminIOException - if a remote or network exception occurspublic List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException
AdminlistReplicationPeers in interface Adminpattern - The compiled regular expression to match peer idIOException - if a remote or network exception occurspublic void decommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException
AdmindecommissionRegionServers in interface Adminservers - The list of servers to decommission.offload - True to offload the regions from the decommissioned serversIOExceptionpublic List<ServerName> listDecommissionedRegionServers() throws IOException
AdminlistDecommissionedRegionServers in interface AdminIOExceptionpublic void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) throws IOException
AdminrecommissionRegionServer in interface Adminserver - The server to recommission.encodedRegionNames - Regions to load onto the server.IOExceptionpublic List<TableCFs> listReplicatedTableCFs() throws IOException
AdminlistReplicatedTableCFs in interface AdminIOExceptionpublic void enableTableReplication(TableName tableName) throws IOException
AdminenableTableReplication in interface AdmintableName - name of the tableIOException - if a remote or network exception occurspublic void disableTableReplication(TableName tableName) throws IOException
AdmindisableTableReplication in interface AdmintableName - name of the tableIOException - if a remote or network exception occursprivate void checkAndSyncTableDescToPeers(TableName tableName, byte[][] splits) throws IOException
tableName - name of the table to sync to the peersplits - table split keysIOExceptionprivate void setTableRep(TableName tableName, boolean enableRep) throws IOException
tableName - name of the tableenableRep - is replication switch enable or disableIOException - if a remote or network exception occurspublic void clearCompactionQueues(ServerName sn, Set<String> queues) throws IOException, InterruptedException
AdminclearCompactionQueues in interface Adminsn - the region server namequeues - the set of queue nameIOException - if a remote or network exception occursInterruptedExceptionpublic List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException
AdminclearDeadServers in interface Adminservers - list of dead region servers.IOException - if a remote or network exception occursCopyright © 2007–2019 The Apache Software Foundation. All rights reserved.