Interface Admin
- All Superinterfaces:
Abortable
,AutoCloseable
,Closeable
- All Known Implementing Classes:
AdminOverAsyncAdmin
,ThriftAdmin
Connection.getAdmin()
and call
close()
when done.
Admin can be used to create, drop, list, enable and disable and otherwise modify tables, as well as perform other administrative operations.
- Since:
- 0.99.0
- See Also:
-
Method Summary
Modifier and TypeMethodDescriptionvoid
Abort the server or client.default boolean
abortProcedure
(long procId, boolean mayInterruptIfRunning) Deprecated.since 2.1.1 and will be removed in 4.0.0.abortProcedureAsync
(long procId, boolean mayInterruptIfRunning) Deprecated.since 2.1.1 and will be removed in 4.0.0.default void
addColumnFamily
(TableName tableName, ColumnFamilyDescriptor columnFamily) Add a column family to an existing table.addColumnFamilyAsync
(TableName tableName, ColumnFamilyDescriptor columnFamily) Add a column family to an existing table.default void
addReplicationPeer
(String peerId, ReplicationPeerConfig peerConfig) Add a new replication peer for replicating data to slave cluster.default void
addReplicationPeer
(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) Add a new replication peer for replicating data to slave cluster.addReplicationPeerAsync
(String peerId, ReplicationPeerConfig peerConfig) Add a new replication peer but does not block and wait for it.addReplicationPeerAsync
(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) Add a new replication peer but does not block and wait for it.void
addRSGroup
(String groupName) Creates a new RegionServer group with the given namedefault void
Append the replicable table column family config from the specified peer.void
assign
(byte[] regionName) Assign a Region.default boolean
balance()
Invoke the balancer.default boolean
balance
(boolean force) Deprecated.Since 2.5.0.balance
(BalanceRequest request) Invoke the balancer with the given balance request.default BalanceResponse
balanceRSGroup
(String groupName) Balance regions in the given RegionServer groupbalanceRSGroup
(String groupName, BalanceRequest request) Balance regions in the given RegionServer group, running based on the givenBalanceRequest
.boolean
balancerSwitch
(boolean onOrOff, boolean synchronous) Turn the load balancer on or off.boolean
catalogJanitorSwitch
(boolean onOrOff) Enable/Disable the catalog janitor/boolean
cleanerChoreSwitch
(boolean onOrOff) Enable/Disable the cleaner chore.clearBlockCache
(TableName tableName) Clear all the blocks corresponding to this table from BlockCache.void
clearCompactionQueues
(ServerName serverName, Set<String> queues) Clear compacting queues on a regionserver.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 RegionServersdefault void
cloneSnapshot
(String snapshotName, TableName tableName) Create a new table by cloning the snapshot content.default void
cloneSnapshot
(String snapshotName, TableName tableName, boolean restoreAcl) Create a new table by cloning the snapshot content.default void
cloneSnapshot
(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Create a new table by cloning the snapshot content.cloneSnapshotAsync
(String snapshotName, TableName tableName) Create a new table by cloning the snapshot content, but does not block and wait for it to be completely cloned.cloneSnapshotAsync
(String snapshotName, TableName tableName, boolean restoreAcl) Create a new table by cloning the snapshot content.cloneSnapshotAsync
(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Create a new table by cloning the snapshot content.void
cloneTableSchema
(TableName tableName, TableName newTableName, boolean preserveSplits) Create a new table by cloning the existent table schema.void
close()
void
Compact a table.void
Compact a column family within a table.void
compact
(TableName tableName, byte[] columnFamily, CompactType compactType) Compact a column family within a table.void
compact
(TableName tableName, CompactType compactType) Compact a table.compactionSwitch
(boolean switchState, List<String> serverNamesList) Turn the compaction on or off.void
compactRegion
(byte[] regionName) Compact an individual region.void
compactRegion
(byte[] regionName, byte[] columnFamily) Compact a column family within a region.void
compactRegionServer
(ServerName serverName) Compact all regions on the region server.Deprecated.since 3.0.0, will removed in 4.0.0.coprocessorService
(ServerName serverName) Deprecated.since 3.0.0, will removed in 4.0.0.default void
createNamespace
(NamespaceDescriptor descriptor) Create a new namespace.createNamespaceAsync
(NamespaceDescriptor descriptor) Create a new namespace.default void
createTable
(TableDescriptor desc) Creates a new table.default void
createTable
(TableDescriptor desc, byte[][] splitKeys) Creates a new table with an initial set of empty regions defined by the specified split keys.void
createTable
(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) Creates a new table with the specified number of regions.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.default void
deleteColumnFamily
(TableName tableName, byte[] columnFamily) Delete a column family from a table.deleteColumnFamilyAsync
(TableName tableName, byte[] columnFamily) Delete a column family from a table.default void
deleteNamespace
(String name) Delete an existing namespace.deleteNamespaceAsync
(String name) Delete an existing namespace.void
deleteSnapshot
(String snapshotName) Delete an existing snapshot.void
deleteSnapshots
(Pattern pattern) Delete existing snapshots whose names match the pattern passed.default void
deleteTable
(TableName tableName) Deletes a table.deleteTableAsync
(TableName tableName) Deletes the table but does not block and wait for it to be completely removed.void
deleteTableSnapshots
(Pattern tableNamePattern, Pattern snapshotNamePattern) Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.default void
disableReplicationPeer
(String peerId) Stop the replication stream to the specified peer.disableReplicationPeerAsync
(String peerId) Disable a replication peer but does not block and wait for it.default void
disableTable
(TableName tableName) Disable table and wait on completion.disableTableAsync
(TableName tableName) Disable the table but does not block and wait for it to be completely disabled.void
disableTableReplication
(TableName tableName) Disable a table's replication switch.default void
enableReplicationPeer
(String peerId) Restart the replication stream to the specified peer.enableReplicationPeerAsync
(String peerId) Enable a replication peer but does not block and wait for it.default void
enableTable
(TableName tableName) Enable a table.enableTableAsync
(TableName tableName) Enable the table but does not block and wait for it to be completely enabled.void
enableTableReplication
(TableName tableName) Enable a table's replication switch.boolean
exceedThrottleQuotaSwitch
(boolean enable) Switch the exceed throttle quota.void
Execute a distributed procedure on a cluster.byte[]
Execute a distributed procedure on a cluster.void
Flush a table.void
Flush the specified column family stores on all regions of the passed table.void
Flush the specified column family stores on all regions of the passed table.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.default Collection<ServerName>
Get a list of current backup masters.getCachedFilesList
(ServerName serverName) Get the list of cached filesdefault ClusterMetrics
Get whole cluster metrics, containing status about:getClusterMetrics
(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.getConfiguredNamespacesAndTablesInRSGroup
(String groupName) Get the namespaces and tables which have this RegionServer group in descriptor.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.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.default ServerName
Get the current active master.Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().default int
Get the info port of the current master if one is available.getNamespaceDescriptor
(String name) Get a namespace descriptor by name.int
Return the operation timeout for a rpc call.Get procedures.getQuota
(QuotaFilter filter) List the quotas based on the filter.getRegionMetrics
(ServerName serverName) GetRegionMetrics
of all regions hosted on a regionserver.getRegionMetrics
(ServerName serverName, TableName tableName) GetRegionMetrics
of all regions hosted on a regionserver for a table.getRegions
(ServerName serverName) Get all the online regions on a region server.getRegions
(TableName tableName) Get the regions of a given table.default Collection<ServerName>
Get the live server list.default Collection<ServerName>
getRegionServers
(boolean excludeDecommissionedRS) Retrieve all current live region servers including decommissioned if excludeDecommissionedRS is false, else non-decommissioned ones onlyMap<TableName,
? extends SpaceQuotaSnapshotView> getRegionServerSpaceQuotaSnapshots
(ServerName serverName) Fetches the observedSpaceQuotaSnapshotView
s observed by a RegionServer.getReplicationPeerConfig
(String peerId) Returns the configured ReplicationPeerConfig for the specified peer.default SyncReplicationState
Get the current cluster state in a synchronous replication peer.getRSGroup
(String groupName) Get group info for the given group namegetRSGroup
(Address hostPort) Get group info for the given hostPortgetRSGroup
(TableName tableName) Get group info for the given tableReturn the set of supported security capabilities.default List<OnlineLogRecord>
getSlowLogResponses
(Set<ServerName> serverNames, LogQueryFilter logQueryFilter) Deprecated.since 2.4.0 and will be removed in 4.0.0.Fetches the table sizes on the filesystem as tracked by the HBase Master.int
Return the blocking wait time for an asynchronous operation.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 permissionshasUserPermissions
(List<Permission> permissions) Check if call user has specific permissionsboolean
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 snapshot) 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
isTableDisabled
(TableName tableName) Check if a table is disabled.boolean
isTableEnabled
(TableName tableName) Check if a table is enabled.default List<ServerName>
List dead region servers.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.Lists current set of RegionServer groupsList completed snapshots.listSnapshots
(Pattern pattern) List all the completed snapshots matching the given pattern.List all the userspace tables.listTableDescriptors
(boolean includeSysTables) List all userspace tables and whether or not include system tables.listTableDescriptors
(List<TableName> tableNames) Get tableDescriptors.default List<TableDescriptor>
listTableDescriptors
(Pattern pattern) List all the userspace tables that match the given pattern.listTableDescriptors
(Pattern pattern, boolean includeSysTables) List all the tables matching the given pattern.listTableDescriptorsByNamespace
(byte[] name) Get list of table descriptors by namespace.listTableDescriptorsByState
(boolean isEnabled) List all enabled or disabled tablesList all of the names of userspace tables.default TableName[]
listTableNames
(Pattern pattern) 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 nameslistTablesInRSGroup
(String groupName) Get all tables in this RegionServer group.listTableSnapshots
(Pattern tableNamePattern, Pattern snapshotNamePattern) List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.default List<ServerName>
List unknown region servers.void
majorCompact
(TableName tableName) Major compact a table.void
majorCompact
(TableName tableName, byte[] columnFamily) Major compact a column family within a table.void
majorCompact
(TableName tableName, byte[] columnFamily, CompactType compactType) Major compact a column family within a table.void
majorCompact
(TableName tableName, CompactType compactType) Major compact a table.void
majorCompactRegion
(byte[] regionName) Major compact a table or an individual region.void
majorCompactRegion
(byte[] regionName, byte[] columnFamily) Major compact a column family within region.void
majorCompactRegionServer
(ServerName serverName) Major compact all regions on the region server.mergeRegionsAsync
(byte[][] nameofRegionsToMerge, boolean forcible) Merge multiple regions (>=2).mergeRegionsAsync
(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) Deprecated.since 2.3.0 and will be removed in 4.0.0.boolean
mergeSwitch
(boolean enabled, boolean synchronous) Turn the merge switch on or off.default void
modifyColumnFamily
(TableName tableName, ColumnFamilyDescriptor columnFamily) Modify an existing column family on a table.modifyColumnFamilyAsync
(TableName tableName, ColumnFamilyDescriptor columnFamily) Modify an existing column family on a table.default void
modifyColumnFamilyStoreFileTracker
(TableName tableName, byte[] family, String dstSFT) Change the store file tracker of the given table's given family.modifyColumnFamilyStoreFileTrackerAsync
(TableName tableName, byte[] family, String dstSFT) Change the store file tracker of the given table's given family.default void
modifyNamespace
(NamespaceDescriptor descriptor) Modify an existing namespace.modifyNamespaceAsync
(NamespaceDescriptor descriptor) Modify an existing namespace.default void
Modify an existing table, more IRB friendly version.Modify an existing table, more IRB (ruby) friendly version.modifyTableAsync
(TableDescriptor td, boolean reopenRegions) The same asmodifyTableAsync(TableDescriptor td)
, except for the reopenRegions parameter, which controls whether the process of modifying the table should reopen all regions.default void
modifyTableStoreFileTracker
(TableName tableName, String dstSFT) Change the store file tracker of the given table.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.default void
move
(byte[] encodedRegionName, byte[] destServerName) Deprecated.since 2.2.0 and will be removed in 4.0.0.void
move
(byte[] encodedRegionName, ServerName destServerName) Move the regionencodedRegionName
todestServerName
.void
moveServersToRSGroup
(Set<Address> servers, String targetGroup) Move given set of servers to the specified target RegionServer groupdefault boolean
Invoke region normalizer.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.default void
removeReplicationPeer
(String peerId) Remove a peer and stop the replication.removeReplicationPeerAsync
(String peerId) Remove a replication peer but does not block and wait for it.default void
Remove some table-cfs from config of the specified peer.void
removeRSGroup
(String groupName) Remove RegionServer group associated with the given namevoid
removeServersFromRSGroup
(Set<Address> servers) Remove decommissioned servers from group 1.void
renameRSGroup
(String oldName, String newName) Rename rsgroupdefault boolean
replicationPeerModificationSwitch
(boolean on) Enable or disable replication peer modification.boolean
replicationPeerModificationSwitch
(boolean on, boolean drainProcedures) Enable or disable replication peer modification.void
restoreSnapshot
(String snapshotName) Restore the specified snapshot on the original table.default void
restoreSnapshot
(String snapshotName, boolean takeFailSafeSnapshot) Restore the specified snapshot on the original table.void
restoreSnapshot
(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) Restore the specified snapshot on the original table.void
revoke
(UserPermission userPermission) Revokes user specific permissionsvoid
rollWALWriter
(ServerName serverName) Roll the log writer.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.void
setRSGroup
(Set<TableName> tables, String groupName) Set the RegionServer group for tablesvoid
shutdown()
Shuts down the HBase cluster.default void
Take a snapshot for the given table.default void
Create typed snapshot of the table.default void
snapshot
(String snapshotName, TableName tableName, SnapshotType type) Create typed snapshot of the table.default void
snapshot
(String snapshotName, TableName tableName, SnapshotType type, Map<String, Object> snapshotProps) Create typed snapshot of the table.void
snapshot
(SnapshotDescription snapshot) Take a snapshot and wait for the server to complete that snapshot (blocking).snapshotAsync
(SnapshotDescription snapshot) Take a snapshot without waiting for the server to complete that snapshot (asynchronous).boolean
snapshotCleanupSwitch
(boolean on, boolean synchronous) Turn on or off the auto snapshot cleanup based on TTL.void
Split a table.void
Split a table.splitRegionAsync
(byte[] regionName) Split an individual region.splitRegionAsync
(byte[] regionName, byte[] splitPoint) Split an individual 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.boolean
switchRpcThrottle
(boolean enable) Switch the rpc throttle enable state.boolean
tableExists
(TableName tableName) Check if a table exists.default void
transitReplicationPeerSyncReplicationState
(String peerId, SyncReplicationState state) Transit current cluster to a new state in a synchronous replication peer.Transit current cluster to a new state in a synchronous replication peer.void
truncateRegion
(byte[] regionName) Truncate an individual region.truncateRegionAsync
(byte[] regionName) Truncate an individual region.default void
truncateTable
(TableName tableName, boolean preserveSplits) Truncate a table.truncateTableAsync
(TableName tableName, boolean preserveSplits) Truncate the table but does not block and wait for it to be completely enabled.void
unassign
(byte[] regionName) Unassign a Region.default void
unassign
(byte[] regionName, boolean force) Deprecated.since 2.4.0 and will be removed in 4.0.0.void
Update the configuration and trigger an online config change on all the regionservers.void
updateConfiguration
(String groupName) Update the configuration and trigger an online config change on all the regionservers in the RSGroup.void
updateConfiguration
(ServerName server) Update the configuration and trigger an online config change on the regionserver.default void
updateReplicationPeerConfig
(String peerId, ReplicationPeerConfig peerConfig) Update the peerConfig for the specified peer.updateReplicationPeerConfigAsync
(String peerId, ReplicationPeerConfig peerConfig) Update the peerConfig for the specified peer but does not block and wait for it.void
updateRSGroupConfig
(String groupName, Map<String, String> configuration) Update RSGroup configuration
-
Method Details
-
getOperationTimeout
int getOperationTimeout()Return the operation timeout for a rpc call.- See Also:
-
getSyncWaitTimeout
int getSyncWaitTimeout()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 abovegetOperationTimeout()
. 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).- See Also:
-
abort
Description copied from interface:Abortable
Abort the server or client. -
isAborted
boolean isAborted()Description copied from interface:Abortable
Check if the server or client was aborted. -
getConnection
Returns Connection used by this object. -
tableExists
Check if a table exists.- Parameters:
tableName
- Table to check.- Returns:
true
if table exists already.- Throws:
IOException
- if a remote or network exception occurs
-
listTableDescriptors
List all the userspace tables.- Returns:
- a list of TableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
listTableDescriptors
List all userspace tables and whether or not include system tables.- Returns:
- a list of TableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
listTableDescriptors
List all the userspace tables that match the given pattern.- Parameters:
pattern
- The compiled regular expression to match against- Returns:
- a list of TableDescriptors
- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
listTableDescriptors
List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) throws IOException List all the tables matching the given pattern.- 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:
-
listTableDescriptorsByState
List all enabled or disabled tables- Parameters:
isEnabled
- is true means return enabled tables, false means return disabled tables- Returns:
- a list of enabled or disabled tables
- Throws:
IOException
-
listTableNames
List all of the names of userspace tables.- Returns:
- TableName[] table names
- Throws:
IOException
- if a remote or network exception occurs
-
listTableNames
List all of the names of userspace tables.- Parameters:
pattern
- The regular expression to match against- Returns:
- array of table names
- Throws:
IOException
- if a remote or network exception occurs
-
listTableNames
List all of the names of userspace tables.- 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
-
listTableNamesByState
List all enabled or disabled table names- 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
-
getDescriptor
Get a table descriptor.- Parameters:
tableName
- as aTableName
- Returns:
- the tableDescriptor
- Throws:
TableNotFoundException
- if the table was not foundIOException
- if a remote or network exception occurs
-
createTable
Creates a new table. Synchronous operation.- Parameters:
desc
- table descriptor for table- Throws:
IllegalArgumentException
- if the table name is reservedMasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If concurrent threads, the table may have been created between test-for-existence and attempt-at-creation).IOException
- if a remote or network exception occurs
-
createTable
void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) throws IOException 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.- 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 occursIllegalArgumentException
- if the table name is reservedMasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If concurrent threads, the table may have been created between test-for-existence and attempt-at-creation).
-
createTable
Creates a new table with an initial set of empty regions defined by the specified split keys. The total number of regions created will be the number of split keys plus one. Synchronous operation. Note : Avoid passing empty split key.- Parameters:
desc
- table descriptor for tablesplitKeys
- array of split keys for the initial regions of the table- Throws:
IllegalArgumentException
- if the table name is reserved, if the split keys are repeated and if the split key has empty byte array.MasterNotRunningException
- if master is not runningTableExistsException
- if table already exists (If concurrent threads, the table may have been created between test-for-existence and attempt-at-creation).IOException
- if a remote or network exception occurs
-
createTableAsync
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.- 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
-
createTableAsync
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.- 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
-
deleteTable
Deletes a table. Synchronous operation.- Parameters:
tableName
- name of table to delete- Throws:
IOException
- if a remote or network exception occurs
-
deleteTableAsync
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.- 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
-
truncateTable
Truncate a table. Synchronous operation.- Parameters:
tableName
- name of table to truncatepreserveSplits
-true
if the splits should be preserved- Throws:
IOException
- if a remote or network exception occurs
-
truncateTableAsync
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.- 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
-
enableTable
Enable a table. May timeout. UseenableTableAsync(org.apache.hadoop.hbase.TableName)
andisTableEnabled(org.apache.hadoop.hbase.TableName)
instead. The table has to be in disabled state for it to be enabled.- Parameters:
tableName
- name of the table- Throws:
IOException
- There could be couple types of IOException TableNotFoundException means the table doesn't exist. TableNotDisabledException means the table isn't in disabled state.- See Also:
-
enableTableAsync
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.- 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
-
disableTableAsync
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.- 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
-
disableTable
Disable table and wait on completion. May timeout eventually. UsedisableTableAsync(org.apache.hadoop.hbase.TableName)
andisTableDisabled(org.apache.hadoop.hbase.TableName)
instead. The table has to be in enabled state for it to be disabled.- Throws:
IOException
- There could be couple types of IOException TableNotFoundException means the table doesn't exist. TableNotEnabledException means the table isn't in enabled state.
-
isTableEnabled
Check if a table is enabled.- Parameters:
tableName
- name of table to check- Returns:
true
if table is on-line- Throws:
IOException
- if a remote or network exception occurs
-
isTableDisabled
Check if a table is disabled.- Parameters:
tableName
- name of table to check- Returns:
true
if table is off-line- Throws:
IOException
- if a remote or network exception occurs
-
isTableAvailable
Check if a table is available.- 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
-
addColumnFamily
default void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException Add a column family to an existing table. Synchronous operation. UseaddColumnFamilyAsync(TableName, ColumnFamilyDescriptor)
instead because it returns aFuture
from which you can learn whether success or failure.- Parameters:
tableName
- name of the table to add column family tocolumnFamily
- column family descriptor of column family to be added- Throws:
IOException
- if a remote or network exception occurs
-
addColumnFamilyAsync
Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException 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.- 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
-
deleteColumnFamily
Delete a column family from a table. Synchronous operation. UsedeleteColumnFamily(TableName, byte[])
instead because it returns aFuture
from which you can learn whether success or failure.- Parameters:
tableName
- name of tablecolumnFamily
- name of column family to be deleted- Throws:
IOException
- if a remote or network exception occurs
-
deleteColumnFamilyAsync
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.- 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
-
modifyColumnFamily
default void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException Modify an existing column family on a table. Synchronous operation. UsemodifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor)
instead because it returns aFuture
from which you can learn whether success or failure.- Parameters:
tableName
- name of tablecolumnFamily
- new column family descriptor to use- Throws:
IOException
- if a remote or network exception occurs
-
modifyColumnFamilyAsync
Future<Void> modifyColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException 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.- 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
-
modifyColumnFamilyStoreFileTracker
default void modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT) throws IOException Change the store file tracker of the given table's given family.- Parameters:
tableName
- the table you want to changefamily
- the family you want to changedstSFT
- the destination store file tracker- Throws:
IOException
- if a remote or network exception occurs
-
modifyColumnFamilyStoreFileTrackerAsync
Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family, String dstSFT) throws IOException Change the store file tracker of the given table's given family.- 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
-
getRegions
Get all the online regions on a region server.- Returns:
- List of
RegionInfo
- Throws:
IOException
- if a remote or network exception occurs
-
flush
Flush a table. Synchronous operation.- Parameters:
tableName
- table to flush- Throws:
IOException
- if a remote or network exception occurs
-
flush
Flush the specified column family stores on all regions of the passed table. This runs as a synchronous operation.- Parameters:
tableName
- table to flushcolumnFamily
- column family within a table- Throws:
IOException
- if a remote or network exception occurs
-
flush
Flush the specified column family stores on all regions of the passed table. This runs as a synchronous operation.- Parameters:
tableName
- table to flushcolumnFamilies
- column families within a table- Throws:
IOException
- if a remote or network exception occurs
-
flushRegion
Flush an individual region. Synchronous operation.- Parameters:
regionName
- region to flush- Throws:
IOException
- if a remote or network exception occurs
-
flushRegion
Flush a column family within a region. Synchronous operation.- Parameters:
regionName
- region to flushcolumnFamily
- column family within a region- Throws:
IOException
- if a remote or network exception occurs
-
flushRegionServer
Flush all regions on the region server. Synchronous operation.- 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).- Parameters:
tableName
- table to compact- Throws:
IOException
- if a remote or network exception occurs
-
compactRegion
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).- 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).- 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).- Parameters:
regionName
- region to compactcolumnFamily
- column family within a region- 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).- Parameters:
tableName
- table to compactcompactType
-CompactType
- Throws:
IOException
- if a remote or network exception occursInterruptedException
-
compact
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).- 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
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).- Parameters:
tableName
- table to major compact- Throws:
IOException
- if a remote or network exception occurs
-
majorCompactRegion
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).- 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).- Parameters:
tableName
- table to major compactcolumnFamily
- column family within a table- Throws:
IOException
- if a remote or network exception occurs
-
majorCompactRegion
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).- Parameters:
regionName
- egion to major compactcolumnFamily
- column family within a region- Throws:
IOException
- if a remote or network exception occurs
-
majorCompact
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).- Parameters:
tableName
- table to compactcompactType
-CompactType
- Throws:
IOException
- if a remote or network exception occursInterruptedException
-
majorCompact
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).- 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
-
compactionSwitch
Map<ServerName,Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList) throws IOException 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.- Parameters:
switchState
- Set totrue
to enable,false
to disable.serverNamesList
- list of region servers.- Returns:
- Previous compaction states for region servers
- Throws:
IOException
- if a remote or network exception occurs
-
compactRegionServer
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).- Parameters:
serverName
- the region server name- Throws:
IOException
- if a remote or network exception occurs
-
majorCompactRegionServer
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).- Parameters:
serverName
- the region server name- Throws:
IOException
- if a remote or network exception occurs
-
move
Move the regionencodedRegionName
to a random server.- 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
Deprecated.since 2.2.0 and will be removed in 4.0.0. Usemove(byte[], ServerName)
instead. And if you want to move the region to a random server, please usemove(byte[])
.Move the regionrencodedRegionName
todestServerName
.- 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. If passed the empty byte array we'll assign to a random server. A server name is made of host, port and startcode. Here is an example:host187.example.com,60020,1289493121758
- Throws:
IOException
- if we can't find a region namedencodedRegionName
- See Also:
-
move
Move the regionencodedRegionName
todestServerName
.- 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
Assign a Region.- Parameters:
regionName
- Region name to assign.- Throws:
IOException
- if a remote or network exception occurs
-
unassign
Unassign a Region.- Parameters:
regionName
- Region name to unassign.- Throws:
IOException
- if a remote or network exception occurs
-
unassign
Deprecated.since 2.4.0 and will be removed in 4.0.0. Useunassign(byte[])
instead.Unassign a region from current hosting regionserver. Region will then be assigned to a regionserver chosen at random. Region could be reassigned back to the same server. Usemove(byte[], ServerName)
if you want to control the region movement.- Parameters:
regionName
- Region to unassign. Will clear any existing RegionPlan if one found.force
- Iftrue
, force unassign (Will remove region from regions-in-transition too if present. If results in double assignment use hbck -fix to resolve. To be used by experts).- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
offline
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.- Parameters:
regionName
- Region to offline.- Throws:
IOException
- if a remote or network exception occurs
-
balancerSwitch
Turn the load balancer on or off.- Parameters:
onOrOff
- 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
Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the reassignments. Can NOT run for various reasons. Check logs.- Returns:
true
if balancer ran,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
balance
Invoke the balancer with the given balance request. The BalanceRequest defines how the balancer will run. SeeBalanceRequest
for more details.- 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
-
balance
Deprecated.Since 2.5.0. Will be removed in 4.0.0. Usebalance(BalanceRequest)
instead.Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the reassignments. If there is region in transition, force parameter of true would still run balancer. Can *not* run for other reasons. Check logs.- Parameters:
force
- whether we should force balance even if there is region in transition- Returns:
true
if balancer ran,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
isBalancerEnabled
Query the current state of the balancer.- 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.- Parameters:
tableName
- table to clear block cache- Returns:
- CacheEvictionStats related to the eviction
- Throws:
IOException
- if a remote or network exception occurs
-
normalize
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.- Returns:
true
if region normalizer ran,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
normalize
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.- 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
Query the current state of the region normalizer.- Returns:
true
if region normalizer is enabled,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
normalizerSwitch
Turn region normalizer on or off.- Returns:
- Previous normalizer value
- Throws:
IOException
- if a remote or network exception occurs
-
catalogJanitorSwitch
Enable/Disable the catalog janitor/- Parameters:
onOrOff
- iftrue
enables the catalog janitor- Returns:
- the previous state
- Throws:
IOException
- if a remote or network exception occurs
-
runCatalogJanitor
Ask for a scan of the catalog table.- Returns:
- the number of entries cleaned. Returns -1 if previous run is in progress.
- Throws:
IOException
- if a remote or network exception occurs
-
isCatalogJanitorEnabled
Query on the catalog janitor state (Enabled/Disabled?).- Throws:
IOException
- if a remote or network exception occurs
-
cleanerChoreSwitch
Enable/Disable the cleaner chore.- Parameters:
onOrOff
- iftrue
enables the cleaner chore- Returns:
- the previous state
- Throws:
IOException
- if a remote or network exception occurs
-
runCleanerChore
Ask for cleaner chore to run.- Returns:
true
if cleaner chore ran,false
otherwise- Throws:
IOException
- if a remote or network exception occurs
-
isCleanerChoreEnabled
Query on the cleaner chore state (Enabled/Disabled?).- Throws:
IOException
- if a remote or network exception occurs
-
mergeRegionsAsync
@Deprecated default Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) throws IOException Deprecated.since 2.3.0 and will be removed in 4.0.0. Multi-region merge feature is now supported. UsemergeRegionsAsync(byte[][], boolean)
instead.Merge two regions. Asynchronous operation.- 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
Merge multiple regions (>=2). Asynchronous operation.- 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
-
split
Split a table. The method will execute split action for each region in table.- Parameters:
tableName
- table to split- Throws:
IOException
- if a remote or network exception occurs
-
split
Split a table.- Parameters:
tableName
- table to splitsplitPoint
- the explicit position to split on- Throws:
IOException
- if a remote or network exception occurs
-
splitRegionAsync
Split an individual region. Asynchronous operation.- Parameters:
regionName
- region to split- Throws:
IOException
- if a remote or network exception occurs
-
splitRegionAsync
Split an individual region. Asynchronous operation.- Parameters:
regionName
- region to splitsplitPoint
- the explicit position to split on- Throws:
IOException
- if a remote or network exception occurs
-
modifyTable
Modify an existing table, more IRB friendly version.- Parameters:
td
- modified description of the table- Throws:
IOException
- if a remote or network exception occurs
-
truncateRegion
Truncate an individual region.- Parameters:
regionName
- region to truncate- Throws:
IOException
- if a remote or network exception occurs
-
truncateRegionAsync
Truncate an individual region. Asynchronous operation.- Parameters:
regionName
- region to truncate- Throws:
IOException
- if a remote or network exception occurs
-
modifyTableAsync
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.- Parameters:
td
- description of the table- 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
-
modifyTableAsync
The same asmodifyTableAsync(TableDescriptor td)
, except for the reopenRegions parameter, which controls whether the process of modifying the table should reopen all regions.- Parameters:
td
- description of the tablereopenRegions
- By default, 'modifyTable' 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
-
modifyTableStoreFileTracker
Change the store file tracker of the given table.- Parameters:
tableName
- the table you want to changedstSFT
- the destination store file tracker- Throws:
IOException
- if a remote or network exception occurs
-
modifyTableStoreFileTrackerAsync
Future<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT) throws IOException Change the store file tracker of the given table.- 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
-
shutdown
Shuts down the HBase cluster. Notice that, a success shutdown call may ends with an error since the remote server has already been shutdown.- Throws:
IOException
- if a remote or network exception occurs
-
stopMaster
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.- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
isMasterInMaintenanceMode
Check whether Master is in maintenance mode.- Throws:
IOException
- if a remote or network exception occurs
-
stopRegionServer
Stop the designated regionserver.- Parameters:
hostnamePort
- Hostname and port delimited by a:
as inexample.org:1234
- Throws:
IOException
- if a remote or network exception occurs
-
getClusterMetrics
Get whole cluster metrics, containing status about:hbase version cluster id primary/backup master(s) master's coprocessors live/dead regionservers balancer regions in transition
- Returns:
- cluster metrics
- Throws:
IOException
- if a remote or network exception occurs
-
getClusterMetrics
Get cluster status with a set ofClusterMetrics.Option
to get desired status.- Returns:
- cluster status
- Throws:
IOException
- if a remote or network exception occurs
-
getMaster
Get the current active master.- Returns:
- current master server name
- Throws:
IOException
- if a remote or network exception occurs
-
getBackupMasters
Get a list of current backup masters.- Returns:
- current backup master list
- Throws:
IOException
- if a remote or network exception occurs
-
getRegionServers
Get the live server list.- Returns:
- current live region servers list
- Throws:
IOException
- if a remote or network exception occurs
-
getRegionServers
Retrieve all current live region servers including decommissioned if excludeDecommissionedRS is false, else non-decommissioned ones only- Parameters:
excludeDecommissionedRS
- should we exclude decommissioned RS nodes- Returns:
- all current live region servers including/excluding decommissioned hosts
- Throws:
IOException
- if a remote or network exception occurs
-
getRegionMetrics
GetRegionMetrics
of all regions hosted on a regionserver.- Parameters:
serverName
- region server from whichRegionMetrics
is required.- Returns:
- a
RegionMetrics
list of all regions hosted on a region server - Throws:
IOException
- if a remote or network exception occurs
-
getRegionMetrics
GetRegionMetrics
of all regions hosted on a regionserver for a table.- 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
org.apache.hadoop.conf.Configuration getConfiguration()Returns Configuration used by the instance. -
createNamespace
Create a new namespace. Blocks until namespace has been successfully created or an exception is thrown.- Parameters:
descriptor
- descriptor which describes the new namespace.- Throws:
IOException
- if a remote or network exception occurs
-
createNamespaceAsync
Create a new namespace.- 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
-
modifyNamespace
Modify an existing namespace. Blocks until namespace has been successfully modified or an exception is thrown.- Parameters:
descriptor
- descriptor which describes the new namespace- Throws:
IOException
- if a remote or network exception occurs
-
modifyNamespaceAsync
Modify an existing namespace.- 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
-
deleteNamespace
Delete an existing namespace. Only empty namespaces (no tables) can be removed. Blocks until namespace has been successfully deleted or an exception is thrown.- Parameters:
name
- namespace name- Throws:
IOException
- if a remote or network exception occurs
-
deleteNamespaceAsync
Delete an existing namespace. Only empty namespaces (no tables) can be removed.- 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
NamespaceDescriptor getNamespaceDescriptor(String name) throws NamespaceNotFoundException, IOException Get a namespace descriptor by name.- 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- Returns:
- List of namespace names
- Throws:
IOException
- if a remote or network exception occurs
-
listNamespaceDescriptors
List available namespace descriptors- Returns:
- List of descriptors
- Throws:
IOException
- if a remote or network exception occurs
-
listTableDescriptorsByNamespace
Get list of table descriptors by namespace.- Parameters:
name
- namespace name- Returns:
- returns a list of TableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
listTableNamesByNamespace
Get list of table names by namespace.- Parameters:
name
- namespace name- Returns:
- The list of table names in the namespace
- Throws:
IOException
- if a remote or network exception occurs
-
getRegions
Get the regions of a given table.- Parameters:
tableName
- the name of the table- Returns:
- List of
RegionInfo
. - Throws:
IOException
- if a remote or network exception occurs
-
close
void close()- Specified by:
close
in interfaceAutoCloseable
- Specified by:
close
in interfaceCloseable
-
listTableDescriptors
Get tableDescriptors.- Parameters:
tableNames
- List of table names- Returns:
- returns a list of TableDescriptors
- Throws:
IOException
- if a remote or network exception occurs
-
abortProcedure
@Deprecated default boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException Deprecated.since 2.1.1 and will be removed in 4.0.0.Abort a procedure. Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.- 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
@Deprecated Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning) throws IOException Deprecated.since 2.1.1 and will be removed in 4.0.0.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.- 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:
-
getProcedures
Get procedures.- Returns:
- procedure list in JSON
- Throws:
IOException
- if a remote or network exception occurs
-
getLocks
Get locks.- Returns:
- lock list in JSON
- Throws:
IOException
- if a remote or network exception occurs
-
rollWALWriter
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.- Parameters:
serverName
- The servername of the regionserver.- Throws:
IOException
- if a remote or network exception occursFailedLogCloseException
- if we failed to close the WAL
-
getMasterCoprocessorNames
Helper that delegates to getClusterMetrics().getMasterCoprocessorNames().- Returns:
- an array of master coprocessors
- Throws:
IOException
- See Also:
-
getCompactionState
Get the current compaction state of a table. It could be in a major compaction, a minor compaction, both, or none.- Parameters:
tableName
- table to examine- Returns:
- the current compaction state
- Throws:
IOException
- if a remote or network exception occurs
-
getCompactionState
Get the current compaction state of a table. It could be in a compaction, or none.- Parameters:
tableName
- table to examinecompactType
-CompactType
- Returns:
- the current compaction state
- Throws:
IOException
- if a remote or network exception occurs
-
getCompactionStateForRegion
Get the current compaction state of region. It could be in a major compaction, a minor compaction, both, or none.- Parameters:
regionName
- region to examine- Returns:
- the current compaction state
- Throws:
IOException
- if a remote or network exception occurs
-
getLastMajorCompactionTimestamp
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.- Parameters:
tableName
- table to examine- Returns:
- the last major compaction timestamp or 0
- Throws:
IOException
- if a remote or network exception occurs
-
getLastMajorCompactionTimestampForRegion
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.- Parameters:
regionName
- region to examine- Returns:
- the last major compaction timestamp or 0
- Throws:
IOException
- if a remote or network exception occurs
-
snapshot
default void snapshot(String snapshotName, TableName tableName) throws IOException, SnapshotCreationException, IllegalArgumentException Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be taken. If the table is disabled, an offline snapshot is taken. Snapshots are taken sequentially even when requested concurrently, across all tables. Snapshots are considered unique based on the name of the snapshot. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with aSnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. SeeTableName.isLegalFullyQualifiedTableName(byte[])
.- Parameters:
snapshotName
- name of the snapshot to be createdtableName
- name of the table for which snapshot is created- Throws:
IOException
- if a remote or network exception occursSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectly
-
snapshot
default void snapshot(String snapshotName, TableName tableName, SnapshotType type) throws IOException, SnapshotCreationException, IllegalArgumentException Create typed snapshot of the table. Snapshots are considered unique based on the name of the snapshot. Snapshots are taken sequentially even when requested concurrently, across all tables. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with aSnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. SeeTableName.isLegalFullyQualifiedTableName(byte[])
.- Parameters:
snapshotName
- name to give the snapshot on the filesystem. Must be unique from all other snapshots stored on the clustertableName
- name of the table to snapshottype
- type of snapshot to take- Throws:
IOException
- we fail to reach the masterSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectly
-
snapshot
default void snapshot(String snapshotName, TableName tableName, SnapshotType type, Map<String, Object> snapshotProps) throws IOException, SnapshotCreationException, IllegalArgumentExceptionCreate typed snapshot of the table. Snapshots are considered unique based on the name of the snapshot. Snapshots are taken sequentially even when requested concurrently, across all tables. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with aSnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. SeeTableName.isLegalFullyQualifiedTableName(byte[])
. Snapshot can live with ttl seconds.- Parameters:
snapshotName
- name to give the snapshot on the filesystem. Must be unique from all other snapshots stored on the clustertableName
- name of the table to snapshottype
- type of snapshot to takesnapshotProps
- snapshot additional properties e.g. TTL- Throws:
IOException
- we fail to reach the masterSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectly
-
snapshot
default void snapshot(String snapshotName, TableName tableName, Map<String, Object> snapshotProps) throws IOException, SnapshotCreationException, IllegalArgumentExceptionCreate typed snapshot of the table. Snapshots are considered unique based on the name of the snapshot. Snapshots are taken sequentially even when requested concurrently, across all tables. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with aSnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. SeeTableName.isLegalFullyQualifiedTableName(byte[])
. Snapshot can live with ttl seconds.- Parameters:
snapshotName
- name to give the snapshot on the filesystem. Must be unique from all other snapshots stored on the clustertableName
- name of the table to snapshotsnapshotProps
- snapshot additional properties e.g. TTL- Throws:
IOException
- we fail to reach the masterSnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectly
-
snapshot
void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException, IllegalArgumentException Take a snapshot and wait for the server to complete that snapshot (blocking). Snapshots are considered unique based on the name of the snapshot. Snapshots are taken sequentially even when requested concurrently, across all tables. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with aSnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. SeeTableName.isLegalFullyQualifiedTableName(byte[])
. You should probably usesnapshot(String, org.apache.hadoop.hbase.TableName)
unless you are sure about the type of snapshot that you want to take.- Parameters:
snapshot
- snapshot to take- Throws:
IOException
- or we lose contact with the master.SnapshotCreationException
- if snapshot failed to be takenIllegalArgumentException
- if the snapshot request is formatted incorrectly
-
snapshotAsync
Future<Void> snapshotAsync(SnapshotDescription snapshot) throws IOException, SnapshotCreationException Take a snapshot without waiting for the server to complete that snapshot (asynchronous). Snapshots are considered unique based on the name of the snapshot. Snapshots are taken sequentially even when requested concurrently, across all tables.- Parameters:
snapshot
- snapshot to take- Throws:
IOException
- if the snapshot did not succeed or we lose contact with the master.SnapshotCreationException
- if snapshot creation failedIllegalArgumentException
- if the snapshot request is formatted incorrectly
-
isSnapshotFinished
boolean isSnapshotFinished(SnapshotDescription snapshot) throws IOException, HBaseSnapshotException, UnknownSnapshotException 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
.- Parameters:
snapshot
- 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
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.- Parameters:
snapshotName
- name of the snapshot to restore- Throws:
IOException
- if a remote or network exception occursRestoreSnapshotException
- if snapshot failed to be restoredIllegalArgumentException
- if the restore request is formatted incorrectly
-
restoreSnapshot
default void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) throws IOException, RestoreSnapshotException 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".- 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 restoredIllegalArgumentException
- if the restore request is formatted incorrectly
-
restoreSnapshot
void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) throws IOException, RestoreSnapshotException 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".- 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 restoredIllegalArgumentException
- if the restore request is formatted incorrectly
-
cloneSnapshot
default void cloneSnapshot(String snapshotName, TableName tableName) throws IOException, TableExistsException, RestoreSnapshotException Create a new table by cloning the snapshot content.- Parameters:
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restored- Throws:
IOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid name
-
cloneSnapshot
default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException Create a new table by cloning the snapshot content.- 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 StoreFileTracker 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 clonedIllegalArgumentException
- if the specified table has not a valid name
-
cloneSnapshot
default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException Create a new table by cloning the snapshot content.- 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 table- Throws:
IOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid name
-
cloneSnapshotAsync
default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName) throws IOException, TableExistsException Create a new table by cloning the snapshot content, but does not block and wait for it to be completely cloned. 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.- Parameters:
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restored- Returns:
- the result of the async clone snapshot. You can use Future.get(long, TimeUnit) to wait on the operation to complete.
- Throws:
IOException
- if a remote or network exception occursTableExistsException
- if table to be cloned already exists
-
cloneSnapshotAsync
default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) throws IOException, TableExistsException, RestoreSnapshotException Create a new table by cloning the snapshot content.- 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 table- Throws:
IOException
- if a remote or network exception occursTableExistsException
- if table to be created already existsRestoreSnapshotException
- if snapshot failed to be clonedIllegalArgumentException
- if the specified table has not a valid name
-
cloneSnapshotAsync
Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) throws IOException, TableExistsException, RestoreSnapshotException Create a new table by cloning the snapshot content.- 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 clonedIllegalArgumentException
- if the specified table has not a valid name
-
execProcedure
Execute a distributed procedure on a cluster.- 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
-
execProcedureWithReturn
byte[] execProcedureWithReturn(String signature, String instance, Map<String, String> props) throws IOExceptionExecute a distributed procedure on a cluster.- 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
-
isProcedureFinished
boolean isProcedureFinished(String signature, String instance, Map<String, String> props) throws IOExceptionCheck 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
- 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
-
listSnapshots
List completed snapshots.- Returns:
- a list of snapshot descriptors for completed snapshots
- Throws:
IOException
- if a network error occurs
-
listSnapshots
List all the completed snapshots matching the given pattern.- Parameters:
pattern
- The compiled regular expression to match against- Returns:
- list of SnapshotDescription
- Throws:
IOException
- if a remote or network exception occurs
-
listTableSnapshots
List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) throws IOException List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.- 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
Delete an existing snapshot.- Parameters:
snapshotName
- name of the snapshot- Throws:
IOException
- if a remote or network exception occurs
-
deleteSnapshots
Delete existing snapshots whose names match the pattern passed.- Parameters:
pattern
- pattern for names of the snapshot to match- Throws:
IOException
- if a remote or network exception occurs
-
deleteTableSnapshots
Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.- 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
Apply the new quota settings.- Parameters:
quota
- the quota settings- Throws:
IOException
- if a remote or network exception occurs
-
getQuota
List the quotas based on the filter.- Parameters:
filter
- the quota settings filter- Returns:
- the QuotaSetting list
- Throws:
IOException
- if a remote or network exception occurs
-
coprocessorService
Deprecated.since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any more. Use the coprocessorService methods inAsyncAdmin
instead.Creates and returns aRpcChannel
instance connected to the active master. The obtainedRpcChannel
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);
- Returns:
- A MasterCoprocessorRpcChannel instance
-
coprocessorService
Deprecated.since 3.0.0, will removed in 4.0.0. This is too low level, please stop using it any more. Use the coprocessorService methods inAsyncAdmin
instead.Creates and returns aRpcChannel
instance connected to the passed region server. The obtainedRpcChannel
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);
- Parameters:
serverName
- the server name to which the endpoint call is made- Returns:
- A RegionServerCoprocessorRpcChannel instance
-
updateConfiguration
Update the configuration and trigger an online config change on the regionserver.- Parameters:
server
- : The server whose config needs to be updated.- Throws:
IOException
- if a remote or network exception occurs
-
updateConfiguration
Update the configuration and trigger an online config change on all the regionservers.- Throws:
IOException
- if a remote or network exception occurs
-
updateConfiguration
Update the configuration and trigger an online config change on all the regionservers in the RSGroup.- Parameters:
groupName
- the group name- Throws:
IOException
- if a remote or network exception occurs
-
getMasterInfoPort
Get the info port of the current master if one is available.- Returns:
- master info port
- Throws:
IOException
- if a remote or network exception occurs
-
getSecurityCapabilities
Return the set of supported security capabilities.- Throws:
IOException
- if a remote or network exception occurs
-
splitSwitch
Turn the split switch on or off.- 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
Turn the merge switch on or off.- 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
-
isSplitEnabled
Query the current state of the split switch.- Returns:
true
if the switch is enabled,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
isMergeEnabled
Query the current state of the merge switch.- Returns:
true
if the switch is enabled,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
addReplicationPeer
Add a new replication peer for replicating data to slave cluster.- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peer- Throws:
IOException
- if a remote or network exception occurs
-
addReplicationPeer
default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException Add a new replication peer for replicating data to slave cluster.- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peerenabled
- peer state, true if ENABLED and false if DISABLED- Throws:
IOException
- if a remote or network exception occurs
-
addReplicationPeerAsync
default Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig) throws IOException 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.- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication peer- Returns:
- the result of the async operation
- Throws:
IOException
- IOException if a remote or network exception occurs
-
addReplicationPeerAsync
Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws IOException 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.
- 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
-
removeReplicationPeer
Remove a peer and stop the replication.- Parameters:
peerId
- a short name that identifies the peer- Throws:
IOException
- if a remote or network exception occurs
-
removeReplicationPeerAsync
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.
- 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
-
enableReplicationPeer
Restart the replication stream to the specified peer.- Parameters:
peerId
- a short name that identifies the peer- Throws:
IOException
- if a remote or network exception occurs
-
enableReplicationPeerAsync
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.
- 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
-
disableReplicationPeer
Stop the replication stream to the specified peer.- Parameters:
peerId
- a short name that identifies the peer- Throws:
IOException
- if a remote or network exception occurs
-
disableReplicationPeerAsync
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.- 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
Returns the configured ReplicationPeerConfig for the specified peer.- Parameters:
peerId
- a short name that identifies the peer- Returns:
- ReplicationPeerConfig for the peer
- Throws:
IOException
- if a remote or network exception occurs
-
updateReplicationPeerConfig
default void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) throws IOException Update the peerConfig for the specified peer.- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- new config for the replication peer- Throws:
IOException
- if a remote or network exception occurs
-
updateReplicationPeerConfigAsync
Future<Void> updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig) throws IOException 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.- 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
-
appendReplicationPeerTableCFs
default void appendReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs) throws ReplicationException, IOExceptionAppend the replicable table column family config from the specified peer.- Parameters:
id
- a short that identifies the clustertableCfs
- A map from tableName to column family names- Throws:
ReplicationException
- if tableCfs has conflict with existing configIOException
- if a remote or network exception occurs
-
removeReplicationPeerTableCFs
default void removeReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs) throws ReplicationException, IOExceptionRemove some table-cfs from config of the specified peer.- Parameters:
id
- a short name that identifies the clustertableCfs
- A map from tableName to column family names- Throws:
ReplicationException
- if tableCfs has conflict with existing configIOException
- if a remote or network exception occurs
-
listReplicationPeers
Return a list of replication peers.- Returns:
- a list of replication peers description
- Throws:
IOException
- if a remote or network exception occurs
-
listReplicationPeers
Return a list of replication peers.- 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
-
transitReplicationPeerSyncReplicationState
default void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state) throws IOException Transit current cluster to a new state in a synchronous replication peer.- Parameters:
peerId
- a short name that identifies the peerstate
- a new state of current cluster- Throws:
IOException
- if a remote or network exception occurs
-
transitReplicationPeerSyncReplicationStateAsync
Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId, SyncReplicationState state) throws IOException Transit current cluster to a new state in a synchronous 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.
- Parameters:
peerId
- a short name that identifies the peerstate
- a new state of current cluster- Throws:
IOException
- if a remote or network exception occurs
-
getReplicationPeerSyncReplicationState
default SyncReplicationState getReplicationPeerSyncReplicationState(String peerId) throws IOException Get the current cluster state in a synchronous replication peer.- Parameters:
peerId
- a short name that identifies the peer- Returns:
- the current cluster state
- Throws:
IOException
- if a remote or network exception occurs
-
isReplicationPeerEnabled
Check if a replication peer is enabled.- Parameters:
peerId
- id of replication peer to check- Returns:
true
if replication peer is enabled- Throws:
IOException
- if a remote or network exception occurs
-
replicationPeerModificationSwitch
Enable or disable replication peer modification. This is especially useful when you want to change the replication peer storage.- Parameters:
on
-true
means enable, otherwise disable- Returns:
- the previous enable/disable state
- Throws:
IOException
-
replicationPeerModificationSwitch
Enable or disable replication peer modification. This is especially useful when you want to change the replication peer storage.- 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
Check whether replication peer modification is enabled.- Returns:
true
if modification is enabled, otherwisefalse
- Throws:
IOException
-
decommissionRegionServers
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.- 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
List region servers marked as decommissioned, which can not be assigned regions.- Returns:
- List of decommissioned region servers.
- Throws:
IOException
- if a remote or network exception occurs
-
recommissionRegionServer
void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) throws IOException 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.- Parameters:
server
- The server to recommission.encodedRegionNames
- Regions to load onto the server.- Throws:
IOException
- if a remote or network exception occurs
-
listReplicatedTableCFs
Find all table and column families that are replicated from this cluster- Returns:
- the replicated table-cfs list of this cluster.
- Throws:
IOException
- if a remote or network exception occurs
-
enableTableReplication
Enable a table's replication switch.- Parameters:
tableName
- name of the table- Throws:
IOException
- if a remote or network exception occurs
-
disableTableReplication
Disable a table's replication switch.- Parameters:
tableName
- name of the table- Throws:
IOException
- if a remote or network exception occurs
-
clearCompactionQueues
void clearCompactionQueues(ServerName serverName, Set<String> queues) throws IOException, InterruptedException Clear compacting queues on a regionserver.- Parameters:
serverName
- the region server namequeues
- the set of queue name- Throws:
IOException
- if a remote or network exception occursInterruptedException
-
listDeadServers
List dead region servers.- Returns:
- List of dead region servers.
- Throws:
IOException
-
listUnknownServers
List unknown region servers.- Returns:
- List of unknown region servers.
- Throws:
IOException
-
clearDeadServers
Clear dead region servers from master.- 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
void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) throws IOException Create a new table by cloning the existent table schema.- 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
Switch the rpc throttle enable state.- 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
Get if the rpc throttle is enabled.- Returns:
- True if rpc throttle is enabled
- Throws:
IOException
- if a remote or network exception occurs
-
exceedThrottleQuotaSwitch
Switch the exceed throttle quota. If enabled, user/table/namespace throttle quota can be exceeded if region server has availble quota.- 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
Fetches the table sizes on the filesystem as tracked by the HBase Master.- Throws:
IOException
- if a remote or network exception occurs
-
getRegionServerSpaceQuotaSnapshots
Map<TableName,? extends SpaceQuotaSnapshotView> getRegionServerSpaceQuotaSnapshots(ServerName serverName) throws IOException Fetches the observedSpaceQuotaSnapshotView
s observed by a RegionServer.- Throws:
IOException
- if a remote or network exception occurs
-
getCurrentSpaceQuotaSnapshot
Returns the Master's view of a quota on the givennamespace
or null if the Master has no quota information on that namespace.- Throws:
IOException
- if a remote or network exception occurs
-
getCurrentSpaceQuotaSnapshot
Returns the Master's view of a quota on the giventableName
or null if the Master has no quota information on that table.- Throws:
IOException
- if a remote or network exception occurs
-
grant
Grants user specific permissions- 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
Revokes user specific permissions- Parameters:
userPermission
- user name and the specific permission- Throws:
IOException
- if a remote or network exception occurs
-
getUserPermissions
List<UserPermission> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) throws IOException Get the global/namespace/table permissions for user- 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
-
hasUserPermissions
Check if the user has specific permissions- 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
-
hasUserPermissions
Check if call user has specific permissions- Parameters:
permissions
- the specific permission list- Returns:
- True if user has the specific permissions
- Throws:
IOException
- if a remote or network exception occurs
-
snapshotCleanupSwitch
Turn on or off the auto snapshot cleanup based on TTL.- 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
Query the current state of the auto snapshot cleanup based on TTL.- Returns:
true
if the auto snapshot cleanup is enabled,false
otherwise.- Throws:
IOException
- if a remote or network exception occurs
-
getSlowLogResponses
@Deprecated default List<OnlineLogRecord> getSlowLogResponses(Set<ServerName> serverNames, LogQueryFilter logQueryFilter) throws IOException Deprecated.since 2.4.0 and will be removed in 4.0.0. UsegetLogEntries(Set, String, ServerType, int, Map)
instead.Retrieves online slow/large RPC logs from the provided list of RegionServers- Parameters:
serverNames
- Server names to get slowlog responses fromlogQueryFilter
- filter to be used if provided (determines slow / large RPC logs)- Returns:
- online slowlog response list
- Throws:
IOException
- if a remote or network exception occurs
-
clearSlowLogResponses
Clears online slow/large RPC logs from the provided list of RegionServers- 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
-
addRSGroup
Creates a new RegionServer group with the given name- Parameters:
groupName
- the name of the group- Throws:
IOException
- if a remote or network exception occurs
-
getRSGroup
Get group info for the given group name- Parameters:
groupName
- the group name- Returns:
- group info
- Throws:
IOException
- if a remote or network exception occurs
-
getRSGroup
Get group info for the given hostPort- Parameters:
hostPort
- HostPort to get RSGroupInfo for- Throws:
IOException
- if a remote or network exception occurs
-
getRSGroup
Get group info for the given table- Parameters:
tableName
- table name to get RSGroupInfo for- Throws:
IOException
- if a remote or network exception occurs
-
listRSGroups
Lists current set of RegionServer groups- Throws:
IOException
- if a remote or network exception occurs
-
listTablesInRSGroup
Get all tables in this RegionServer group.- Parameters:
groupName
- the group name- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
getConfiguredNamespacesAndTablesInRSGroup
Pair<List<String>,List<TableName>> getConfiguredNamespacesAndTablesInRSGroup(String groupName) throws IOException Get the namespaces and tables which have this RegionServer group in descriptor. The difference between this method andlistTablesInRSGroup(String)
is that, this method will not include the table which is actually in this RegionServr group but without the RegionServer group configuration in itsTableDescriptor
. For example, we have a group 'A', and we make namespace 'nsA' in this group, then all the tables under this namespace will in the group 'A', but this method will not return these tables but only the namespace 'nsA', while thelistTablesInRSGroup(String)
will return all these tables.- Parameters:
groupName
- the group name- Throws:
IOException
- if a remote or network exception occurs- See Also:
-
removeRSGroup
Remove RegionServer group associated with the given name- Parameters:
groupName
- the group name- Throws:
IOException
- if a remote or network exception occurs
-
removeServersFromRSGroup
Remove decommissioned servers from group 1. Sometimes we may find the server aborted due to some hardware failure and we must offline the server for repairing. Or we need to move some servers to join other clusters. So we need to remove these servers from the group. 2. Dead/recovering/live servers will be disallowed.- Parameters:
servers
- set of servers to remove- Throws:
IOException
- if a remote or network exception occurs
-
moveServersToRSGroup
Move given set of servers to the specified target RegionServer group- Parameters:
servers
- set of servers to movetargetGroup
- the group to move servers to- Throws:
IOException
- if a remote or network exception occurs
-
setRSGroup
Set the RegionServer group for tables- Parameters:
tables
- tables to set group forgroupName
- group name for tables- Throws:
IOException
- if a remote or network exception occurs
-
balanceRSGroup
Balance regions in the given RegionServer group- Parameters:
groupName
- the group name- Returns:
- BalanceResponse details about the balancer run
- Throws:
IOException
- if a remote or network exception occurs
-
balanceRSGroup
Balance regions in the given RegionServer group, running based on the givenBalanceRequest
.- Returns:
- BalanceResponse details about the balancer run
- Throws:
IOException
-
renameRSGroup
Rename rsgroup- Parameters:
oldName
- old rsgroup namenewName
- new rsgroup name- Throws:
IOException
- if a remote or network exception occurs
-
updateRSGroupConfig
Update RSGroup configuration- Parameters:
groupName
- the group nameconfiguration
- new configuration of the group name to be set- Throws:
IOException
- if a remote or network exception occurs
-
getLogEntries
List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String, Object> filterParams) throws IOExceptionRetrieve recent online records from HMaster / RegionServers. Examples include slow/large RPC logs, balancer decisions by master.- 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
Flush master local region- Throws:
IOException
-
getCachedFilesList
Get the list of cached files- Throws:
IOException
-