@InterfaceAudience.Public public interface AsyncAdmin
Modifier and Type | Method and Description |
---|---|
CompletableFuture<Boolean> |
abortProcedure(long procId,
boolean mayInterruptIfRunning)
Deprecated.
since 2.1.1 and will be removed in 4.0.0.
|
CompletableFuture<Void> |
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 cluster
|
CompletableFuture<Void> |
addReplicationPeer(String peerId,
ReplicationPeerConfig peerConfig,
boolean enabled)
Add a new replication peer for replicating data to slave cluster
|
CompletableFuture<Void> |
appendReplicationPeerTableCFs(String peerId,
Map<TableName,List<String>> tableCfs)
Append the replicable table-cf config of the specified peer
|
CompletableFuture<Void> |
assign(byte[] regionName) |
default CompletableFuture<Boolean> |
balance()
Invoke the balancer.
|
CompletableFuture<Boolean> |
balance(boolean forcible)
Invoke the balancer.
|
default CompletableFuture<Boolean> |
balancerSwitch(boolean on)
Turn the load balancer on or off.
|
CompletableFuture<Boolean> |
balancerSwitch(boolean on,
boolean drainRITs)
Turn the load balancer on or off.
|
CompletableFuture<Boolean> |
catalogJanitorSwitch(boolean on)
Turn the catalog janitor on/off.
|
CompletableFuture<Boolean> |
cleanerChoreSwitch(boolean on)
Turn the cleaner chore on/off.
|
CompletableFuture<CacheEvictionStats> |
clearBlockCache(TableName tableName)
Clear all the blocks corresponding to this table from BlockCache.
|
CompletableFuture<Void> |
clearCompactionQueues(ServerName serverName,
Set<String> queues)
Clear compacting queues on a region server.
|
CompletableFuture<List<ServerName>> |
clearDeadServers(List<ServerName> servers)
Clear dead region servers from master.
|
CompletableFuture<List<Boolean>> |
clearSlowLogResponses(Set<ServerName> serverNames)
Clears online slow RPC logs from the provided list of
RegionServers
|
default CompletableFuture<Void> |
cloneSnapshot(String snapshotName,
TableName tableName)
Create a new table by cloning the snapshot content.
|
CompletableFuture<Void> |
cloneSnapshot(String snapshotName,
TableName tableName,
boolean restoreAcl)
Create a new table by cloning the snapshot content.
|
CompletableFuture<Void> |
cloneTableSchema(TableName tableName,
TableName newTableName,
boolean preserveSplits)
Create a new table by cloning the existent table schema.
|
default CompletableFuture<Void> |
compact(TableName tableName)
Compact a table.
|
default CompletableFuture<Void> |
compact(TableName tableName,
byte[] columnFamily)
Compact a column family within a table.
|
CompletableFuture<Void> |
compact(TableName tableName,
byte[] columnFamily,
CompactType compactType)
Compact a column family within a table.
|
CompletableFuture<Void> |
compact(TableName tableName,
CompactType compactType)
Compact a table.
|
CompletableFuture<Map<ServerName,Boolean>> |
compactionSwitch(boolean switchState,
List<String> serverNamesList)
Turn the compaction on or off.
|
CompletableFuture<Void> |
compactRegion(byte[] regionName)
Compact an individual region.
|
CompletableFuture<Void> |
compactRegion(byte[] regionName,
byte[] columnFamily)
Compact a column family within a region.
|
CompletableFuture<Void> |
compactRegionServer(ServerName serverName)
Compact all regions on the region server.
|
<S,R> CompletableFuture<R> |
coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker,
ServiceCaller<S,R> callable)
Execute the given coprocessor call on the master.
|
<S,R> CompletableFuture<R> |
coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker,
ServiceCaller<S,R> callable,
ServerName serverName)
Execute the given coprocessor call on the given region server.
|
CompletableFuture<Void> |
createNamespace(NamespaceDescriptor descriptor)
Create a new namespace.
|
CompletableFuture<Void> |
createTable(TableDescriptor desc)
Creates a new table.
|
CompletableFuture<Void> |
createTable(TableDescriptor desc,
byte[][] splitKeys)
Creates a new table with an initial set of empty regions defined by the specified split keys.
|
CompletableFuture<Void> |
createTable(TableDescriptor desc,
byte[] startKey,
byte[] endKey,
int numRegions)
Creates a new table with the specified number of regions.
|
CompletableFuture<Void> |
decommissionRegionServers(List<ServerName> servers,
boolean offload)
Mark region server(s) as decommissioned to prevent additional regions from getting
assigned to them.
|
CompletableFuture<Void> |
deleteColumnFamily(TableName tableName,
byte[] columnFamily)
Delete a column family from a table.
|
CompletableFuture<Void> |
deleteNamespace(String name)
Delete an existing namespace.
|
CompletableFuture<Void> |
deleteSnapshot(String snapshotName)
Delete an existing snapshot.
|
CompletableFuture<Void> |
deleteSnapshots()
Delete all existing snapshots.
|
CompletableFuture<Void> |
deleteSnapshots(Pattern pattern)
Delete existing snapshots whose names match the pattern passed.
|
CompletableFuture<Void> |
deleteTable(TableName tableName)
Deletes a table.
|
CompletableFuture<Void> |
deleteTableSnapshots(Pattern tableNamePattern)
Delete all existing snapshots matching the given table name pattern.
|
CompletableFuture<Void> |
deleteTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern)
Delete all existing snapshots matching the given table name regular expression and snapshot
name regular expression.
|
CompletableFuture<Void> |
disableReplicationPeer(String peerId)
Stop the replication stream to the specified peer
|
CompletableFuture<Void> |
disableTable(TableName tableName)
Disable a table.
|
CompletableFuture<Void> |
disableTableReplication(TableName tableName)
Disable a table's replication switch.
|
CompletableFuture<Void> |
enableReplicationPeer(String peerId)
Restart the replication stream to the specified peer
|
CompletableFuture<Void> |
enableTable(TableName tableName)
Enable a table.
|
CompletableFuture<Void> |
enableTableReplication(TableName tableName)
Enable a table's replication switch.
|
CompletableFuture<Boolean> |
exceedThrottleQuotaSwitch(boolean enable)
Switch the exceed throttle quota.
|
CompletableFuture<Void> |
execProcedure(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster.
|
CompletableFuture<byte[]> |
execProcedureWithReturn(String signature,
String instance,
Map<String,String> props)
Execute a distributed procedure on a cluster.
|
CompletableFuture<Void> |
flush(TableName tableName)
Flush a table.
|
CompletableFuture<Void> |
flushRegion(byte[] regionName)
Flush an individual region.
|
CompletableFuture<Void> |
flushRegionServer(ServerName serverName)
Flush all region on the region server.
|
default CompletableFuture<Collection<ServerName>> |
getBackupMasters() |
CompletableFuture<ClusterMetrics> |
getClusterMetrics() |
CompletableFuture<ClusterMetrics> |
getClusterMetrics(EnumSet<ClusterMetrics.Option> options) |
default CompletableFuture<CompactionState> |
getCompactionState(TableName tableName)
Get the current compaction state of a table.
|
CompletableFuture<CompactionState> |
getCompactionState(TableName tableName,
CompactType compactType)
Get the current compaction state of a table.
|
CompletableFuture<CompactionState> |
getCompactionStateForRegion(byte[] regionName)
Get the current compaction state of region.
|
CompletableFuture<? extends SpaceQuotaSnapshotView> |
getCurrentSpaceQuotaSnapshot(String namespace)
Returns the Master's view of a quota on the given
namespace or null if the Master has
no quota information on that namespace. |
CompletableFuture<? extends SpaceQuotaSnapshotView> |
getCurrentSpaceQuotaSnapshot(TableName tableName)
Returns the Master's view of a quota on the given
tableName or null if the Master has
no quota information on that table. |
CompletableFuture<TableDescriptor> |
getDescriptor(TableName tableName)
Method for getting the tableDescriptor
|
CompletableFuture<Optional<Long>> |
getLastMajorCompactionTimestamp(TableName tableName)
Get the timestamp of the last major compaction for the passed table.
|
CompletableFuture<Optional<Long>> |
getLastMajorCompactionTimestampForRegion(byte[] regionName)
Get the timestamp of the last major compaction for the passed region.
|
CompletableFuture<String> |
getLocks()
List locks.
|
default CompletableFuture<ServerName> |
getMaster() |
default CompletableFuture<List<String>> |
getMasterCoprocessorNames() |
default CompletableFuture<Integer> |
getMasterInfoPort()
Get the info port of the current master if one is available.
|
CompletableFuture<NamespaceDescriptor> |
getNamespaceDescriptor(String name)
Get a namespace descriptor by name
|
CompletableFuture<String> |
getProcedures()
List procedures
|
CompletableFuture<List<QuotaSettings>> |
getQuota(QuotaFilter filter)
List the quotas based on the filter.
|
CompletableFuture<List<RegionMetrics>> |
getRegionMetrics(ServerName serverName)
Get a list of
RegionMetrics of all regions hosted on a region seerver. |
CompletableFuture<List<RegionMetrics>> |
getRegionMetrics(ServerName serverName,
TableName tableName)
Get a list of
RegionMetrics of all regions hosted on a region seerver for a table. |
CompletableFuture<List<RegionInfo>> |
getRegions(ServerName serverName)
Get all the online regions on a region server.
|
CompletableFuture<List<RegionInfo>> |
getRegions(TableName tableName)
Get the regions of a given table.
|
default CompletableFuture<Collection<ServerName>> |
getRegionServers() |
CompletableFuture<? extends Map<TableName,? extends SpaceQuotaSnapshotView>> |
getRegionServerSpaceQuotaSnapshots(ServerName serverName)
Fetches the observed
SpaceQuotaSnapshotView s observed by a RegionServer. |
CompletableFuture<ReplicationPeerConfig> |
getReplicationPeerConfig(String peerId)
Returns the configured ReplicationPeerConfig for the specified peer
|
CompletableFuture<List<SecurityCapability>> |
getSecurityCapabilities() |
CompletableFuture<List<OnlineLogRecord>> |
getSlowLogResponses(Set<ServerName> serverNames,
LogQueryFilter slowLogQueryFilter)
Retrieves online slow RPC logs from the provided list of
RegionServers
|
CompletableFuture<Map<TableName,Long>> |
getSpaceQuotaTableSizes()
Fetches the table sizes on the filesystem as tracked by the HBase Master.
|
CompletableFuture<List<UserPermission>> |
getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest)
Get the global/namespace/table permissions for user
|
CompletableFuture<Void> |
grant(UserPermission userPermission,
boolean mergeExistingPermissions)
Grants user specific permissions
|
default CompletableFuture<List<Boolean>> |
hasUserPermissions(List<Permission> permissions)
Check if call user has specific permissions
|
CompletableFuture<List<Boolean>> |
hasUserPermissions(String userName,
List<Permission> permissions)
Check if the user has specific permissions
|
CompletableFuture<Boolean> |
isBalancerEnabled()
Query the current state of the balancer.
|
CompletableFuture<Boolean> |
isCatalogJanitorEnabled()
Query on the catalog janitor state.
|
CompletableFuture<Boolean> |
isCleanerChoreEnabled()
Query the current state of the cleaner chore.
|
CompletableFuture<Boolean> |
isMasterInMaintenanceMode()
Check whether master is in maintenance mode
|
CompletableFuture<Boolean> |
isMergeEnabled()
Query the current state of the Merge switch.
|
CompletableFuture<Boolean> |
isNormalizerEnabled()
Query the current state of the region normalizer
|
CompletableFuture<Boolean> |
isProcedureFinished(String signature,
String instance,
Map<String,String> props)
Check the current state of the specified procedure.
|
CompletableFuture<Boolean> |
isRpcThrottleEnabled()
Get if the rpc throttle is enabled.
|
CompletableFuture<Boolean> |
isSnapshotCleanupEnabled()
Query the current state of the auto snapshot cleanup based on TTL.
|
CompletableFuture<Boolean> |
isSnapshotFinished(SnapshotDescription snapshot)
Check the current state of the passed snapshot.
|
CompletableFuture<Boolean> |
isSplitEnabled()
Query the current state of the Split switch.
|
CompletableFuture<Boolean> |
isTableAvailable(TableName tableName) |
CompletableFuture<Boolean> |
isTableAvailable(TableName tableName,
byte[][] splitKeys)
Deprecated.
Since 2.2.0. Will be removed in 3.0.0. Use
isTableAvailable(TableName) |
CompletableFuture<Boolean> |
isTableDisabled(TableName tableName) |
CompletableFuture<Boolean> |
isTableEnabled(TableName tableName) |
default CompletableFuture<List<ServerName>> |
listDeadServers()
List all the dead region servers.
|
CompletableFuture<List<ServerName>> |
listDecommissionedRegionServers()
List region servers marked as decommissioned, which can not be assigned regions.
|
CompletableFuture<List<NamespaceDescriptor>> |
listNamespaceDescriptors()
List available namespace descriptors
|
CompletableFuture<List<String>> |
listNamespaces()
List available namespaces
|
CompletableFuture<List<TableCFs>> |
listReplicatedTableCFs()
Find all table and column families that are replicated from this cluster
|
CompletableFuture<List<ReplicationPeerDescription>> |
listReplicationPeers()
Return a list of replication peers.
|
CompletableFuture<List<ReplicationPeerDescription>> |
listReplicationPeers(Pattern pattern)
Return a list of replication peers.
|
CompletableFuture<List<SnapshotDescription>> |
listSnapshots()
List completed snapshots.
|
CompletableFuture<List<SnapshotDescription>> |
listSnapshots(Pattern pattern)
List all the completed snapshots matching the given pattern.
|
default CompletableFuture<List<TableDescriptor>> |
listTableDescriptors()
List all the userspace tables.
|
CompletableFuture<List<TableDescriptor>> |
listTableDescriptors(boolean includeSysTables)
List all the tables.
|
CompletableFuture<List<TableDescriptor>> |
listTableDescriptors(List<TableName> tableNames)
List specific tables including system tables.
|
CompletableFuture<List<TableDescriptor>> |
listTableDescriptors(Pattern pattern,
boolean includeSysTables)
List all the tables matching the given pattern.
|
CompletableFuture<List<TableDescriptor>> |
listTableDescriptorsByNamespace(String name)
Get list of table descriptors by namespace.
|
default CompletableFuture<List<TableName>> |
listTableNames()
List all of the names of userspace tables.
|
CompletableFuture<List<TableName>> |
listTableNames(boolean includeSysTables)
List all of the names of tables.
|
CompletableFuture<List<TableName>> |
listTableNames(Pattern pattern,
boolean includeSysTables)
List all of the names of userspace tables.
|
CompletableFuture<List<TableName>> |
listTableNamesByNamespace(String name)
Get list of table names by namespace.
|
CompletableFuture<List<SnapshotDescription>> |
listTableSnapshots(Pattern tableNamePattern)
List all the completed snapshots matching the given table name pattern.
|
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.
|
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.
|
CompletableFuture<Void> |
majorCompact(TableName tableName,
byte[] columnFamily,
CompactType compactType)
Major compact a column family within a table.
|
CompletableFuture<Void> |
majorCompact(TableName tableName,
CompactType compactType)
Major compact a table.
|
CompletableFuture<Void> |
majorCompactRegion(byte[] regionName)
Major compact a region.
|
CompletableFuture<Void> |
majorCompactRegion(byte[] regionName,
byte[] columnFamily)
Major compact a column family within region.
|
CompletableFuture<Void> |
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.Use
mergeRegions(List, boolean)
instead. |
CompletableFuture<Void> |
mergeRegions(List<byte[]> nameOfRegionsToMerge,
boolean forcible)
Merge multiple regions (>=2).
|
default CompletableFuture<Boolean> |
mergeSwitch(boolean enabled)
Turn the Merge switch on or off.
|
CompletableFuture<Boolean> |
mergeSwitch(boolean enabled,
boolean drainMerges)
Turn the Merge switch on or off.
|
CompletableFuture<Void> |
modifyColumnFamily(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Modify an existing column family on a table.
|
CompletableFuture<Void> |
modifyNamespace(NamespaceDescriptor descriptor)
Modify an existing namespace.
|
CompletableFuture<Void> |
modifyTable(TableDescriptor desc)
Modify an existing table, more IRB friendly version.
|
CompletableFuture<Void> |
move(byte[] regionName)
Move the region
r to a random server. |
CompletableFuture<Void> |
move(byte[] regionName,
ServerName destServerName)
Move the region
r to dest . |
CompletableFuture<Boolean> |
normalize()
Invoke region normalizer.
|
CompletableFuture<Boolean> |
normalizerSwitch(boolean on)
Set region normalizer on/off.
|
CompletableFuture<Void> |
offline(byte[] regionName)
Offline specified region from master's in-memory state.
|
CompletableFuture<Void> |
recommissionRegionServer(ServerName server,
List<byte[]> encodedRegionNames)
Remove decommission marker from a region server to allow regions assignments.
|
CompletableFuture<Void> |
removeReplicationPeer(String peerId)
Remove a peer and stop the replication
|
CompletableFuture<Void> |
removeReplicationPeerTableCFs(String peerId,
Map<TableName,List<String>> tableCfs)
Remove some table-cfs from config of the specified peer
|
CompletableFuture<Void> |
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.
|
CompletableFuture<Void> |
restoreSnapshot(String snapshotName,
boolean takeFailSafeSnapshot,
boolean restoreAcl)
Restore the specified snapshot on the original table.
|
CompletableFuture<Void> |
revoke(UserPermission userPermission)
Revokes user specific permissions
|
CompletableFuture<Void> |
rollWALWriter(ServerName serverName)
Roll the log writer.
|
CompletableFuture<Integer> |
runCatalogJanitor()
Ask for a scan of the catalog table.
|
CompletableFuture<Boolean> |
runCleanerChore()
Ask for cleaner chore to run.
|
CompletableFuture<Void> |
setQuota(QuotaSettings quota)
Apply the new quota settings.
|
CompletableFuture<Void> |
shutdown()
Shuts down the HBase cluster.
|
CompletableFuture<Void> |
snapshot(SnapshotDescription snapshot)
Take a snapshot and wait for the server to complete that snapshot asynchronously.
|
default CompletableFuture<Void> |
snapshot(String snapshotName,
TableName tableName)
Take a snapshot for the given table.
|
default CompletableFuture<Void> |
snapshot(String snapshotName,
TableName tableName,
SnapshotType type)
Create typed snapshot of the table.
|
CompletableFuture<Boolean> |
snapshotCleanupSwitch(boolean on,
boolean sync)
Turn on or off the auto snapshot cleanup based on TTL.
|
CompletableFuture<Void> |
split(TableName tableName)
Split a table.
|
CompletableFuture<Void> |
split(TableName tableName,
byte[] splitPoint)
Split a table.
|
CompletableFuture<Void> |
splitRegion(byte[] regionName)
Split an individual region.
|
CompletableFuture<Void> |
splitRegion(byte[] regionName,
byte[] splitPoint)
Split an individual region.
|
default CompletableFuture<Boolean> |
splitSwitch(boolean enabled)
Turn the Split switch on or off.
|
CompletableFuture<Boolean> |
splitSwitch(boolean enabled,
boolean drainSplits)
Turn the Split switch on or off.
|
CompletableFuture<Void> |
stopMaster()
Shuts down the current HBase master only.
|
CompletableFuture<Void> |
stopRegionServer(ServerName serverName)
Stop the designated regionserver.
|
CompletableFuture<Boolean> |
switchRpcThrottle(boolean enable)
Switch the rpc throttle enabled state.
|
CompletableFuture<Boolean> |
tableExists(TableName tableName) |
CompletableFuture<Void> |
truncateTable(TableName tableName,
boolean preserveSplits)
Truncate a table.
|
CompletableFuture<Void> |
unassign(byte[] regionName,
boolean forcible)
Unassign a region from current hosting regionserver.
|
CompletableFuture<Void> |
updateConfiguration()
Update the configuration and trigger an online config change on all the masters and
regionservers.
|
CompletableFuture<Void> |
updateConfiguration(ServerName serverName)
Update the configuration and trigger an online config change on the regionserver.
|
CompletableFuture<Void> |
updateReplicationPeerConfig(String peerId,
ReplicationPeerConfig peerConfig)
Update the peerConfig for the specified peer
|
CompletableFuture<Boolean> tableExists(TableName tableName)
tableName
- Table to check.CompletableFuture
.default CompletableFuture<List<TableDescriptor>> listTableDescriptors()
CompletableFuture
.CompletableFuture<List<TableDescriptor>> listTableDescriptors(boolean includeSysTables)
includeSysTables
- False to match only against userspace tablesCompletableFuture
.CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern, boolean includeSysTables)
pattern
- The compiled regular expression to match againstincludeSysTables
- False to match only against userspace tablesCompletableFuture
.CompletableFuture<List<TableDescriptor>> listTableDescriptors(List<TableName> tableNames)
tableNames
- the table list to match againstCompletableFuture
.CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name)
name
- namespace nameCompletableFuture
.default CompletableFuture<List<TableName>> listTableNames()
CompletableFuture
.listTableNames(Pattern, boolean)
CompletableFuture<List<TableName>> listTableNames(boolean includeSysTables)
includeSysTables
- False to match only against userspace tablesCompletableFuture
.CompletableFuture<List<TableName>> listTableNames(Pattern pattern, boolean includeSysTables)
pattern
- The regular expression to match againstincludeSysTables
- False to match only against userspace tablesCompletableFuture
.CompletableFuture<List<TableName>> listTableNamesByNamespace(String name)
name
- namespace nameCompletableFuture
.CompletableFuture<TableDescriptor> getDescriptor(TableName tableName)
tableName
- as a TableName
CompletableFuture
.CompletableFuture<Void> createTable(TableDescriptor desc)
desc
- table descriptor for tableCompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
desc
- table descriptor for tablestartKey
- beginning of key rangeendKey
- end of key rangenumRegions
- the total number of regions to createCompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys)
desc
- table descriptor for tablesplitKeys
- array of split keys for the initial regions of the tableCompletableFuture<Void> modifyTable(TableDescriptor desc)
desc
- modified description of the tableCompletableFuture<Void> deleteTable(TableName tableName)
tableName
- name of table to deleteCompletableFuture<Void> truncateTable(TableName tableName, boolean preserveSplits)
tableName
- name of table to truncatepreserveSplits
- True if the splits should be preservedCompletableFuture<Void> enableTable(TableName tableName)
tableName
- name of the tableCompletableFuture<Void> disableTable(TableName tableName)
tableName
- CompletableFuture<Boolean> isTableEnabled(TableName tableName)
tableName
- name of table to checkCompletableFuture
.CompletableFuture<Boolean> isTableDisabled(TableName tableName)
tableName
- name of table to checkCompletableFuture
.CompletableFuture<Boolean> isTableAvailable(TableName tableName)
tableName
- name of table to checkCompletableFuture
.@Deprecated CompletableFuture<Boolean> isTableAvailable(TableName tableName, byte[][] splitKeys)
isTableAvailable(TableName)
CompletableFuture
.tableName
- name of table to checksplitKeys
- keys to check if the table has been created with all split keysCompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
tableName
- name of the table to add column family tocolumnFamily
- column family descriptor of column family to be addedCompletableFuture<Void> deleteColumnFamily(TableName tableName, byte[] columnFamily)
tableName
- name of tablecolumnFamily
- name of column family to be deletedCompletableFuture<Void> modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
tableName
- name of tablecolumnFamily
- new column family descriptor to useCompletableFuture<Void> createNamespace(NamespaceDescriptor descriptor)
descriptor
- descriptor which describes the new namespaceCompletableFuture<Void> modifyNamespace(NamespaceDescriptor descriptor)
descriptor
- descriptor which describes the new namespaceCompletableFuture<Void> deleteNamespace(String name)
name
- namespace nameCompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(String name)
name
- name of namespace descriptorCompletableFuture
.CompletableFuture<List<String>> listNamespaces()
CompletableFuture
.CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors()
CompletableFuture
.CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName)
CompletableFuture<List<RegionInfo>> getRegions(TableName tableName)
CompletableFuture<Void> flush(TableName tableName)
tableName
- table to flushCompletableFuture<Void> flushRegion(byte[] regionName)
regionName
- region to flushCompletableFuture<Void> flushRegionServer(ServerName serverName)
serverName
- server to flushdefault CompletableFuture<Void> compact(TableName tableName)
TableNotFoundException
if table not found.tableName
- table to compactdefault CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily)
TableNotFoundException
if table not found.tableName
- table to compactcolumnFamily
- column family within a table. If not present, compact the table's all
column families.CompletableFuture<Void> compact(TableName tableName, CompactType compactType)
TableNotFoundException
if table not found for
normal compaction type.tableName
- table to compactcompactType
- CompactType
CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, CompactType compactType)
TableNotFoundException
if table not found for
normal compaction type.tableName
- table to compactcolumnFamily
- column family within a tablecompactType
- CompactType
CompletableFuture<Void> compactRegion(byte[] regionName)
regionName
- region to compactCompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily)
regionName
- region to compactcolumnFamily
- column family within a region. If not present, compact the region's all
column families.default CompletableFuture<Void> majorCompact(TableName tableName)
TableNotFoundException
if table not found.tableName
- table to major compactdefault CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily)
TableNotFoundException
if table not found for
normal compaction. type.tableName
- table to major compactcolumnFamily
- column family within a table. If not present, major compact the table's all
column families.CompletableFuture<Void> majorCompact(TableName tableName, CompactType compactType)
TableNotFoundException
if table not found for
normal compaction type.tableName
- table to major compactcompactType
- CompactType
CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType)
TableNotFoundException
if table not found.tableName
- table to major compactcolumnFamily
- column family within a table. If not present, major compact the table's all
column families.compactType
- CompactType
CompletableFuture<Void> majorCompactRegion(byte[] regionName)
regionName
- region to major compactCompletableFuture<Void> majorCompactRegion(byte[] regionName, byte[] columnFamily)
regionName
- region to major compactcolumnFamily
- column family within a region. If not present, major compact the region's
all column families.CompletableFuture<Void> compactRegionServer(ServerName serverName)
serverName
- the region server nameCompletableFuture<Void> majorCompactRegionServer(ServerName serverName)
serverName
- the region server namedefault CompletableFuture<Boolean> mergeSwitch(boolean enabled)
enabled
- enabled or notCompletableFuture
CompletableFuture<Boolean> mergeSwitch(boolean enabled, boolean drainMerges)
drainMerges
parameter only effects when will we complete the returned
CompletableFuture
.enabled
- enabled or notdrainMerges
- If true
, it waits until current merge() call, if outstanding,
to return.CompletableFuture
CompletableFuture<Boolean> isMergeEnabled()
CompletableFuture
default CompletableFuture<Boolean> splitSwitch(boolean enabled)
enabled
- enabled or notCompletableFuture
CompletableFuture<Boolean> splitSwitch(boolean enabled, boolean drainSplits)
drainSplits
parameter only effects when will we complete the returned
CompletableFuture
.enabled
- enabled or notdrainSplits
- If true
, it waits until current split() call, if outstanding,
to return.CompletableFuture
CompletableFuture<Boolean> isSplitEnabled()
CompletableFuture
@Deprecated default CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, boolean forcible)
mergeRegions(List, boolean)
instead.nameOfRegionA
- encoded or full name of region anameOfRegionB
- encoded or full name of region bforcible
- true if do a compulsory merge, otherwise we will only merge two adjacent
regionsCompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible)
nameOfRegionsToMerge
- encoded or full name of daughter regionsforcible
- true if do a compulsory merge, otherwise we will only merge two adjacent
regionsCompletableFuture<Void> split(TableName tableName)
tableName
- table to splitCompletableFuture<Void> splitRegion(byte[] regionName)
regionName
- region to splitCompletableFuture<Void> split(TableName tableName, byte[] splitPoint)
tableName
- table to splitsplitPoint
- the explicit position to split onCompletableFuture<Void> splitRegion(byte[] regionName, byte[] splitPoint)
regionName
- region to splitsplitPoint
- the explicit position to split on. If not present, it will decide by region
server.CompletableFuture<Void> assign(byte[] regionName)
regionName
- Encoded or full name of region to assign.CompletableFuture<Void> unassign(byte[] regionName, boolean forcible)
move(byte[], ServerName)
if you want to control the region movement.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).CompletableFuture<Void> offline(byte[] regionName)
regionName
- Encoded or full name of region to offlineCompletableFuture<Void> move(byte[] regionName)
r
to a random server.regionName
- Encoded or full name of region to move.CompletableFuture<Void> move(byte[] regionName, ServerName destServerName)
r
to dest
.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
CompletableFuture<Void> setQuota(QuotaSettings quota)
quota
- the quota settingsCompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter)
filter
- the quota settings filterdefault CompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig)
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication slave clusterCompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication slave clusterenabled
- peer state, true if ENABLED and false if DISABLEDCompletableFuture<Void> removeReplicationPeer(String peerId)
peerId
- a short name that identifies the peerCompletableFuture<Void> enableReplicationPeer(String peerId)
peerId
- a short name that identifies the peerCompletableFuture<Void> disableReplicationPeer(String peerId)
peerId
- a short name that identifies the peerCompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId)
peerId
- a short name that identifies the peerCompletableFuture
.CompletableFuture<Void> updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
peerId
- a short name that identifies the peerpeerConfig
- new config for the peerCompletableFuture<Void> appendReplicationPeerTableCFs(String peerId, Map<TableName,List<String>> tableCfs)
peerId
- a short that identifies the clustertableCfs
- A map from tableName to column family namesCompletableFuture<Void> removeReplicationPeerTableCFs(String peerId, Map<TableName,List<String>> tableCfs)
peerId
- a short name that identifies the clustertableCfs
- A map from tableName to column family namesCompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers()
CompletableFuture
.CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(Pattern pattern)
pattern
- The compiled regular expression to match peer idCompletableFuture
.CompletableFuture<List<TableCFs>> listReplicatedTableCFs()
CompletableFuture
.CompletableFuture<Void> enableTableReplication(TableName tableName)
tableName
- name of the tableCompletableFuture<Void> disableTableReplication(TableName tableName)
tableName
- name of the tabledefault CompletableFuture<Void> snapshot(String snapshotName, TableName tableName)
SnapshotCreationException
indicating the duplicate
naming. Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
.snapshotName
- name of the snapshot to be createdtableName
- name of the table for which snapshot is createddefault CompletableFuture<Void> snapshot(String snapshotName, TableName tableName, SnapshotType type)
SnapshotCreationException
indicating the duplicate
naming. Snapshot names follow the same naming constraints as tables in HBase. See
TableName.isLegalFullyQualifiedTableName(byte[])
.snapshotName
- name to give the snapshot on the filesystem. Must be unique from all other
snapshots stored on the clustertableName
- name of the table to snapshottype
- type of snapshot to takeCompletableFuture<Void> snapshot(SnapshotDescription snapshot)
SnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
HBase. See TableName.isLegalFullyQualifiedTableName(byte[])
.
You should probably use snapshot(String, org.apache.hadoop.hbase.TableName)
unless you
are sure about the type of snapshot that you want to take.snapshot
- snapshot to takeCompletableFuture<Boolean> isSnapshotFinished(SnapshotDescription snapshot)
UnknownSnapshotException
.snapshot
- description of the snapshot to checkCompletableFuture<Void> restoreSnapshot(String snapshotName)
snapshotName
- name of the snapshot to restoredefault CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot)
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true if the failsafe snapshot should be takenCompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true if the failsafe snapshot should be takenrestoreAcl
- true
to restore acl of snapshotdefault CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName)
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredCompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl)
snapshotName
- name of the snapshot to be clonedtableName
- name of the table where the snapshot will be restoredrestoreAcl
- true
to restore acl of snapshotCompletableFuture<List<SnapshotDescription>> listSnapshots()
CompletableFuture
CompletableFuture<List<SnapshotDescription>> listSnapshots(Pattern pattern)
pattern
- The compiled regular expression to match againstCompletableFuture
CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern)
tableNamePattern
- The compiled table name regular expression to match againstCompletableFuture
CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match againstCompletableFuture
CompletableFuture<Void> deleteSnapshot(String snapshotName)
snapshotName
- name of the snapshotCompletableFuture<Void> deleteSnapshots()
CompletableFuture<Void> deleteSnapshots(Pattern pattern)
pattern
- pattern for names of the snapshot to matchCompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern)
tableNamePattern
- The compiled table name regular expression to match againstCompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match againstCompletableFuture<Void> execProcedure(String signature, String instance, Map<String,String> props)
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 procedureCompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance, Map<String,String> props)
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 procedureCompletableFuture<Boolean> isProcedureFinished(String signature, String instance, Map<String,String> props)
signature
- The signature that uniquely identifies a procedureinstance
- The instance name of the procedureprops
- Property/Value pairs of properties passing to the procedureCompletableFuture
@Deprecated CompletableFuture<Boolean> abortProcedure(long procId, boolean mayInterruptIfRunning)
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?CompletableFuture
CompletableFuture<String> getProcedures()
CompletableFuture
CompletableFuture<String> getLocks()
CompletableFuture
CompletableFuture<Void> decommissionRegionServers(List<ServerName> servers, boolean offload)
servers
- The list of servers to decommission.offload
- True to offload the regions from the decommissioned serversCompletableFuture<List<ServerName>> listDecommissionedRegionServers()
CompletableFuture
CompletableFuture<Void> recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
server
- The server to recommission.encodedRegionNames
- Regions to load onto the server.CompletableFuture<ClusterMetrics> getClusterMetrics()
CompletableFuture
CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<ClusterMetrics.Option> options)
CompletableFuture
default CompletableFuture<ServerName> getMaster()
CompletableFuture
default CompletableFuture<Collection<ServerName>> getBackupMasters()
CompletableFuture
default CompletableFuture<Collection<ServerName>> getRegionServers()
CompletableFuture
default CompletableFuture<List<String>> getMasterCoprocessorNames()
CompletableFuture
default CompletableFuture<Integer> getMasterInfoPort()
CompletableFuture<Void> shutdown()
CompletableFuture<Void> stopMaster()
CompletableFuture<Void> stopRegionServer(ServerName serverName)
serverName
- CompletableFuture<Void> updateConfiguration(ServerName serverName)
serverName
- : The server whose config needs to be updated.CompletableFuture<Void> updateConfiguration()
CompletableFuture<Void> rollWALWriter(ServerName serverName)
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.
serverName
- The servername of the region server.CompletableFuture<Void> clearCompactionQueues(ServerName serverName, Set<String> queues)
serverName
- queues
- the set of queue nameCompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName)
RegionMetrics
of all regions hosted on a region seerver.serverName
- RegionMetrics
wrapped by CompletableFuture
CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName, TableName tableName)
RegionMetrics
of all regions hosted on a region seerver for a table.serverName
- tableName
- RegionMetrics
wrapped by CompletableFuture
CompletableFuture<Boolean> isMasterInMaintenanceMode()
CompletableFuture
default CompletableFuture<CompactionState> getCompactionState(TableName tableName)
tableName
- table to examineCompletableFuture
CompletableFuture<CompactionState> getCompactionState(TableName tableName, CompactType compactType)
tableName
- table to examinecompactType
- CompactType
CompletableFuture
CompletableFuture<CompactionState> getCompactionStateForRegion(byte[] regionName)
regionName
- region to examineCompletableFuture
CompletableFuture<Optional<Long>> getLastMajorCompactionTimestamp(TableName tableName)
The timestamp of the oldest HFile resulting from a major compaction of that table, or not present if no such HFile could be found.
tableName
- table to examineCompletableFuture
CompletableFuture<Optional<Long>> getLastMajorCompactionTimestampForRegion(byte[] regionName)
The timestamp of the oldest HFile resulting from a major compaction of that region, or not present if no such HFile could be found.
regionName
- region to examineCompletableFuture
CompletableFuture<List<SecurityCapability>> getSecurityCapabilities()
CompletableFuture
.default CompletableFuture<Boolean> balancerSwitch(boolean on)
on
- Set to true
to enable, false
to disable.CompletableFuture
.CompletableFuture<Boolean> balancerSwitch(boolean on, boolean drainRITs)
drainRITs
parameter only effects when will we complete the returned
CompletableFuture
.on
- Set to true
to enable, false
to disable.drainRITs
- If true
, it waits until current balance() call, if outstanding,
to return.CompletableFuture
.default CompletableFuture<Boolean> balance()
CompletableFuture
.CompletableFuture<Boolean> balance(boolean forcible)
forcible
- whether we should force balance even if there is region in transition.CompletableFuture
.CompletableFuture<Boolean> isBalancerEnabled()
CompletableFuture
.CompletableFuture<Boolean> normalizerSwitch(boolean on)
on
- whether normalizer should be on or offCompletableFuture
CompletableFuture<Boolean> isNormalizerEnabled()
CompletableFuture
CompletableFuture<Boolean> normalize()
CompletableFuture
CompletableFuture<Boolean> cleanerChoreSwitch(boolean on)
on
- CompletableFuture
CompletableFuture<Boolean> isCleanerChoreEnabled()
CompletableFuture
CompletableFuture<Boolean> runCleanerChore()
CompletableFuture
CompletableFuture<Boolean> catalogJanitorSwitch(boolean on)
on
- CompletableFuture
CompletableFuture<Boolean> isCatalogJanitorEnabled()
CompletableFuture
CompletableFuture<Integer> runCatalogJanitor()
CompletableFuture
<S,R> CompletableFuture<R> coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker, ServiceCaller<S,R> callable)
The stubMaker
is just a delegation to the newStub
call. Usually it is only a
one line lambda expression, like:
channel -> xxxService.newStub(channel)
S
- the type of the asynchronous stubR
- the type of the return valuestubMaker
- a delegation to the actual newStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment of
ServiceCaller
for more details.CompletableFuture
.ServiceCaller
<S,R> CompletableFuture<R> coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker, ServiceCaller<S,R> callable, ServerName serverName)
The stubMaker
is just a delegation to the newStub
call. Usually it is only a
one line lambda expression, like:
channel -> xxxService.newStub(channel)
S
- the type of the asynchronous stubR
- the type of the return valuestubMaker
- a delegation to the actual newStub
call.callable
- a delegation to the actual protobuf rpc call. See the comment of
ServiceCaller
for more details.serverName
- the given region serverCompletableFuture
.ServiceCaller
default CompletableFuture<List<ServerName>> listDeadServers()
CompletableFuture<List<ServerName>> clearDeadServers(List<ServerName> servers)
servers
- list of dead region servers.CompletableFuture
.CompletableFuture<CacheEvictionStats> clearBlockCache(TableName tableName)
tableName
- table to clear block cacheCompletableFuture
.CompletableFuture<Void> cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
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 preservedCompletableFuture<Map<ServerName,Boolean>> compactionSwitch(boolean switchState, List<String> serverNamesList)
switchState
- Set to true
to enable, false
to disable.serverNamesList
- list of region servers.CompletableFuture<Boolean> switchRpcThrottle(boolean enable)
enable
- Set to true
to enable, false
to disable.CompletableFuture<Boolean> isRpcThrottleEnabled()
CompletableFuture<Boolean> exceedThrottleQuotaSwitch(boolean enable)
enable
- Set to true
to enable, false
to disable.CompletableFuture<Map<TableName,Long>> getSpaceQuotaTableSizes()
CompletableFuture<? extends Map<TableName,? extends SpaceQuotaSnapshotView>> getRegionServerSpaceQuotaSnapshots(ServerName serverName)
SpaceQuotaSnapshotView
s observed by a RegionServer.CompletableFuture<? extends SpaceQuotaSnapshotView> getCurrentSpaceQuotaSnapshot(String namespace)
namespace
or null if the Master has
no quota information on that namespace.CompletableFuture<? extends SpaceQuotaSnapshotView> getCurrentSpaceQuotaSnapshot(TableName tableName)
tableName
or null if the Master has
no quota information on that table.CompletableFuture<Void> grant(UserPermission userPermission, boolean mergeExistingPermissions)
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.CompletableFuture<Void> revoke(UserPermission userPermission)
userPermission
- user name and the specific permissionCompletableFuture<List<UserPermission>> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest)
getUserPermissionsRequest
- A request contains which user, global, namespace or table
permissions neededCompletableFuture<List<Boolean>> hasUserPermissions(String userName, List<Permission> permissions)
userName
- the user namepermissions
- the specific permission listdefault CompletableFuture<List<Boolean>> hasUserPermissions(List<Permission> permissions)
permissions
- the specific permission listCompletableFuture<Boolean> snapshotCleanupSwitch(boolean on, boolean sync)
sync
parameter only effects when will we complete the returned
CompletableFuture
.on
- Set to true
to enable, false
to disable.sync
- If true
, it waits until current snapshot cleanup is completed,
if outstanding.CompletableFuture
.CompletableFuture<Boolean> isSnapshotCleanupEnabled()
CompletableFuture
.CompletableFuture<List<OnlineLogRecord>> getSlowLogResponses(Set<ServerName> serverNames, LogQueryFilter slowLogQueryFilter)
serverNames
- Server names to get slowlog responses fromslowLogQueryFilter
- filter to be used if providedCompletableFuture
CompletableFuture<List<Boolean>> clearSlowLogResponses(Set<ServerName> serverNames)
serverNames
- Set of Server names to clean slowlog responses fromCompletableFuture
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.