@InterfaceAudience.Private class AsyncHBaseAdmin extends Object implements AsyncAdmin
RawAsyncHBaseAdmin
. The difference is that users need to provide a
thread pool when constructing this class, and the callback methods registered to the returned
CompletableFuture
will be executed in this thread pool. So usually it is safe for users
to do anything they want in the callbacks without breaking the rpc framework.RawAsyncHBaseAdmin
,
AsyncConnection.getAdmin(ExecutorService)
,
AsyncConnection.getAdminBuilder(ExecutorService)
Modifier and Type | Field and Description |
---|---|
private ExecutorService |
pool |
private RawAsyncHBaseAdmin |
rawAdmin |
Constructor and Description |
---|
AsyncHBaseAdmin(RawAsyncHBaseAdmin rawAdmin,
ExecutorService pool) |
Modifier and Type | Method and Description |
---|---|
CompletableFuture<Boolean> |
abortProcedure(long procId,
boolean mayInterruptIfRunning)
Abort a procedure Do not use.
|
CompletableFuture<Void> |
addColumnFamily(TableName tableName,
ColumnFamilyDescriptor columnFamily)
Add a column family to an existing table.
|
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)
Assign an individual region.
|
CompletableFuture<BalanceResponse> |
balance(BalanceRequest request)
Invoke the balancer with the given balance request.
|
CompletableFuture<Boolean> |
balancerSwitch(boolean on,
boolean drainRITs)
Turn the load balancer on or off.
|
CompletableFuture<Boolean> |
catalogJanitorSwitch(boolean enabled)
Turn the catalog janitor on/off.
|
CompletableFuture<Boolean> |
cleanerChoreSwitch(boolean enabled)
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
|
CompletableFuture<Void> |
cloneSnapshot(String snapshotName,
TableName tableName,
boolean restoreAcl,
String customSFT)
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.
|
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> |
flush(TableName tableName,
byte[] columnFamily)
Flush the specified column family stores on all regions of the passed table.
|
CompletableFuture<Void> |
flushMasterStore()
Flush master local region
|
CompletableFuture<Void> |
flushRegion(byte[] regionName)
Flush an individual region.
|
CompletableFuture<Void> |
flushRegion(byte[] regionName,
byte[] columnFamily)
Flush a column family within a region.
|
CompletableFuture<Void> |
flushRegionServer(ServerName sn)
Flush all region on the region server.
|
CompletableFuture<ClusterMetrics> |
getClusterMetrics()
Returns cluster status wrapped by
CompletableFuture |
CompletableFuture<ClusterMetrics> |
getClusterMetrics(EnumSet<ClusterMetrics.Option> options)
Returns cluster status wrapped by
CompletableFuture |
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<SpaceQuotaSnapshot> |
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<SpaceQuotaSnapshot> |
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.
|
CompletableFuture<List<LogEntry>> |
getLogEntries(Set<ServerName> serverNames,
String logType,
ServerType serverType,
int limit,
Map<String,Object> filterParams)
Retrieve recent online records from HMaster / RegionServers.
|
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 server. |
CompletableFuture<List<RegionMetrics>> |
getRegionMetrics(ServerName serverName,
TableName tableName)
Get a list of
RegionMetrics of all regions hosted on a region server 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.
|
CompletableFuture<Map<TableName,SpaceQuotaSnapshot>> |
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()
Returns the list of supported security capabilities.
|
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
|
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)
Check if a table is available.
|
CompletableFuture<Boolean> |
isTableAvailable(TableName tableName,
byte[][] splitKeys)
Use this api to check if the table has been created with the specified number of splitkeys
which was used while creating the given table.
|
CompletableFuture<Boolean> |
isTableDisabled(TableName tableName)
Check if a table is disabled.
|
CompletableFuture<Boolean> |
isTableEnabled(TableName tableName)
Check if a table is enabled.
|
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.
|
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.
|
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.
|
CompletableFuture<List<ServerName>> |
listUnknownServers()
List all the unknown region servers.
|
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.
|
CompletableFuture<Void> |
mergeRegions(List<byte[]> nameOfRegionsToMerge,
boolean forcible)
Merge multiple regions (>=2).
|
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> |
modifyColumnFamilyStoreFileTracker(TableName tableName,
byte[] family,
String dstSFT)
Change the store file tracker of the given table's given family.
|
CompletableFuture<Void> |
modifyNamespace(NamespaceDescriptor descriptor)
Modify an existing namespace.
|
CompletableFuture<Void> |
modifyTable(TableDescriptor desc)
Modify an existing table, more IRB friendly version.
|
CompletableFuture<Void> |
modifyTableStoreFileTracker(TableName tableName,
String dstSFT)
Change the store file tracker of the given table.
|
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(NormalizeTableFilterParams ntfp)
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.
|
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.
|
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.
|
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)
Check if a table exists.
|
CompletableFuture<Void> |
truncateTable(TableName tableName,
boolean preserveSplits)
Truncate a table.
|
CompletableFuture<Void> |
unassign(byte[] regionName)
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
|
private <T> CompletableFuture<T> |
wrap(CompletableFuture<T> future) |
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
addReplicationPeer, balance, balance, balancerSwitch, cloneSnapshot, cloneSnapshot, compact, compact, getBackupMasters, getCompactionState, getMaster, getMasterCoprocessorNames, getMasterInfoPort, getRegionServers, getRegionServers, getSlowLogResponses, hasUserPermissions, listTableDescriptors, listTableNames, majorCompact, majorCompact, mergeRegions, mergeSwitch, normalize, restoreSnapshot, snapshot, snapshot, splitSwitch, unassign
private final RawAsyncHBaseAdmin rawAdmin
private final ExecutorService pool
AsyncHBaseAdmin(RawAsyncHBaseAdmin rawAdmin, ExecutorService pool)
private <T> CompletableFuture<T> wrap(CompletableFuture<T> future)
public CompletableFuture<Boolean> tableExists(TableName tableName)
AsyncAdmin
tableExists
in interface AsyncAdmin
tableName
- Table to check.CompletableFuture
.public CompletableFuture<List<TableDescriptor>> listTableDescriptors(boolean includeSysTables)
AsyncAdmin
listTableDescriptors
in interface AsyncAdmin
includeSysTables
- False to match only against userspace tablesCompletableFuture
.public CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern, boolean includeSysTables)
AsyncAdmin
listTableDescriptors
in interface AsyncAdmin
pattern
- The compiled regular expression to match againstincludeSysTables
- False to match only against userspace tablesCompletableFuture
.public CompletableFuture<List<TableDescriptor>> listTableDescriptors(List<TableName> tableNames)
AsyncAdmin
listTableDescriptors
in interface AsyncAdmin
tableNames
- the table list to match againstCompletableFuture
.public CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name)
AsyncAdmin
listTableDescriptorsByNamespace
in interface AsyncAdmin
name
- namespace nameCompletableFuture
.public CompletableFuture<List<TableName>> listTableNames(boolean includeSysTables)
AsyncAdmin
listTableNames
in interface AsyncAdmin
includeSysTables
- False to match only against userspace tablesCompletableFuture
.public CompletableFuture<List<TableName>> listTableNames(Pattern pattern, boolean includeSysTables)
AsyncAdmin
listTableNames
in interface AsyncAdmin
pattern
- The regular expression to match againstincludeSysTables
- False to match only against userspace tablesCompletableFuture
.public CompletableFuture<List<TableName>> listTableNamesByNamespace(String name)
AsyncAdmin
listTableNamesByNamespace
in interface AsyncAdmin
name
- namespace nameCompletableFuture
.public CompletableFuture<TableDescriptor> getDescriptor(TableName tableName)
AsyncAdmin
getDescriptor
in interface AsyncAdmin
tableName
- as a TableName
CompletableFuture
.public CompletableFuture<Void> createTable(TableDescriptor desc)
AsyncAdmin
createTable
in interface AsyncAdmin
desc
- table descriptor for tablepublic CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
AsyncAdmin
createTable
in interface AsyncAdmin
desc
- table descriptor for tablestartKey
- beginning of key rangeendKey
- end of key rangenumRegions
- the total number of regions to createpublic CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys)
AsyncAdmin
createTable
in interface AsyncAdmin
desc
- table descriptor for tablesplitKeys
- array of split keys for the initial regions of the tablepublic CompletableFuture<Void> modifyTable(TableDescriptor desc)
AsyncAdmin
modifyTable
in interface AsyncAdmin
desc
- modified description of the tablepublic CompletableFuture<Void> modifyTableStoreFileTracker(TableName tableName, String dstSFT)
AsyncAdmin
modifyTableStoreFileTracker
in interface AsyncAdmin
tableName
- the table you want to changedstSFT
- the destination store file trackerpublic CompletableFuture<Void> deleteTable(TableName tableName)
AsyncAdmin
deleteTable
in interface AsyncAdmin
tableName
- name of table to deletepublic CompletableFuture<Void> truncateTable(TableName tableName, boolean preserveSplits)
AsyncAdmin
truncateTable
in interface AsyncAdmin
tableName
- name of table to truncatepreserveSplits
- True if the splits should be preservedpublic CompletableFuture<Void> enableTable(TableName tableName)
AsyncAdmin
enableTable
in interface AsyncAdmin
tableName
- name of the tablepublic CompletableFuture<Void> disableTable(TableName tableName)
AsyncAdmin
disableTable
in interface AsyncAdmin
public CompletableFuture<Boolean> isTableEnabled(TableName tableName)
AsyncAdmin
isTableEnabled
in interface AsyncAdmin
tableName
- name of table to checkCompletableFuture
.public CompletableFuture<Boolean> isTableDisabled(TableName tableName)
AsyncAdmin
isTableDisabled
in interface AsyncAdmin
tableName
- name of table to checkCompletableFuture
.public CompletableFuture<Boolean> isTableAvailable(TableName tableName)
AsyncAdmin
isTableAvailable
in interface AsyncAdmin
tableName
- name of table to checkCompletableFuture
.public CompletableFuture<Boolean> isTableAvailable(TableName tableName, byte[][] splitKeys)
AsyncAdmin
CompletableFuture
.isTableAvailable
in interface AsyncAdmin
tableName
- name of table to checksplitKeys
- keys to check if the table has been created with all split keyspublic CompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
AsyncAdmin
addColumnFamily
in interface AsyncAdmin
tableName
- name of the table to add column family tocolumnFamily
- column family descriptor of column family to be addedpublic CompletableFuture<Void> deleteColumnFamily(TableName tableName, byte[] columnFamily)
AsyncAdmin
deleteColumnFamily
in interface AsyncAdmin
tableName
- name of tablecolumnFamily
- name of column family to be deletedpublic CompletableFuture<Void> modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily)
AsyncAdmin
modifyColumnFamily
in interface AsyncAdmin
tableName
- name of tablecolumnFamily
- new column family descriptor to usepublic CompletableFuture<Void> modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT)
AsyncAdmin
modifyColumnFamilyStoreFileTracker
in interface AsyncAdmin
tableName
- the table you want to changefamily
- the family you want to changedstSFT
- the destination store file trackerpublic CompletableFuture<Void> createNamespace(NamespaceDescriptor descriptor)
AsyncAdmin
createNamespace
in interface AsyncAdmin
descriptor
- descriptor which describes the new namespacepublic CompletableFuture<Void> modifyNamespace(NamespaceDescriptor descriptor)
AsyncAdmin
modifyNamespace
in interface AsyncAdmin
descriptor
- descriptor which describes the new namespacepublic CompletableFuture<Void> deleteNamespace(String name)
AsyncAdmin
deleteNamespace
in interface AsyncAdmin
name
- namespace namepublic CompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(String name)
AsyncAdmin
getNamespaceDescriptor
in interface AsyncAdmin
name
- name of namespace descriptorCompletableFuture
.public CompletableFuture<List<String>> listNamespaces()
AsyncAdmin
listNamespaces
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors()
AsyncAdmin
listNamespaceDescriptors
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName)
AsyncAdmin
getRegions
in interface AsyncAdmin
public CompletableFuture<List<RegionInfo>> getRegions(TableName tableName)
AsyncAdmin
getRegions
in interface AsyncAdmin
public CompletableFuture<Void> flush(TableName tableName)
AsyncAdmin
flush
in interface AsyncAdmin
tableName
- table to flushpublic CompletableFuture<Void> flush(TableName tableName, byte[] columnFamily)
AsyncAdmin
flush
in interface AsyncAdmin
tableName
- table to flushcolumnFamily
- column family within a tablepublic CompletableFuture<Void> flushRegion(byte[] regionName)
AsyncAdmin
flushRegion
in interface AsyncAdmin
regionName
- region to flushpublic CompletableFuture<Void> flushRegion(byte[] regionName, byte[] columnFamily)
AsyncAdmin
flushRegion
in interface AsyncAdmin
regionName
- region to flushcolumnFamily
- column family within a region. If not present, flush the region's all
column families.public CompletableFuture<Void> flushRegionServer(ServerName sn)
AsyncAdmin
flushRegionServer
in interface AsyncAdmin
sn
- server to flushpublic CompletableFuture<Void> compact(TableName tableName, CompactType compactType)
AsyncAdmin
TableNotFoundException
if table not found for normal compaction
type.compact
in interface AsyncAdmin
tableName
- table to compactcompactType
- CompactType
public CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, CompactType compactType)
AsyncAdmin
TableNotFoundException
if table not found for
normal compaction type.compact
in interface AsyncAdmin
tableName
- table to compactcolumnFamily
- column family within a tablecompactType
- CompactType
public CompletableFuture<Void> compactRegion(byte[] regionName)
AsyncAdmin
compactRegion
in interface AsyncAdmin
regionName
- region to compactpublic CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily)
AsyncAdmin
compactRegion
in interface AsyncAdmin
regionName
- region to compactcolumnFamily
- column family within a region. If not present, compact the region's all
column families.public CompletableFuture<Void> majorCompact(TableName tableName, CompactType compactType)
AsyncAdmin
TableNotFoundException
if table not found for normal compaction
type.majorCompact
in interface AsyncAdmin
tableName
- table to major compactcompactType
- CompactType
public CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType)
AsyncAdmin
TableNotFoundException
if table not found.majorCompact
in interface AsyncAdmin
tableName
- table to major compactcolumnFamily
- column family within a table. If not present, major compact the table's all
column families.compactType
- CompactType
public CompletableFuture<Void> majorCompactRegion(byte[] regionName)
AsyncAdmin
majorCompactRegion
in interface AsyncAdmin
regionName
- region to major compactpublic CompletableFuture<Void> majorCompactRegion(byte[] regionName, byte[] columnFamily)
AsyncAdmin
majorCompactRegion
in interface AsyncAdmin
regionName
- region to major compactcolumnFamily
- column family within a region. If not present, major compact the region's
all column families.public CompletableFuture<Void> compactRegionServer(ServerName serverName)
AsyncAdmin
compactRegionServer
in interface AsyncAdmin
serverName
- the region server namepublic CompletableFuture<Void> majorCompactRegionServer(ServerName serverName)
AsyncAdmin
majorCompactRegionServer
in interface AsyncAdmin
serverName
- the region server namepublic CompletableFuture<Boolean> mergeSwitch(boolean enabled, boolean drainMerges)
AsyncAdmin
drainMerges
parameter only effects when will we complete the returned
CompletableFuture
.mergeSwitch
in interface AsyncAdmin
enabled
- enabled or notdrainMerges
- If true
, it waits until current merge() call, if outstanding,
to return.CompletableFuture
public CompletableFuture<Boolean> isMergeEnabled()
AsyncAdmin
isMergeEnabled
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> splitSwitch(boolean enabled, boolean drainSplits)
AsyncAdmin
drainSplits
parameter only effects when will we complete the returned
CompletableFuture
.splitSwitch
in interface AsyncAdmin
enabled
- enabled or notdrainSplits
- If true
, it waits until current split() call, if outstanding,
to return.CompletableFuture
public CompletableFuture<Boolean> isSplitEnabled()
AsyncAdmin
isSplitEnabled
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible)
AsyncAdmin
mergeRegions
in interface AsyncAdmin
nameOfRegionsToMerge
- encoded or full name of daughter regionsforcible
- true if do a compulsory merge, otherwise we will only merge two
adjacent regionspublic CompletableFuture<Void> split(TableName tableName)
AsyncAdmin
split
in interface AsyncAdmin
tableName
- table to splitpublic CompletableFuture<Void> split(TableName tableName, byte[] splitPoint)
AsyncAdmin
split
in interface AsyncAdmin
tableName
- table to splitsplitPoint
- the explicit position to split onpublic CompletableFuture<Void> splitRegion(byte[] regionName)
AsyncAdmin
splitRegion
in interface AsyncAdmin
regionName
- region to splitpublic CompletableFuture<Void> splitRegion(byte[] regionName, byte[] splitPoint)
AsyncAdmin
splitRegion
in interface AsyncAdmin
regionName
- region to splitsplitPoint
- the explicit position to split on. If not present, it will decide by region
server.public CompletableFuture<Void> assign(byte[] regionName)
AsyncAdmin
assign
in interface AsyncAdmin
regionName
- Encoded or full name of region to assign.public CompletableFuture<Void> unassign(byte[] regionName)
AsyncAdmin
AsyncAdmin.move(byte[], ServerName)
if you want to control the region movement.unassign
in interface AsyncAdmin
regionName
- Encoded or full name of region to unassign.public CompletableFuture<Void> offline(byte[] regionName)
AsyncAdmin
offline
in interface AsyncAdmin
regionName
- Encoded or full name of region to offlinepublic CompletableFuture<Void> move(byte[] regionName)
AsyncAdmin
r
to a random server.move
in interface AsyncAdmin
regionName
- Encoded or full name of region to move.public CompletableFuture<Void> move(byte[] regionName, ServerName destServerName)
AsyncAdmin
r
to dest
.move
in interface AsyncAdmin
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
public CompletableFuture<Void> setQuota(QuotaSettings quota)
AsyncAdmin
setQuota
in interface AsyncAdmin
quota
- the quota settingspublic CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter)
AsyncAdmin
getQuota
in interface AsyncAdmin
filter
- the quota settings filterpublic CompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
AsyncAdmin
addReplicationPeer
in interface AsyncAdmin
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication slave clusterenabled
- peer state, true if ENABLED and false if DISABLEDpublic CompletableFuture<Void> removeReplicationPeer(String peerId)
AsyncAdmin
removeReplicationPeer
in interface AsyncAdmin
peerId
- a short name that identifies the peerpublic CompletableFuture<Void> enableReplicationPeer(String peerId)
AsyncAdmin
enableReplicationPeer
in interface AsyncAdmin
peerId
- a short name that identifies the peerpublic CompletableFuture<Void> disableReplicationPeer(String peerId)
AsyncAdmin
disableReplicationPeer
in interface AsyncAdmin
peerId
- a short name that identifies the peerpublic CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId)
AsyncAdmin
getReplicationPeerConfig
in interface AsyncAdmin
peerId
- a short name that identifies the peerCompletableFuture
.public CompletableFuture<Void> updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
AsyncAdmin
updateReplicationPeerConfig
in interface AsyncAdmin
peerId
- a short name that identifies the peerpeerConfig
- new config for the peerpublic CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId, Map<TableName,List<String>> tableCfs)
AsyncAdmin
appendReplicationPeerTableCFs
in interface AsyncAdmin
peerId
- a short that identifies the clustertableCfs
- A map from tableName to column family namespublic CompletableFuture<Void> removeReplicationPeerTableCFs(String peerId, Map<TableName,List<String>> tableCfs)
AsyncAdmin
removeReplicationPeerTableCFs
in interface AsyncAdmin
peerId
- a short name that identifies the clustertableCfs
- A map from tableName to column family namespublic CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers()
AsyncAdmin
listReplicationPeers
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(Pattern pattern)
AsyncAdmin
listReplicationPeers
in interface AsyncAdmin
pattern
- The compiled regular expression to match peer idCompletableFuture
.public CompletableFuture<List<TableCFs>> listReplicatedTableCFs()
AsyncAdmin
listReplicatedTableCFs
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<Void> enableTableReplication(TableName tableName)
AsyncAdmin
enableTableReplication
in interface AsyncAdmin
tableName
- name of the tablepublic CompletableFuture<Void> disableTableReplication(TableName tableName)
AsyncAdmin
disableTableReplication
in interface AsyncAdmin
tableName
- name of the tablepublic CompletableFuture<Void> snapshot(SnapshotDescription snapshot)
AsyncAdmin
SnapshotCreationException
indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
HBase. See TableName.isLegalFullyQualifiedTableName(byte[])
.
You should probably use AsyncAdmin.snapshot(String, org.apache.hadoop.hbase.TableName)
unless you
are sure about the type of snapshot that you want to take.snapshot
in interface AsyncAdmin
snapshot
- snapshot to takepublic CompletableFuture<Boolean> isSnapshotFinished(SnapshotDescription snapshot)
AsyncAdmin
UnknownSnapshotException
.isSnapshotFinished
in interface AsyncAdmin
snapshot
- description of the snapshot to checkpublic CompletableFuture<Void> restoreSnapshot(String snapshotName)
AsyncAdmin
restoreSnapshot
in interface AsyncAdmin
snapshotName
- name of the snapshot to restorepublic CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
AsyncAdmin
restoreSnapshot
in interface AsyncAdmin
snapshotName
- name of the snapshot to restoretakeFailSafeSnapshot
- true if the failsafe snapshot should be takenrestoreAcl
- true
to restore acl of snapshotpublic CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT)
AsyncAdmin
cloneSnapshot
in interface AsyncAdmin
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 tablepublic CompletableFuture<List<SnapshotDescription>> listSnapshots()
AsyncAdmin
listSnapshots
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<List<SnapshotDescription>> listSnapshots(Pattern pattern)
AsyncAdmin
listSnapshots
in interface AsyncAdmin
pattern
- The compiled regular expression to match againstCompletableFuture
public CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern)
AsyncAdmin
listTableSnapshots
in interface AsyncAdmin
tableNamePattern
- The compiled table name regular expression to match againstCompletableFuture
public CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
AsyncAdmin
listTableSnapshots
in interface AsyncAdmin
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match againstCompletableFuture
public CompletableFuture<Void> deleteSnapshot(String snapshotName)
AsyncAdmin
deleteSnapshot
in interface AsyncAdmin
snapshotName
- name of the snapshotpublic CompletableFuture<Void> deleteSnapshots()
AsyncAdmin
deleteSnapshots
in interface AsyncAdmin
public CompletableFuture<Void> deleteSnapshots(Pattern pattern)
AsyncAdmin
deleteSnapshots
in interface AsyncAdmin
pattern
- pattern for names of the snapshot to matchpublic CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern)
AsyncAdmin
deleteTableSnapshots
in interface AsyncAdmin
tableNamePattern
- The compiled table name regular expression to match againstpublic CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
AsyncAdmin
deleteTableSnapshots
in interface AsyncAdmin
tableNamePattern
- The compiled table name regular expression to match againstsnapshotNamePattern
- The compiled snapshot name regular expression to match againstpublic CompletableFuture<Void> execProcedure(String signature, String instance, Map<String,String> props)
AsyncAdmin
execProcedure
in interface AsyncAdmin
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 procedurepublic CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance, Map<String,String> props)
AsyncAdmin
execProcedureWithReturn
in interface AsyncAdmin
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 procedurepublic CompletableFuture<Boolean> isProcedureFinished(String signature, String instance, Map<String,String> props)
AsyncAdmin
isProcedureFinished
in interface AsyncAdmin
signature
- The signature that uniquely identifies a procedureinstance
- The instance name of the procedureprops
- Property/Value pairs of properties passing to the procedureCompletableFuture
public CompletableFuture<Boolean> abortProcedure(long procId, boolean mayInterruptIfRunning)
AsyncAdmin
abortProcedure
in interface AsyncAdmin
procId
- ID of the procedure to abortmayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?CompletableFuture
public CompletableFuture<String> getProcedures()
AsyncAdmin
getProcedures
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<String> getLocks()
AsyncAdmin
getLocks
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Void> decommissionRegionServers(List<ServerName> servers, boolean offload)
AsyncAdmin
decommissionRegionServers
in interface AsyncAdmin
servers
- The list of servers to decommission.offload
- True to offload the regions from the decommissioned serverspublic CompletableFuture<List<ServerName>> listDecommissionedRegionServers()
AsyncAdmin
listDecommissionedRegionServers
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Void> recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
AsyncAdmin
recommissionRegionServer
in interface AsyncAdmin
server
- The server to recommission.encodedRegionNames
- Regions to load onto the server.public CompletableFuture<ClusterMetrics> getClusterMetrics()
AsyncAdmin
CompletableFuture
getClusterMetrics
in interface AsyncAdmin
public CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<ClusterMetrics.Option> options)
AsyncAdmin
CompletableFuture
getClusterMetrics
in interface AsyncAdmin
public CompletableFuture<Void> shutdown()
AsyncAdmin
shutdown
in interface AsyncAdmin
public CompletableFuture<Void> stopMaster()
AsyncAdmin
stopMaster
in interface AsyncAdmin
public CompletableFuture<Void> stopRegionServer(ServerName serverName)
AsyncAdmin
stopRegionServer
in interface AsyncAdmin
public CompletableFuture<Void> updateConfiguration(ServerName serverName)
AsyncAdmin
updateConfiguration
in interface AsyncAdmin
serverName
- : The server whose config needs to be updated.public CompletableFuture<Void> updateConfiguration()
AsyncAdmin
updateConfiguration
in interface AsyncAdmin
public CompletableFuture<Void> rollWALWriter(ServerName serverName)
AsyncAdmin
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.
rollWALWriter
in interface AsyncAdmin
serverName
- The servername of the region server.public CompletableFuture<Void> clearCompactionQueues(ServerName serverName, Set<String> queues)
AsyncAdmin
clearCompactionQueues
in interface AsyncAdmin
serverName
- The servername of the region server.queues
- the set of queue namepublic CompletableFuture<List<SecurityCapability>> getSecurityCapabilities()
AsyncAdmin
CompletableFuture
.getSecurityCapabilities
in interface AsyncAdmin
public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName)
AsyncAdmin
RegionMetrics
of all regions hosted on a region server.getRegionMetrics
in interface AsyncAdmin
RegionMetrics
wrapped by CompletableFuture
public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName, TableName tableName)
AsyncAdmin
RegionMetrics
of all regions hosted on a region server for a table.getRegionMetrics
in interface AsyncAdmin
RegionMetrics
wrapped by CompletableFuture
public CompletableFuture<Boolean> isMasterInMaintenanceMode()
AsyncAdmin
isMasterInMaintenanceMode
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<CompactionState> getCompactionState(TableName tableName, CompactType compactType)
AsyncAdmin
getCompactionState
in interface AsyncAdmin
tableName
- table to examinecompactType
- CompactType
CompletableFuture
public CompletableFuture<CompactionState> getCompactionStateForRegion(byte[] regionName)
AsyncAdmin
getCompactionStateForRegion
in interface AsyncAdmin
regionName
- region to examineCompletableFuture
public CompletableFuture<Optional<Long>> getLastMajorCompactionTimestamp(TableName tableName)
AsyncAdmin
The timestamp of the oldest HFile resulting from a major compaction of that table, or not present if no such HFile could be found.
getLastMajorCompactionTimestamp
in interface AsyncAdmin
tableName
- table to examineCompletableFuture
public CompletableFuture<Optional<Long>> getLastMajorCompactionTimestampForRegion(byte[] regionName)
AsyncAdmin
The timestamp of the oldest HFile resulting from a major compaction of that region, or not present if no such HFile could be found.
getLastMajorCompactionTimestampForRegion
in interface AsyncAdmin
regionName
- region to examineCompletableFuture
public CompletableFuture<Boolean> balancerSwitch(boolean on, boolean drainRITs)
AsyncAdmin
drainRITs
parameter only effects when will we complete the returned
CompletableFuture
.balancerSwitch
in interface AsyncAdmin
on
- Set to true
to enable, false
to disable.drainRITs
- If true
, it waits until current balance() call, if outstanding,
to return.CompletableFuture
.public CompletableFuture<BalanceResponse> balance(BalanceRequest request)
AsyncAdmin
BalanceRequest
for more details.balance
in interface AsyncAdmin
request
- defines how the balancer should runBalanceResponse
with details about the results of the invocation.public CompletableFuture<Boolean> isBalancerEnabled()
AsyncAdmin
isBalancerEnabled
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<Boolean> normalizerSwitch(boolean on)
AsyncAdmin
normalizerSwitch
in interface AsyncAdmin
on
- whether normalizer should be on or offCompletableFuture
public CompletableFuture<Boolean> isNormalizerEnabled()
AsyncAdmin
isNormalizerEnabled
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> normalize(NormalizeTableFilterParams ntfp)
AsyncAdmin
normalize
in interface AsyncAdmin
ntfp
- limit to tables matching the specified filter.CompletableFuture
public CompletableFuture<Boolean> cleanerChoreSwitch(boolean enabled)
AsyncAdmin
CompletableFuture
cleanerChoreSwitch
in interface AsyncAdmin
public CompletableFuture<Boolean> isCleanerChoreEnabled()
AsyncAdmin
isCleanerChoreEnabled
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> runCleanerChore()
AsyncAdmin
runCleanerChore
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Boolean> catalogJanitorSwitch(boolean enabled)
AsyncAdmin
CompletableFuture
catalogJanitorSwitch
in interface AsyncAdmin
public CompletableFuture<Boolean> isCatalogJanitorEnabled()
AsyncAdmin
isCatalogJanitorEnabled
in interface AsyncAdmin
CompletableFuture
public CompletableFuture<Integer> runCatalogJanitor()
AsyncAdmin
runCatalogJanitor
in interface AsyncAdmin
CompletableFuture
public <S,R> CompletableFuture<R> coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker, ServiceCaller<S,R> callable)
AsyncAdmin
The stubMaker
is just a delegation to the newStub
call. Usually it is only a
one line lambda expression, like:
channel -> xxxService.newStub(channel)
coprocessorService
in interface AsyncAdmin
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
public <S,R> CompletableFuture<R> coprocessorService(Function<com.google.protobuf.RpcChannel,S> stubMaker, ServiceCaller<S,R> callable, ServerName serverName)
AsyncAdmin
The stubMaker
is just a delegation to the newStub
call. Usually it is only a
one line lambda expression, like:
channel -> xxxService.newStub(channel)
coprocessorService
in interface AsyncAdmin
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
public CompletableFuture<List<ServerName>> listDeadServers()
AsyncAdmin
listDeadServers
in interface AsyncAdmin
public CompletableFuture<List<ServerName>> listUnknownServers()
AsyncAdmin
listUnknownServers
in interface AsyncAdmin
public CompletableFuture<List<ServerName>> clearDeadServers(List<ServerName> servers)
AsyncAdmin
clearDeadServers
in interface AsyncAdmin
servers
- list of dead region servers.CompletableFuture
.public CompletableFuture<CacheEvictionStats> clearBlockCache(TableName tableName)
AsyncAdmin
clearBlockCache
in interface AsyncAdmin
tableName
- table to clear block cacheCompletableFuture
.public CompletableFuture<Void> cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
AsyncAdmin
cloneTableSchema
in interface AsyncAdmin
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 preservedpublic CompletableFuture<Map<ServerName,Boolean>> compactionSwitch(boolean switchState, List<String> serverNamesList)
AsyncAdmin
compactionSwitch
in interface AsyncAdmin
switchState
- Set to true
to enable, false
to disable.serverNamesList
- list of region servers.public CompletableFuture<Boolean> switchRpcThrottle(boolean enable)
AsyncAdmin
switchRpcThrottle
in interface AsyncAdmin
enable
- Set to true
to enable, false
to disable.public CompletableFuture<Boolean> isRpcThrottleEnabled()
AsyncAdmin
isRpcThrottleEnabled
in interface AsyncAdmin
public CompletableFuture<Boolean> exceedThrottleQuotaSwitch(boolean enable)
AsyncAdmin
exceedThrottleQuotaSwitch
in interface AsyncAdmin
enable
- Set to true
to enable, false
to disable.public CompletableFuture<Map<TableName,Long>> getSpaceQuotaTableSizes()
AsyncAdmin
getSpaceQuotaTableSizes
in interface AsyncAdmin
public CompletableFuture<Map<TableName,SpaceQuotaSnapshot>> getRegionServerSpaceQuotaSnapshots(ServerName serverName)
AsyncAdmin
SpaceQuotaSnapshotView
s observed by a RegionServer.getRegionServerSpaceQuotaSnapshots
in interface AsyncAdmin
public CompletableFuture<SpaceQuotaSnapshot> getCurrentSpaceQuotaSnapshot(String namespace)
AsyncAdmin
namespace
or null if the Master has
no quota information on that namespace.getCurrentSpaceQuotaSnapshot
in interface AsyncAdmin
public CompletableFuture<SpaceQuotaSnapshot> getCurrentSpaceQuotaSnapshot(TableName tableName)
AsyncAdmin
tableName
or null if the Master has
no quota information on that table.getCurrentSpaceQuotaSnapshot
in interface AsyncAdmin
public CompletableFuture<Void> grant(UserPermission userPermission, boolean mergeExistingPermissions)
AsyncAdmin
grant
in interface AsyncAdmin
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.public CompletableFuture<Void> revoke(UserPermission userPermission)
AsyncAdmin
revoke
in interface AsyncAdmin
userPermission
- user name and the specific permissionpublic CompletableFuture<List<UserPermission>> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest)
AsyncAdmin
getUserPermissions
in interface AsyncAdmin
getUserPermissionsRequest
- A request contains which user, global, namespace or table
permissions neededpublic CompletableFuture<List<Boolean>> hasUserPermissions(String userName, List<Permission> permissions)
AsyncAdmin
hasUserPermissions
in interface AsyncAdmin
userName
- the user namepermissions
- the specific permission listpublic CompletableFuture<Boolean> snapshotCleanupSwitch(boolean on, boolean sync)
AsyncAdmin
sync
parameter only effects when will we complete the returned
CompletableFuture
.snapshotCleanupSwitch
in interface AsyncAdmin
on
- Set to true
to enable, false
to disable.sync
- If true
, it waits until current snapshot cleanup is completed, if
outstanding.CompletableFuture
.public CompletableFuture<Boolean> isSnapshotCleanupEnabled()
AsyncAdmin
isSnapshotCleanupEnabled
in interface AsyncAdmin
CompletableFuture
.public CompletableFuture<List<Boolean>> clearSlowLogResponses(Set<ServerName> serverNames)
AsyncAdmin
clearSlowLogResponses
in interface AsyncAdmin
serverNames
- Set of Server names to clean slowlog responses fromCompletableFuture
public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String,Object> filterParams)
AsyncAdmin
getLogEntries
in interface AsyncAdmin
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 paramspublic CompletableFuture<Void> flushMasterStore()
AsyncAdmin
flushMasterStore
in interface AsyncAdmin
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.