Class HBaseAdmin
- All Implemented Interfaces:
Closeable
,AutoCloseable
,Abortable
,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)
- See Also:
-
Nested Class Summary
Modifier and TypeClassDescriptionprivate static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
private static class
protected static class
protected static class
Future that waits on a procedure result.private static class
private static class
private static final class
private static class
protected static class
private static class
private static class
-
Field Summary
Modifier and TypeFieldDescriptionprivate boolean
private final org.apache.hadoop.conf.Configuration
private ClusterConnection
private int
private static final org.slf4j.Logger
private NonceGenerator
private final int
private int
private final long
private RpcRetryingCallerFactory
private RpcControllerFactory
private int
private final int
-
Constructor Summary
-
Method Summary
Modifier and TypeMethodDescriptionvoid
Abort the server or client.boolean
abortProcedure
(long procId, boolean mayInterruptIfRunning) Abort a procedure.abortProcedureAsync
(long procId, boolean mayInterruptIfRunning) Abort a procedure but does not block and wait for completion.addColumnFamilyAsync
(TableName tableName, ColumnFamilyDescriptor columnFamily) Add a column family to an existing table.addReplicationPeerAsync
(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) Add a new replication peer but does not block and wait for it.void
assign
(byte[] regionName) Assign a Region.private org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse
asyncSnapshot
(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) static void
available
(org.apache.hadoop.conf.Configuration conf) Is HBase available? Throw an exception if not.balance
(BalanceRequest request) Invoke the balancer with the given balance request.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 notboolean
cleanerChoreSwitch
(boolean on) Enable/Disable the cleaner chore.private CacheEvictionStats
clearBlockCache
(ServerName sn, List<RegionInfo> hris) 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.clearDeadServers
(List<ServerName> servers) Clear dead region servers from master.clearSlowLogResponses
(Set<ServerName> serverNames) Clears online slow/large RPC logs from the provided list of RegionServersprivate Boolean
clearSlowLogsResponses
(ServerName serverName) cloneSnapshotAsync
(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Create a new table by cloning the snapshot content.void
cloneTableSchema
(TableName tableName, TableName newTableName, boolean preserveSplits) Create a new table by cloning the existent table schema.void
close()
void
closeRegion
(byte[] regionName, String unused) Deprecated.void
closeRegion
(String regionName, String unused) Deprecated.void
closeRegion
(ServerName unused, HRegionInfo hri) 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 a table.void
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.compactionSwitch
(boolean switchState, List<String> serverNamesList) Turn the compaction on or off.void
compactRegion
(byte[] regionName) Compact an individual region.void
compactRegion
(byte[] regionName, byte[] columnFamily) Compact a column family within a region.private void
compactRegion
(byte[] regionName, byte[] columnFamily, boolean major) Compact an individual region.void
compactRegionServer
(ServerName serverName) Compact all regions on the region server.Creates and returns aRpcChannel
instance connected to the active master.coprocessorService
(ServerName serverName) Creates and returns aRpcChannel
instance connected to the passed region server.createNamespaceAsync
(NamespaceDescriptor descriptor) Create a new namespace.void
createTable
(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) Creates a new table with the specified number of regions.Creates a new table but does not block and wait for it to come online.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.deleteColumnFamilyAsync
(TableName tableName, byte[] columnFamily) Delete a column family from a table.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
(String regex) Delete existing snapshots whose names match the pattern passed.void
deleteSnapshots
(Pattern pattern) Delete existing snapshots whose names match the pattern passed.deleteTableAsync
(TableName tableName) Deletes the table but does not block and wait for it to be completely removed.deleteTables
(String regex) Deletes tables matching the passed in pattern and wait on completion.deleteTables
(Pattern pattern) Delete tables matching the passed in pattern and wait on completion.void
deleteTableSnapshots
(String tableNameRegex, String snapshotNameRegex) Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.void
deleteTableSnapshots
(Pattern tableNamePattern, Pattern snapshotNamePattern) Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.disableReplicationPeerAsync
(String peerId) Disable a replication peer but does not block and wait for it.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.disableTables
(String regex) Disable tables matching the passed in pattern and wait on completion.disableTables
(Pattern pattern) Disable tables matching the passed in pattern and wait on completion.enableReplicationPeerAsync
(String peerId) Enable a replication peer but does not block and wait for it.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.enableTables
(String regex) Enable tables matching the passed in pattern and wait on completion.enableTables
(Pattern pattern) Enable tables matching the passed in pattern and wait on completion.boolean
exceedThrottleQuotaSwitch
(boolean enable) Switch the exceed throttle quota.void
Execute a distributed procedure on a cluster.byte[]
Execute a distributed procedure on a cluster.private <C extends RetryingCallable<V> & Closeable,
V>
VexecuteCallable
(C callable) private static <C extends RetryingCallable<V> & Closeable,
V>
VexecuteCallable
(C callable, RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout) private void
flush
(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, RegionInfo info, byte[] columnFamily) void
Flush a table.void
Flush the specified column family stores on all regions of the passed table.flushAsync
(TableName tableName, List<byte[]> columnFamilies) Flush a table but does not block and wait for it to finish.void
Flush master local regionvoid
flushRegion
(byte[] regionName) Flush an individual region.void
flushRegion
(byte[] regionName, byte[] columnFamily) Flush a column family within a region.void
flushRegionServer
(ServerName serverName) Flush all regions on the region server.private static <T> T
Do a get with a timeout against the passed infuture
.getAlterStatus
(byte[] tableName) Get the status ofalter
(a.k.amodify
) command - indicates how many regions have received the updated schema Asynchronous operation.getAlterStatus
(TableName tableName) Get the status of analter
(a.k.amodify
) command - indicates how many regions have received the updated schema Asynchronous operation.getBalancerDecisions
(int limit) getBalancerRejections
(int limit) getCachedFilesList
(ServerName serverName) Get the list of cached filesgetClusterMetrics
(EnumSet<ClusterMetrics.Option> options) Get cluster status with a set ofClusterMetrics.Option
to get desired status.getCompactionState
(TableName tableName) Get the current compaction state of a table.getCompactionState
(TableName tableName, CompactType compactType) Get the current compaction state of a table.getCompactionStateForRegion
(byte[] regionName) Get the current compaction state of region.org.apache.hadoop.conf.Configuration
Returns Configuration used by the instance.Returns Connection used by this object.getCurrentSpaceQuotaSnapshot
(String namespace) Returns the Master's view of a quota on the givennamespace
or null if the Master has no quota information on that namespace.getCurrentSpaceQuotaSnapshot
(TableName tableName) Returns the Master's view of a quota on the giventableName
or null if the Master has no quota information on that table.getDescriptor
(TableName tableName) 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.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.getLocks()
Get locks.getLogEntries
(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String, Object> filterParams) Retrieve recent online records from HMaster / RegionServers.getNamespaceDescriptor
(String name) Get a namespace descriptor by name.Deprecated.As of release 2.0.0, this will be removed in HBase 3.0.0 UsegetRegions(ServerName)
.int
Return the operation timeout for a rpc call.private long
getPauseTime
(int tries) Get procedures.getQuota
(QuotaFilter filter) List the quotas based on the filter.getQuotaRetriever
(QuotaFilter filter) Return a QuotaRetriever to list the quotas based on the filter.(package private) Pair<RegionInfo,
ServerName> getRegion
(byte[] regionName) getRegionMetrics
(ServerName serverName, TableName tableName) GetRegionMetrics
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.getRegions
(ServerName sn) Get all the online regions on a region server.getRegions
(TableName tableName) Get the regions of a given table.getRegionServerSpaceQuotaSnapshots
(ServerName serverName) Fetches the observedSpaceQuotaSnapshotView
s observed by a RegionServer.getReplicationPeerConfig
(String peerId) Returns the configured ReplicationPeerConfig for the specified peer.private RpcControllerFactory
Return the set of supported security capabilities.getSlowLogResponseFromServer
(ServerName serverName, Map<String, Object> filterParams, int limit, String logType) getSlowLogResponses
(Map<String, Object> filterParams, Set<ServerName> serverNames, int limit, String logType) Fetches the table sizes on the filesystem as tracked by the HBase Master.int
Return the blocking wait time for an asynchronous operation.getTableDescriptor
(TableName tableName) Get a table descriptor.(package private) static TableDescriptor
getTableDescriptor
(TableName tableName, Connection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout) getTableDescriptors
(List<String> names) Get tableDescriptors.getTableDescriptorsByTableName
(List<TableName> tableNames) Get tableDescriptors.private TableName
getTableNameBeforeRestoreSnapshot
(String snapshotName) Check whether the snapshot exists and contains disabled tablegetTableRegions
(TableName tableName) Deprecated.As of release 2.0.0, this will be removed in HBase 3.0.0 UsegetRegions(TableName)
.private byte[][]
getTableSplits
(TableName tableName) private MasterCallable<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateRegionResponse>
getTruncateRegionCallable
(TableName tableName, RegionInfo hri) getUserPermissions
(GetUserPermissionsRequest getUserPermissionsRequest) Get the global/namespace/table permissions for uservoid
grant
(UserPermission userPermission, boolean mergeExistingPermissions) Grants user specific permissionshasUserPermissions
(String userName, List<Permission> permissions) Check if the user has specific permissionsprivate void
internalDeleteSnapshot
(SnapshotDescription snapshot) internalRestoreSnapshotAsync
(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Execute Restore/Clone snapshot and wait for the server to complete (blocking).boolean
Check if the server or client was aborted.boolean
Query the current state of the balancer.boolean
Query on the catalog janitor state (Enabled/Disabled?).boolean
Query on the cleaner chore state (Enabled/Disabled?).boolean
Check whether Master is in maintenance mode.boolean
Query the current state of the merge switch.boolean
Query the current state of the region normalizer.boolean
Check the current state of the specified procedure.boolean
isReplicationPeerEnabled
(String peerId) Check if a replication peer is enabled.boolean
Check whether replication peer modification is enabled.boolean
Get if the rpc throttle is enabled.boolean
Query the current state of the auto snapshot cleanup based on TTL.boolean
isSnapshotFinished
(SnapshotDescription snapshotDesc) Check the current state of the passed snapshot.boolean
Query the current state of the split switch.boolean
isTableAvailable
(TableName tableName) Check if a table is available.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) Check if a table is disabled.boolean
isTableEnabled
(TableName tableName) Check if a table is enabled.List region servers marked as decommissioned, which can not be assigned regions.List available namespace descriptorsString[]
List available namespacesFind all table and column families that are replicated from this clusterReturn a list of replication peers.listReplicationPeers
(Pattern pattern) Return a list of replication peers.List completed snapshots.listSnapshots
(String regex) List all the completed snapshots matching the given regular expression.listSnapshots
(Pattern pattern) List all the completed snapshots matching the given pattern.List all the userspace tables.listTableDescriptors
(List<TableName> tableNames) Get tableDescriptors.listTableDescriptors
(Pattern pattern, boolean includeSysTables) List all the tables matching the given pattern.listTableDescriptorsByNamespace
(byte[] name) Get list of table descriptors by namespace.Get list of table descriptors by namespace.listTableDescriptorsByState
(boolean isEnabled) List all enabled or disabled tablesList all of the names of userspace tables.listTableNames
(String regex) List all of the names of userspace tables.listTableNames
(String regex, boolean includeSysTables) List all of the names of userspace tables.listTableNames
(Pattern pattern, boolean includeSysTables) List all of the names of userspace tables.Get list of table names by namespace.listTableNamesByState
(boolean isEnabled) List all enabled or disabled table namesList all the userspace tables.listTables
(String regex) List all the userspace tables matching the given regular expression.listTables
(String regex, boolean includeSysTables) List all the tables matching the given pattern.listTables
(Pattern pattern) List all the userspace tables that match the given pattern.listTables
(Pattern pattern, boolean includeSysTables) List all the tables matching the given pattern.listTableSnapshots
(String tableNameRegex, String snapshotNameRegex) List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.listTableSnapshots
(Pattern tableNamePattern, Pattern snapshotNamePattern) 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.mergeRegionsAsync
(byte[][] nameofRegionsToMerge, boolean forcible) Merge two regions.void
mergeRegionsSync
(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) Merge two regions.boolean
mergeSwitch
(boolean enabled, boolean synchronous) Turn the merge switch on or off.modifyColumnFamilyAsync
(TableName tableName, ColumnFamilyDescriptor columnFamily) Modify an existing column family on a table.modifyColumnFamilyStoreFileTrackerAsync
(TableName tableName, byte[] family, String dstSFT) Change the store file tracker of the given table's given family.modifyNamespaceAsync
(NamespaceDescriptor descriptor) Modify an existing namespace.modifyTableAsync
(TableDescriptor td, boolean reopenRegions) Modify an existing table, more IRB (ruby) friendly version.modifyTableStoreFileTrackerAsync
(TableName tableName, String dstSFT) Change the store file tracker of the given table.void
move
(byte[] encodedRegionName) Move the regionencodedRegionName
to a random server.void
move
(byte[] encodedRegionName, ServerName destServerName) Move the regionrencodedRegionName
todestServerName
.boolean
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.removeReplicationPeerAsync
(String peerId) Remove a replication peer but does not block and wait for it.boolean
replicationPeerModificationSwitch
(boolean on, boolean drainProcedures) Enable or disable replication peer modification.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.restoreSnapshotAsync
(String snapshotName) Restore the specified snapshot on the original table.void
revoke
(UserPermission userPermission) Revokes user specific permissionsvoid
rollWALWriter
(ServerName serverName) Roll the log writer.private org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse
int
Ask for a scan of the catalog table.boolean
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.snapshotAsync
(SnapshotDescription snapshotDesc) Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a single snapshot should be taken at a time, or results may be undefined.boolean
snapshotCleanupSwitch
(boolean on, boolean synchronous) Turn on or off the auto snapshot cleanup based on TTL.void
Split a table.void
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.splitRegionAsync
(byte[] regionName) Split an individual region.splitRegionAsync
(byte[] regionName, byte[] splitPoint) Split an individual region.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
Shuts down the current HBase master only.void
stopRegionServer
(String hostnamePort) Stop the designated regionserver.private Boolean
switchCompact
(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, boolean onOrOff) boolean
switchRpcThrottle
(boolean enable) Switch the rpc throttle enable state.boolean
tableExists
(TableName tableName) Check if a table exists.void
truncateRegion
(byte[] regionName) Truncate an individual region.truncateRegionAsync
(byte[] regionName) Truncate an individual region.truncateTableAsync
(TableName tableName, boolean preserveSplits) Truncate the table but does not block and wait for it to be completely enabled.void
unassign
(byte[] regionName) Unassign a Region.void
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.updateReplicationPeerConfigAsync
(String peerId, ReplicationPeerConfig peerConfig) Update the peerConfig for the specified peer but does not block and wait for it.Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
Methods inherited from interface org.apache.hadoop.hbase.client.Admin
addColumn, addColumnFamily, addReplicationPeer, addReplicationPeer, addReplicationPeerAsync, appendReplicationPeerTableCFs, balance, balance, balancer, balancer, cloneSnapshot, cloneSnapshot, cloneSnapshot, cloneSnapshot, cloneSnapshotAsync, cloneSnapshotAsync, compactRegionServer, createNamespace, createTable, createTable, deleteColumnFamily, deleteNamespace, deleteTable, disableReplicationPeer, disableTable, enableCatalogJanitor, enableReplicationPeer, enableTable, execProcedureWithRet, flush, getBackupMasters, getClusterMetrics, getClusterStatus, getMaster, getMasterCoprocessorNames, getMasterCoprocessors, getMasterInfoPort, getRegionMetrics, getRegionServers, getRegionServers, getSlowLogResponses, hasUserPermissions, isSplitOrMergeEnabled, listDeadServers, listTableDescriptors, listTableNames, listUnknownServers, mergeRegionsAsync, modifyColumn, modifyColumnFamily, modifyColumnFamilyStoreFileTracker, modifyNamespace, modifyTable, modifyTable, modifyTable, modifyTableAsync, modifyTableAsync, modifyTableStoreFileTracker, move, normalize, removeReplicationPeer, removeReplicationPeerTableCFs, replicationPeerModificationSwitch, runCatalogScan, setBalancerRunning, setCleanerChoreRunning, setNormalizerRunning, setSplitOrMergeEnabled, snapshot, snapshot, snapshot, snapshot, snapshot, snapshot, takeSnapshotAsync, truncateTable, unassign, updateReplicationPeerConfig
-
Field Details
-
LOG
-
connection
-
conf
-
pause
-
numRetries
-
syncWaitTimeout
-
aborted
-
operationTimeout
-
rpcTimeout
-
getProcedureTimeout
-
rpcCallerFactory
-
rpcControllerFactory
-
ng
-
-
Constructor Details
-
HBaseAdmin
HBaseAdmin(ClusterConnection connection) throws IOException - Throws:
IOException
-
-
Method Details
-
getOperationTimeout
Description copied from interface:Admin
Return the operation timeout for a rpc call.- Specified by:
getOperationTimeout
in interfaceAdmin
- See Also:
-
getSyncWaitTimeout
Description copied from interface:Admin
Return the blocking wait time for an asynchronous operation. Can be configured byhbase.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 aboveAdmin.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).- Specified by:
getSyncWaitTimeout
in interfaceAdmin
- See Also:
-
abort
Description copied from interface:Abortable
Abort the server or client. -
isAborted
Description copied from interface:Abortable
Check if the server or client was aborted. -
abortProcedure
Description copied from interface:Admin
Abort a procedure. Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.- Specified by:
abortProcedure
in interfaceAdmin
- Parameters:
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?- Returns:
true
if aborted,false
if procedure already completed or does not exist- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
abortProcedureAsync
public Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning) throws IOException Description copied from interface:Admin
Abort a procedure but does not block and wait for completion. 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. Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.- Specified by:
abortProcedureAsync
in interfaceAdmin
- Parameters:
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?- Returns:
true
if aborted,false
if procedure already completed or does not exist- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
listTableDescriptors
Description copied from interface:Admin
List all the userspace tables.- Specified by:
listTableDescriptors
in interfaceAdmin
- Returns:
- a list of TableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
listTableDescriptorsByState
Description copied from interface:Admin
List all enabled or disabled tables- Specified by:
listTableDescriptorsByState
in interfaceAdmin
- Parameters:
isEnabled
- is true means return enabled tables, false means return disabled tables- Returns:
- a list of enabled or disabled tables
- Throws:
IOException
-
listTableDescriptors
public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) throws IOException Description copied from interface:Admin
List all the tables matching the given pattern.- Specified by:
listTableDescriptors
in interfaceAdmin
- Parameters:
pattern
- The compiled regular expression to match againstincludeSysTables
-false
to match only against userspace tables- Returns:
- a list of TableDescriptors
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
getDescriptor
public TableDescriptor getDescriptor(TableName tableName) throws TableNotFoundException, IOException Description copied from interface:Admin
Get a table descriptor.- Specified by:
getDescriptor
in interfaceAdmin
- Parameters:
tableName
- as aTableName
- Returns:
- the tableDescriptor
- Throws:
TableNotFoundException
- if the table was not foundIOException
- if a remote or network exception occurs
-
modifyTableAsync
Description copied from interface:Admin
Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means that it may be a while before your schema change is updated across all of the table. 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.- Specified by:
modifyTableAsync
in interfaceAdmin
- Parameters:
td
- description of the tablereopenRegions
- By default, 'modifyTableAsync' reopens all regions, potentially causing a RIT(Region In Transition) storm in large tables. If set to 'false', regions will remain unaware of the modification until they are individually reopened. Please note that this may temporarily result in configuration inconsistencies among regions.- Returns:
- the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the operation to complete
- Throws:
IOException
- if a remote or network exception occurs
-
modifyTableStoreFileTrackerAsync
public Future<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT) throws IOException Description copied from interface:Admin
Change the store file tracker of the given table.- Specified by:
modifyTableStoreFileTrackerAsync
in interfaceAdmin
- Parameters:
tableName
- the table you want to changedstSFT
- the destination store file tracker- Returns:
- the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the operation to complete
- Throws:
IOException
- if a remote or network exception occurs
-
listTableDescriptorsByNamespace
Description copied from interface:Admin
Get list of table descriptors by namespace.- Specified by:
listTableDescriptorsByNamespace
in interfaceAdmin
- Parameters:
name
- namespace name- Returns:
- returns a list of TableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
listTableDescriptors
Description copied from interface:Admin
Get tableDescriptors.- Specified by:
listTableDescriptors
in interfaceAdmin
- Parameters:
tableNames
- List of table names- Returns:
- returns a list of TableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
getRegions
Description copied from interface:Admin
Get all the online regions on a region server.- Specified by:
getRegions
in interfaceAdmin
- Returns:
- List of
RegionInfo
- Throws:
IOException
- if a remote or network exception occurs
-
getRegions
Description copied from interface:Admin
Get the regions of a given table.- Specified by:
getRegions
in interfaceAdmin
- Parameters:
tableName
- the name of the table- Returns:
- List of
RegionInfo
. - Throws:
IOException
- if a remote or network exception occurs
-
getConnection
Returns Connection used by this object.- Specified by:
getConnection
in interfaceAdmin
-
tableExists
Description copied from interface:Admin
Check if a table exists.- Specified by:
tableExists
in interfaceAdmin
- Parameters:
tableName
- Table to check.- Returns:
true
if table exists already.- Throws:
IOException
- if a remote or network exception occurs
-
listTables
Description copied from interface:Admin
List all the userspace tables.- Specified by:
listTables
in interfaceAdmin
- Returns:
- an array of read-only HTableDescriptors
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
listTables
Description copied from interface:Admin
List all the userspace tables that match the given pattern.- Specified by:
listTables
in interfaceAdmin
- Parameters:
pattern
- The compiled regular expression to match against- Returns:
- an array of read-only HTableDescriptors
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
listTables
Description copied from interface:Admin
List all the userspace tables matching the given regular expression.- Specified by:
listTables
in interfaceAdmin
- Parameters:
regex
- The regular expression to match against- Returns:
- a list of read-only HTableDescriptors
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
listTables
Description copied from interface:Admin
List all the tables matching the given pattern.- Specified by:
listTables
in interfaceAdmin
- Parameters:
pattern
- The compiled regular expression to match againstincludeSysTables
-false
to match only against userspace tables- Returns:
- an array of read-only HTableDescriptors
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
listTables
Description copied from interface:Admin
List all the tables matching the given pattern.- Specified by:
listTables
in interfaceAdmin
- Parameters:
regex
- The regular expression to match againstincludeSysTables
-false
to match only against userspace tables- Returns:
- an array of read-only HTableDescriptors
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
listTableNames
Description copied from interface:Admin
List all of the names of userspace tables.- Specified by:
listTableNames
in interfaceAdmin
- Returns:
- TableName[] table names
- Throws:
IOException
- if a remote or network exception occurs
-
listTableNames
Description copied from interface:Admin
List all of the names of userspace tables.- Specified by:
listTableNames
in interfaceAdmin
- Parameters:
regex
- The regular expression to match against- Returns:
- TableName[] table names
- Throws:
IOException
- if a remote or network exception occurs
-
listTableNames
Description copied from interface:Admin
List all of the names of userspace tables.- Specified by:
listTableNames
in interfaceAdmin
- Parameters:
pattern
- The regular expression to match againstincludeSysTables
-false
to match only against userspace tables- Returns:
- TableName[] table names
- Throws:
IOException
- if a remote or network exception occurs
-
listTableNames
Description copied from interface:Admin
List all of the names of userspace tables.- Specified by:
listTableNames
in interfaceAdmin
- Parameters:
regex
- The regular expression to match againstincludeSysTables
-false
to match only against userspace tables- Returns:
- TableName[] table names
- Throws:
IOException
- if a remote or network exception occurs
-
listTableNamesByState
Description copied from interface:Admin
List all enabled or disabled table names- Specified by:
listTableNamesByState
in interfaceAdmin
- Parameters:
isEnabled
- is true means return enabled table names, false means return disabled table names- Returns:
- a list of enabled or disabled table names
- Throws:
IOException
-
getTableDescriptor
Description copied from interface:Admin
Get a table descriptor.- Specified by:
getTableDescriptor
in interfaceAdmin
- Parameters:
tableName
- as aTableName
- Returns:
- the read-only tableDescriptor
- Throws:
TableNotFoundException
- if the table was not foundIOException
- if a remote or network exception occurs
-
getTableDescriptor
static TableDescriptor getTableDescriptor(TableName tableName, Connection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout) throws IOException - Throws:
IOException
-
getHTableDescriptor
@Deprecated static HTableDescriptor getHTableDescriptor(TableName tableName, Connection connection, RpcRetryingCallerFactory rpcCallerFactory, RpcControllerFactory rpcControllerFactory, int operationTimeout, int rpcTimeout) throws IOException Deprecated.since 2.0 version and will be removed in 3.0 version. usegetTableDescriptor(TableName, Connection, RpcRetryingCallerFactory,RpcControllerFactory,int,int)
- Throws:
IOException
-
getPauseTime
-
createTable
public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) throws IOException Description copied from interface:Admin
Creates a new table with the specified number of regions. The start key specified will become the end key of the first region of the table, and the end key specified will become the start key of the last region of the table (the first region has a null start key and the last region has a null end key). BigInteger math will be used to divide the key range specified into enough segments to make the required number of total regions. Synchronous operation.- Specified by:
createTable
in interfaceAdmin
- Parameters:
desc
- table descriptor for tablestartKey
- beginning of key rangeendKey
- end of key rangenumRegions
- the total number of regions to create- Throws:
IOException
- 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).
-
createTableAsync
Description copied from interface:Admin
Creates a new table but does not block and wait for it to come online. 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. Throws IllegalArgumentException Bad table name, if the split keys are repeated and if the split key has empty byte array.- Specified by:
createTableAsync
in interfaceAdmin
- Parameters:
desc
- table descriptor for tablesplitKeys
- keys to check if the table has been created with all split keys- Returns:
- the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
deleteTableAsync
Description copied from interface:Admin
Deletes the table but does not block and wait for it to be completely removed. 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.- Specified by:
deleteTableAsync
in interfaceAdmin
- Parameters:
tableName
- name of table to delete- Returns:
- the result of the async delete. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
deleteTables
Description copied from interface:Admin
Deletes tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider usingAdmin.listTableDescriptors(Pattern)
andAdmin.deleteTable(org.apache.hadoop.hbase.TableName)
- Specified by:
deleteTables
in interfaceAdmin
- Parameters:
regex
- The regular expression to match table names against- Returns:
- Table descriptors for tables that couldn't be deleted. The return htds are read-only
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
deleteTables
Delete tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider usinglistTables(java.util.regex.Pattern)
andAdmin.deleteTable(TableName)
- Specified by:
deleteTables
in interfaceAdmin
- Parameters:
pattern
- The pattern to match table names against- Returns:
- Table descriptors for tables that couldn't be deleted
- Throws:
IOException
- if a remote or network exception occurs
-
truncateTableAsync
public Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits) throws IOException Description copied from interface:Admin
Truncate the table but does not block and wait for it to be completely enabled. 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.- Specified by:
truncateTableAsync
in interfaceAdmin
- Parameters:
tableName
- name of table to deletepreserveSplits
-true
if the splits should be preserved- Returns:
- the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
getTableSplits
- Throws:
IOException
-
enableTableAsync
Description copied from interface:Admin
Enable the table but does not block and wait for it to be completely enabled. 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.- Specified by:
enableTableAsync
in interfaceAdmin
- Parameters:
tableName
- name of table to delete- Returns:
- the result of the async enable. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
enableTables
Description copied from interface:Admin
Enable tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider usingAdmin.listTableDescriptors(Pattern)
andAdmin.enableTable(org.apache.hadoop.hbase.TableName)
- Specified by:
enableTables
in interfaceAdmin
- Parameters:
regex
- The regular expression to match table names against- Returns:
- Table descriptors for tables that couldn't be enabled. The return HTDs are read-only.
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
enableTables
Description copied from interface:Admin
Enable tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider usingAdmin.listTableDescriptors(java.util.regex.Pattern)
andAdmin.enableTable(org.apache.hadoop.hbase.TableName)
- Specified by:
enableTables
in interfaceAdmin
- Parameters:
pattern
- The pattern to match table names against- Returns:
- Table descriptors for tables that couldn't be enabled. The return HTDs are read-only.
- Throws:
IOException
- if a remote or network exception occurs
-
disableTableAsync
Description copied from interface:Admin
Disable the table but does not block and wait for it to be completely disabled. 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.- Specified by:
disableTableAsync
in interfaceAdmin
- Parameters:
tableName
- name of table to delete- Returns:
- the result of the async disable. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
disableTables
Description copied from interface:Admin
Disable tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider usingAdmin.listTableDescriptors(Pattern)
andAdmin.disableTable(org.apache.hadoop.hbase.TableName)
- Specified by:
disableTables
in interfaceAdmin
- Parameters:
regex
- The regular expression to match table names against- Returns:
- Table descriptors for tables that couldn't be disabled The return htds are read-only
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
disableTables
Description copied from interface:Admin
Disable tables matching the passed in pattern and wait on completion. Warning: Use this method carefully, there is no prompting and the effect is immediate. Consider usingAdmin.listTableDescriptors(java.util.regex.Pattern)
andAdmin.disableTable(org.apache.hadoop.hbase.TableName)
- Specified by:
disableTables
in interfaceAdmin
- Parameters:
pattern
- The pattern to match table names against- Returns:
- Table descriptors for tables that couldn't be disabled The return htds are read-only
- Throws:
IOException
- if a remote or network exception occurs
-
isTableEnabled
Description copied from interface:Admin
Check if a table is enabled.- Specified by:
isTableEnabled
in interfaceAdmin
- Parameters:
tableName
- name of table to check- Returns:
true
if table is on-line- Throws:
IOException
- if a remote or network exception occurs
-
isTableDisabled
Description copied from interface:Admin
Check if a table is disabled.- Specified by:
isTableDisabled
in interfaceAdmin
- Parameters:
tableName
- name of table to check- Returns:
true
if table is off-line- Throws:
IOException
- if a remote or network exception occurs
-
isTableAvailable
Description copied from interface:Admin
Check if a table is available.- Specified by:
isTableAvailable
in interfaceAdmin
- Parameters:
tableName
- name of table to check- Returns:
true
if all regions of the table are available- Throws:
IOException
- if a remote or network exception occurs
-
isTableAvailable
Description copied from interface:Admin
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. Note : If this api is used after a table's region gets splitted, the api may returnfalse
.- Specified by:
isTableAvailable
in interfaceAdmin
- Parameters:
tableName
- name of table to checksplitKeys
- keys to check if the table has been created with all split keys- Throws:
IOException
- if a remote or network excpetion occurs
-
getAlterStatus
Description copied from interface:Admin
Get the status of analter
(a.k.amodify
) command - indicates how many regions have received the updated schema Asynchronous operation.- Specified by:
getAlterStatus
in interfaceAdmin
- Parameters:
tableName
- TableName instance- Returns:
- Pair indicating the number of regions updated Pair.getFirst() is the regions that are yet to be updated Pair.getSecond() is the total number of regions of the table
- Throws:
IOException
- if a remote or network exception occurs
-
getAlterStatus
Description copied from interface:Admin
Get the status ofalter
(a.k.amodify
) command - indicates how many regions have received the updated schema Asynchronous operation.- Specified by:
getAlterStatus
in interfaceAdmin
- Parameters:
tableName
- name of the table to get the status of- Returns:
- Pair indicating the number of regions updated Pair.getFirst() is the regions that are yet to be updated Pair.getSecond() is the total number of regions of the table
- Throws:
IOException
- if a remote or network exception occurs
-
addColumnFamilyAsync
public Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException Description copied from interface:Admin
Add a column family to an existing table. Asynchronous operation. 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.- Specified by:
addColumnFamilyAsync
in interfaceAdmin
- Parameters:
tableName
- name of the table to add column family tocolumnFamily
- column family descriptor of column family to be added- Returns:
- the result of the async add column family. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
deleteColumn
Deprecated.Since 2.0. Will be removed in 3.0. UseAdmin.deleteColumnFamily(TableName, byte[])
instead.Delete a column family from a table. Synchronous operation. UseAdmin.deleteColumnFamily(TableName, byte[])
instead because it returns aFuture
from which you can learn whether success or failure.- Specified by:
deleteColumn
in interfaceAdmin
- Parameters:
tableName
- name of tablecolumnFamily
- name of column family to be deleted- Throws:
IOException
- if a remote or network exception occurs
-
deleteColumnFamilyAsync
public Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily) throws IOException Description copied from interface:Admin
Delete a column family from a table. Asynchronous operation. 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.- Specified by:
deleteColumnFamilyAsync
in interfaceAdmin
- Parameters:
tableName
- name of tablecolumnFamily
- name of column family to be deleted- Returns:
- the result of the async delete column family. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
modifyColumnFamilyAsync
public Future<Void> modifyColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException Description copied from interface:Admin
Modify an existing column family on a table. Asynchronous operation. 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.- Specified by:
modifyColumnFamilyAsync
in interfaceAdmin
- Parameters:
tableName
- name of tablecolumnFamily
- new column family descriptor to use- Returns:
- the result of the async modify column family. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
modifyColumnFamilyStoreFileTrackerAsync
public Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family, String dstSFT) throws IOException Description copied from interface:Admin
Change the store file tracker of the given table's given family.- Specified by:
modifyColumnFamilyStoreFileTrackerAsync
in interfaceAdmin
- Parameters:
tableName
- the table you want to changefamily
- the family you want to changedstSFT
- the destination store file tracker- Returns:
- the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the operation to complete
- Throws:
IOException
- if a remote or network exception occurs
-
closeRegion
Deprecated.Description copied from interface:Admin
UsesAdmin.unassign(byte[], boolean)
to unassign the region. For expert-admins.- Specified by:
closeRegion
in interfaceAdmin
- Parameters:
regionName
- region name to closeunused
- Deprecated. Not used.- Throws:
IOException
- if a remote or network exception occurs
-
closeRegion
Deprecated.Description copied from interface:Admin
UsesAdmin.unassign(byte[], boolean)
to unassign the region. For expert-admins.- Specified by:
closeRegion
in interfaceAdmin
- Parameters:
regionName
- region name to closeunused
- Deprecated. Not used.- Throws:
IOException
- if a remote or network exception occurs
-
closeRegionWithEncodedRegionName
@Deprecated public boolean closeRegionWithEncodedRegionName(String encodedRegionName, String unused) throws IOException Deprecated.Description copied from interface:Admin
UsesAdmin.unassign(byte[], boolean)
to unassign the region. For expert-admins.- Specified by:
closeRegionWithEncodedRegionName
in interfaceAdmin
- Parameters:
encodedRegionName
- The encoded region name; i.e. the hash that makes up the region name suffix: e.g. if regionname isTestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.
, then the encoded region name is:527db22f95c8a9e0116f0cc13c680396
.unused
- Deprecated. Not used.- Returns:
- Deprecated. Returns
true
always. - Throws:
IOException
- if a remote or network exception occurs
-
closeRegion
Deprecated.Description copied from interface:Admin
UsedAdmin.unassign(byte[], boolean)
to unassign the region. For expert-admins.- Specified by:
closeRegion
in interfaceAdmin
- Parameters:
unused
- Deprecated. Not used.- Throws:
IOException
- if a remote or network exception occurs
-
getOnlineRegions
Deprecated.As of release 2.0.0, this will be removed in HBase 3.0.0 UsegetRegions(ServerName)
.Description copied from interface:Admin
Get all the online regions on a region server.- Specified by:
getOnlineRegions
in interfaceAdmin
- Returns:
- List of
HRegionInfo
. - Throws:
IOException
- if a remote or network exception occurs
-
flush
Description copied from interface:Admin
Flush a table. Synchronous operation.- Specified by:
flush
in interfaceAdmin
- Parameters:
tableName
- table to flush- Throws:
IOException
- if a remote or network exception occurs
-
flush
Description copied from interface:Admin
Flush the specified column family stores on all regions of the passed table. This runs as a synchronous operation.- Specified by:
flush
in interfaceAdmin
- Parameters:
tableName
- table to flushcolumnFamily
- column family within a table- Throws:
IOException
- if a remote or network exception occurs
-
flushAsync
Description copied from interface:Admin
Flush a table but does not block and wait for it to finish. 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.- Specified by:
flushAsync
in interfaceAdmin
- Parameters:
tableName
- table to flushcolumnFamilies
- column families within a table- Returns:
- the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
flushRegion
Description copied from interface:Admin
Flush an individual region. Synchronous operation.- Specified by:
flushRegion
in interfaceAdmin
- Parameters:
regionName
- region to flush- Throws:
IOException
- if a remote or network exception occurs
-
flushRegion
Description copied from interface:Admin
Flush a column family within a region. Synchronous operation.- Specified by:
flushRegion
in interfaceAdmin
- Parameters:
regionName
- region to flushcolumnFamily
- column family within a region- Throws:
IOException
- if a remote or network exception occurs
-
flush
private void flush(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, RegionInfo info, byte[] columnFamily) throws IOException - Throws:
IOException
-
flushRegionServer
Description copied from interface:Admin
Flush all regions on the region server. Synchronous operation.- Specified by:
flushRegionServer
in interfaceAdmin
- Parameters:
serverName
- the region server name to flush- Throws:
IOException
- if a remote or network exception occurs
-
compact
Compact a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
compact
in interfaceAdmin
- Parameters:
tableName
- table to compact- Throws:
IOException
- if a remote or network exception occurs
-
compactRegion
Description copied from interface:Admin
Compact an individual region. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
compactRegion
in interfaceAdmin
- Parameters:
regionName
- region to compact- Throws:
IOException
- if a remote or network exception occurs
-
compact
Compact a column family within a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
compact
in interfaceAdmin
- Parameters:
tableName
- table to compactcolumnFamily
- column family within a table- Throws:
IOException
- if a remote or network exception occurs
-
compactRegion
Compact a column family within a region. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
compactRegion
in interfaceAdmin
- Parameters:
regionName
- region to compactcolumnFamily
- column family within a region- Throws:
IOException
- if a remote or network exception occurs
-
compactionSwitch
public Map<ServerName,Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList) throws IOException Description copied from interface:Admin
Turn the compaction on or off. Disabling compactions will also interrupt any currently ongoing compactions. This state is ephemeral. The setting will be lost on restart. Compaction can also be enabled/disabled by modifying configuration hbase.regionserver.compaction.enabled in hbase-site.xml.- Specified by:
compactionSwitch
in interfaceAdmin
- Parameters:
switchState
- Set totrue
to enable,false
to disable.serverNamesList
- list of region servers.- Returns:
- Previous compaction states for region servers
- Throws:
IOException
-
switchCompact
private Boolean switchCompact(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, boolean onOrOff) throws IOException - Throws:
IOException
-
compactRegionServer
Description copied from interface:Admin
Compact all regions on the region server. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
compactRegionServer
in interfaceAdmin
- Parameters:
serverName
- the region server name- Throws:
IOException
- if a remote or network exception occurs
-
majorCompactRegionServer
Description copied from interface:Admin
Major compact all regions on the region server. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
majorCompactRegionServer
in interfaceAdmin
- Parameters:
serverName
- the region server name- Throws:
IOException
- if a remote or network exception occurs
-
majorCompact
Description copied from interface:Admin
Major compact a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
majorCompact
in interfaceAdmin
- Parameters:
tableName
- table to major compact- Throws:
IOException
- if a remote or network exception occurs
-
majorCompactRegion
Description copied from interface:Admin
Major compact a table or an individual region. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
majorCompactRegion
in interfaceAdmin
- Parameters:
regionName
- region to major compact- Throws:
IOException
- if a remote or network exception occurs
-
majorCompact
Major compact a column family within a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
majorCompact
in interfaceAdmin
- Parameters:
tableName
- table to major compactcolumnFamily
- column family within a table- Throws:
IOException
- if a remote or network exception occurs
-
majorCompactRegion
Description copied from interface:Admin
Major compact a column family within region. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
majorCompactRegion
in interfaceAdmin
- Parameters:
regionName
- egion to major compactcolumnFamily
- column family within a region- Throws:
IOException
- if a remote or network exception occurs
-
compact
private void compact(TableName tableName, byte[] columnFamily, boolean major, CompactType compactType) throws IOException Compact a table. Asynchronous operation.- Parameters:
tableName
- table or region to compactcolumnFamily
- column family within a table or regionmajor
- True if we are to do a major compaction.compactType
-CompactType
- Throws:
IOException
- if a remote or network exception occurs
-
compactRegion
private void compactRegion(byte[] regionName, byte[] columnFamily, boolean major) throws IOException Compact an individual region. Asynchronous operation.- Parameters:
regionName
- region to compactcolumnFamily
- column family within a table or regionmajor
- True if we are to do a major compaction.- Throws:
IOException
- if a remote or network exception occurs
-
compact
private void compact(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface admin, RegionInfo hri, boolean major, byte[] family) throws IOException - Throws:
IOException
-
move
Description copied from interface:Admin
Move the regionencodedRegionName
to a random server.- Specified by:
move
in interfaceAdmin
- Parameters:
encodedRegionName
- The encoded region name; i.e. the hash that makes up the region name suffix: e.g. if regionname isTestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.
, then the encoded region name is:527db22f95c8a9e0116f0cc13c680396
.- Throws:
IOException
- if we can't find a region namedencodedRegionName
-
move
Description copied from interface:Admin
Move the regionrencodedRegionName
todestServerName
.- Specified by:
move
in interfaceAdmin
- Parameters:
encodedRegionName
- The encoded region name; i.e. the hash that makes up the region name suffix: e.g. if regionname isTestTable,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
- Throws:
IOException
- if we can't find a region namedencodedRegionName
-
assign
public void assign(byte[] regionName) throws MasterNotRunningException, ZooKeeperConnectionException, IOException Description copied from interface:Admin
Assign a Region.- Specified by:
assign
in interfaceAdmin
- Parameters:
regionName
- Region name to assign.- Throws:
IOException
- if a remote or network exception occursMasterNotRunningException
ZooKeeperConnectionException
-
unassign
Description copied from interface:Admin
Unassign a Region.- Specified by:
unassign
in interfaceAdmin
- Parameters:
regionName
- Region name to unassign.- Throws:
IOException
- if a remote or network exception occurs
-
offline
Description copied from interface:Admin
Offline specified region from master's in-memory state. It will not attempt to reassign the region as in unassign. This API can be used when a region not served by any region server and still online as per Master's in memory state. If this API is incorrectly used on active region then master will loose track of that region. This is a special method that should be used by experts or hbck.- Specified by:
offline
in interfaceAdmin
- Parameters:
regionName
- Region to offline.- Throws:
IOException
- if a remote or network exception occurs
-
balancerSwitch
Description copied from interface:Admin
Turn the load balancer on or off.- Specified by:
balancerSwitch
in interfaceAdmin
- Parameters:
on
- Set totrue
to enable,false
to disable.synchronous
- Iftrue
, it waits until current balance() call, if outstanding, to return.- Returns:
- Previous balancer value
- Throws:
IOException
- if a remote or network exception occurs
-
balance
Description copied from interface:Admin
Invoke the balancer with the given balance request. The BalanceRequest defines how the balancer will run. SeeBalanceRequest
for more details.- Specified by:
balance
in interfaceAdmin
- Parameters:
request
- defines how the balancer should run- Returns:
BalanceResponse
with details about the results of the invocation.- Throws:
IOException
- if a remote or network exception occurs
-
isBalancerEnabled
Description copied from interface:Admin
Query the current state of the balancer.- Specified by:
isBalancerEnabled
in interfaceAdmin
- Returns:
true
if the balancer is enabled,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
clearBlockCache
Clear all the blocks corresponding to this table from BlockCache. For expert-admins. Calling this API will drop all the cached blocks specific to a table from BlockCache. This can significantly impact the query performance as the subsequent queries will have to retrieve the blocks from underlying filesystem.- Specified by:
clearBlockCache
in interfaceAdmin
- Parameters:
tableName
- table to clear block cache- Returns:
- CacheEvictionStats related to the eviction
- Throws:
IOException
- if a remote or network exception occurs
-
clearBlockCache
- Throws:
IOException
-
normalize
Description copied from interface:Admin
Invoke region normalizer. Can NOT run for various reasons. Check logs. This is a non-blocking invocation to region normalizer. If return value is true, it means the request was submitted successfully. We need to check logs for the details of which regions were split/merged.- Specified by:
normalize
in interfaceAdmin
- Parameters:
ntfp
- limit to tables matching the specified filter.- Returns:
true
if region normalizer ran,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
isNormalizerEnabled
Description copied from interface:Admin
Query the current state of the region normalizer.- Specified by:
isNormalizerEnabled
in interfaceAdmin
- Returns:
true
if region normalizer is enabled,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
normalizerSwitch
Description copied from interface:Admin
Turn region normalizer on or off.- Specified by:
normalizerSwitch
in interfaceAdmin
- Returns:
- Previous normalizer value
- Throws:
IOException
- if a remote or network exception occurs
-
catalogJanitorSwitch
Description copied from interface:Admin
Enable/Disable the catalog janitor/- Specified by:
catalogJanitorSwitch
in interfaceAdmin
- Parameters:
enable
- iftrue
enables the catalog janitor- Returns:
- the previous state
- Throws:
IOException
- if a remote or network exception occurs
-
runCatalogJanitor
Description copied from interface:Admin
Ask for a scan of the catalog table.- Specified by:
runCatalogJanitor
in interfaceAdmin
- Returns:
- the number of entries cleaned
- Throws:
IOException
- if a remote or network exception occurs
-
isCatalogJanitorEnabled
Description copied from interface:Admin
Query on the catalog janitor state (Enabled/Disabled?).- Specified by:
isCatalogJanitorEnabled
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
cleanerChoreSwitch
Description copied from interface:Admin
Enable/Disable the cleaner chore.- Specified by:
cleanerChoreSwitch
in interfaceAdmin
- Parameters:
on
- iftrue
enables the cleaner chore- Returns:
- the previous state
- Throws:
IOException
- if a remote or network exception occurs
-
runCleanerChore
Description copied from interface:Admin
Ask for cleaner chore to run.- Specified by:
runCleanerChore
in interfaceAdmin
- Returns:
true
if cleaner chore ran,false
otherwise- Throws:
IOException
- if a remote or network exception occurs
-
isCleanerChoreEnabled
Description copied from interface:Admin
Query on the cleaner chore state (Enabled/Disabled?).- Specified by:
isCleanerChoreEnabled
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
mergeRegionsSync
public void mergeRegionsSync(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException Merge two regions. Synchronous operation. Note: It is not feasible to predict the length of merge. Therefore, this is for internal testing only.- Parameters:
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 regions- Throws:
IOException
- if a remote or network exception occurs
-
mergeRegions
@Deprecated public void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException Deprecated.Since 2.0. Will be removed in 3.0. UseAdmin.mergeRegionsAsync(byte[], byte[], boolean)
instead.Merge two regions. Asynchronous operation.- Specified by:
mergeRegions
in interfaceAdmin
- Parameters:
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 regions- Throws:
IOException
- if a remote or network exception occurs
-
mergeRegionsAsync
public Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible) throws IOException Merge two regions. Asynchronous operation.- Specified by:
mergeRegionsAsync
in interfaceAdmin
- Parameters:
nameofRegionsToMerge
- encoded or full name of daughter regionsforcible
- true if do a compulsory merge, otherwise we will only merge adjacent regions- Throws:
IOException
- if a remote or network exception occurs
-
splitRegionSync
Split one region. Synchronous operation. Note: It is not feasible to predict the length of split. Therefore, this is for internal testing only.- Parameters:
regionName
- encoded or full name of regionsplitPoint
- key where region splits- Throws:
IOException
- if a remote or network exception occurs
-
splitRegionSync
public void splitRegionSync(byte[] regionName, byte[] splitPoint, long timeout, TimeUnit units) throws IOException Split one region. Synchronous operation.- Parameters:
regionName
- region to be splitsplitPoint
- split pointtimeout
- how long to wait on splitunits
- time units- Throws:
IOException
- if a remote or network exception occurs
-
splitRegionAsync
Description copied from interface:Admin
Split an individual region. Asynchronous operation.- Specified by:
splitRegionAsync
in interfaceAdmin
- Parameters:
regionName
- region to splitsplitPoint
- the explicit position to split on- Throws:
IOException
- if a remote or network exception occurs
-
splitRegionAsync
- Throws:
IOException
-
split
Description copied from interface:Admin
Split a table. The method will execute split action for each region in table. Asynchronous operation.- Specified by:
split
in interfaceAdmin
- Parameters:
tableName
- table to split- Throws:
IOException
- if a remote or network exception occurs
-
splitRegion
Description copied from interface:Admin
Split an individual region. Asynchronous operation.- Specified by:
splitRegion
in interfaceAdmin
- Parameters:
regionName
- region to split- Throws:
IOException
- if a remote or network exception occurs
-
split
Description copied from interface:Admin
Split a table. Asynchronous operation.- Specified by:
split
in interfaceAdmin
- Parameters:
tableName
- table to splitsplitPoint
- the explicit position to split on- Throws:
IOException
- if a remote or network exception occurs
-
splitRegion
Description copied from interface:Admin
Split an individual region. Asynchronous operation.- Specified by:
splitRegion
in interfaceAdmin
- Parameters:
regionName
- region to splitsplitPoint
- the explicit position to split on- Throws:
IOException
- if a remote or network exception occurs
-
truncateRegion
Description copied from interface:Admin
Truncate an individual region.- Specified by:
truncateRegion
in interfaceAdmin
- Parameters:
regionName
- region to truncate- Throws:
IOException
- if a remote or network exception occurs
-
truncateRegionAsync
Description copied from interface:Admin
Truncate an individual region. Asynchronous operation.- Specified by:
truncateRegionAsync
in interfaceAdmin
- Parameters:
regionName
- region to truncate- Throws:
IOException
- if a remote or network exception occurs
-
getCachedFilesList
Get the list of cached files- Specified by:
getCachedFilesList
in interfaceAdmin
- Throws:
IOException
-
getTruncateRegionCallable
private MasterCallable<org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateRegionResponse> getTruncateRegionCallable(TableName tableName, RegionInfo hri) -
getRegion
- Parameters:
regionName
- Name of a region.- Returns:
- a pair of HRegionInfo and ServerName if
regionName
is a verified region name (we callMetaTableAccessor.getRegionLocation(Connection, byte[])
else null. Throw IllegalArgumentException ifregionName
is null. - Throws:
IOException
- if a remote or network exception occurs
-
getRegionName
If the input is a region name, it is returned as is. If it's an encoded region name, the corresponding region is found from meta and its region name is returned. If we can't find any region in meta matching the input as either region name or encoded region name, the input is returned as is. We don't throw unknown region exception.- Throws:
IOException
-
checkTableExists
Check if table exists or not- Parameters:
tableName
- Name of a table.- Returns:
- tableName instance
- Throws:
IOException
- if a remote or network exception occurs.TableNotFoundException
- if table does not exist.
-
shutdown
Description copied from interface:Admin
Shuts down the HBase cluster. Notice that, a success shutdown call may ends with an error since the remote server has already been shutdown.- Specified by:
shutdown
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
stopMaster
Description copied from interface:Admin
Shuts 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.- Specified by:
stopMaster
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
stopRegionServer
Description copied from interface:Admin
Stop the designated regionserver.- Specified by:
stopRegionServer
in interfaceAdmin
- Parameters:
hostnamePort
- Hostname and port delimited by a:
as inexample.org:1234
- Throws:
IOException
- if a remote or network exception occurs
-
isMasterInMaintenanceMode
Description copied from interface:Admin
Check whether Master is in maintenance mode.- Specified by:
isMasterInMaintenanceMode
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
getClusterMetrics
Description copied from interface:Admin
Get cluster status with a set ofClusterMetrics.Option
to get desired status.- Specified by:
getClusterMetrics
in interfaceAdmin
- Returns:
- cluster status
- Throws:
IOException
- if a remote or network exception occurs
-
getRegionMetrics
public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName) throws IOException Description copied from interface:Admin
GetRegionMetrics
of all regions hosted on a regionserver for a table.- Specified by:
getRegionMetrics
in interfaceAdmin
- Parameters:
serverName
- region server from whichRegionMetrics
is required.tableName
- getRegionMetrics
of regions belonging to the table- Returns:
- region metrics map of all regions of a table hosted on a region server
- Throws:
IOException
- if a remote or network exception occurs
-
getConfiguration
Description copied from interface:Admin
Returns Configuration used by the instance.- Specified by:
getConfiguration
in interfaceAdmin
-
get
Do a get with a timeout against the passed infuture
.- Throws:
IOException
-
createNamespaceAsync
Description copied from interface:Admin
Create a new namespace.- Specified by:
createNamespaceAsync
in interfaceAdmin
- Parameters:
descriptor
- descriptor which describes the new namespace- Returns:
- the result of the async create namespace operation. Use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
modifyNamespaceAsync
Description copied from interface:Admin
Modify an existing namespace.- Specified by:
modifyNamespaceAsync
in interfaceAdmin
- Parameters:
descriptor
- descriptor which describes the new namespace- Returns:
- the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
deleteNamespaceAsync
Description copied from interface:Admin
Delete an existing namespace. Only empty namespaces (no tables) can be removed.- Specified by:
deleteNamespaceAsync
in interfaceAdmin
- Parameters:
name
- namespace name- Returns:
- the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
getNamespaceDescriptor
public NamespaceDescriptor getNamespaceDescriptor(String name) throws NamespaceNotFoundException, IOException Description copied from interface:Admin
Get a namespace descriptor by name.- Specified by:
getNamespaceDescriptor
in interfaceAdmin
- Parameters:
name
- name of namespace descriptor- Returns:
- A descriptor
- Throws:
NamespaceNotFoundException
- if the namespace was not foundIOException
- if a remote or network exception occurs
-
listNamespaces
List available namespaces- Specified by:
listNamespaces
in interfaceAdmin
- Returns:
- List of namespace names
- Throws:
IOException
- if a remote or network exception occurs
-
listNamespaceDescriptors
List available namespace descriptors- Specified by:
listNamespaceDescriptors
in interfaceAdmin
- Returns:
- List of descriptors
- Throws:
IOException
- if a remote or network exception occurs
-
getProcedures
Description copied from interface:Admin
Get procedures.- Specified by:
getProcedures
in interfaceAdmin
- Returns:
- procedure list in JSON
- Throws:
IOException
- if a remote or network exception occurs
-
getLocks
Description copied from interface:Admin
Get locks.- Specified by:
getLocks
in interfaceAdmin
- Returns:
- lock list in JSON
- Throws:
IOException
- if a remote or network exception occurs
-
listTableDescriptorsByNamespace
Description copied from interface:Admin
Get list of table descriptors by namespace.- Specified by:
listTableDescriptorsByNamespace
in interfaceAdmin
- Parameters:
name
- namespace name- Returns:
- HTD[] the read-only tableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
listTableNamesByNamespace
Description copied from interface:Admin
Get list of table names by namespace.- Specified by:
listTableNamesByNamespace
in interfaceAdmin
- Parameters:
name
- namespace name- Returns:
- The list of table names in the namespace
- Throws:
IOException
- if a remote or network exception occurs
-
available
public static void available(org.apache.hadoop.conf.Configuration conf) throws MasterNotRunningException, ZooKeeperConnectionException, IOException Is HBase available? Throw an exception if not.- Parameters:
conf
- system configuration- Throws:
MasterNotRunningException
- if the master is not running.ZooKeeperConnectionException
- if unable to connect to zookeeper. // TODO do not expose ZKConnectionException.IOException
-
getTableRegions
Deprecated.As of release 2.0.0, this will be removed in HBase 3.0.0 UsegetRegions(TableName)
.Description copied from interface:Admin
Get the regions of a given table.- Specified by:
getTableRegions
in interfaceAdmin
- Parameters:
tableName
- the name of the table- Returns:
- List of
HRegionInfo
. - Throws:
IOException
- if a remote or network exception occurs
-
close
- Specified by:
close
in interfaceAdmin
- Specified by:
close
in interfaceAutoCloseable
- Specified by:
close
in interfaceCloseable
- Throws:
IOException
-
getTableDescriptorsByTableName
public HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames) throws IOException Description copied from interface:Admin
Get tableDescriptors.- Specified by:
getTableDescriptorsByTableName
in interfaceAdmin
- Parameters:
tableNames
- List of table names- Returns:
- HTD[] the read-only tableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
getTableDescriptors
Description copied from interface:Admin
Get tableDescriptors.- Specified by:
getTableDescriptors
in interfaceAdmin
- Parameters:
names
- List of table names- Returns:
- HTD[] the read-only tableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
rollWALWriterImpl
private org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriterImpl(ServerName sn) throws IOException, FailedLogCloseException - Throws:
IOException
FailedLogCloseException
-
rollWALWriter
Description copied from interface:Admin
Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file. Note that the actual rolling of the log writer is asynchronous and may not be complete when this method returns. As a side effect of this call, the named region server may schedule store flushes at the request of the wal.- Specified by:
rollWALWriter
in interfaceAdmin
- Parameters:
serverName
- The servername of the regionserver.- Throws:
IOException
- if a remote or network exception occursFailedLogCloseException
- if we failed to close the WAL
-
getCompactionState
Description copied from interface:Admin
Get the current compaction state of a table. It could be in a major compaction, a minor compaction, both, or none.- Specified by:
getCompactionState
in interfaceAdmin
- Parameters:
tableName
- table to examine- Returns:
- the current compaction state
- Throws:
IOException
- if a remote or network exception occurs
-
getCompactionStateForRegion
Description copied from interface:Admin
Get the current compaction state of region. It could be in a major compaction, a minor compaction, both, or none.- Specified by:
getCompactionStateForRegion
in interfaceAdmin
- Parameters:
regionName
- region to examine- Returns:
- the current compaction state
- Throws:
IOException
- if a remote or network exception occurs
-
snapshotAsync
public Future<Void> snapshotAsync(SnapshotDescription snapshotDesc) throws IOException, SnapshotCreationException Description copied from interface:Admin
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.- Specified by:
snapshotAsync
in interfaceAdmin
- Parameters:
snapshotDesc
- snapshot to take- Throws:
IOException
- if the snapshot did not succeed or we lose contact with the master.SnapshotCreationException
- if snapshot creation failed
-
asyncSnapshot
private org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse asyncSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshot) throws IOException - Throws:
IOException
-
isSnapshotFinished
public boolean isSnapshotFinished(SnapshotDescription snapshotDesc) throws IOException, HBaseSnapshotException, UnknownSnapshotException Description copied from interface:Admin
Check the current state of the passed snapshot. There are three possible states:- running - returns false
- finished - returns true
- finished with error - throws the exception that caused the snapshot to fail
UnknownSnapshotException
.- Specified by:
isSnapshotFinished
in interfaceAdmin
- Parameters:
snapshotDesc
- description of the snapshot to check- Returns:
- true if the snapshot is completed, false if the snapshot is still running
- Throws:
IOException
- if we have a network issueHBaseSnapshotException
- if the snapshot failedUnknownSnapshotException
- if the requested snapshot is unknown
-
restoreSnapshot
Description copied from interface:Admin
Restore the specified snapshot on the original table. (The table must be disabled) If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set totrue
, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted.- Specified by:
restoreSnapshot
in interfaceAdmin
- Parameters:
snapshotName
- name of the snapshot to restore- Throws:
IOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restored
-
restoreSnapshot
Description copied from interface:Admin
Restore the specified snapshot on the original table. (The table must be disabled) If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set totrue
, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted.- Specified by:
restoreSnapshot
in interfaceAdmin
- Parameters:
snapshotName
- name of the snapshot to restore- Throws:
IOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restored
-
restoreSnapshot
public void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException Description copied from interface:Admin
Restore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set totrue
, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".- Specified by:
restoreSnapshot
in interfaceAdmin
- Parameters:
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
-true
if the failsafe snapshot should be taken- Throws:
IOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restored
-
getTableNameBeforeRestoreSnapshot
private TableName getTableNameBeforeRestoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException Check whether the snapshot exists and contains disabled table- Parameters:
snapshotName
- name of the snapshot to restore- Throws:
IOException
- if a remote or network exception occursRestoreSnapshotException
- if no valid snapshot is found
-
restoreSnapshot
public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException Description copied from interface:Admin
Restore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set totrue
, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".- Specified by:
restoreSnapshot
in interfaceAdmin
- Parameters:
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
-true
if the failsafe snapshot should be taken- Throws:
IOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restored
-
restoreSnapshot
public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) throws IOException, RestoreSnapshotException Description copied from interface:Admin
Restore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set totrue
, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".- Specified by:
restoreSnapshot
in interfaceAdmin
- Parameters:
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
-true
if the failsafe snapshot should be takenrestoreAcl
-true
to restore acl of snapshot- Throws:
IOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restored
-
restoreSnapshotAsync
public Future<Void> restoreSnapshotAsync(String snapshotName) throws IOException, RestoreSnapshotException Description copied from interface:Admin
Restore the specified snapshot on the original table. (The table must be disabled) If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set totrue
, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted.- Specified by:
restoreSnapshotAsync
in interfaceAdmin
- Parameters:
snapshotName
- name of the snapshot to restore- Returns:
- the result of the async restore snapshot. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restored
-
cloneSnapshotAsync
public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException Description copied from interface:Admin
Create a new table by cloning the snapshot content.- Specified by:
cloneSnapshotAsync
in interfaceAdmin
- Parameters:
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredrestoreAcl
-true
to clone acl into newly created tablecustomSFT
- specify the StroreFileTracker used for the table- Throws:
IOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be cloned
-
execProcedureWithReturn
public byte[] execProcedureWithReturn(String signature, String instance, Map<String, String> props) throws IOExceptionDescription copied from interface:Admin
Execute a distributed procedure on a cluster.- Specified by:
execProcedureWithReturn
in interfaceAdmin
- Parameters:
signature
- A distributed procedure is uniquely identified by its signature (default the root ZK node name of the procedure).instance
- The instance name of the procedure. For some procedures, this parameter is optional.props
- Property/Value pairs of properties passing to the procedure- Returns:
- data returned after procedure execution. null if no return data.
- Throws:
IOException
- if a remote or network exception occurs
-
execProcedure
public void execProcedure(String signature, String instance, Map<String, String> props) throws IOExceptionDescription copied from interface:Admin
Execute a distributed procedure on a cluster.- Specified by:
execProcedure
in interfaceAdmin
- Parameters:
signature
- A distributed procedure is uniquely identified by its signature (default the root ZK node name of the procedure).instance
- The instance name of the procedure. For some procedures, this parameter is optional.props
- Property/Value pairs of properties passing to the procedure- Throws:
IOException
- if a remote or network exception occurs
-
isProcedureFinished
public boolean isProcedureFinished(String signature, String instance, Map<String, String> props) throws IOExceptionDescription copied from interface:Admin
Check the current state of the specified procedure. There are three possible states:- running - returns false
- finished - returns true
- finished with error - throws the exception that caused the procedure to fail
- Specified by:
isProcedureFinished
in interfaceAdmin
- Parameters:
signature
- The signature that uniquely identifies a procedureinstance
- The instance name of the procedureprops
- Property/Value pairs of properties passing to the procedure- Returns:
true
if the specified procedure is finished successfully,false
if it is still running- Throws:
IOException
- if the specified procedure finished with error
-
internalRestoreSnapshotAsync
private Future<Void> internalRestoreSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, RestoreSnapshotException Execute Restore/Clone snapshot and wait for the server to complete (blocking). To check if the cloned table exists, useisTableAvailable(org.apache.hadoop.hbase.TableName)
-- it is not safe to create an HTable instance to this table before it is available.- Parameters:
snapshotName
- snapshot to restoretableName
- table name to restore the snapshot on- Throws:
IOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectly
-
listSnapshots
Description copied from interface:Admin
List completed snapshots.- Specified by:
listSnapshots
in interfaceAdmin
- Returns:
- a list of snapshot descriptors for completed snapshots
- Throws:
IOException
- if a network error occurs
-
listSnapshots
Description copied from interface:Admin
List all the completed snapshots matching the given regular expression.- Specified by:
listSnapshots
in interfaceAdmin
- Parameters:
regex
- The regular expression to match against- Returns:
- list of SnapshotDescription
- Throws:
IOException
- if a remote or network exception occurs
-
listSnapshots
Description copied from interface:Admin
List all the completed snapshots matching the given pattern.- Specified by:
listSnapshots
in interfaceAdmin
- Parameters:
pattern
- The compiled regular expression to match against- Returns:
- list of SnapshotDescription
- Throws:
IOException
- if a remote or network exception occurs
-
listTableSnapshots
public List<SnapshotDescription> listTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException Description copied from interface:Admin
List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.- Specified by:
listTableSnapshots
in interfaceAdmin
- Parameters:
tableNameRegex
- The table name regular expression to match againstsnapshotNameRegex
- The snapshot name regular expression to match against- Returns:
- list of completed SnapshotDescription
- Throws:
IOException
- if a remote or network exception occurs
-
listTableSnapshots
public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException Description copied from interface:Admin
List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.- Specified by:
listTableSnapshots
in interfaceAdmin
- Parameters:
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match against- Returns:
- list of completed SnapshotDescription
- Throws:
IOException
- if a remote or network exception occurs
-
deleteSnapshot
Description copied from interface:Admin
Delete an existing snapshot.- Specified by:
deleteSnapshot
in interfaceAdmin
- Parameters:
snapshotName
- name of the snapshot- Throws:
IOException
- if a remote or network exception occurs
-
deleteSnapshot
Description copied from interface:Admin
Delete an existing snapshot.- Specified by:
deleteSnapshot
in interfaceAdmin
- Parameters:
snapshotName
- name of the snapshot- Throws:
IOException
- if a remote or network exception occurs
-
deleteSnapshots
Description copied from interface:Admin
Delete existing snapshots whose names match the pattern passed.- Specified by:
deleteSnapshots
in interfaceAdmin
- Parameters:
regex
- The regular expression to match against- Throws:
IOException
- if a remote or network exception occurs
-
deleteSnapshots
Description copied from interface:Admin
Delete existing snapshots whose names match the pattern passed.- Specified by:
deleteSnapshots
in interfaceAdmin
- Parameters:
pattern
- pattern for names of the snapshot to match- Throws:
IOException
- if a remote or network exception occurs
-
internalDeleteSnapshot
- Throws:
IOException
-
deleteTableSnapshots
public void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException Description copied from interface:Admin
Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.- Specified by:
deleteTableSnapshots
in interfaceAdmin
- Parameters:
tableNameRegex
- The table name regular expression to match againstsnapshotNameRegex
- The snapshot name regular expression to match against- Throws:
IOException
- if a remote or network exception occurs
-
deleteTableSnapshots
public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException Description copied from interface:Admin
Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.- Specified by:
deleteTableSnapshots
in interfaceAdmin
- Parameters:
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match against- Throws:
IOException
- if a remote or network exception occurs
-
setQuota
Description copied from interface:Admin
Apply the new quota settings.- Specified by:
setQuota
in interfaceAdmin
- Parameters:
quota
- the quota settings- Throws:
IOException
- if a remote or network exception occurs
-
getQuotaRetriever
Description copied from interface:Admin
Return a QuotaRetriever to list the quotas based on the filter.- Specified by:
getQuotaRetriever
in interfaceAdmin
- Parameters:
filter
- the quota settings filter- Returns:
- the quota retriever
- Throws:
IOException
- if a remote or network exception occurs
-
getQuota
Description copied from interface:Admin
List the quotas based on the filter.- Specified by:
getQuota
in interfaceAdmin
- Parameters:
filter
- the quota settings filter- Returns:
- the QuotaSetting list
- Throws:
IOException
- if a remote or network exception occurs
-
executeCallable
private <C extends RetryingCallable<V> & Closeable,V> V executeCallable(C callable) throws IOException - Throws:
IOException
-
executeCallable
private static <C extends RetryingCallable<V> & Closeable,V> V executeCallable(C callable, RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout, int rpcTimeout) throws IOException - Throws:
IOException
-
coprocessorService
Description copied from interface:Admin
Creates and returns aRpcChannel
instance connected to the active master.The obtained
RpcChannel
instance can be used to access a published coprocessorService
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);
- Specified by:
coprocessorService
in interfaceAdmin
- Returns:
- A MasterCoprocessorRpcChannel instance
-
coprocessorService
Description copied from interface:Admin
Creates and returns aRpcChannel
instance connected to the passed region server.The obtained
RpcChannel
instance can be used to access a published coprocessorService
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);
- Specified by:
coprocessorService
in interfaceAdmin
- Parameters:
serverName
- the server name to which the endpoint call is made- Returns:
- A RegionServerCoprocessorRpcChannel instance
-
updateConfiguration
Description copied from interface:Admin
Update the configuration and trigger an online config change on the regionserver.- Specified by:
updateConfiguration
in interfaceAdmin
- Parameters:
server
- : The server whose config needs to be updated.- Throws:
IOException
- if a remote or network exception occurs
-
updateConfiguration
Description copied from interface:Admin
Update the configuration and trigger an online config change on all the regionservers.- Specified by:
updateConfiguration
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
getLastMajorCompactionTimestamp
Description copied from interface:Admin
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.- Specified by:
getLastMajorCompactionTimestamp
in interfaceAdmin
- Parameters:
tableName
- table to examine- Returns:
- the last major compaction timestamp or 0
- Throws:
IOException
- if a remote or network exception occurs
-
getLastMajorCompactionTimestampForRegion
Description copied from interface:Admin
Get the timestamp of the last major compaction for the passed region. The timestamp of the oldest HFile resulting from a major compaction of that region, or 0 if no such HFile could be found.- Specified by:
getLastMajorCompactionTimestampForRegion
in interfaceAdmin
- Parameters:
regionName
- region to examine- Returns:
- the last major compaction timestamp or 0
- Throws:
IOException
- if a remote or network exception occurs
-
compact
public void compact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException Compact a column family within a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
compact
in interfaceAdmin
- Parameters:
tableName
- table to compactcolumnFamily
- column family within a tablecompactType
-CompactType
- Throws:
IOException
- if not a mob column family or if a remote or network exception occursInterruptedException
-
compact
public void compact(TableName tableName, CompactType compactType) throws IOException, InterruptedException Compact a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
compact
in interfaceAdmin
- Parameters:
tableName
- table to compactcompactType
-CompactType
- Throws:
IOException
- if a remote or network exception occursInterruptedException
-
majorCompact
public void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) throws IOException, InterruptedException Major compact a column family within a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
majorCompact
in interfaceAdmin
- Parameters:
tableName
- table to compactcolumnFamily
- column family within a tablecompactType
-CompactType
- Throws:
IOException
- if not a mob column family or if a remote or network exception occursInterruptedException
-
majorCompact
public void majorCompact(TableName tableName, CompactType compactType) throws IOException, InterruptedException Major compact a table. Asynchronous operation in that this method requests that a Compaction run and then it returns. It does not wait on the completion of Compaction (it can take a while).- Specified by:
majorCompact
in interfaceAdmin
- Parameters:
tableName
- table to compactcompactType
-CompactType
- Throws:
IOException
- if a remote or network exception occursInterruptedException
-
getCompactionState
public CompactionState getCompactionState(TableName tableName, CompactType compactType) throws IOException Get the current compaction state of a table. It could be in a compaction, or none.- Specified by:
getCompactionState
in interfaceAdmin
- Parameters:
tableName
- table to examinecompactType
-CompactType
- Returns:
- the current compaction state
- Throws:
IOException
- if a remote or network exception occurs
-
getSecurityCapabilities
Description copied from interface:Admin
Return the set of supported security capabilities.- Specified by:
getSecurityCapabilities
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
splitSwitch
Description copied from interface:Admin
Turn the split switch on or off.- Specified by:
splitSwitch
in interfaceAdmin
- Parameters:
enabled
- enabled or notsynchronous
- Iftrue
, it waits until current split() call, if outstanding, to return.- Returns:
- Previous switch value
- Throws:
IOException
- if a remote or network exception occurs
-
mergeSwitch
Description copied from interface:Admin
Turn the merge switch on or off.- Specified by:
mergeSwitch
in interfaceAdmin
- Parameters:
enabled
- enabled or notsynchronous
- Iftrue
, it waits until current merge() call, if outstanding, to return.- Returns:
- Previous switch value
- Throws:
IOException
- if a remote or network exception occurs
-
splitOrMergeSwitch
private boolean splitOrMergeSwitch(boolean enabled, boolean synchronous, MasterSwitchType switchType) throws IOException - Throws:
IOException
-
isSplitEnabled
Description copied from interface:Admin
Query the current state of the split switch.- Specified by:
isSplitEnabled
in interfaceAdmin
- Returns:
true
if the switch is enabled,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
isMergeEnabled
Description copied from interface:Admin
Query the current state of the merge switch.- Specified by:
isMergeEnabled
in interfaceAdmin
- Returns:
true
if the switch is enabled,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
getRpcControllerFactory
-
addReplicationPeerAsync
public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException Description copied from interface:Admin
Add a new replication peer but does not block and wait for it.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.
- Specified by:
addReplicationPeerAsync
in interfaceAdmin
- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peerenabled
- peer state, true if ENABLED and false if DISABLED- Returns:
- the result of the async operation
- Throws:
IOException
- IOException if a remote or network exception occurs
-
removeReplicationPeerAsync
Description copied from interface:Admin
Remove a replication peer but does not block and wait for it.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.
- Specified by:
removeReplicationPeerAsync
in interfaceAdmin
- Parameters:
peerId
- a short name that identifies the peer- Returns:
- the result of the async operation
- Throws:
IOException
- IOException if a remote or network exception occurs
-
enableReplicationPeerAsync
Description copied from interface:Admin
Enable a replication peer but does not block and wait for it.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.
- Specified by:
enableReplicationPeerAsync
in interfaceAdmin
- Parameters:
peerId
- a short name that identifies the peer- Returns:
- the result of the async operation
- Throws:
IOException
- IOException if a remote or network exception occurs
-
disableReplicationPeerAsync
Description copied from interface:Admin
Disable a replication peer but does not block and wait for it. 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.- Specified by:
disableReplicationPeerAsync
in interfaceAdmin
- Parameters:
peerId
- a short name that identifies the peer- Returns:
- the result of the async operation
- Throws:
IOException
- IOException if a remote or network exception occurs
-
getReplicationPeerConfig
Description copied from interface:Admin
Returns the configured ReplicationPeerConfig for the specified peer.- Specified by:
getReplicationPeerConfig
in interfaceAdmin
- Parameters:
peerId
- a short name that identifies the peer- Returns:
- ReplicationPeerConfig for the peer
- Throws:
IOException
- if a remote or network exception occurs
-
updateReplicationPeerConfigAsync
public Future<Void> updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig) throws IOException Description copied from interface:Admin
Update the peerConfig for the specified peer but does not block and wait for it. 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.- Specified by:
updateReplicationPeerConfigAsync
in interfaceAdmin
- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- new config for the replication peer- Returns:
- the result of the async operation
- Throws:
IOException
- IOException if a remote or network exception occurs
-
listReplicationPeers
Description copied from interface:Admin
Return a list of replication peers.- Specified by:
listReplicationPeers
in interfaceAdmin
- Returns:
- a list of replication peers description
- Throws:
IOException
- if a remote or network exception occurs
-
listReplicationPeers
Description copied from interface:Admin
Return a list of replication peers.- Specified by:
listReplicationPeers
in interfaceAdmin
- Parameters:
pattern
- The compiled regular expression to match peer id- Returns:
- a list of replication peers description
- Throws:
IOException
- if a remote or network exception occurs
-
decommissionRegionServers
Description copied from interface:Admin
Mark region server(s) as decommissioned to prevent additional regions from getting assigned to them. Optionally unload the regions on the servers. If there are multiple servers to be decommissioned, decommissioning them at the same time can prevent wasteful region movements. Region unloading is asynchronous.- Specified by:
decommissionRegionServers
in interfaceAdmin
- Parameters:
servers
- The list of servers to decommission.offload
- True to offload the regions from the decommissioned servers- Throws:
IOException
- if a remote or network exception occurs
-
listDecommissionedRegionServers
Description copied from interface:Admin
List region servers marked as decommissioned, which can not be assigned regions.- Specified by:
listDecommissionedRegionServers
in interfaceAdmin
- Returns:
- List of decommissioned region servers.
- Throws:
IOException
- if a remote or network exception occurs
-
recommissionRegionServer
public void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) throws IOException Description copied from interface:Admin
Remove decommission marker from a region server to allow regions assignments. Load regions onto the server if a list of regions is given. Region loading is asynchronous.- Specified by:
recommissionRegionServer
in interfaceAdmin
- Parameters:
server
- The server to recommission.encodedRegionNames
- Regions to load onto the server.- Throws:
IOException
- if a remote or network exception occurs
-
listReplicatedTableCFs
Description copied from interface:Admin
Find all table and column families that are replicated from this cluster- Specified by:
listReplicatedTableCFs
in interfaceAdmin
- Returns:
- the replicated table-cfs list of this cluster.
- Throws:
IOException
- if a remote or network exception occurs
-
enableTableReplication
Description copied from interface:Admin
Enable a table's replication switch.- Specified by:
enableTableReplication
in interfaceAdmin
- Parameters:
tableName
- name of the table- Throws:
IOException
- if a remote or network exception occurs
-
disableTableReplication
Description copied from interface:Admin
Disable a table's replication switch.- Specified by:
disableTableReplication
in interfaceAdmin
- Parameters:
tableName
- name of the table- Throws:
IOException
- if a remote or network exception occurs
-
isReplicationPeerEnabled
Description copied from interface:Admin
Check if a replication peer is enabled.- Specified by:
isReplicationPeerEnabled
in interfaceAdmin
- Parameters:
peerId
- id of replication peer to check- Returns:
true
if replication peer is enabled- Throws:
IOException
- if a remote or network exception occurs
-
checkAndSyncTableDescToPeers
Connect to peer and check the table descriptor on peer:- Create the same table on peer when not exist.
- Throw an exception if the table already has replication enabled on any of the column families.
- Throw an exception if the table exists on peer cluster but descriptors are not same.
- Parameters:
tableName
- name of the table to sync to the peersplits
- table split keys- Throws:
IOException
- if a remote or network exception occurs
-
setTableRep
Set the table's replication switch if the table's replication switch is already not set.- Parameters:
tableName
- name of the tableenableRep
- is replication switch enable or disable- Throws:
IOException
- if a remote or network exception occurs
-
clearCompactionQueues
public void clearCompactionQueues(ServerName sn, Set<String> queues) throws IOException, InterruptedException Description copied from interface:Admin
Clear compacting queues on a regionserver.- Specified by:
clearCompactionQueues
in interfaceAdmin
- Parameters:
sn
- the region server namequeues
- the set of queue name- Throws:
IOException
- if a remote or network exception occursInterruptedException
-
clearDeadServers
Description copied from interface:Admin
Clear dead region servers from master.- Specified by:
clearDeadServers
in interfaceAdmin
- Parameters:
servers
- list of dead region servers.- Returns:
- List of servers that are not cleared
- Throws:
IOException
- if a remote or network exception occurs
-
cloneTableSchema
public void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) throws IOException Description copied from interface:Admin
Create a new table by cloning the existent table schema.- Specified by:
cloneTableSchema
in interfaceAdmin
- Parameters:
tableName
- name of the table to be clonednewTableName
- name of the new table where the table will be createdpreserveSplits
- True if the splits should be preserved- Throws:
IOException
- if a remote or network exception occurs
-
switchRpcThrottle
Description copied from interface:Admin
Switch the rpc throttle enable state.- Specified by:
switchRpcThrottle
in interfaceAdmin
- Parameters:
enable
- Set totrue
to enable,false
to disable.- Returns:
- Previous rpc throttle enabled value
- Throws:
IOException
- if a remote or network exception occurs
-
isRpcThrottleEnabled
Description copied from interface:Admin
Get if the rpc throttle is enabled.- Specified by:
isRpcThrottleEnabled
in interfaceAdmin
- Returns:
- True if rpc throttle is enabled
- Throws:
IOException
- if a remote or network exception occurs
-
exceedThrottleQuotaSwitch
Description copied from interface:Admin
Switch the exceed throttle quota. If enabled, user/table/namespace throttle quota can be exceeded if region server has availble quota.- Specified by:
exceedThrottleQuotaSwitch
in interfaceAdmin
- Parameters:
enable
- Set totrue
to enable,false
to disable.- Returns:
- Previous exceed throttle enabled value
- Throws:
IOException
- if a remote or network exception occurs
-
getSpaceQuotaTableSizes
Description copied from interface:Admin
Fetches the table sizes on the filesystem as tracked by the HBase Master.- Specified by:
getSpaceQuotaTableSizes
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
getRegionServerSpaceQuotaSnapshots
public Map<TableName,SpaceQuotaSnapshot> getRegionServerSpaceQuotaSnapshots(ServerName serverName) throws IOException Description copied from interface:Admin
Fetches the observedSpaceQuotaSnapshotView
s observed by a RegionServer.- Specified by:
getRegionServerSpaceQuotaSnapshots
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
getCurrentSpaceQuotaSnapshot
Description copied from interface:Admin
Returns the Master's view of a quota on the givennamespace
or null if the Master has no quota information on that namespace.- Specified by:
getCurrentSpaceQuotaSnapshot
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
getCurrentSpaceQuotaSnapshot
Description copied from interface:Admin
Returns the Master's view of a quota on the giventableName
or null if the Master has no quota information on that table.- Specified by:
getCurrentSpaceQuotaSnapshot
in interfaceAdmin
- Throws:
IOException
- if a remote or network exception occurs
-
grant
public void grant(UserPermission userPermission, boolean mergeExistingPermissions) throws IOException Description copied from interface:Admin
Grants user specific permissions- Specified by:
grant
in interfaceAdmin
- Parameters:
userPermission
- user name and the specific permissionmergeExistingPermissions
- If set to false, later granted permissions will override previous granted permissions. otherwise, it'll merge with previous granted permissions.- Throws:
IOException
- if a remote or network exception occurs
-
revoke
Description copied from interface:Admin
Revokes user specific permissions- Specified by:
revoke
in interfaceAdmin
- Parameters:
userPermission
- user name and the specific permission- Throws:
IOException
- if a remote or network exception occurs
-
getUserPermissions
public List<UserPermission> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) throws IOException Description copied from interface:Admin
Get the global/namespace/table permissions for user- Specified by:
getUserPermissions
in interfaceAdmin
- Parameters:
getUserPermissionsRequest
- A request contains which user, global, namespace or table permissions needed- Returns:
- The user and permission list
- Throws:
IOException
- if a remote or network exception occurs
-
splitRegionAsync
Description copied from interface:Admin
Split an individual region. Asynchronous operation.- Specified by:
splitRegionAsync
in interfaceAdmin
- Parameters:
regionName
- region to split- Throws:
IOException
- if a remote or network exception occurs
-
createTableAsync
Description copied from interface:Admin
Creates a new table but does not block and wait for it to come online. 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. Throws IllegalArgumentException Bad table name, if the split keys are repeated and if the split key has empty byte array.- Specified by:
createTableAsync
in interfaceAdmin
- Parameters:
desc
- table descriptor for table- Returns:
- the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occurs
-
hasUserPermissions
public List<Boolean> hasUserPermissions(String userName, List<Permission> permissions) throws IOException Description copied from interface:Admin
Check if the user has specific permissions- Specified by:
hasUserPermissions
in interfaceAdmin
- Parameters:
userName
- the user namepermissions
- the specific permission list- Returns:
- True if user has the specific permissions
- Throws:
IOException
- if a remote or network exception occurs
-
snapshotCleanupSwitch
Description copied from interface:Admin
Turn on or off the auto snapshot cleanup based on TTL.- Specified by:
snapshotCleanupSwitch
in interfaceAdmin
- Parameters:
on
- Set totrue
to enable,false
to disable.synchronous
- Iftrue
, it waits until current snapshot cleanup is completed, if outstanding.- Returns:
- Previous auto snapshot cleanup value
- Throws:
IOException
- if a remote or network exception occurs
-
isSnapshotCleanupEnabled
Description copied from interface:Admin
Query the current state of the auto snapshot cleanup based on TTL.- Specified by:
isSnapshotCleanupEnabled
in interfaceAdmin
- Returns:
true
if the auto snapshot cleanup is enabled,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
getSlowLogResponses
private List<LogEntry> getSlowLogResponses(Map<String, Object> filterParams, Set<ServerName> serverNames, int limit, String logType) -
getSlowLogResponseFromServer
private List<LogEntry> getSlowLogResponseFromServer(ServerName serverName, Map<String, Object> filterParams, int limit, String logType) throws IOException- Throws:
IOException
-
clearSlowLogResponses
public List<Boolean> clearSlowLogResponses(@Nullable Set<ServerName> serverNames) throws IOException Description copied from interface:Admin
Clears online slow/large RPC logs from the provided list of RegionServers- Specified by:
clearSlowLogResponses
in interfaceAdmin
- Parameters:
serverNames
- Set of Server names to clean slowlog responses from- Returns:
- List of booleans representing if online slowlog response buffer is cleaned from each RegionServer
- Throws:
IOException
- if a remote or network exception occurs
-
getLogEntries
public List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String, Object> filterParams) throws IOExceptionDescription copied from interface:Admin
Retrieve recent online records from HMaster / RegionServers. Examples include slow/large RPC logs, balancer decisions by master.- Specified by:
getLogEntries
in interfaceAdmin
- Parameters:
serverNames
- servers to retrieve records from, useful in case of records maintained by RegionServer as we can select specific server. In case of servertype=MASTER, logs will only come from the currently active master.logType
- string representing type of log recordsserverType
- enum for server type: HMaster or RegionServerlimit
- put a limit to list of records that server should send in responsefilterParams
- additional filter params- Returns:
- Log entries representing online records from servers
- Throws:
IOException
- if a remote or network exception occurs
-
flushMasterStore
Description copied from interface:Admin
Flush master local region- Specified by:
flushMasterStore
in interfaceAdmin
- Throws:
IOException
-
getBalancerDecisions
- Throws:
IOException
-
getBalancerRejections
- Throws:
IOException
-
clearSlowLogsResponses
- Throws:
IOException
-
replicationPeerModificationSwitch
public boolean replicationPeerModificationSwitch(boolean on, boolean drainProcedures) throws IOException Description copied from interface:Admin
Enable or disable replication peer modification. This is especially useful when you want to change the replication peer storage.- Specified by:
replicationPeerModificationSwitch
in interfaceAdmin
- Parameters:
on
-true
means enable, otherwise disabledrainProcedures
- iftrue
, will wait until all the running replication peer modification procedures finish- Returns:
- the previous enable/disable state
- Throws:
IOException
-
isReplicationPeerModificationEnabled
Description copied from interface:Admin
Check whether replication peer modification is enabled.- Specified by:
isReplicationPeerModificationEnabled
in interfaceAdmin
- Returns:
true
if modification is enabled, otherwisefalse
- Throws:
IOException
-