Interface AsyncAdmin
- All Known Implementing Classes:
AsyncHBaseAdmin
,RawAsyncHBaseAdmin
- Since:
- 2.0.0
-
Method Summary
Modifier and TypeMethodDescriptionabortProcedure
(long procId, boolean mayInterruptIfRunning) Deprecated.since 2.1.1 and will be removed in 4.0.0.addColumnFamily
(TableName tableName, ColumnFamilyDescriptor columnFamily) Add a column family to an existing table.default CompletableFuture<Void>
addReplicationPeer
(String peerId, ReplicationPeerConfig peerConfig) Add a new replication peer for replicating data to slave clusteraddReplicationPeer
(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) Add a new replication peer for replicating data to slave clusteraddRSGroup
(String groupName) Creates a new RegionServer group with the given nameAppend the replicable table-cf config of the specified peerassign
(byte[] regionName) Assign an individual region.default CompletableFuture<Boolean>
balance()
Invoke the balancer.default CompletableFuture<Boolean>
balance
(boolean forcible) Deprecated.Since 2.5.0.balance
(BalanceRequest request) Invoke the balancer with the given balance request.default CompletableFuture<BalanceResponse>
balanceRSGroup
(String groupName) Balance regions in the given RegionServer groupbalanceRSGroup
(String groupName, BalanceRequest request) Balance regions in the given RegionServer groupdefault CompletableFuture<Boolean>
balancerSwitch
(boolean on) Turn the load balancer on or off.balancerSwitch
(boolean on, boolean drainRITs) Turn the load balancer on or off.catalogJanitorSwitch
(boolean on) Turn the catalog janitor on/off.cleanerChoreSwitch
(boolean on) Turn the cleaner chore on/off.clearBlockCache
(TableName tableName) Clear all the blocks corresponding to this table from BlockCache.clearCompactionQueues
(ServerName serverName, Set<String> queues) Clear compacting queues on a region server.clearDeadServers
(List<ServerName> servers) Clear dead region servers from master.clearSlowLogResponses
(Set<ServerName> serverNames) Clears online slow RPC logs from the provided list of RegionServersdefault CompletableFuture<Void>
cloneSnapshot
(String snapshotName, TableName tableName) Create a new table by cloning the snapshot content.default CompletableFuture<Void>
cloneSnapshot
(String snapshotName, TableName tableName, boolean restoreAcl) Create a new table by cloning the snapshot content.cloneSnapshot
(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Create a new table by cloning the snapshot content.cloneTableSchema
(TableName tableName, TableName newTableName, boolean preserveSplits) Create a new table by cloning the existent table schema.default CompletableFuture<Void>
Compact a table.default CompletableFuture<Void>
Compact a column family within a table.compact
(TableName tableName, byte[] columnFamily, CompactType compactType) Compact a column family within a table.compact
(TableName tableName, CompactType compactType) Compact a table.compactionSwitch
(boolean switchState, List<String> serverNamesList) Turn the compaction on or off.compactRegion
(byte[] regionName) Compact an individual region.compactRegion
(byte[] regionName, byte[] columnFamily) Compact a column family within a region.compactRegionServer
(ServerName serverName) Compact all regions on the region server.<S,
R> CompletableFuture<R> coprocessorService
(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable) Execute the given coprocessor call on the master.<S,
R> CompletableFuture<R> coprocessorService
(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, ServerName serverName) Execute the given coprocessor call on the given region server.createNamespace
(NamespaceDescriptor descriptor) Create a new namespace.createTable
(TableDescriptor desc) Creates a new table.createTable
(TableDescriptor desc, byte[][] splitKeys) Creates a new table with an initial set of empty regions defined by the specified split keys.createTable
(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) Creates a new table with the specified number of regions.decommissionRegionServers
(List<ServerName> servers, boolean offload) Mark region server(s) as decommissioned to prevent additional regions from getting assigned to them.deleteColumnFamily
(TableName tableName, byte[] columnFamily) Delete a column family from a table.deleteNamespace
(String name) Delete an existing namespace.deleteSnapshot
(String snapshotName) Delete an existing snapshot.Delete all existing snapshots.deleteSnapshots
(Pattern pattern) Delete existing snapshots whose names match the pattern passed.deleteTable
(TableName tableName) Deletes a table.deleteTableSnapshots
(Pattern tableNamePattern) Delete all existing snapshots matching the given table name pattern.deleteTableSnapshots
(Pattern tableNamePattern, Pattern snapshotNamePattern) Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.disableReplicationPeer
(String peerId) Stop the replication stream to the specified peerdisableTable
(TableName tableName) Disable a table.disableTableReplication
(TableName tableName) Disable a table's replication switch.enableReplicationPeer
(String peerId) Restart the replication stream to the specified peerenableTable
(TableName tableName) Enable a table.enableTableReplication
(TableName tableName) Enable a table's replication switch.exceedThrottleQuotaSwitch
(boolean enable) Switch the exceed throttle quota.Execute a distributed procedure on a cluster.CompletableFuture<byte[]>
Execute a distributed procedure on a cluster.Flush a table.Flush the specified column family stores on all regions of the passed table.Flush the specified column family stores on all regions of the passed table.Flush master local regionflushRegion
(byte[] regionName) Flush an individual region.flushRegion
(byte[] regionName, byte[] columnFamily) Flush a column family within a region.flushRegionServer
(ServerName serverName) Flush all region on the region server.default CompletableFuture<Collection<ServerName>>
Returns current backup master list wrapped byCompletableFuture
getCachedFilesList
(ServerName serverName) Get the list of cached filesReturns cluster status wrapped byCompletableFuture
getClusterMetrics
(EnumSet<ClusterMetrics.Option> options) Returns cluster status wrapped byCompletableFuture
default CompletableFuture<CompactionState>
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.getConfiguredNamespacesAndTablesInRSGroup
(String groupName) Get the namespaces and tables which have this RegionServer group in descriptor.CompletableFuture<? extends SpaceQuotaSnapshotView>
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.CompletableFuture<? extends SpaceQuotaSnapshotView>
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) Method for getting the tableDescriptorgetLastMajorCompactionTimestamp
(TableName tableName) Get the timestamp of the last major compaction for the passed table.getLastMajorCompactionTimestampForRegion
(byte[] regionName) Get the timestamp of the last major compaction for the passed region.getLocks()
List locks.getLogEntries
(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String, Object> filterParams) Retrieve recent online records from HMaster / RegionServers.default CompletableFuture<ServerName>
Returns current master server name wrapped byCompletableFuture
default CompletableFuture<List<String>>
Returns a list of master coprocessors wrapped byCompletableFuture
default CompletableFuture<Integer>
Get the info port of the current master if one is available.getNamespaceDescriptor
(String name) Get a namespace descriptor by nameList proceduresgetQuota
(QuotaFilter filter) List the quotas based on the filter.getRegionMetrics
(ServerName serverName) Get a list ofRegionMetrics
of all regions hosted on a region server.getRegionMetrics
(ServerName serverName, TableName tableName) Get a list ofRegionMetrics
of all regions hosted on a region server 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 CompletableFuture<Collection<ServerName>>
Returns current live region servers list wrapped byCompletableFuture
default CompletableFuture<Collection<ServerName>>
getRegionServers
(boolean excludeDecommissionedRS) CompletableFuture<? extends Map<TableName,
? extends SpaceQuotaSnapshotView>> getRegionServerSpaceQuotaSnapshots
(ServerName serverName) Fetches the observedSpaceQuotaSnapshotView
s observed by a RegionServer.getReplicationPeerConfig
(String peerId) Returns the configured ReplicationPeerConfig for the specified peerdefault CompletableFuture<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 tableReturns the list of supported security capabilities.default CompletableFuture<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.getUserPermissions
(GetUserPermissionsRequest getUserPermissionsRequest) Get the global/namespace/table permissions for usergrant
(UserPermission userPermission, boolean mergeExistingPermissions) Grants user specific permissionshasUserPermissions
(String userName, List<Permission> permissions) Check if the user has specific permissionsdefault CompletableFuture<List<Boolean>>
hasUserPermissions
(List<Permission> permissions) Check if call user has specific permissionsQuery the current state of the balancer.Query on the catalog janitor state.Query the current state of the cleaner chore.Check whether master is in maintenance modeQuery the current state of the Merge switch.Query the current state of the region normalizerCheck the current state of the specified procedure.isReplicationPeerEnabled
(String peerId) Check if a replication peer is enabled.Check whether replication peer modification is enabled.Get if the rpc throttle is enabled.Query the current state of the auto snapshot cleanup based on TTL.isSnapshotFinished
(SnapshotDescription snapshot) Check the current state of the passed snapshot.Query the current state of the Split switch.isTableAvailable
(TableName tableName) Check if a table is available.isTableDisabled
(TableName tableName) Check if a table is disabled.isTableEnabled
(TableName tableName) Check if a table is enabled.default CompletableFuture<List<ServerName>>
List all the dead region servers.List region servers marked as decommissioned, which can not be assigned regions.List available namespace descriptorsList 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.default CompletableFuture<List<TableDescriptor>>
List all the userspace tables.listTableDescriptors
(boolean includeSysTables) List all the tables.listTableDescriptors
(List<TableName> tableNames) List specific tables including system tables.listTableDescriptors
(Pattern pattern, boolean includeSysTables) List all the tables matching the given pattern.Get list of table descriptors by namespace.listTableDescriptorsByState
(boolean isEnabled) List all enabled or disabled table descriptorsdefault CompletableFuture<List<TableName>>
List all of the names of userspace tables.listTableNames
(boolean includeSysTables) List all of the names of 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) List all the completed snapshots matching the given table name pattern.listTableSnapshots
(Pattern tableNamePattern, Pattern snapshotNamePattern) List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.default CompletableFuture<List<ServerName>>
List all the unknown region servers.default CompletableFuture<Void>
majorCompact
(TableName tableName) Major compact a table.default CompletableFuture<Void>
majorCompact
(TableName tableName, byte[] columnFamily) Major compact a column family within a table.majorCompact
(TableName tableName, byte[] columnFamily, CompactType compactType) Major compact a column family within a table.majorCompact
(TableName tableName, CompactType compactType) Major compact a table.majorCompactRegion
(byte[] regionName) Major compact a region.majorCompactRegion
(byte[] regionName, byte[] columnFamily) Major compact a column family within region.majorCompactRegionServer
(ServerName serverName) Compact all regions on the region server.default CompletableFuture<Void>
mergeRegions
(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) Deprecated.since 2.3.0 and will be removed in 4.0.0.UsemergeRegions(List, boolean)
instead.mergeRegions
(List<byte[]> nameOfRegionsToMerge, boolean forcible) Merge multiple regions (>=2).default CompletableFuture<Boolean>
mergeSwitch
(boolean enabled) Turn the Merge switch on or off.mergeSwitch
(boolean enabled, boolean drainMerges) Turn the Merge switch on or off.modifyColumnFamily
(TableName tableName, ColumnFamilyDescriptor columnFamily) Modify an existing column family on a table.modifyColumnFamilyStoreFileTracker
(TableName tableName, byte[] family, String dstSFT) Change the store file tracker of the given table's given family.modifyNamespace
(NamespaceDescriptor descriptor) Modify an existing namespace.default CompletableFuture<Void>
modifyTable
(TableDescriptor desc) Modify an existing table, more IRB friendly version.modifyTable
(TableDescriptor desc, boolean reopenRegions) Modify an existing table, more IRB friendly version.modifyTableStoreFileTracker
(TableName tableName, String dstSFT) Change the store file tracker of the given table.move
(byte[] regionName) Move the regionr
to a random server.move
(byte[] regionName, ServerName destServerName) Move the regionr
todest
.moveServersToRSGroup
(Set<Address> servers, String groupName) Move given set of servers to the specified target RegionServer groupdefault CompletableFuture<Boolean>
Invoke region normalizer.Invoke region normalizer.normalizerSwitch
(boolean on) Set region normalizer on/off.offline
(byte[] regionName) Offline specified region from master's in-memory state.recommissionRegionServer
(ServerName server, List<byte[]> encodedRegionNames) Remove decommission marker from a region server to allow regions assignments.removeReplicationPeer
(String peerId) Remove a peer and stop the replicationRemove some table-cfs from config of the specified peerremoveRSGroup
(String groupName) Remove RegionServer group associated with the given nameremoveServersFromRSGroup
(Set<Address> servers) Remove decommissioned servers from group 1.renameRSGroup
(String oldName, String newName) Rename rsgroupdefault CompletableFuture<Boolean>
replicationPeerModificationSwitch
(boolean on) Enable or disable replication peer modification.replicationPeerModificationSwitch
(boolean on, boolean drainProcedures) Enable or disable replication peer modification.restoreSnapshot
(String snapshotName) Restore the specified snapshot on the original table.default CompletableFuture<Void>
restoreSnapshot
(String snapshotName, boolean takeFailSafeSnapshot) Restore the specified snapshot on the original table.restoreSnapshot
(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) Restore the specified snapshot on the original table.revoke
(UserPermission userPermission) Revokes user specific permissionsrollWALWriter
(ServerName serverName) Roll the log writer.Ask for a scan of the catalog table.Ask for cleaner chore to run.setQuota
(QuotaSettings quota) Apply the new quota settings.setRSGroup
(Set<TableName> tables, String groupName) Set the RegionServer group for tablesshutdown()
Shuts down the HBase cluster.default CompletableFuture<Void>
Take a snapshot for the given table.default CompletableFuture<Void>
snapshot
(String snapshotName, TableName tableName, SnapshotType type) Create typed snapshot of the table.snapshot
(SnapshotDescription snapshot) Take a snapshot and wait for the server to complete that snapshot asynchronously.snapshotCleanupSwitch
(boolean on, boolean sync) Turn on or off the auto snapshot cleanup based on TTL.Split a table.Split a table.splitRegion
(byte[] regionName) Split an individual region.splitRegion
(byte[] regionName, byte[] splitPoint) Split an individual region.default CompletableFuture<Boolean>
splitSwitch
(boolean enabled) Turn the Split switch on or off.splitSwitch
(boolean enabled, boolean drainSplits) Turn the Split switch on or off.Shuts down the current HBase master only.stopRegionServer
(ServerName serverName) Stop the designated regionserver.switchRpcThrottle
(boolean enable) Switch the rpc throttle enabled state.tableExists
(TableName tableName) Check if a table exists.transitReplicationPeerSyncReplicationState
(String peerId, SyncReplicationState state) Transit current cluster to a new state in a synchronous replication peer.truncateRegion
(byte[] regionName) Truncate an individual region.truncateTable
(TableName tableName, boolean preserveSplits) Truncate a table.unassign
(byte[] regionName) Unassign a region from current hosting regionserver.default CompletableFuture<Void>
unassign
(byte[] regionName, boolean forcible) Deprecated.since 2.4.0 and will be removed in 4.0.0.Update the configuration and trigger an online config change on all the masters and regionservers.updateConfiguration
(String groupName) Update the configuration and trigger an online config change on all the regionservers in the RSGroup.updateConfiguration
(ServerName serverName) Update the configuration and trigger an online config change on the regionserver.updateReplicationPeerConfig
(String peerId, ReplicationPeerConfig peerConfig) Update the peerConfig for the specified peerupdateRSGroupConfig
(String groupName, Map<String, String> configuration) Update RSGroup configuration
-
Method Details
-
tableExists
Check if a table exists.- Parameters:
tableName
- Table to check.- Returns:
- True if table exists already. The return value will be wrapped by a
CompletableFuture
.
-
listTableDescriptors
List all the userspace tables.- Returns:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture
.
-
listTableDescriptors
List all the tables.- Parameters:
includeSysTables
- False to match only against userspace tables- Returns:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture
.
-
listTableDescriptors
CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern, boolean includeSysTables) 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:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture
.
-
listTableDescriptors
List specific tables including system tables.- Parameters:
tableNames
- the table list to match against- Returns:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture
.
-
listTableDescriptorsByNamespace
Get list of table descriptors by namespace.- Parameters:
name
- namespace name- Returns:
- returns a list of TableDescriptors wrapped by a
CompletableFuture
.
-
listTableDescriptorsByState
List all enabled or disabled table descriptors- Parameters:
isEnabled
- is true means return enabled table descriptors, false means return disabled table descriptors- Returns:
- a list of table names wrapped by a
CompletableFuture
.
-
listTableNames
List all of the names of userspace tables.- Returns:
- a list of table names wrapped by a
CompletableFuture
. - See Also:
-
listTableNames
List all of the names of tables.- Parameters:
includeSysTables
- False to match only against userspace tables- Returns:
- a list of table names wrapped by a
CompletableFuture
.
-
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:
- a list of table names wrapped by a
CompletableFuture
.
-
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 table names wrapped by a
CompletableFuture
.
-
listTableNamesByNamespace
Get list of table names by namespace.- Parameters:
name
- namespace name- Returns:
- The list of table names in the namespace wrapped by a
CompletableFuture
.
-
getDescriptor
Method for getting the tableDescriptor- Parameters:
tableName
- as aTableName
- Returns:
- the read-only tableDescriptor wrapped by a
CompletableFuture
.
-
createTable
Creates a new table.- Parameters:
desc
- table descriptor for table
-
createTable
CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) 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.- Parameters:
desc
- table descriptor for tablestartKey
- beginning of key rangeendKey
- end of key rangenumRegions
- the total number of regions to create
-
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. Note : Avoid passing empty split key.- Parameters:
desc
- table descriptor for tablesplitKeys
- array of split keys for the initial regions of the table
-
modifyTable
Modify an existing table, more IRB friendly version.- Parameters:
desc
- modified description of the table
-
modifyTable
Modify an existing table, more IRB friendly version.- Parameters:
desc
- 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.
-
modifyTableStoreFileTracker
Change the store file tracker of the given table.- Parameters:
tableName
- the table you want to changedstSFT
- the destination store file tracker
-
deleteTable
Deletes a table.- Parameters:
tableName
- name of table to delete
-
truncateTable
Truncate a table.- Parameters:
tableName
- name of table to truncatepreserveSplits
- True if the splits should be preserved
-
enableTable
Enable a table. The table has to be in disabled state for it to be enabled.- Parameters:
tableName
- name of the table
-
disableTable
Disable a table. The table has to be in enabled state for it to be disabled. -
isTableEnabled
Check if a table is enabled.- Parameters:
tableName
- name of table to check- Returns:
- true if table is on-line. The return value will be wrapped by a
CompletableFuture
.
-
isTableDisabled
Check if a table is disabled.- Parameters:
tableName
- name of table to check- Returns:
- true if table is off-line. The return value will be wrapped by a
CompletableFuture
.
-
isTableAvailable
Check if a table is available.- Parameters:
tableName
- name of table to check- Returns:
- true if all regions of the table are available. The return value will be wrapped by a
CompletableFuture
.
-
addColumnFamily
Add a column family to an existing table.- Parameters:
tableName
- name of the table to add column family tocolumnFamily
- column family descriptor of column family to be added
-
deleteColumnFamily
Delete a column family from a table.- Parameters:
tableName
- name of tablecolumnFamily
- name of column family to be deleted
-
modifyColumnFamily
CompletableFuture<Void> modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) Modify an existing column family on a table.- Parameters:
tableName
- name of tablecolumnFamily
- new column family descriptor to use
-
modifyColumnFamilyStoreFileTracker
CompletableFuture<Void> modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT) 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
-
createNamespace
Create a new namespace.- Parameters:
descriptor
- descriptor which describes the new namespace
-
modifyNamespace
Modify an existing namespace.- Parameters:
descriptor
- descriptor which describes the new namespace
-
deleteNamespace
Delete an existing namespace. Only empty namespaces (no tables) can be removed.- Parameters:
name
- namespace name
-
getNamespaceDescriptor
Get a namespace descriptor by name- Parameters:
name
- name of namespace descriptor- Returns:
- A descriptor wrapped by a
CompletableFuture
.
-
listNamespaces
List available namespaces- Returns:
- List of namespaces wrapped by a
CompletableFuture
.
-
listNamespaceDescriptors
List available namespace descriptors- Returns:
- List of descriptors wrapped by a
CompletableFuture
.
-
getRegions
Get all the online regions on a region server. -
getRegions
Get the regions of a given table. -
flush
Flush a table.- Parameters:
tableName
- table to flush
-
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
-
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
-
flushRegion
Flush an individual region.- Parameters:
regionName
- region to flush
-
flushRegion
Flush a column family within a region.- Parameters:
regionName
- region to flushcolumnFamily
- column family within a region. If not present, flush the region's all column families.
-
flushRegionServer
Flush all region on the region server.- Parameters:
serverName
- server to flush
-
compact
Compact a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found.- Parameters:
tableName
- table to compact
-
compact
Compact a column family within a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found.- Parameters:
tableName
- table to compactcolumnFamily
- column family within a table. If not present, compact the table's all column families.
-
compact
Compact a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found for normal compaction type.- Parameters:
tableName
- table to compactcompactType
-CompactType
-
compact
Compact a column family within a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found for normal compaction type.- Parameters:
tableName
- table to compactcolumnFamily
- column family within a tablecompactType
-CompactType
-
compactRegion
Compact an individual region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Parameters:
regionName
- region to compact
-
compactRegion
Compact a column family within a region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Parameters:
regionName
- region to compactcolumnFamily
- column family within a region. If not present, compact the region's all column families.
-
majorCompact
Major compact a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found.- Parameters:
tableName
- table to major compact
-
majorCompact
Major compact a column family within a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found for normal compaction. type.- Parameters:
tableName
- table to major compactcolumnFamily
- column family within a table. If not present, major compact the table's all column families.
-
majorCompact
Major compact a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found for normal compaction type.- Parameters:
tableName
- table to major compactcompactType
-CompactType
-
majorCompact
CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) Major compact a column family within a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundException
if table not found.- Parameters:
tableName
- table to major compactcolumnFamily
- column family within a table. If not present, major compact the table's all column families.compactType
-CompactType
-
majorCompactRegion
Major compact a region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Parameters:
regionName
- region to major compact
-
majorCompactRegion
Major compact a column family within region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Parameters:
regionName
- region to major compactcolumnFamily
- column family within a region. If not present, major compact the region's all column families.
-
compactRegionServer
Compact all regions on the region server.- Parameters:
serverName
- the region server name
-
majorCompactRegionServer
Compact all regions on the region server.- Parameters:
serverName
- the region server name
-
mergeSwitch
Turn the Merge switch on or off.- Parameters:
enabled
- enabled or not- Returns:
- Previous switch value wrapped by a
CompletableFuture
-
mergeSwitch
Turn the Merge switch on or off. Notice that, the method itself is always non-blocking, which means it will always return immediately. ThedrainMerges
parameter only effects when will we complete the returnedCompletableFuture
.- Parameters:
enabled
- enabled or notdrainMerges
- Iftrue
, it waits until current merge() call, if outstanding, to return.- Returns:
- Previous switch value wrapped by a
CompletableFuture
-
isMergeEnabled
Query the current state of the Merge switch.- Returns:
- true if the switch is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
splitSwitch
Turn the Split switch on or off.- Parameters:
enabled
- enabled or not- Returns:
- Previous switch value wrapped by a
CompletableFuture
-
splitSwitch
Turn the Split switch on or off. Notice that, the method itself is always non-blocking, which means it will always return immediately. ThedrainSplits
parameter only effects when will we complete the returnedCompletableFuture
.- Parameters:
enabled
- enabled or notdrainSplits
- Iftrue
, it waits until current split() call, if outstanding, to return.- Returns:
- Previous switch value wrapped by a
CompletableFuture
-
isSplitEnabled
Query the current state of the Split switch.- Returns:
- true if the switch is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
mergeRegions
@Deprecated default CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible) Deprecated.since 2.3.0 and will be removed in 4.0.0.UsemergeRegions(List, boolean)
instead.Merge two regions.- 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
-
mergeRegions
Merge multiple regions (>=2).- Parameters:
nameOfRegionsToMerge
- encoded or full name of daughter regionsforcible
- true if do a compulsory merge, otherwise we will only merge two adjacent regions
-
split
Split a table. The method will execute split action for each region in table.- Parameters:
tableName
- table to split
-
splitRegion
Split an individual region.- Parameters:
regionName
- region to split
-
split
Split a table.- Parameters:
tableName
- table to splitsplitPoint
- the explicit position to split on
-
splitRegion
Split an individual region.- Parameters:
regionName
- region to splitsplitPoint
- the explicit position to split on. If not present, it will decide by region server.
-
truncateRegion
Truncate an individual region.- Parameters:
regionName
- region to truncate
-
assign
Assign an individual region.- Parameters:
regionName
- Encoded or full name of region to assign.
-
unassign
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
- Encoded or full name of region to unassign.
-
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
- Encoded or full name of region to unassign. Will clear any existing RegionPlan if one found.forcible
- If true, force unassign (Will remove region from regions-in-transition too if present. If results in double assignment use hbck -fix to resolve. To be used by experts).- 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
- Encoded or full name of region to offline
-
move
Move the regionr
to a random server.- Parameters:
regionName
- Encoded or full name of region to move.
-
move
Move the regionr
todest
.- Parameters:
regionName
- Encoded or full name of region to move.destServerName
- The servername of the destination regionserver. If not present, 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
-
setQuota
Apply the new quota settings.- Parameters:
quota
- the quota settings
-
getQuota
List the quotas based on the filter.- Parameters:
filter
- the quota settings filter- Returns:
- the QuotaSetting list, which wrapped by a CompletableFuture.
-
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 slave cluster
-
addReplicationPeer
CompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) Add a new replication peer for replicating data to slave cluster- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication slave clusterenabled
- peer state, true if ENABLED and false if DISABLED
-
removeReplicationPeer
Remove a peer and stop the replication- Parameters:
peerId
- a short name that identifies the peer
-
enableReplicationPeer
Restart the replication stream to the specified peer- Parameters:
peerId
- a short name that identifies the peer
-
disableReplicationPeer
Stop the replication stream to the specified peer- Parameters:
peerId
- a short name that identifies the peer
-
getReplicationPeerConfig
Returns the configured ReplicationPeerConfig for the specified peer- Parameters:
peerId
- a short name that identifies the peer- Returns:
- ReplicationPeerConfig for the peer wrapped by a
CompletableFuture
.
-
updateReplicationPeerConfig
CompletableFuture<Void> updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) Update the peerConfig for the specified peer- Parameters:
peerId
- a short name that identifies the peerpeerConfig
- new config for the peer
-
transitReplicationPeerSyncReplicationState
CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state) 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
-
getReplicationPeerSyncReplicationState
default CompletableFuture<SyncReplicationState> getReplicationPeerSyncReplicationState(String peerId) Get the current cluster state in a synchronous replication peer.- Parameters:
peerId
- a short name that identifies the peer- Returns:
- the current cluster state wrapped by a
CompletableFuture
.
-
appendReplicationPeerTableCFs
CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId, Map<TableName, List<String>> tableCfs) Append the replicable table-cf config of the specified peer- Parameters:
peerId
- a short that identifies the clustertableCfs
- A map from tableName to column family names
-
removeReplicationPeerTableCFs
CompletableFuture<Void> removeReplicationPeerTableCFs(String peerId, Map<TableName, List<String>> tableCfs) Remove some table-cfs from config of the specified peer- Parameters:
peerId
- a short name that identifies the clustertableCfs
- A map from tableName to column family names
-
listReplicationPeers
Return a list of replication peers.- Returns:
- a list of replication peers description. The return value will be wrapped by a
CompletableFuture
.
-
listReplicationPeers
Return a list of replication peers.- Parameters:
pattern
- The compiled regular expression to match peer id- Returns:
- a list of replication peers description. The return value will be wrapped by a
CompletableFuture
.
-
listReplicatedTableCFs
Find all table and column families that are replicated from this cluster- Returns:
- the replicated table-cfs list of this cluster. The return value will be wrapped by a
CompletableFuture
.
-
enableTableReplication
Enable a table's replication switch.- Parameters:
tableName
- name of the table
-
disableTableReplication
Disable a table's replication switch.- Parameters:
tableName
- name of the table
-
isReplicationPeerEnabled
Check if a replication peer is enabled.- Parameters:
peerId
- id of replication peer to check- Returns:
- true if replication peer is enabled. The return value will be wrapped by a
CompletableFuture
-
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 wrapped by a
CompletableFuture
-
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 wrapped by a
CompletableFuture
-
isReplicationPeerModificationEnabled
Check whether replication peer modification is enabled.- Returns:
true
if modification is enabled, otherwisefalse
, wrapped by aCompletableFuture
-
snapshot
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
-
snapshot
default CompletableFuture<Void> snapshot(String snapshotName, TableName tableName, SnapshotType type) 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
-
snapshot
Take a snapshot and wait for the server to complete that snapshot asynchronously. 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[])
. 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
-
isSnapshotFinished
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
-
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 to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted.- Parameters:
snapshotName
- name of the snapshot to restore
-
restoreSnapshot
Restore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".- Parameters:
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true if the failsafe snapshot should be taken
-
restoreSnapshot
CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) Restore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".- Parameters:
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true if the failsafe snapshot should be takenrestoreAcl
-true
to restore acl of snapshot
-
cloneSnapshot
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
-
cloneSnapshot
default CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl) 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 restore acl of snapshot
-
cloneSnapshot
CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) 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 restore acl of snapshotcustomSFT
- specify the StroreFileTracker used for the table
-
listSnapshots
List completed snapshots.- Returns:
- a list of snapshot descriptors for completed snapshots wrapped by a
CompletableFuture
-
listSnapshots
List all the completed snapshots matching the given pattern.- Parameters:
pattern
- The compiled regular expression to match against- Returns:
- - returns a List of SnapshotDescription wrapped by a
CompletableFuture
-
listTableSnapshots
List all the completed snapshots matching the given table name pattern.- Parameters:
tableNamePattern
- The compiled table name regular expression to match against- Returns:
- - returns a List of completed SnapshotDescription wrapped by a
CompletableFuture
-
listTableSnapshots
CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) 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:
- - returns a List of completed SnapshotDescription wrapped by a
CompletableFuture
-
deleteSnapshot
Delete an existing snapshot.- Parameters:
snapshotName
- name of the snapshot
-
deleteSnapshots
Delete all existing snapshots. -
deleteSnapshots
Delete existing snapshots whose names match the pattern passed.- Parameters:
pattern
- pattern for names of the snapshot to match
-
deleteTableSnapshots
Delete all existing snapshots matching the given table name pattern.- Parameters:
tableNamePattern
- The compiled table name regular expression to match against
-
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
-
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
-
execProcedureWithReturn
CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance, Map<String, String> props) 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- Returns:
- data returned after procedure execution. null if no return data.
-
isProcedureFinished
CompletableFuture<Boolean> isProcedureFinished(String signature, String instance, Map<String, String> props) Check the current state of the specified procedure. There are three possible states:- running - returns false
- finished - returns true
- finished with error - throws the exception that caused the procedure to fail
- 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.
The value is wrapped by
CompletableFuture
-
abortProcedure
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. the value is
wrapped by
CompletableFuture
- See Also:
-
getProcedures
List procedures- Returns:
- procedure list JSON wrapped by
CompletableFuture
-
getLocks
List locks.- Returns:
- lock list JSON wrapped by
CompletableFuture
-
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
-
listDecommissionedRegionServers
List region servers marked as decommissioned, which can not be assigned regions.- Returns:
- List of decommissioned region servers wrapped by
CompletableFuture
-
recommissionRegionServer
CompletableFuture<Void> recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) 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.
-
getClusterMetrics
Returns cluster status wrapped byCompletableFuture
-
getClusterMetrics
Returns cluster status wrapped byCompletableFuture
-
getMaster
Returns current master server name wrapped byCompletableFuture
-
getBackupMasters
Returns current backup master list wrapped byCompletableFuture
-
getRegionServers
Returns current live region servers list wrapped byCompletableFuture
-
getRegionServers
-
getMasterCoprocessorNames
Returns a list of master coprocessors wrapped byCompletableFuture
-
getMasterInfoPort
Get the info port of the current master if one is available.- Returns:
- master info port
-
shutdown
Shuts down the HBase cluster. -
stopMaster
Shuts down the current HBase master only. -
stopRegionServer
Stop the designated regionserver. -
updateConfiguration
Update the configuration and trigger an online config change on the regionserver.- Parameters:
serverName
- : The server whose config needs to be updated.
-
updateConfiguration
Update the configuration and trigger an online config change on all the masters and regionservers. -
updateConfiguration
Update the configuration and trigger an online config change on all the regionservers in the RSGroup.- Parameters:
groupName
- the group name
-
rollWALWriter
Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.When the returned CompletableFuture is done, it only means the rollWALWriter request was sent to the region server and may need some time to finish the rollWALWriter operation. 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 region server.
-
clearCompactionQueues
Clear compacting queues on a region server.- Parameters:
serverName
- The servername of the region server.queues
- the set of queue name
-
getRegionMetrics
Get a list ofRegionMetrics
of all regions hosted on a region server.- Returns:
- list of
RegionMetrics
wrapped byCompletableFuture
-
getRegionMetrics
Get a list ofRegionMetrics
of all regions hosted on a region server for a table.- Returns:
- a list of
RegionMetrics
wrapped byCompletableFuture
-
isMasterInMaintenanceMode
Check whether master is in maintenance mode- Returns:
- true if master is in maintenance mode, false otherwise. The return value will be
wrapped by a
CompletableFuture
-
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 wrapped by a
CompletableFuture
-
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 examinecompactType
-CompactType
- Returns:
- the current compaction state wrapped by a
CompletableFuture
-
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 wrapped by a
CompletableFuture
-
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 not present if no such HFile could be found.
- Parameters:
tableName
- table to examine- Returns:
- the last major compaction timestamp wrapped by a
CompletableFuture
-
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 not present if no such HFile could be found.
- Parameters:
regionName
- region to examine- Returns:
- the last major compaction timestamp wrapped by a
CompletableFuture
-
getSecurityCapabilities
Returns the list of supported security capabilities. The return value will be wrapped by aCompletableFuture
. -
balancerSwitch
Turn the load balancer on or off.- Parameters:
on
- Set totrue
to enable,false
to disable.- Returns:
- Previous balancer value wrapped by a
CompletableFuture
.
-
balancerSwitch
Turn the load balancer on or off. Notice that, the method itself is always non-blocking, which means it will always return immediately. ThedrainRITs
parameter only effects when will we complete the returnedCompletableFuture
.- Parameters:
on
- Set totrue
to enable,false
to disable.drainRITs
- Iftrue
, it waits until current balance() call, if outstanding, to return.- Returns:
- Previous balancer value wrapped by a
CompletableFuture
.
-
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. The return value will be wrapped by a
CompletableFuture
.
-
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:
forcible
- whether we should force balance even if there is region in transition.- Returns:
- True if balancer ran, false otherwise. The return value will be wrapped by a
CompletableFuture
.
-
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.
-
isBalancerEnabled
Query the current state of the balancer.- Returns:
- true if the balance switch is on, false otherwise. The return value will be wrapped by
a
CompletableFuture
.
-
normalizerSwitch
Set region normalizer on/off.- Parameters:
on
- whether normalizer should be on or off- Returns:
- Previous normalizer value wrapped by a
CompletableFuture
-
isNormalizerEnabled
Query the current state of the region normalizer- Returns:
- true if region normalizer is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
normalize
Invoke region normalizer. Can NOT run for various reasons. Check logs.- Returns:
- true if region normalizer ran, false otherwise. The return value will be wrapped by a
CompletableFuture
-
normalize
Invoke region normalizer. Can NOT run for various reasons. Check logs.- Parameters:
ntfp
- limit to tables matching the specified filter.- Returns:
- true if region normalizer ran, false otherwise. The return value will be wrapped by a
CompletableFuture
-
cleanerChoreSwitch
Turn the cleaner chore on/off.- Returns:
- Previous cleaner state wrapped by a
CompletableFuture
-
isCleanerChoreEnabled
Query the current state of the cleaner chore.- Returns:
- true if cleaner chore is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
runCleanerChore
Ask for cleaner chore to run.- Returns:
- true if cleaner chore ran, false otherwise. The return value will be wrapped by a
CompletableFuture
-
catalogJanitorSwitch
Turn the catalog janitor on/off.- Returns:
- the previous state wrapped by a
CompletableFuture
-
isCatalogJanitorEnabled
Query on the catalog janitor state.- Returns:
- true if the catalog janitor is on, false otherwise. The return value will be wrapped by
a
CompletableFuture
-
runCatalogJanitor
Ask for a scan of the catalog table.- Returns:
- the number of entries cleaned. The return value will be wrapped by a
CompletableFuture
-
coprocessorService
<S,R> CompletableFuture<R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable) Execute the given coprocessor call on the master.The
stubMaker
is just a delegation to thenewStub
call. Usually it is only a one line lambda expression, like:channel -> xxxService.newStub(channel)
- Type Parameters:
S
- the type of the asynchronous stubR
- the type of the return value- Parameters:
stubMaker
- a delegation to the actualnewStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment ofServiceCaller
for more details.- Returns:
- the return value of the protobuf rpc call, wrapped by a
CompletableFuture
. - See Also:
-
coprocessorService
<S,R> CompletableFuture<R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, ServerName serverName) Execute the given coprocessor call on the given region server.The
stubMaker
is just a delegation to thenewStub
call. Usually it is only a one line lambda expression, like:channel -> xxxService.newStub(channel)
- Type Parameters:
S
- the type of the asynchronous stubR
- the type of the return value- Parameters:
stubMaker
- a delegation to the actualnewStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment ofServiceCaller
for more details.serverName
- the given region server- Returns:
- the return value of the protobuf rpc call, wrapped by a
CompletableFuture
. - See Also:
-
listDeadServers
List all the dead region servers. -
listUnknownServers
List all the unknown region servers. -
clearDeadServers
Clear dead region servers from master.- Parameters:
servers
- list of dead region servers.- Returns:
- - returns a list of servers that not cleared wrapped by a
CompletableFuture
.
-
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 wrapped by a
CompletableFuture
.
-
cloneTableSchema
CompletableFuture<Void> cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) 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
-
compactionSwitch
CompletableFuture<Map<ServerName,Boolean>> compactionSwitch(boolean switchState, List<String> serverNamesList) 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
-
switchRpcThrottle
Switch the rpc throttle enabled state.- Parameters:
enable
- Set totrue
to enable,false
to disable.- Returns:
- Previous rpc throttle enabled value
-
isRpcThrottleEnabled
Get if the rpc throttle is enabled.- Returns:
- True if rpc throttle is enabled
-
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
-
getSpaceQuotaTableSizes
Fetches the table sizes on the filesystem as tracked by the HBase Master. -
getRegionServerSpaceQuotaSnapshots
CompletableFuture<? extends Map<TableName,? extends SpaceQuotaSnapshotView>> getRegionServerSpaceQuotaSnapshots(ServerName serverName) Fetches the observedSpaceQuotaSnapshotView
s observed by a RegionServer. -
getCurrentSpaceQuotaSnapshot
Returns the Master's view of a quota on the givennamespace
or null if the Master has no quota information on that namespace. -
getCurrentSpaceQuotaSnapshot
CompletableFuture<? extends SpaceQuotaSnapshotView> 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. -
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.
-
revoke
Revokes user specific permissions- Parameters:
userPermission
- user name and the specific permission
-
getUserPermissions
CompletableFuture<List<UserPermission>> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) 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
-
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
-
hasUserPermissions
Check if call user has specific permissions- Parameters:
permissions
- the specific permission list- Returns:
- True if user has the specific permissions
-
snapshotCleanupSwitch
Turn on or off the auto snapshot cleanup based on TTL. Notice that, the method itself is always non-blocking, which means it will always return immediately. Thesync
parameter only effects when will we complete the returnedCompletableFuture
.- Parameters:
on
- Set totrue
to enable,false
to disable.sync
- Iftrue
, it waits until current snapshot cleanup is completed, if outstanding.- Returns:
- Previous auto snapshot cleanup value wrapped by a
CompletableFuture
.
-
isSnapshotCleanupEnabled
Query the current state of the auto snapshot cleanup based on TTL.- Returns:
- true if the auto snapshot cleanup is enabled, false otherwise. The return value will be
wrapped by a
CompletableFuture
.
-
getSlowLogResponses
@Deprecated default CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(Set<ServerName> serverNames, LogQueryFilter logQueryFilter) Deprecated.since 2.4.0 and will be removed in 4.0.0. UsegetLogEntries(Set, String, ServerType, int, Map)
instead.Retrieves online slow RPC logs from the provided list of RegionServers- Parameters:
serverNames
- Server names to get slowlog responses fromlogQueryFilter
- filter to be used if provided- Returns:
- Online slowlog response list. The return value wrapped by a
CompletableFuture
-
clearSlowLogResponses
Clears online slow 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. The return value wrapped by a
CompletableFuture
-
addRSGroup
Creates a new RegionServer group with the given name- Parameters:
groupName
- the name of the group
-
getRSGroup
Get group info for the given group name- Parameters:
groupName
- the group name- Returns:
- group info
-
getRSGroup
Get group info for the given hostPort- Parameters:
hostPort
- HostPort to get RSGroupInfo for
-
getRSGroup
Get group info for the given table- Parameters:
tableName
- table name to get RSGroupInfo for
-
listRSGroups
Lists current set of RegionServer groups -
listTablesInRSGroup
Get all tables in this RegionServer group.- Parameters:
groupName
- the group name- See Also:
-
getConfiguredNamespacesAndTablesInRSGroup
CompletableFuture<Pair<List<String>,List<TableName>>> getConfiguredNamespacesAndTablesInRSGroup(String groupName) 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- See Also:
-
removeRSGroup
Remove RegionServer group associated with the given name- Parameters:
groupName
- the group name
-
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
-
moveServersToRSGroup
Move given set of servers to the specified target RegionServer group- Parameters:
servers
- set of servers to movegroupName
- the group to move servers to
-
setRSGroup
Set the RegionServer group for tables- Parameters:
tables
- tables to set group forgroupName
- group name for tables
-
balanceRSGroup
Balance regions in the given RegionServer group- Parameters:
groupName
- the group name- Returns:
- BalanceResponse details about the balancer run
-
balanceRSGroup
Balance regions in the given RegionServer group- Parameters:
groupName
- the group namerequest
- options to define how the balancer should run- Returns:
- BalanceResponse details about the balancer run
-
renameRSGroup
Rename rsgroup- Parameters:
oldName
- old rsgroup namenewName
- new rsgroup name
-
updateRSGroupConfig
Update RSGroup configuration- Parameters:
groupName
- the group nameconfiguration
- new configuration of the group name to be set
-
getLogEntries
CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String, Object> filterParams) Retrieve 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
-
flushMasterStore
Flush master local region -
getCachedFilesList
Get the list of cached files
-