Class RawAsyncHBaseAdmin
- All Implemented Interfaces:
AsyncAdmin
The word 'Raw' means that this is a low level class. The returned CompletableFuture will
be finished inside the rpc framework thread, which means that the callbacks registered to the
CompletableFuture will also be executed inside the rpc framework thread. So users who use
this class should not try to do time consuming tasks in the callbacks.
- Since:
- 2.0.0
- See Also:
-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionprivate static classprivate static interfaceprivate static interfaceprivate static classprivate static classprivate static classprivate static classprivate classprivate static classprivate static classprivate static classprivate static interfaceprivate static classprivate static classprivate static classprivate static classprivate static classprivate static classprivate static classprivate static classprivate static classprivate static classprivate static final classprivate static classprivate static classprivate static classprivate static classprivate static class -
Field Summary
FieldsModifier and TypeFieldDescriptionprivate final AsyncConnectionImplstatic final Stringprivate static final org.slf4j.Loggerprivate final intprivate final AsyncTable<AdvancedScanResultConsumer>private final NonceGeneratorprivate final longprivate final longprivate final longprivate final org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimerprivate final longprivate final int -
Constructor Summary
ConstructorsConstructorDescriptionRawAsyncHBaseAdmin(AsyncConnectionImpl connection, org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer retryTimer, AsyncAdminBuilderBase builder) -
Method Summary
Modifier and TypeMethodDescriptionabortProcedure(long procId, boolean mayInterruptIfRunning) Abort a procedure Do not use.addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) Add a column family to an existing table.addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) Add a new replication peer for replicating data to slave clusteraddRSGroup(String groupName) Creates a new RegionServer group with the given nameprivate <PREQ,PRESP, RESP>
CompletableFuture<RESP>adminCall(HBaseRpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.Interface stub, PREQ preq, RawAsyncHBaseAdmin.AdminRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<RESP, PRESP> respConverter) Append the replicable table-cf config of the specified peerassign(byte[] regionName) Assign an individual region.balance(BalanceRequest request) Invoke the balancer with the given balance request.balanceRSGroup(String groupName, BalanceRequest request) Balance regions in the given RegionServer groupbalancerSwitch(boolean on, boolean drainRITs) Turn the load balancer on or off.private <PREQ,PRESP, RESP>
CompletableFuture<RESP>call(HBaseRpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.Interface stub, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<RESP, PRESP> respConverter) catalogJanitorSwitch(boolean enabled) Turn the catalog janitor on/off.private voidcheckAndGetTableName(byte[] encodeRegionName, AtomicReference<TableName> tableName, CompletableFuture<TableName> result) private CompletableFuture<Void>checkAndSyncTableToPeerClusters(TableName tableName, byte[][] splits) Connect to peer and check the table descriptor on peer: Create the same table on peer when not exist. Throw an exception if the table already has replication enabled on any of the column families. Throw an exception if the table exists on peer cluster but descriptors are not same.private CompletableFuture<TableName>checkRegionsAndGetTableName(byte[][] encodedRegionNames) cleanerChoreSwitch(boolean enabled) Turn the cleaner chore on/off.private CompletableFuture<CacheEvictionStats>clearBlockCache(ServerName serverName, List<RegionInfo> hris) clearBlockCache(TableName tableName) Clear all the blocks corresponding to this table from BlockCache.clearCompactionQueues(ServerName serverName, Set<String> queues) Clear compacting queues on a region server.clearDeadServers(List<ServerName> servers) Clear dead region servers from master.clearSlowLogResponses(Set<ServerName> serverNames) Clears online slow RPC logs from the provided list of RegionServersprivate CompletableFuture<Boolean>clearSlowLogsResponses(ServerName serverName) cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Create a new table by cloning the snapshot content.cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) Create a new table by cloning the existent table schema.private CompletableFuture<Void>compact(ServerName sn, RegionInfo hri, boolean major, byte[] columnFamily) Compact the region at specific region server.private CompletableFuture<Void>compact(TableName tableName, byte[] columnFamily, boolean major, CompactType compactType) Compact column family of a table, Asynchronous operation even if CompletableFuture.get()compact(TableName tableName, byte[] columnFamily, CompactType compactType) Compact a column family within a table.compact(TableName tableName, CompactType compactType) Compact a table.compactionSwitch(boolean switchState, List<String> serverNamesList) Turn the compaction on or off.compactRegion(byte[] regionName) Compact an individual region.compactRegion(byte[] regionName, byte[] columnFamily) Compact a column family within a region.private CompletableFuture<Void>compactRegion(byte[] regionName, byte[] columnFamily, boolean major) Compact all regions on the region server.private CompletableFuture<Void>compactRegionServer(ServerName sn, boolean major) private CompletableFuture<Void>compareTableWithPeerCluster(TableName tableName, TableDescriptor tableDesc, ReplicationPeerDescription peer, AsyncAdmin peerAdmin) private static CompletableFuture<Boolean>completeCheckTableState(CompletableFuture<Boolean> future, Optional<TableState> tableState, Throwable error, TableState.State targetState, TableName tableName) Utility for completing passed TableStateCompletableFuturefutureusing passed parameters.private <T> voidcompleteConditionalOnFuture(CompletableFuture<T> dependentFuture, CompletableFuture<T> parentFuture) private <T> booleancompleteExceptionally(CompletableFuture<T> future, Throwable error) private static <T> CompletableFuture<List<T>>convertToFutureOfList(List<CompletableFuture<T>> futures) <S,R> CompletableFuture<R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable) Execute the given coprocessor call on the master.<S,R> CompletableFuture<R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, ServerName serverName) Execute the given coprocessor call on the given region server.createNamespace(NamespaceDescriptor descriptor) Create a new namespace.createTable(TableDescriptor desc) Creates a new table.createTable(TableDescriptor desc, byte[][] splitKeys) Creates a new table with an initial set of empty regions defined by the specified split keys.createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) Creates a new table with the specified number of regions.private CompletableFuture<Void>createTable(TableName tableName, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request) decommissionRegionServers(List<ServerName> servers, boolean offload) Mark region server(s) as decommissioned to prevent additional regions from getting assigned to them.deleteColumnFamily(TableName tableName, byte[] columnFamily) Delete a column family from a table.deleteNamespace(String name) Delete an existing namespace.deleteSnapshot(String snapshotName) Delete an existing snapshot.Delete all existing snapshots.deleteSnapshots(Pattern snapshotNamePattern) Delete existing snapshots whose names match the pattern passed.deleteTable(TableName tableName) Deletes a table.deleteTableSnapshots(Pattern tableNamePattern) Delete all existing snapshots matching the given table name pattern.deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) Delete all existing snapshots matching the given table name regular expression and snapshot name regular expression.disableReplicationPeer(String peerId) Stop the replication stream to the specified peerdisableTable(TableName tableName) Disable a table.disableTableReplication(TableName tableName) Disable a table's replication switch.enableReplicationPeer(String peerId) Restart the replication stream to the specified peerenableTable(TableName tableName) Enable a table.enableTableReplication(TableName tableName) Enable a table's replication switch.exceedThrottleQuotaSwitch(boolean enable) Switch the exceed throttle quota.Execute a distributed procedure on a cluster.CompletableFuture<byte[]>Execute a distributed procedure on a cluster.private <T> CompletableFuture<T>failedFuture(Throwable error) private CompletableFuture<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse>flush(ServerName serverName, RegionInfo regionInfo, byte[] columnFamily, boolean writeFlushWALMarker) Flush a table.Flush the specified column family stores on all regions of the passed table.Flush the specified column family stores on all regions of the passed table.Flush master local regionflushRegion(byte[] regionName) Flush an individual region.flushRegion(byte[] regionName, byte[] columnFamily) Flush a column family within a region.(package private) CompletableFuture<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse>flushRegionInternal(byte[] regionName, byte[] columnFamily, boolean writeFlushWALMarker) This method is for internal use only, where we need the response of the flush.Flush all region on the region server.private CompletableFuture<List<LogEntry>>getBalancerDecisions(int limit) private CompletableFuture<List<LogEntry>>getBalancerRejections(int limit) getCachedFilesList(ServerName serverName) Get the list of cached filesReturns cluster status wrapped byCompletableFuturegetClusterMetrics(EnumSet<ClusterMetrics.Option> options) Returns cluster status wrapped byCompletableFuturegetCompactionState(TableName tableName, CompactType compactType) Get the current compaction state of a table.getCompactionStateForRegion(byte[] regionName) Get the current compaction state of region.private CompletableFuture<List<SnapshotDescription>>getCompletedSnapshots(Pattern pattern) private CompletableFuture<List<SnapshotDescription>>getCompletedSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) getConfiguredNamespacesAndTablesInRSGroup(String groupName) Get the namespaces and tables which have this RegionServer group in descriptor.getCurrentSpaceQuotaSnapshot(String namespace) Returns the Master's view of a quota on the givennamespaceor null if the Master has no quota information on that namespace.private CompletableFuture<SpaceQuotaSnapshot>getCurrentSpaceQuotaSnapshot(RawAsyncHBaseAdmin.Converter<SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse> converter) getCurrentSpaceQuotaSnapshot(TableName tableName) Returns the Master's view of a quota on the giventableNameor null if the Master has no quota information on that table.getDescriptor(TableName tableName) Method for getting the tableDescriptorgetLastMajorCompactionTimestamp(TableName tableName) Get the timestamp of the last major compaction for the passed table.getLastMajorCompactionTimestampForRegion(byte[] regionName) Get the timestamp of the last major compaction for the passed region.getLocks()List locks.getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String, Object> filterParams) Retrieve recent online records from HMaster / RegionServers.getNamespaceDescriptor(String name) Get a namespace descriptor by nameprivate <T> voidgetProcedureResult(long procId, RawAsyncHBaseAdmin.Converter<T, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> converter, CompletableFuture<T> future, int retries) List proceduresgetQuota(QuotaFilter filter) List the quotas based on the filter.private CompletableFuture<RegionInfo>getRegionInfo(byte[] regionNameOrEncodedRegionName) Get the region info for the passed region name.(package private) CompletableFuture<HRegionLocation>getRegionLocation(byte[] regionNameOrEncodedRegionName) Get the region location for the passed region name.getRegionMetrics(ServerName serverName) Get a list ofRegionMetricsof all regions hosted on a region server.getRegionMetrics(ServerName serverName, TableName tableName) Get a list ofRegionMetricsof all regions hosted on a region server for a table.private CompletableFuture<List<RegionMetrics>>getRegionMetrics(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request, ServerName serverName) getRegions(ServerName serverName) Get all the online regions on a region server.getRegions(TableName tableName) Get the regions of a given table.private CompletableFuture<List<ServerName>>getRegionServerList(List<String> serverNamesList) getRegionServerSpaceQuotaSnapshots(ServerName serverName) Fetches the observedSpaceQuotaSnapshotViews observed by a RegionServer.getReplicationPeerConfig(String peerId) Returns the configured ReplicationPeerConfig for the specified peergetRSGroup(String groupName) Get group info for the given group namegetRSGroup(Address hostPort) Get group info for the given hostPortgetRSGroup(TableName table) Get group info for the given tableReturns the list of supported security capabilities.private CompletableFuture<List<LogEntry>>getSlowLogResponseFromServer(ServerName serverName, Map<String, Object> filterParams, int limit, String logType) private CompletableFuture<List<LogEntry>>getSlowLogResponses(Map<String, Object> filterParams, Set<ServerName> serverNames, int limit, String logType) Fetches the table sizes on the filesystem as tracked by the HBase Master.private byte[][]getSplitKeys(byte[] startKey, byte[] endKey, int numRegions) private CompletableFuture<List<TableDescriptor>>getTableDescriptors(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request) private CompletableFuture<List<HRegionLocation>>getTableHRegionLocations(TableName tableName) List all region locations for the specific table.private CompletableFuture<List<TableName>>getTableNames(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request) private CompletableFuture<byte[][]>getTableSplits(TableName tableName) getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) Get the global/namespace/table permissions for usergrant(UserPermission userPermission, boolean mergeExistingPermissions) Grants user specific permissionshasUserPermissions(String userName, List<Permission> permissions) Check if the user has specific permissionsprivate CompletableFuture<Void>internalDeleteSnapshot(SnapshotDescription snapshot) private CompletableFuture<Void>internalDeleteSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) private CompletableFuture<Void>internalRestoreSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Query the current state of the balancer.Query on the catalog janitor state.Query the current state of the cleaner chore.Check whether master is in maintenance modeQuery the current state of the Merge switch.Query the current state of the region normalizerCheck the current state of the specified procedure.isReplicationPeerEnabled(String peerId) Check if a replication peer is enabled.Check whether replication peer modification is enabled.Get if the rpc throttle is enabled.Query the current state of the auto snapshot cleanup based on TTL.isSnapshotFinished(SnapshotDescription snapshot) Check the current state of the passed snapshot.Query the current state of the Split switch.private CompletableFuture<Boolean>isSplitOrMergeOn(MasterSwitchType switchType) isTableAvailable(TableName tableName) Check if a table is available.isTableDisabled(TableName tableName) Check if a table is disabled.isTableEnabled(TableName tableName) Check if a table is enabled.private voidlegacyFlush(CompletableFuture<Void> future, TableName tableName, List<byte[]> columnFamilies) List region servers marked as decommissioned, which can not be assigned regions.List available namespace descriptorsList available namespacesFind all table and column families that are replicated from this clusterReturn a list of replication peers.listReplicationPeers(Pattern pattern) Return a list of replication peers.private CompletableFuture<List<ReplicationPeerDescription>>listReplicationPeers(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request) Lists current set of RegionServer groupsList completed snapshots.listSnapshots(Pattern pattern) List all the completed snapshots matching the given pattern.listTableDescriptors(boolean includeSysTables) List all the tables.listTableDescriptors(List<TableName> tableNames) List specific tables including system tables.listTableDescriptors(Pattern pattern, boolean includeSysTables) Get list of table descriptors by namespace.listTableDescriptorsByState(boolean isEnabled) List all enabled or disabled table descriptorslistTableNames(boolean includeSysTables) List all of the names of tables.listTableNames(Pattern pattern, boolean includeSysTables) List all of the names of userspace tables.Get list of table names by namespace.listTableNamesByState(boolean isEnabled) List all enabled or disabled table nameslistTablesInRSGroup(String groupName) Get all tables in this RegionServer group.listTableSnapshots(Pattern tableNamePattern) List all the completed snapshots matching the given table name pattern.listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) List all the completed snapshots matching the given table name regular expression and snapshot name regular expression.majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) Major compact a column family within a table.majorCompact(TableName tableName, CompactType compactType) Major compact a table.majorCompactRegion(byte[] regionName) Major compact a region.majorCompactRegion(byte[] regionName, byte[] columnFamily) Major compact a column family within region.Compact all regions on the region server.mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible) Merge multiple regions (>=2).mergeSwitch(boolean enabled, boolean drainMerges) Turn the Merge switch on or off.modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) Modify an existing column family on a table.modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT) Change the store file tracker of the given table's given family.modifyNamespace(NamespaceDescriptor descriptor) Modify an existing namespace.modifyTable(TableDescriptor desc) Modify an existing table, more IRB friendly version.modifyTable(TableDescriptor desc, boolean reopenRegions) Modify an existing table, more IRB friendly version.modifyTableStoreFileTracker(TableName tableName, String dstSFT) Change the store file tracker of the given table.move(byte[] regionName) Move the regionrto a random server.move(byte[] regionName, ServerName destServerName) Move the regionrtodest.private CompletableFuture<Void>moveRegion(RegionInfo regionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request) moveServersToRSGroup(Set<Address> servers, String groupName) Move given set of servers to the specified target RegionServer groupprivate <T> AsyncRpcRetryingCallerFactory.AdminRequestCallerBuilder<T>(package private) <T> AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder<T>(package private) <T> AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder<T>Invoke region normalizer.private CompletableFuture<Boolean>normalize(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request) normalizerSwitch(boolean on) Set region normalizer on/off.offline(byte[] regionName) Offline specified region from master's in-memory state.private <PREQ,PRESP, PRES>
CompletableFuture<PRES>procedureCall(Consumer<AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder<?>> prioritySetter, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.Converter<PRES, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> resultConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<PRES> consumer) private <PREQ,PRESP, PRES>
CompletableFuture<PRES>procedureCall(TableName tableName, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.Converter<PRES, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> resultConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<PRES> consumer) short-circuit call for procedureCall(Consumer, Object, MasterRpcCall, Converter, Converter, ProcedureBiConsumer) by skip setting priority for requestprivate <PREQ,PRESP>
CompletableFuture<Void>procedureCall(TableName tableName, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<Void> consumer) short-circuit call for procedureCall(TableName, Object, MasterRpcCall, Converter, Converter, ProcedureBiConsumer) by ignoring procedure resultprivate <PREQ,PRESP, PRES>
CompletableFuture<PRES>procedureCall(PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.Converter<PRES, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> resultConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<PRES> consumer) short-circuit call for procedureCall(Consumer, Object, MasterRpcCall, Converter, Converter, ProcedureBiConsumer) by skip setting priority for requestprivate <PREQ,PRESP>
CompletableFuture<Void>procedureCall(PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<Void> consumer) short-circuit call forprocedureCall(Object, MasterRpcCall, Converter, Converter, ProcedureBiConsumer)by ignoring procedure resultrecommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) Remove decommission marker from a region server to allow regions assignments.removeReplicationPeer(String peerId) Remove a peer and stop the replicationRemove some table-cfs from config of the specified peerremoveRSGroup(String groupName) Remove RegionServer group associated with the given nameremoveServersFromRSGroup(Set<Address> servers) Remove decommissioned servers from group 1.renameRSGroup(String oldName, String newName) Rename rsgroupreplicationPeerModificationSwitch(boolean on, boolean drainProcedures) Enable or disable replication peer modification.restoreBackupSystemTable(String snapshotName) restoreSnapshot(String snapshotName) Restore the specified snapshot on the original table.restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) Restore the specified snapshot on the original table.private CompletableFuture<Void>restoreSnapshot(String snapshotName, TableName tableName, boolean takeFailSafeSnapshot, boolean restoreAcl) revoke(UserPermission userPermission) Revokes user specific permissionsRoll log writer for all RegionServers.rollWALWriter(ServerName serverName) Roll the log writer.Ask for a scan of the catalog table.Ask for cleaner chore to run.setQuota(QuotaSettings quota) Apply the new quota settings.setRSGroup(Set<TableName> tables, String groupName) Set the RegionServer group for tablesprivate CompletableFuture<Boolean>setSplitOrMergeOn(boolean enabled, boolean synchronous, MasterSwitchType switchType) private CompletableFuture<Void>setTableReplication(TableName tableName, boolean enableRep) Set the table's replication switch if the table's replication switch is already not set.shutdown()Shuts down the HBase cluster.snapshot(SnapshotDescription snapshotDesc) Take a snapshot and wait for the server to complete that snapshot asynchronously.snapshotCleanupSwitch(boolean on, boolean sync) Turn on or off the auto snapshot cleanup based on TTL.private CompletableFuture<Void>split(RegionInfo hri, byte[] splitPoint) Split a table.Split a table.splitRegion(byte[] regionName) Split an individual region.splitRegion(byte[] regionName, byte[] splitPoint) Split an individual region.splitSwitch(boolean enabled, boolean drainSplits) Turn the Split switch on or off.Shuts down the current HBase master only.stopRegionServer(ServerName serverName) Stop the designated regionserver.private CompletableFuture<Boolean>switchCompact(ServerName serverName, boolean onOrOff) switchRpcThrottle(boolean enable) Switch the rpc throttle enabled state.tableExists(TableName tableName) Check if a table exists.private byte[]toEncodeRegionName(byte[] regionName) transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState clusterState) Transit current cluster to a new state in a synchronous replication peer.truncateRegion(byte[] regionName) Truncate an individual region.private CompletableFuture<Void>truncateRegion(RegionInfo hri) truncateTable(TableName tableName, boolean preserveSplits) Truncate a table.private CompletableFuture<Void>trySyncTableToPeerCluster(TableName tableName, byte[][] splits, ReplicationPeerDescription peer) unassign(byte[] regionName) Unassign a region from current hosting regionserver.Update the configuration and trigger an online config change on all the masters and regionservers.updateConfiguration(String groupName) Update the configuration and trigger an online config change on all the regionservers in the RSGroup.updateConfiguration(ServerName serverName) Update the configuration and trigger an online config change on the regionserver.updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) Update the peerConfig for the specified peerupdateRSGroupConfig(String groupName, Map<String, String> configuration) Update RSGroup configurationprivate voidverifySplitKeys(byte[][] splitKeys) private <T> CompletableFuture<T>waitProcedureResult(CompletableFuture<Long> procFuture, RawAsyncHBaseAdmin.Converter<T, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> converter) private voidwaitSnapshotFinish(SnapshotDescription snapshot, CompletableFuture<Void> future, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse resp) private voidwaitUntilAllReplicationPeerModificationProceduresDone(CompletableFuture<Boolean> future, boolean prevOn, int retries) Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, waitMethods inherited from interface org.apache.hadoop.hbase.client.AsyncAdmin
addReplicationPeer, balance, balance, balanceRSGroup, balancerSwitch, cloneSnapshot, cloneSnapshot, compact, compact, getBackupMasters, getCompactionState, getMaster, getMasterCoprocessorNames, getMasterInfoPort, getRegionServers, getRegionServers, getReplicationPeerSyncReplicationState, getSlowLogResponses, hasUserPermissions, listDeadServers, listTableDescriptors, listTableNames, listUnknownServers, majorCompact, majorCompact, mergeRegions, mergeSwitch, normalize, replicationPeerModificationSwitch, restoreSnapshot, snapshot, snapshot, splitSwitch, unassign
-
Field Details
-
FLUSH_TABLE_PROCEDURE_SIGNATURE
- See Also:
-
LOG
-
connection
-
retryTimer
-
metaTable
-
rpcTimeoutNs
-
operationTimeoutNs
-
pauseNs
-
pauseNsForServerOverloaded
-
maxAttempts
-
startLogErrorsCnt
-
ng
-
-
Constructor Details
-
RawAsyncHBaseAdmin
RawAsyncHBaseAdmin(AsyncConnectionImpl connection, org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer retryTimer, AsyncAdminBuilderBase builder)
-
-
Method Details
-
newMasterCaller
-
newAdminCaller
-
call
private <PREQ,PRESP, CompletableFuture<RESP> callRESP> (HBaseRpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService.Interface stub, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<RESP, PRESP> respConverter) -
adminCall
private <PREQ,PRESP, CompletableFuture<RESP> adminCallRESP> (HBaseRpcController controller, org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.Interface stub, PREQ preq, RawAsyncHBaseAdmin.AdminRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<RESP, PRESP> respConverter) -
procedureCall
private <PREQ,PRESP> CompletableFuture<Void> procedureCall(PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<Void> consumer) short-circuit call forprocedureCall(Object, MasterRpcCall, Converter, Converter, ProcedureBiConsumer)by ignoring procedure result -
procedureCall
private <PREQ,PRESP, CompletableFuture<PRES> procedureCallPRES> (PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.Converter<PRES, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> resultConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<PRES> consumer) short-circuit call for procedureCall(Consumer, Object, MasterRpcCall, Converter, Converter, ProcedureBiConsumer) by skip setting priority for request -
procedureCall
private <PREQ,PRESP> CompletableFuture<Void> procedureCall(TableName tableName, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<Void> consumer) short-circuit call for procedureCall(TableName, Object, MasterRpcCall, Converter, Converter, ProcedureBiConsumer) by ignoring procedure result -
procedureCall
private <PREQ,PRESP, CompletableFuture<PRES> procedureCallPRES> (TableName tableName, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.Converter<PRES, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> resultConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<PRES> consumer) short-circuit call for procedureCall(Consumer, Object, MasterRpcCall, Converter, Converter, ProcedureBiConsumer) by skip setting priority for request -
procedureCall
private <PREQ,PRESP, CompletableFuture<PRES> procedureCallPRES> (Consumer<AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder<?>> prioritySetter, PREQ preq, RawAsyncHBaseAdmin.MasterRpcCall<PRESP, PREQ> rpcCall, RawAsyncHBaseAdmin.Converter<Long, PRESP> respConverter, RawAsyncHBaseAdmin.Converter<PRES, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> resultConverter, RawAsyncHBaseAdmin.ProcedureBiConsumer<PRES> consumer) - Type Parameters:
PREQ- type of requestPRESP- type of responsePRES- type of procedure call result- Parameters:
prioritySetter- prioritySetter set priority by table for requestpreq- procedure call requestrpcCall- procedure rpc callrespConverter- extract proc id from procedure call responseresultConverter- extract result from procedure call resultconsumer- action performs on result- Returns:
- procedure call result, null if procedure is void
-
tableExists
Description copied from interface:AsyncAdminCheck if a table exists.- Specified by:
tableExistsin interfaceAsyncAdmin- Parameters:
tableName- Table to check.- Returns:
- True if table exists already. The return value will be wrapped by a
CompletableFuture.
-
listTableDescriptors
Description copied from interface:AsyncAdminList all the tables.- Specified by:
listTableDescriptorsin interfaceAsyncAdmin- Parameters:
includeSysTables- False to match only against userspace tables- Returns:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture.
-
listTableDescriptors
public CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern, boolean includeSysTables) - Specified by:
listTableDescriptorsin interfaceAsyncAdmin- Parameters:
pattern- The compiled regular expression to match againstincludeSysTables- False to match only against userspace tables- Returns:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture.
-
listTableDescriptors
Description copied from interface:AsyncAdminList specific tables including system tables.- Specified by:
listTableDescriptorsin interfaceAsyncAdmin- Parameters:
tableNames- the table list to match against- Returns:
- - returns a list of TableDescriptors wrapped by a
CompletableFuture.
-
getTableDescriptors
private CompletableFuture<List<TableDescriptor>> getTableDescriptors(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest request) -
listTableNames
Description copied from interface:AsyncAdminList all of the names of tables.- Specified by:
listTableNamesin interfaceAsyncAdmin- Parameters:
includeSysTables- False to match only against userspace tables- Returns:
- a list of table names wrapped by a
CompletableFuture.
-
listTableNames
Description copied from interface:AsyncAdminList all of the names of userspace tables.- Specified by:
listTableNamesin interfaceAsyncAdmin- Parameters:
pattern- The regular expression to match againstincludeSysTables- False to match only against userspace tables- Returns:
- a list of table names wrapped by a
CompletableFuture.
-
listTableNamesByState
Description copied from interface:AsyncAdminList all enabled or disabled table names- Specified by:
listTableNamesByStatein interfaceAsyncAdmin- Parameters:
isEnabled- is true means return enabled table names, false means return disabled table names- Returns:
- a list of table names wrapped by a
CompletableFuture.
-
getTableNames
private CompletableFuture<List<TableName>> getTableNames(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest request) -
listTableDescriptorsByNamespace
Description copied from interface:AsyncAdminGet list of table descriptors by namespace.- Specified by:
listTableDescriptorsByNamespacein interfaceAsyncAdmin- Parameters:
name- namespace name- Returns:
- returns a list of TableDescriptors wrapped by a
CompletableFuture.
-
listTableDescriptorsByState
Description copied from interface:AsyncAdminList all enabled or disabled table descriptors- Specified by:
listTableDescriptorsByStatein interfaceAsyncAdmin- Parameters:
isEnabled- is true means return enabled table descriptors, false means return disabled table descriptors- Returns:
- a list of table names wrapped by a
CompletableFuture.
-
listTableNamesByNamespace
Description copied from interface:AsyncAdminGet list of table names by namespace.- Specified by:
listTableNamesByNamespacein interfaceAsyncAdmin- Parameters:
name- namespace name- Returns:
- The list of table names in the namespace wrapped by a
CompletableFuture.
-
getDescriptor
Description copied from interface:AsyncAdminMethod for getting the tableDescriptor- Specified by:
getDescriptorin interfaceAsyncAdmin- Parameters:
tableName- as aTableName- Returns:
- the read-only tableDescriptor wrapped by a
CompletableFuture.
-
createTable
Description copied from interface:AsyncAdminCreates a new table.- Specified by:
createTablein interfaceAsyncAdmin- Parameters:
desc- table descriptor for table
-
createTable
public CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) Description copied from interface:AsyncAdminCreates a new table with the specified number of regions. The start key specified will become the end key of the first region of the table, and the end key specified will become the start key of the last region of the table (the first region has a null start key and the last region has a null end key). BigInteger math will be used to divide the key range specified into enough segments to make the required number of total regions.- Specified by:
createTablein interfaceAsyncAdmin- Parameters:
desc- table descriptor for tablestartKey- beginning of key rangeendKey- end of key rangenumRegions- the total number of regions to create
-
createTable
Description copied from interface:AsyncAdminCreates a new table with an initial set of empty regions defined by the specified split keys. The total number of regions created will be the number of split keys plus one. Note : Avoid passing empty split key.- Specified by:
createTablein interfaceAsyncAdmin- Parameters:
desc- table descriptor for tablesplitKeys- array of split keys for the initial regions of the table
-
createTable
private CompletableFuture<Void> createTable(TableName tableName, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.CreateTableRequest request) -
modifyTable
Description copied from interface:AsyncAdminModify an existing table, more IRB friendly version.- Specified by:
modifyTablein interfaceAsyncAdmin- Parameters:
desc- modified description of the table
-
modifyTable
Description copied from interface:AsyncAdminModify an existing table, more IRB friendly version.- Specified by:
modifyTablein interfaceAsyncAdmin- Parameters:
desc- description of the tablereopenRegions- By default, 'modifyTable' reopens all regions, potentially causing a RIT (Region In Transition) storm in large tables. If set to 'false', regions will remain unaware of the modification until they are individually reopened. Please note that this may temporarily result in configuration inconsistencies among regions.
-
modifyTableStoreFileTracker
Description copied from interface:AsyncAdminChange the store file tracker of the given table.- Specified by:
modifyTableStoreFileTrackerin interfaceAsyncAdmin- Parameters:
tableName- the table you want to changedstSFT- the destination store file tracker
-
deleteTable
Description copied from interface:AsyncAdminDeletes a table.- Specified by:
deleteTablein interfaceAsyncAdmin- Parameters:
tableName- name of table to delete
-
truncateTable
Description copied from interface:AsyncAdminTruncate a table.- Specified by:
truncateTablein interfaceAsyncAdmin- Parameters:
tableName- name of table to truncatepreserveSplits- True if the splits should be preserved
-
enableTable
Description copied from interface:AsyncAdminEnable a table. The table has to be in disabled state for it to be enabled.- Specified by:
enableTablein interfaceAsyncAdmin- Parameters:
tableName- name of the table
-
disableTable
Description copied from interface:AsyncAdminDisable a table. The table has to be in enabled state for it to be disabled.- Specified by:
disableTablein interfaceAsyncAdmin
-
completeCheckTableState
private static CompletableFuture<Boolean> completeCheckTableState(CompletableFuture<Boolean> future, Optional<TableState> tableState, Throwable error, TableState.State targetState, TableName tableName) Utility for completing passed TableStateCompletableFuturefutureusing passed parameters. Sets error or boolean result ('true' if table matches the passed-in targetState). -
isTableEnabled
Description copied from interface:AsyncAdminCheck if a table is enabled.- Specified by:
isTableEnabledin interfaceAsyncAdmin- Parameters:
tableName- name of table to check- Returns:
- true if table is on-line. The return value will be wrapped by a
CompletableFuture.
-
isTableDisabled
Description copied from interface:AsyncAdminCheck if a table is disabled.- Specified by:
isTableDisabledin interfaceAsyncAdmin- Parameters:
tableName- name of table to check- Returns:
- true if table is off-line. The return value will be wrapped by a
CompletableFuture.
-
isTableAvailable
Description copied from interface:AsyncAdminCheck if a table is available.- Specified by:
isTableAvailablein interfaceAsyncAdmin- Parameters:
tableName- name of table to check- Returns:
- true if all regions of the table are available. The return value will be wrapped by a
CompletableFuture.
-
addColumnFamily
public CompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) Description copied from interface:AsyncAdminAdd a column family to an existing table.- Specified by:
addColumnFamilyin interfaceAsyncAdmin- Parameters:
tableName- name of the table to add column family tocolumnFamily- column family descriptor of column family to be added
-
deleteColumnFamily
Description copied from interface:AsyncAdminDelete a column family from a table.- Specified by:
deleteColumnFamilyin interfaceAsyncAdmin- Parameters:
tableName- name of tablecolumnFamily- name of column family to be deleted
-
modifyColumnFamily
public CompletableFuture<Void> modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) Description copied from interface:AsyncAdminModify an existing column family on a table.- Specified by:
modifyColumnFamilyin interfaceAsyncAdmin- Parameters:
tableName- name of tablecolumnFamily- new column family descriptor to use
-
modifyColumnFamilyStoreFileTracker
public CompletableFuture<Void> modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family, String dstSFT) Description copied from interface:AsyncAdminChange the store file tracker of the given table's given family.- Specified by:
modifyColumnFamilyStoreFileTrackerin interfaceAsyncAdmin- Parameters:
tableName- the table you want to changefamily- the family you want to changedstSFT- the destination store file tracker
-
createNamespace
Description copied from interface:AsyncAdminCreate a new namespace.- Specified by:
createNamespacein interfaceAsyncAdmin- Parameters:
descriptor- descriptor which describes the new namespace
-
modifyNamespace
Description copied from interface:AsyncAdminModify an existing namespace.- Specified by:
modifyNamespacein interfaceAsyncAdmin- Parameters:
descriptor- descriptor which describes the new namespace
-
deleteNamespace
Description copied from interface:AsyncAdminDelete an existing namespace. Only empty namespaces (no tables) can be removed.- Specified by:
deleteNamespacein interfaceAsyncAdmin- Parameters:
name- namespace name
-
getNamespaceDescriptor
Description copied from interface:AsyncAdminGet a namespace descriptor by name- Specified by:
getNamespaceDescriptorin interfaceAsyncAdmin- Parameters:
name- name of namespace descriptor- Returns:
- A descriptor wrapped by a
CompletableFuture.
-
listNamespaces
Description copied from interface:AsyncAdminList available namespaces- Specified by:
listNamespacesin interfaceAsyncAdmin- Returns:
- List of namespaces wrapped by a
CompletableFuture.
-
listNamespaceDescriptors
Description copied from interface:AsyncAdminList available namespace descriptors- Specified by:
listNamespaceDescriptorsin interfaceAsyncAdmin- Returns:
- List of descriptors wrapped by a
CompletableFuture.
-
getRegions
Description copied from interface:AsyncAdminGet all the online regions on a region server.- Specified by:
getRegionsin interfaceAsyncAdmin
-
getRegions
Description copied from interface:AsyncAdminGet the regions of a given table.- Specified by:
getRegionsin interfaceAsyncAdmin
-
flush
Description copied from interface:AsyncAdminFlush a table.- Specified by:
flushin interfaceAsyncAdmin- Parameters:
tableName- table to flush
-
flush
Description copied from interface:AsyncAdminFlush the specified column family stores on all regions of the passed table. This runs as a synchronous operation.- Specified by:
flushin interfaceAsyncAdmin- Parameters:
tableName- table to flushcolumnFamily- column family within a table
-
flush
Description copied from interface:AsyncAdminFlush the specified column family stores on all regions of the passed table. This runs as a synchronous operation.- Specified by:
flushin interfaceAsyncAdmin- Parameters:
tableName- table to flushcolumnFamilyList- column families within a table
-
legacyFlush
private void legacyFlush(CompletableFuture<Void> future, TableName tableName, List<byte[]> columnFamilies) -
flushRegion
Description copied from interface:AsyncAdminFlush an individual region.- Specified by:
flushRegionin interfaceAsyncAdmin- Parameters:
regionName- region to flush
-
flushRegion
Description copied from interface:AsyncAdminFlush a column family within a region.- Specified by:
flushRegionin interfaceAsyncAdmin- Parameters:
regionName- region to flushcolumnFamily- column family within a region. If not present, flush the region's all column families.
-
flushRegionInternal
CompletableFuture<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse> flushRegionInternal(byte[] regionName, byte[] columnFamily, boolean writeFlushWALMarker) This method is for internal use only, where we need the response of the flush. As it exposes the protobuf message, please do NOT try to expose it as a public API. -
flush
private CompletableFuture<org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse> flush(ServerName serverName, RegionInfo regionInfo, byte[] columnFamily, boolean writeFlushWALMarker) -
flushRegionServer
Description copied from interface:AsyncAdminFlush all region on the region server.- Specified by:
flushRegionServerin interfaceAsyncAdmin- Parameters:
sn- server to flush
-
compact
Description copied from interface:AsyncAdminCompact a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundExceptionif table not found for normal compaction type.- Specified by:
compactin interfaceAsyncAdmin- Parameters:
tableName- table to compactcompactType-CompactType
-
compact
public CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, CompactType compactType) Description copied from interface:AsyncAdminCompact a column family within a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundExceptionif table not found for normal compaction type.- Specified by:
compactin interfaceAsyncAdmin- Parameters:
tableName- table to compactcolumnFamily- column family within a tablecompactType-CompactType
-
compactRegion
Description copied from interface:AsyncAdminCompact an individual region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Specified by:
compactRegionin interfaceAsyncAdmin- Parameters:
regionName- region to compact
-
compactRegion
Description copied from interface:AsyncAdminCompact a column family within a region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Specified by:
compactRegionin interfaceAsyncAdmin- Parameters:
regionName- region to compactcolumnFamily- column family within a region. If not present, compact the region's all column families.
-
majorCompact
Description copied from interface:AsyncAdminMajor compact a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundExceptionif table not found for normal compaction type.- Specified by:
majorCompactin interfaceAsyncAdmin- Parameters:
tableName- table to major compactcompactType-CompactType
-
majorCompact
public CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) Description copied from interface:AsyncAdminMajor compact a column family within a table. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation. ThrowsTableNotFoundExceptionif table not found.- Specified by:
majorCompactin interfaceAsyncAdmin- Parameters:
tableName- table to major compactcolumnFamily- column family within a table. If not present, major compact the table's all column families.compactType-CompactType
-
majorCompactRegion
Description copied from interface:AsyncAdminMajor compact a region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Specified by:
majorCompactRegionin interfaceAsyncAdmin- Parameters:
regionName- region to major compact
-
majorCompactRegion
Description copied from interface:AsyncAdminMajor compact a column family within region. When the returned CompletableFuture is done, it only means the compact request was sent to HBase and may need some time to finish the compact operation.- Specified by:
majorCompactRegionin interfaceAsyncAdmin- Parameters:
regionName- region to major compactcolumnFamily- column family within a region. If not present, major compact the region's all column families.
-
compactRegionServer
Description copied from interface:AsyncAdminCompact all regions on the region server.- Specified by:
compactRegionServerin interfaceAsyncAdmin- Parameters:
sn- the region server name
-
majorCompactRegionServer
Description copied from interface:AsyncAdminCompact all regions on the region server.- Specified by:
majorCompactRegionServerin interfaceAsyncAdmin- Parameters:
sn- the region server name
-
compactRegionServer
-
compactRegion
private CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily, boolean major) -
getTableHRegionLocations
List all region locations for the specific table. -
compact
private CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily, boolean major, CompactType compactType) Compact column family of a table, Asynchronous operation even if CompletableFuture.get() -
compact
private CompletableFuture<Void> compact(ServerName sn, RegionInfo hri, boolean major, byte[] columnFamily) Compact the region at specific region server. -
toEncodeRegionName
-
checkAndGetTableName
private void checkAndGetTableName(byte[] encodeRegionName, AtomicReference<TableName> tableName, CompletableFuture<TableName> result) -
checkRegionsAndGetTableName
-
mergeSwitch
Description copied from interface:AsyncAdminTurn the Merge switch on or off. Notice that, the method itself is always non-blocking, which means it will always return immediately. ThedrainMergesparameter only effects when will we complete the returnedCompletableFuture.- Specified by:
mergeSwitchin interfaceAsyncAdmin- Parameters:
enabled- enabled or notdrainMerges- Iftrue, it waits until current merge() call, if outstanding, to return.- Returns:
- Previous switch value wrapped by a
CompletableFuture
-
isMergeEnabled
Description copied from interface:AsyncAdminQuery the current state of the Merge switch.- Specified by:
isMergeEnabledin interfaceAsyncAdmin- Returns:
- true if the switch is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
splitSwitch
Description copied from interface:AsyncAdminTurn the Split switch on or off. Notice that, the method itself is always non-blocking, which means it will always return immediately. ThedrainSplitsparameter only effects when will we complete the returnedCompletableFuture.- Specified by:
splitSwitchin interfaceAsyncAdmin- Parameters:
enabled- enabled or notdrainSplits- Iftrue, it waits until current split() call, if outstanding, to return.- Returns:
- Previous switch value wrapped by a
CompletableFuture
-
isSplitEnabled
Description copied from interface:AsyncAdminQuery the current state of the Split switch.- Specified by:
isSplitEnabledin interfaceAsyncAdmin- Returns:
- true if the switch is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
setSplitOrMergeOn
private CompletableFuture<Boolean> setSplitOrMergeOn(boolean enabled, boolean synchronous, MasterSwitchType switchType) -
isSplitOrMergeOn
-
mergeRegions
Description copied from interface:AsyncAdminMerge multiple regions (>=2).- Specified by:
mergeRegionsin interfaceAsyncAdmin- Parameters:
nameOfRegionsToMerge- encoded or full name of daughter regionsforcible- true if do a compulsory merge, otherwise we will only merge two adjacent regions
-
split
Description copied from interface:AsyncAdminSplit a table. The method will execute split action for each region in table.- Specified by:
splitin interfaceAsyncAdmin- Parameters:
tableName- table to split
-
split
Description copied from interface:AsyncAdminSplit a table.- Specified by:
splitin interfaceAsyncAdmin- Parameters:
tableName- table to splitsplitPoint- the explicit position to split on
-
splitRegion
Description copied from interface:AsyncAdminSplit an individual region.- Specified by:
splitRegionin interfaceAsyncAdmin- Parameters:
regionName- region to split
-
splitRegion
Description copied from interface:AsyncAdminSplit an individual region.- Specified by:
splitRegionin interfaceAsyncAdmin- Parameters:
regionName- region to splitsplitPoint- the explicit position to split on. If not present, it will decide by region server.
-
split
-
truncateRegion
Description copied from interface:AsyncAdminTruncate an individual region.- Specified by:
truncateRegionin interfaceAsyncAdmin- Parameters:
regionName- region to truncate
-
truncateRegion
-
assign
Description copied from interface:AsyncAdminAssign an individual region.- Specified by:
assignin interfaceAsyncAdmin- Parameters:
regionName- Encoded or full name of region to assign.
-
unassign
Description copied from interface:AsyncAdminUnassign a region from current hosting regionserver. Region will then be assigned to a regionserver chosen at random. Region could be reassigned back to the same server. UseAsyncAdmin.move(byte[], ServerName)if you want to control the region movement.- Specified by:
unassignin interfaceAsyncAdmin- Parameters:
regionName- Encoded or full name of region to unassign.
-
offline
Description copied from interface:AsyncAdminOffline specified region from master's in-memory state. It will not attempt to reassign the region as in unassign. This API can be used when a region not served by any region server and still online as per Master's in memory state. If this API is incorrectly used on active region then master will loose track of that region. This is a special method that should be used by experts or hbck.- Specified by:
offlinein interfaceAsyncAdmin- Parameters:
regionName- Encoded or full name of region to offline
-
move
Description copied from interface:AsyncAdminMove the regionrto a random server.- Specified by:
movein interfaceAsyncAdmin- Parameters:
regionName- Encoded or full name of region to move.
-
move
Description copied from interface:AsyncAdminMove the regionrtodest.- Specified by:
movein interfaceAsyncAdmin- Parameters:
regionName- Encoded or full name of region to move.destServerName- The servername of the destination regionserver. If not present, we'll assign to a random server. A server name is made of host, port and startcode. Here is an example:host187.example.com,60020,1289493121758
-
moveRegion
private CompletableFuture<Void> moveRegion(RegionInfo regionInfo, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MoveRegionRequest request) -
setQuota
Description copied from interface:AsyncAdminApply the new quota settings.- Specified by:
setQuotain interfaceAsyncAdmin- Parameters:
quota- the quota settings
-
getQuota
Description copied from interface:AsyncAdminList the quotas based on the filter.- Specified by:
getQuotain interfaceAsyncAdmin- Parameters:
filter- the quota settings filter- Returns:
- the QuotaSetting list, which wrapped by a CompletableFuture.
-
addReplicationPeer
public CompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) Description copied from interface:AsyncAdminAdd a new replication peer for replicating data to slave cluster- Specified by:
addReplicationPeerin interfaceAsyncAdmin- Parameters:
peerId- a short name that identifies the peerpeerConfig- configuration for the replication slave clusterenabled- peer state, true if ENABLED and false if DISABLED
-
removeReplicationPeer
Description copied from interface:AsyncAdminRemove a peer and stop the replication- Specified by:
removeReplicationPeerin interfaceAsyncAdmin- Parameters:
peerId- a short name that identifies the peer
-
enableReplicationPeer
Description copied from interface:AsyncAdminRestart the replication stream to the specified peer- Specified by:
enableReplicationPeerin interfaceAsyncAdmin- Parameters:
peerId- a short name that identifies the peer
-
disableReplicationPeer
Description copied from interface:AsyncAdminStop the replication stream to the specified peer- Specified by:
disableReplicationPeerin interfaceAsyncAdmin- Parameters:
peerId- a short name that identifies the peer
-
getReplicationPeerConfig
Description copied from interface:AsyncAdminReturns the configured ReplicationPeerConfig for the specified peer- Specified by:
getReplicationPeerConfigin interfaceAsyncAdmin- Parameters:
peerId- a short name that identifies the peer- Returns:
- ReplicationPeerConfig for the peer wrapped by a
CompletableFuture.
-
updateReplicationPeerConfig
public CompletableFuture<Void> updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) Description copied from interface:AsyncAdminUpdate the peerConfig for the specified peer- Specified by:
updateReplicationPeerConfigin interfaceAsyncAdmin- Parameters:
peerId- a short name that identifies the peerpeerConfig- new config for the peer
-
transitReplicationPeerSyncReplicationState
public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState clusterState) Description copied from interface:AsyncAdminTransit current cluster to a new state in a synchronous replication peer.- Specified by:
transitReplicationPeerSyncReplicationStatein interfaceAsyncAdmin- Parameters:
peerId- a short name that identifies the peerclusterState- a new state of current cluster
-
appendReplicationPeerTableCFs
public CompletableFuture<Void> appendReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs) Description copied from interface:AsyncAdminAppend the replicable table-cf config of the specified peer- Specified by:
appendReplicationPeerTableCFsin interfaceAsyncAdmin- Parameters:
id- a short that identifies the clustertableCfs- A map from tableName to column family names
-
removeReplicationPeerTableCFs
public CompletableFuture<Void> removeReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs) Description copied from interface:AsyncAdminRemove some table-cfs from config of the specified peer- Specified by:
removeReplicationPeerTableCFsin interfaceAsyncAdmin- Parameters:
id- a short name that identifies the clustertableCfs- A map from tableName to column family names
-
listReplicationPeers
Description copied from interface:AsyncAdminReturn a list of replication peers.- Specified by:
listReplicationPeersin interfaceAsyncAdmin- Returns:
- a list of replication peers description. The return value will be wrapped by a
CompletableFuture.
-
listReplicationPeers
Description copied from interface:AsyncAdminReturn a list of replication peers.- Specified by:
listReplicationPeersin interfaceAsyncAdmin- Parameters:
pattern- The compiled regular expression to match peer id- Returns:
- a list of replication peers description. The return value will be wrapped by a
CompletableFuture.
-
listReplicationPeers
private CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest request) -
listReplicatedTableCFs
Description copied from interface:AsyncAdminFind all table and column families that are replicated from this cluster- Specified by:
listReplicatedTableCFsin interfaceAsyncAdmin- Returns:
- the replicated table-cfs list of this cluster. The return value will be wrapped by a
CompletableFuture.
-
snapshot
Description copied from interface:AsyncAdminTake a snapshot and wait for the server to complete that snapshot asynchronously. Snapshots are taken sequentially even when requested concurrently, across all tables. Snapshots are considered unique based on the name of the snapshot. Attempts to take a snapshot with the same name (even a different type or with different parameters) will fail with aSnapshotCreationExceptionindicating the duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. SeeTableName.isLegalFullyQualifiedTableName(byte[]). You should probably useAsyncAdmin.snapshot(String, org.apache.hadoop.hbase.TableName)unless you are sure about the type of snapshot that you want to take.- Specified by:
snapshotin interfaceAsyncAdmin- Parameters:
snapshotDesc- snapshot to take
-
waitSnapshotFinish
private void waitSnapshotFinish(SnapshotDescription snapshot, CompletableFuture<Void> future, org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse resp) -
isSnapshotFinished
Description copied from interface:AsyncAdminCheck the current state of the passed snapshot. There are three possible states:- running - returns false
- finished - returns true
- finished with error - throws the exception that caused the snapshot to fail
UnknownSnapshotException.- Specified by:
isSnapshotFinishedin interfaceAsyncAdmin- Parameters:
snapshot- description of the snapshot to check- Returns:
- true if the snapshot is completed, false if the snapshot is still running
-
restoreSnapshot
Description copied from interface:AsyncAdminRestore the specified snapshot on the original table. (The table must be disabled) If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted.- Specified by:
restoreSnapshotin interfaceAsyncAdmin- Parameters:
snapshotName- name of the snapshot to restore
-
restoreSnapshot
public CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl) Description copied from interface:AsyncAdminRestore the specified snapshot on the original table. (The table must be disabled) If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before executing the restore operation. In case of restore failure, the failsafe snapshot will be restored. If the restore completes without problem the failsafe snapshot is deleted. The failsafe snapshot name is configurable by using the property "hbase.snapshot.restore.failsafe.name".- Specified by:
restoreSnapshotin interfaceAsyncAdmin- Parameters:
snapshotName- name of the snapshot to restoretakeFailSafeSnapshot- true if the failsafe snapshot should be takenrestoreAcl-trueto restore acl of snapshot
-
restoreSnapshot
private CompletableFuture<Void> restoreSnapshot(String snapshotName, TableName tableName, boolean takeFailSafeSnapshot, boolean restoreAcl) -
completeConditionalOnFuture
private <T> void completeConditionalOnFuture(CompletableFuture<T> dependentFuture, CompletableFuture<T> parentFuture) -
cloneSnapshot
public CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) Description copied from interface:AsyncAdminCreate a new table by cloning the snapshot content.- Specified by:
cloneSnapshotin interfaceAsyncAdmin- Parameters:
snapshotName- name of the snapshot to be clonedtableName- name of the table where the snapshot will be restoredrestoreAcl-trueto restore acl of snapshotcustomSFT- specify the StroreFileTracker used for the table
-
internalRestoreSnapshot
private CompletableFuture<Void> internalRestoreSnapshot(String snapshotName, TableName tableName, boolean restoreAcl, String customSFT) -
listSnapshots
Description copied from interface:AsyncAdminList completed snapshots.- Specified by:
listSnapshotsin interfaceAsyncAdmin- Returns:
- a list of snapshot descriptors for completed snapshots wrapped by a
CompletableFuture
-
listSnapshots
Description copied from interface:AsyncAdminList all the completed snapshots matching the given pattern.- Specified by:
listSnapshotsin interfaceAsyncAdmin- Parameters:
pattern- The compiled regular expression to match against- Returns:
- - returns a List of SnapshotDescription wrapped by a
CompletableFuture
-
getCompletedSnapshots
-
listTableSnapshots
Description copied from interface:AsyncAdminList all the completed snapshots matching the given table name pattern.- Specified by:
listTableSnapshotsin interfaceAsyncAdmin- Parameters:
tableNamePattern- The compiled table name regular expression to match against- Returns:
- - returns a List of completed SnapshotDescription wrapped by a
CompletableFuture
-
listTableSnapshots
public CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) Description copied from interface:AsyncAdminList all the completed snapshots matching the given table name regular expression and snapshot name regular expression.- Specified by:
listTableSnapshotsin interfaceAsyncAdmin- Parameters:
tableNamePattern- The compiled table name regular expression to match againstsnapshotNamePattern- The compiled snapshot name regular expression to match against- Returns:
- - returns a List of completed SnapshotDescription wrapped by a
CompletableFuture
-
getCompletedSnapshots
private CompletableFuture<List<SnapshotDescription>> getCompletedSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) -
deleteSnapshot
Description copied from interface:AsyncAdminDelete an existing snapshot.- Specified by:
deleteSnapshotin interfaceAsyncAdmin- Parameters:
snapshotName- name of the snapshot
-
deleteSnapshots
Description copied from interface:AsyncAdminDelete all existing snapshots.- Specified by:
deleteSnapshotsin interfaceAsyncAdmin
-
deleteSnapshots
Description copied from interface:AsyncAdminDelete existing snapshots whose names match the pattern passed.- Specified by:
deleteSnapshotsin interfaceAsyncAdmin- Parameters:
snapshotNamePattern- pattern for names of the snapshot to match
-
deleteTableSnapshots
Description copied from interface:AsyncAdminDelete all existing snapshots matching the given table name pattern.- Specified by:
deleteTableSnapshotsin interfaceAsyncAdmin- Parameters:
tableNamePattern- The compiled table name regular expression to match against
-
deleteTableSnapshots
public CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) Description copied from interface:AsyncAdminDelete all existing snapshots matching the given table name regular expression and snapshot name regular expression.- Specified by:
deleteTableSnapshotsin interfaceAsyncAdmin- Parameters:
tableNamePattern- The compiled table name regular expression to match againstsnapshotNamePattern- The compiled snapshot name regular expression to match against
-
internalDeleteSnapshots
private CompletableFuture<Void> internalDeleteSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) -
internalDeleteSnapshot
-
execProcedure
public CompletableFuture<Void> execProcedure(String signature, String instance, Map<String, String> props) Description copied from interface:AsyncAdminExecute a distributed procedure on a cluster.- Specified by:
execProcedurein interfaceAsyncAdmin- Parameters:
signature- A distributed procedure is uniquely identified by its signature (default the root ZK node name of the procedure).instance- The instance name of the procedure. For some procedures, this parameter is optional.props- Property/Value pairs of properties passing to the procedure
-
execProcedureWithReturn
public CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance, Map<String, String> props) Description copied from interface:AsyncAdminExecute a distributed procedure on a cluster.- Specified by:
execProcedureWithReturnin interfaceAsyncAdmin- Parameters:
signature- A distributed procedure is uniquely identified by its signature (default the root ZK node name of the procedure).instance- The instance name of the procedure. For some procedures, this parameter is optional.props- Property/Value pairs of properties passing to the procedure- Returns:
- data returned after procedure execution. null if no return data.
-
isProcedureFinished
public CompletableFuture<Boolean> isProcedureFinished(String signature, String instance, Map<String, String> props) Description copied from interface:AsyncAdminCheck the current state of the specified procedure. There are three possible states:- running - returns false
- finished - returns true
- finished with error - throws the exception that caused the procedure to fail
- Specified by:
isProcedureFinishedin interfaceAsyncAdmin- Parameters:
signature- The signature that uniquely identifies a procedureinstance- The instance name of the procedureprops- Property/Value pairs of properties passing to the procedure- Returns:
- true if the specified procedure is finished successfully, false if it is still running.
The value is wrapped by
CompletableFuture
-
abortProcedure
Description copied from interface:AsyncAdminAbort a procedure Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.- Specified by:
abortProcedurein interfaceAsyncAdmin- Parameters:
procId- ID of the procedure to abortmayInterruptIfRunning- if the proc completed at least one step, should it be aborted?- Returns:
- true if aborted, false if procedure already completed or does not exist. the value is
wrapped by
CompletableFuture - See Also:
-
getProcedures
Description copied from interface:AsyncAdminList procedures- Specified by:
getProceduresin interfaceAsyncAdmin- Returns:
- procedure list JSON wrapped by
CompletableFuture
-
getLocks
Description copied from interface:AsyncAdminList locks.- Specified by:
getLocksin interfaceAsyncAdmin- Returns:
- lock list JSON wrapped by
CompletableFuture
-
decommissionRegionServers
Description copied from interface:AsyncAdminMark region server(s) as decommissioned to prevent additional regions from getting assigned to them. Optionally unload the regions on the servers. If there are multiple servers to be decommissioned, decommissioning them at the same time can prevent wasteful region movements. Region unloading is asynchronous.- Specified by:
decommissionRegionServersin interfaceAsyncAdmin- Parameters:
servers- The list of servers to decommission.offload- True to offload the regions from the decommissioned servers
-
listDecommissionedRegionServers
Description copied from interface:AsyncAdminList region servers marked as decommissioned, which can not be assigned regions.- Specified by:
listDecommissionedRegionServersin interfaceAsyncAdmin- Returns:
- List of decommissioned region servers wrapped by
CompletableFuture
-
recommissionRegionServer
public CompletableFuture<Void> recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) Description copied from interface:AsyncAdminRemove decommission marker from a region server to allow regions assignments. Load regions onto the server if a list of regions is given. Region loading is asynchronous.- Specified by:
recommissionRegionServerin interfaceAsyncAdmin- Parameters:
server- The server to recommission.encodedRegionNames- Regions to load onto the server.
-
getRegionLocation
Get the region location for the passed region name. The region name may be a full region name or encoded region name. If the region does not found, then it'll throw an UnknownRegionException wrapped by aCompletableFuture- Parameters:
regionNameOrEncodedRegionName- region name or encoded region name- Returns:
- region location, wrapped by a
CompletableFuture
-
getRegionInfo
Get the region info for the passed region name. The region name may be a full region name or encoded region name. If the region does not found, then it'll throw an UnknownRegionException wrapped by aCompletableFuture- Returns:
- region info, wrapped by a
CompletableFuture
-
getSplitKeys
-
verifySplitKeys
-
waitProcedureResult
private <T> CompletableFuture<T> waitProcedureResult(CompletableFuture<Long> procFuture, RawAsyncHBaseAdmin.Converter<T, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> converter) -
getProcedureResult
private <T> void getProcedureResult(long procId, RawAsyncHBaseAdmin.Converter<T, org.apache.hbase.thirdparty.com.google.protobuf.ByteString> converter, CompletableFuture<T> future, int retries) -
failedFuture
-
completeExceptionally
-
getClusterMetrics
Description copied from interface:AsyncAdminReturns cluster status wrapped byCompletableFuture- Specified by:
getClusterMetricsin interfaceAsyncAdmin
-
getClusterMetrics
Description copied from interface:AsyncAdminReturns cluster status wrapped byCompletableFuture- Specified by:
getClusterMetricsin interfaceAsyncAdmin
-
shutdown
Description copied from interface:AsyncAdminShuts down the HBase cluster.- Specified by:
shutdownin interfaceAsyncAdmin
-
stopMaster
Description copied from interface:AsyncAdminShuts down the current HBase master only.- Specified by:
stopMasterin interfaceAsyncAdmin
-
stopRegionServer
Description copied from interface:AsyncAdminStop the designated regionserver.- Specified by:
stopRegionServerin interfaceAsyncAdmin
-
updateConfiguration
Description copied from interface:AsyncAdminUpdate the configuration and trigger an online config change on the regionserver.- Specified by:
updateConfigurationin interfaceAsyncAdmin- Parameters:
serverName- : The server whose config needs to be updated.
-
updateConfiguration
Description copied from interface:AsyncAdminUpdate the configuration and trigger an online config change on all the masters and regionservers.- Specified by:
updateConfigurationin interfaceAsyncAdmin
-
updateConfiguration
Description copied from interface:AsyncAdminUpdate the configuration and trigger an online config change on all the regionservers in the RSGroup.- Specified by:
updateConfigurationin interfaceAsyncAdmin- Parameters:
groupName- the group name
-
rollWALWriter
Description copied from interface:AsyncAdminRoll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.When the returned CompletableFuture is done, it only means the rollWALWriter request was sent to the region server and may need some time to finish the rollWALWriter operation. As a side effect of this call, the named region server may schedule store flushes at the request of the wal.
- Specified by:
rollWALWriterin interfaceAsyncAdmin- Parameters:
serverName- The servername of the region server.
-
rollAllWALWriters
Description copied from interface:AsyncAdminRoll log writer for all RegionServers. Note that unlikeAdmin.rollWALWriter(ServerName), this method is synchronous, which means it will block until all RegionServers have completed the log roll, or a RegionServer fails due to an exception that retry will not work.- Specified by:
rollAllWALWritersin interfaceAsyncAdmin- Returns:
- server and the highest wal filenum of server before performing log roll
-
clearCompactionQueues
Description copied from interface:AsyncAdminClear compacting queues on a region server.- Specified by:
clearCompactionQueuesin interfaceAsyncAdmin- Parameters:
serverName- The servername of the region server.queues- the set of queue name
-
getSecurityCapabilities
Description copied from interface:AsyncAdminReturns the list of supported security capabilities. The return value will be wrapped by aCompletableFuture.- Specified by:
getSecurityCapabilitiesin interfaceAsyncAdmin
-
getRegionMetrics
Description copied from interface:AsyncAdminGet a list ofRegionMetricsof all regions hosted on a region server.- Specified by:
getRegionMetricsin interfaceAsyncAdmin- Returns:
- list of
RegionMetricswrapped byCompletableFuture
-
getRegionMetrics
public CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName, TableName tableName) Description copied from interface:AsyncAdminGet a list ofRegionMetricsof all regions hosted on a region server for a table.- Specified by:
getRegionMetricsin interfaceAsyncAdmin- Returns:
- a list of
RegionMetricswrapped byCompletableFuture
-
getRegionMetrics
private CompletableFuture<List<RegionMetrics>> getRegionMetrics(org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionLoadRequest request, ServerName serverName) -
isMasterInMaintenanceMode
Description copied from interface:AsyncAdminCheck whether master is in maintenance mode- Specified by:
isMasterInMaintenanceModein interfaceAsyncAdmin- Returns:
- true if master is in maintenance mode, false otherwise. The return value will be
wrapped by a
CompletableFuture
-
getCompactionState
public CompletableFuture<CompactionState> getCompactionState(TableName tableName, CompactType compactType) Description copied from interface:AsyncAdminGet the current compaction state of a table. It could be in a major compaction, a minor compaction, both, or none.- Specified by:
getCompactionStatein interfaceAsyncAdmin- Parameters:
tableName- table to examinecompactType-CompactType- Returns:
- the current compaction state wrapped by a
CompletableFuture
-
getCompactionStateForRegion
Description copied from interface:AsyncAdminGet the current compaction state of region. It could be in a major compaction, a minor compaction, both, or none.- Specified by:
getCompactionStateForRegionin interfaceAsyncAdmin- Parameters:
regionName- region to examine- Returns:
- the current compaction state wrapped by a
CompletableFuture
-
getLastMajorCompactionTimestamp
Description copied from interface:AsyncAdminGet the timestamp of the last major compaction for the passed table.The timestamp of the oldest HFile resulting from a major compaction of that table, or not present if no such HFile could be found.
- Specified by:
getLastMajorCompactionTimestampin interfaceAsyncAdmin- Parameters:
tableName- table to examine- Returns:
- the last major compaction timestamp wrapped by a
CompletableFuture
-
getLastMajorCompactionTimestampForRegion
public CompletableFuture<Optional<Long>> getLastMajorCompactionTimestampForRegion(byte[] regionName) Description copied from interface:AsyncAdminGet the timestamp of the last major compaction for the passed region.The timestamp of the oldest HFile resulting from a major compaction of that region, or not present if no such HFile could be found.
- Specified by:
getLastMajorCompactionTimestampForRegionin interfaceAsyncAdmin- Parameters:
regionName- region to examine- Returns:
- the last major compaction timestamp wrapped by a
CompletableFuture
-
compactionSwitch
public CompletableFuture<Map<ServerName,Boolean>> compactionSwitch(boolean switchState, List<String> serverNamesList) Description copied from interface:AsyncAdminTurn the compaction on or off. Disabling compactions will also interrupt any currently ongoing compactions. This state is ephemeral. The setting will be lost on restart. Compaction can also be enabled/disabled by modifying configuration hbase.regionserver.compaction.enabled in hbase-site.xml.- Specified by:
compactionSwitchin interfaceAsyncAdmin- Parameters:
switchState- Set totrueto enable,falseto disable.serverNamesList- list of region servers.- Returns:
- Previous compaction states for region servers
-
getRegionServerList
-
switchCompact
-
balancerSwitch
Description copied from interface:AsyncAdminTurn the load balancer on or off. Notice that, the method itself is always non-blocking, which means it will always return immediately. ThedrainRITsparameter only effects when will we complete the returnedCompletableFuture.- Specified by:
balancerSwitchin interfaceAsyncAdmin- Parameters:
on- Set totrueto enable,falseto disable.drainRITs- Iftrue, it waits until current balance() call, if outstanding, to return.- Returns:
- Previous balancer value wrapped by a
CompletableFuture.
-
balance
Description copied from interface:AsyncAdminInvoke the balancer with the given balance request. The BalanceRequest defines how the balancer will run. SeeBalanceRequestfor more details.- Specified by:
balancein interfaceAsyncAdmin- Parameters:
request- defines how the balancer should run- Returns:
BalanceResponsewith details about the results of the invocation.
-
isBalancerEnabled
Description copied from interface:AsyncAdminQuery the current state of the balancer.- Specified by:
isBalancerEnabledin interfaceAsyncAdmin- Returns:
- true if the balance switch is on, false otherwise. The return value will be wrapped by
a
CompletableFuture.
-
normalizerSwitch
Description copied from interface:AsyncAdminSet region normalizer on/off.- Specified by:
normalizerSwitchin interfaceAsyncAdmin- Parameters:
on- whether normalizer should be on or off- Returns:
- Previous normalizer value wrapped by a
CompletableFuture
-
isNormalizerEnabled
Description copied from interface:AsyncAdminQuery the current state of the region normalizer- Specified by:
isNormalizerEnabledin interfaceAsyncAdmin- Returns:
- true if region normalizer is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
normalize
Description copied from interface:AsyncAdminInvoke region normalizer. Can NOT run for various reasons. Check logs.- Specified by:
normalizein interfaceAsyncAdmin- Parameters:
ntfp- limit to tables matching the specified filter.- Returns:
- true if region normalizer ran, false otherwise. The return value will be wrapped by a
CompletableFuture
-
normalize
private CompletableFuture<Boolean> normalize(org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.NormalizeRequest request) -
cleanerChoreSwitch
Description copied from interface:AsyncAdminTurn the cleaner chore on/off.- Specified by:
cleanerChoreSwitchin interfaceAsyncAdmin- Returns:
- Previous cleaner state wrapped by a
CompletableFuture
-
isCleanerChoreEnabled
Description copied from interface:AsyncAdminQuery the current state of the cleaner chore.- Specified by:
isCleanerChoreEnabledin interfaceAsyncAdmin- Returns:
- true if cleaner chore is on, false otherwise. The return value will be wrapped by a
CompletableFuture
-
runCleanerChore
Description copied from interface:AsyncAdminAsk for cleaner chore to run.- Specified by:
runCleanerChorein interfaceAsyncAdmin- Returns:
- true if cleaner chore ran, false otherwise. The return value will be wrapped by a
CompletableFuture
-
catalogJanitorSwitch
Description copied from interface:AsyncAdminTurn the catalog janitor on/off.- Specified by:
catalogJanitorSwitchin interfaceAsyncAdmin- Returns:
- the previous state wrapped by a
CompletableFuture
-
isCatalogJanitorEnabled
Description copied from interface:AsyncAdminQuery on the catalog janitor state.- Specified by:
isCatalogJanitorEnabledin interfaceAsyncAdmin- Returns:
- true if the catalog janitor is on, false otherwise. The return value will be wrapped by
a
CompletableFuture
-
runCatalogJanitor
Description copied from interface:AsyncAdminAsk for a scan of the catalog table.- Specified by:
runCatalogJanitorin interfaceAsyncAdmin- Returns:
- the number of entries cleaned. The return value will be wrapped by a
CompletableFuture
-
coprocessorService
public <S,R> CompletableFuture<R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable) Description copied from interface:AsyncAdminExecute the given coprocessor call on the master.The
stubMakeris just a delegation to thenewStubcall. Usually it is only a one line lambda expression, like:channel -> xxxService.newStub(channel)
- Specified by:
coprocessorServicein interfaceAsyncAdmin- Type Parameters:
S- the type of the asynchronous stubR- the type of the return value- Parameters:
stubMaker- a delegation to the actualnewStubcall.callable- a delegation to the actual protobuf rpc call. See the comment ofServiceCallerfor more details.- Returns:
- the return value of the protobuf rpc call, wrapped by a
CompletableFuture. - See Also:
-
coprocessorService
public <S,R> CompletableFuture<R> coprocessorService(Function<org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel, S> stubMaker, ServiceCaller<S, R> callable, ServerName serverName) Description copied from interface:AsyncAdminExecute the given coprocessor call on the given region server.The
stubMakeris just a delegation to thenewStubcall. Usually it is only a one line lambda expression, like:channel -> xxxService.newStub(channel)
- Specified by:
coprocessorServicein interfaceAsyncAdmin- Type Parameters:
S- the type of the asynchronous stubR- the type of the return value- Parameters:
stubMaker- a delegation to the actualnewStubcall.callable- a delegation to the actual protobuf rpc call. See the comment ofServiceCallerfor more details.serverName- the given region server- Returns:
- the return value of the protobuf rpc call, wrapped by a
CompletableFuture. - See Also:
-
clearDeadServers
Description copied from interface:AsyncAdminClear dead region servers from master.- Specified by:
clearDeadServersin interfaceAsyncAdmin- Parameters:
servers- list of dead region servers.- Returns:
- - returns a list of servers that not cleared wrapped by a
CompletableFuture.
-
newServerCaller
-
enableTableReplication
Description copied from interface:AsyncAdminEnable a table's replication switch.- Specified by:
enableTableReplicationin interfaceAsyncAdmin- Parameters:
tableName- name of the table
-
disableTableReplication
Description copied from interface:AsyncAdminDisable a table's replication switch.- Specified by:
disableTableReplicationin interfaceAsyncAdmin- Parameters:
tableName- name of the table
-
getTableSplits
-
checkAndSyncTableToPeerClusters
private CompletableFuture<Void> checkAndSyncTableToPeerClusters(TableName tableName, byte[][] splits) Connect to peer and check the table descriptor on peer:- Create the same table on peer when not exist.
- Throw an exception if the table already has replication enabled on any of the column families.
- Throw an exception if the table exists on peer cluster but descriptors are not same.
- Parameters:
tableName- name of the table to sync to the peersplits- table split keys
-
trySyncTableToPeerCluster
private CompletableFuture<Void> trySyncTableToPeerCluster(TableName tableName, byte[][] splits, ReplicationPeerDescription peer) -
compareTableWithPeerCluster
private CompletableFuture<Void> compareTableWithPeerCluster(TableName tableName, TableDescriptor tableDesc, ReplicationPeerDescription peer, AsyncAdmin peerAdmin) -
setTableReplication
Set the table's replication switch if the table's replication switch is already not set.- Parameters:
tableName- name of the tableenableRep- is replication switch enable or disable
-
isReplicationPeerEnabled
Description copied from interface:AsyncAdminCheck if a replication peer is enabled.- Specified by:
isReplicationPeerEnabledin interfaceAsyncAdmin- Parameters:
peerId- id of replication peer to check- Returns:
- true if replication peer is enabled. The return value will be wrapped by a
CompletableFuture
-
waitUntilAllReplicationPeerModificationProceduresDone
private void waitUntilAllReplicationPeerModificationProceduresDone(CompletableFuture<Boolean> future, boolean prevOn, int retries) -
replicationPeerModificationSwitch
public CompletableFuture<Boolean> replicationPeerModificationSwitch(boolean on, boolean drainProcedures) Description copied from interface:AsyncAdminEnable or disable replication peer modification. This is especially useful when you want to change the replication peer storage.- Specified by:
replicationPeerModificationSwitchin interfaceAsyncAdmin- Parameters:
on-truemeans enable, otherwise disabledrainProcedures- iftrue, will wait until all the running replication peer modification procedures finish- Returns:
- the previous enable/disable state wrapped by a
CompletableFuture
-
isReplicationPeerModificationEnabled
Description copied from interface:AsyncAdminCheck whether replication peer modification is enabled.- Specified by:
isReplicationPeerModificationEnabledin interfaceAsyncAdmin- Returns:
trueif modification is enabled, otherwisefalse, wrapped by aCompletableFuture
-
clearBlockCache
Description copied from interface:AsyncAdminClear all the blocks corresponding to this table from BlockCache. For expert-admins. Calling this API will drop all the cached blocks specific to a table from BlockCache. This can significantly impact the query performance as the subsequent queries will have to retrieve the blocks from underlying filesystem.- Specified by:
clearBlockCachein interfaceAsyncAdmin- Parameters:
tableName- table to clear block cache- Returns:
- CacheEvictionStats related to the eviction wrapped by a
CompletableFuture.
-
cloneTableSchema
public CompletableFuture<Void> cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) Description copied from interface:AsyncAdminCreate a new table by cloning the existent table schema.- Specified by:
cloneTableSchemain interfaceAsyncAdmin- Parameters:
tableName- name of the table to be clonednewTableName- name of the new table where the table will be createdpreserveSplits- True if the splits should be preserved
-
clearBlockCache
private CompletableFuture<CacheEvictionStats> clearBlockCache(ServerName serverName, List<RegionInfo> hris) -
switchRpcThrottle
Description copied from interface:AsyncAdminSwitch the rpc throttle enabled state.- Specified by:
switchRpcThrottlein interfaceAsyncAdmin- Parameters:
enable- Set totrueto enable,falseto disable.- Returns:
- Previous rpc throttle enabled value
-
isRpcThrottleEnabled
Description copied from interface:AsyncAdminGet if the rpc throttle is enabled.- Specified by:
isRpcThrottleEnabledin interfaceAsyncAdmin- Returns:
- True if rpc throttle is enabled
-
exceedThrottleQuotaSwitch
Description copied from interface:AsyncAdminSwitch the exceed throttle quota. If enabled, user/table/namespace throttle quota can be exceeded if region server has availble quota.- Specified by:
exceedThrottleQuotaSwitchin interfaceAsyncAdmin- Parameters:
enable- Set totrueto enable,falseto disable.- Returns:
- Previous exceed throttle enabled value
-
getSpaceQuotaTableSizes
Description copied from interface:AsyncAdminFetches the table sizes on the filesystem as tracked by the HBase Master.- Specified by:
getSpaceQuotaTableSizesin interfaceAsyncAdmin
-
getRegionServerSpaceQuotaSnapshots
public CompletableFuture<Map<TableName,SpaceQuotaSnapshot>> getRegionServerSpaceQuotaSnapshots(ServerName serverName) Description copied from interface:AsyncAdminFetches the observedSpaceQuotaSnapshotViews observed by a RegionServer.- Specified by:
getRegionServerSpaceQuotaSnapshotsin interfaceAsyncAdmin
-
getCurrentSpaceQuotaSnapshot
private CompletableFuture<SpaceQuotaSnapshot> getCurrentSpaceQuotaSnapshot(RawAsyncHBaseAdmin.Converter<SpaceQuotaSnapshot, org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.GetQuotaStatesResponse> converter) -
getCurrentSpaceQuotaSnapshot
Description copied from interface:AsyncAdminReturns the Master's view of a quota on the givennamespaceor null if the Master has no quota information on that namespace.- Specified by:
getCurrentSpaceQuotaSnapshotin interfaceAsyncAdmin
-
getCurrentSpaceQuotaSnapshot
Description copied from interface:AsyncAdminReturns the Master's view of a quota on the giventableNameor null if the Master has no quota information on that table.- Specified by:
getCurrentSpaceQuotaSnapshotin interfaceAsyncAdmin
-
grant
public CompletableFuture<Void> grant(UserPermission userPermission, boolean mergeExistingPermissions) Description copied from interface:AsyncAdminGrants user specific permissions- Specified by:
grantin interfaceAsyncAdmin- Parameters:
userPermission- user name and the specific permissionmergeExistingPermissions- If set to false, later granted permissions will override previous granted permissions. otherwise, it'll merge with previous granted permissions.
-
revoke
Description copied from interface:AsyncAdminRevokes user specific permissions- Specified by:
revokein interfaceAsyncAdmin- Parameters:
userPermission- user name and the specific permission
-
getUserPermissions
public CompletableFuture<List<UserPermission>> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) Description copied from interface:AsyncAdminGet the global/namespace/table permissions for user- Specified by:
getUserPermissionsin interfaceAsyncAdmin- Parameters:
getUserPermissionsRequest- A request contains which user, global, namespace or table permissions needed- Returns:
- The user and permission list
-
hasUserPermissions
public CompletableFuture<List<Boolean>> hasUserPermissions(String userName, List<Permission> permissions) Description copied from interface:AsyncAdminCheck if the user has specific permissions- Specified by:
hasUserPermissionsin interfaceAsyncAdmin- Parameters:
userName- the user namepermissions- the specific permission list- Returns:
- True if user has the specific permissions
-
snapshotCleanupSwitch
Description copied from interface:AsyncAdminTurn on or off the auto snapshot cleanup based on TTL. Notice that, the method itself is always non-blocking, which means it will always return immediately. Thesyncparameter only effects when will we complete the returnedCompletableFuture.- Specified by:
snapshotCleanupSwitchin interfaceAsyncAdmin- Parameters:
on- Set totrueto enable,falseto disable.sync- Iftrue, it waits until current snapshot cleanup is completed, if outstanding.- Returns:
- Previous auto snapshot cleanup value wrapped by a
CompletableFuture.
-
isSnapshotCleanupEnabled
Description copied from interface:AsyncAdminQuery the current state of the auto snapshot cleanup based on TTL.- Specified by:
isSnapshotCleanupEnabledin interfaceAsyncAdmin- Returns:
- true if the auto snapshot cleanup is enabled, false otherwise. The return value will be
wrapped by a
CompletableFuture.
-
moveServersToRSGroup
Description copied from interface:AsyncAdminMove given set of servers to the specified target RegionServer group- Specified by:
moveServersToRSGroupin interfaceAsyncAdmin- Parameters:
servers- set of servers to movegroupName- the group to move servers to
-
addRSGroup
Description copied from interface:AsyncAdminCreates a new RegionServer group with the given name- Specified by:
addRSGroupin interfaceAsyncAdmin- Parameters:
groupName- the name of the group
-
removeRSGroup
Description copied from interface:AsyncAdminRemove RegionServer group associated with the given name- Specified by:
removeRSGroupin interfaceAsyncAdmin- Parameters:
groupName- the group name
-
balanceRSGroup
Description copied from interface:AsyncAdminBalance regions in the given RegionServer group- Specified by:
balanceRSGroupin interfaceAsyncAdmin- Parameters:
groupName- the group namerequest- options to define how the balancer should run- Returns:
- BalanceResponse details about the balancer run
-
listRSGroups
Description copied from interface:AsyncAdminLists current set of RegionServer groups- Specified by:
listRSGroupsin interfaceAsyncAdmin
-
getSlowLogResponses
private CompletableFuture<List<LogEntry>> getSlowLogResponses(Map<String, Object> filterParams, Set<ServerName> serverNames, int limit, String logType) -
getSlowLogResponseFromServer
private CompletableFuture<List<LogEntry>> getSlowLogResponseFromServer(ServerName serverName, Map<String, Object> filterParams, int limit, String logType) -
clearSlowLogResponses
public CompletableFuture<List<Boolean>> clearSlowLogResponses(@Nullable Set<ServerName> serverNames) Description copied from interface:AsyncAdminClears online slow RPC logs from the provided list of RegionServers- Specified by:
clearSlowLogResponsesin interfaceAsyncAdmin- Parameters:
serverNames- Set of Server names to clean slowlog responses from- Returns:
- List of booleans representing if online slowlog response buffer is cleaned from each
RegionServer. The return value wrapped by a
CompletableFuture
-
clearSlowLogsResponses
-
convertToFutureOfList
private static <T> CompletableFuture<List<T>> convertToFutureOfList(List<CompletableFuture<T>> futures) -
listTablesInRSGroup
Description copied from interface:AsyncAdminGet all tables in this RegionServer group.- Specified by:
listTablesInRSGroupin interfaceAsyncAdmin- Parameters:
groupName- the group name- See Also:
-
getConfiguredNamespacesAndTablesInRSGroup
public CompletableFuture<Pair<List<String>,List<TableName>>> getConfiguredNamespacesAndTablesInRSGroup(String groupName) Description copied from interface:AsyncAdminGet the namespaces and tables which have this RegionServer group in descriptor. The difference between this method andAsyncAdmin.listTablesInRSGroup(String)is that, this method will not include the table which is actually in this RegionServr group but without the RegionServer group configuration in itsTableDescriptor. For example, we have a group 'A', and we make namespace 'nsA' in this group, then all the tables under this namespace will in the group 'A', but this method will not return these tables but only the namespace 'nsA', while theAsyncAdmin.listTablesInRSGroup(String)will return all these tables.- Specified by:
getConfiguredNamespacesAndTablesInRSGroupin interfaceAsyncAdmin- Parameters:
groupName- the group name- See Also:
-
getRSGroup
Description copied from interface:AsyncAdminGet group info for the given hostPort- Specified by:
getRSGroupin interfaceAsyncAdmin- Parameters:
hostPort- HostPort to get RSGroupInfo for
-
removeServersFromRSGroup
Description copied from interface:AsyncAdminRemove decommissioned servers from group 1. Sometimes we may find the server aborted due to some hardware failure and we must offline the server for repairing. Or we need to move some servers to join other clusters. So we need to remove these servers from the group. 2. Dead/recovering/live servers will be disallowed.- Specified by:
removeServersFromRSGroupin interfaceAsyncAdmin- Parameters:
servers- set of servers to remove
-
setRSGroup
Description copied from interface:AsyncAdminSet the RegionServer group for tables- Specified by:
setRSGroupin interfaceAsyncAdmin- Parameters:
tables- tables to set group forgroupName- group name for tables
-
getRSGroup
Description copied from interface:AsyncAdminGet group info for the given table- Specified by:
getRSGroupin interfaceAsyncAdmin- Parameters:
table- table name to get RSGroupInfo for
-
getRSGroup
Description copied from interface:AsyncAdminGet group info for the given group name- Specified by:
getRSGroupin interfaceAsyncAdmin- Parameters:
groupName- the group name- Returns:
- group info
-
renameRSGroup
Description copied from interface:AsyncAdminRename rsgroup- Specified by:
renameRSGroupin interfaceAsyncAdmin- Parameters:
oldName- old rsgroup namenewName- new rsgroup name
-
updateRSGroupConfig
public CompletableFuture<Void> updateRSGroupConfig(String groupName, Map<String, String> configuration) Description copied from interface:AsyncAdminUpdate RSGroup configuration- Specified by:
updateRSGroupConfigin interfaceAsyncAdmin- Parameters:
groupName- the group nameconfiguration- new configuration of the group name to be set
-
getBalancerDecisions
-
getBalancerRejections
-
getLogEntries
public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, String logType, ServerType serverType, int limit, Map<String, Object> filterParams) Description copied from interface:AsyncAdminRetrieve recent online records from HMaster / RegionServers. Examples include slow/large RPC logs, balancer decisions by master.- Specified by:
getLogEntriesin interfaceAsyncAdmin- Parameters:
serverNames- servers to retrieve records from, useful in case of records maintained by RegionServer as we can select specific server. In case of servertype=MASTER, logs will only come from the currently active master.logType- string representing type of log recordsserverType- enum for server type: HMaster or RegionServerlimit- put a limit to list of records that server should send in responsefilterParams- additional filter params
-
flushMasterStore
Description copied from interface:AsyncAdminFlush master local region- Specified by:
flushMasterStorein interfaceAsyncAdmin
-
getCachedFilesList
Description copied from interface:AsyncAdminGet the list of cached files- Specified by:
getCachedFilesListin interfaceAsyncAdmin
-
restoreBackupSystemTable
- Specified by:
restoreBackupSystemTablein interfaceAsyncAdmin
-