Interface RegionServerServices
- All Superinterfaces:
Abortable,FavoredNodesForRegion,MutableOnlineRegions,OnlineRegions,Server,Stoppable
- All Known Implementing Classes:
HMaster,HMasterCommandLine.LocalHMaster,HRegionServer
@Private
public interface RegionServerServices
extends Server, MutableOnlineRegions, FavoredNodesForRegion
A curated subset of services provided by
HRegionServer. For use internally only. Passed
to Managers, Services and Chores so can pass less-than-a full-on HRegionServer at test-time. Be
judicious adding API. Changes cause ripples through the code base.-
Nested Class Summary
Nested ClassesModifier and TypeInterfaceDescriptionstatic classContext for postOpenDeployTasks().static class -
Method Summary
Modifier and TypeMethodDescriptionReturns theAccessCheckerReturns The block cache instance.doubleReturns hbase executor servicedoubleDeprecated.Returns the controller to avoid flush too fastReturns heap memory manager instanceReturns The RegionServer's "Leases" serviceReturns the metrics tracker for the region serverReturns The cache for mob files.Only required for "old" log replay; if it's removed, remove this.Returns the RegionServerAccounting for this Region ServerReturns RegionServer's instance ofRegionServerRpcQuotaManagerReturns RegionServer's instance ofRegionServerSpaceQuotaManagerConcurrentMap<byte[],Boolean> Get the regions that are currently being opened or closed in the RSReturns a reference to the region server's RPC serverReturns RegionServer's instance ofSecureBulkLoadManagerReturns Return table descriptors implementation.getWAL(RegionInfo regionInfo) Returns the WAL for a particular region.getWALs()Returns the List of WALs that are used by this server Doesn't include the meta WALReturnsZKPermissionWatcherbooleanReturns True if cluster is up; false if cluster is not up (we are shutting down).voidTasks to perform after region open to complete deploy of region on regionserverregionLock(List<RegionInfo> regionInfos, String description, Abortable abort) Master based locks on namespaces/tables/regions.booleanregisterService(com.google.protobuf.Service service) Registers a new protocol bufferServicesubclass as a coprocessor endpoint to be available for handlingbooleanreportFileArchivalForQuotas(TableName tableName, Collection<Map.Entry<String, Long>> archivedFiles) Reports a collection of files, and their sizes, that belonged to the giventablewere just moved to the archive directory.booleanreportRegionSizesForQuotas(RegionSizeStore sizeStore) Reports the provided Region sizes hosted by this RegionServer to the active Master.booleanNotify master that a handler requests to change a region statevoidunassign(byte[] regionName) Unassign the given region from the current regionserver and assign it randomly.Methods inherited from interface org.apache.hadoop.hbase.regionserver.FavoredNodesForRegion
getFavoredNodesForRegion, updateRegionFavoredNodesMappingMethods inherited from interface org.apache.hadoop.hbase.regionserver.MutableOnlineRegions
addRegion, removeRegionMethods inherited from interface org.apache.hadoop.hbase.regionserver.OnlineRegions
getRegion, getRegions, getRegionsMethods inherited from interface org.apache.hadoop.hbase.Server
createConnection, getChoreService, getClusterConnection, getConfiguration, getConnection, getCoordinatedStateManager, getFileSystem, getServerName, getZooKeeper, isStopping
-
Method Details
-
getWAL
Returns the WAL for a particular region. Pass null for getting the default (common) WAL- Throws:
IOException
-
getWALs
Returns the List of WALs that are used by this server Doesn't include the meta WAL- Throws:
IOException
-
getFlushRequester
- Returns:
- Implementation of
FlushRequesteror null. Usually it will not be null unless during intialization.
-
getCompactionRequestor
- Returns:
- Implementation of
CompactionRequesteror null. Usually it will not be null unless during intialization.
-
getRegionServerAccounting
Returns the RegionServerAccounting for this Region Server -
getRegionServerRpcQuotaManager
Returns RegionServer's instance ofRegionServerRpcQuotaManager -
getSecureBulkLoadManager
Returns RegionServer's instance ofSecureBulkLoadManager -
getRegionServerSpaceQuotaManager
Returns RegionServer's instance ofRegionServerSpaceQuotaManager -
postOpenDeployTasks
Tasks to perform after region open to complete deploy of region on regionserver- Parameters:
context- the context- Throws:
IOException
-
reportRegionStateTransition
Notify master that a handler requests to change a region state -
getRpcServer
Returns a reference to the region server's RPC server -
getRegionsInTransitionInRS
ConcurrentMap<byte[],Boolean> getRegionsInTransitionInRS()Get the regions that are currently being opened or closed in the RS- Returns:
- map of regions in transition in this RS
-
getLeaseManager
Returns The RegionServer's "Leases" service -
getExecutorService
Returns hbase executor service -
getNonceManager
Only required for "old" log replay; if it's removed, remove this.- Returns:
- The RegionServer's NonceManager
-
registerService
Registers a new protocol bufferServicesubclass as a coprocessor endpoint to be available for handling- Parameters:
service- theServicesubclass instance to expose as a coprocessor endpoint- Returns:
trueif the registration was successful,false
-
getHeapMemoryManager
Returns heap memory manager instance -
getCompactionPressure
double getCompactionPressure()- Returns:
- the max compaction pressure of all stores on this regionserver. The value should be greater than or equal to 0.0, and any value greater than 1.0 means we enter the emergency state that some stores have too many store files.
- See Also:
-
getFlushThroughputController
Returns the controller to avoid flush too fast -
getFlushPressure
Deprecated.- Returns:
- the flush pressure of all stores on this regionserver. The value should be greater than or equal to 0.0, and any value greater than 1.0 means we enter the emergency state that global memstore size already exceeds lower limit.
-
getMetrics
Returns the metrics tracker for the region server -
regionLock
EntityLock regionLock(List<RegionInfo> regionInfos, String description, Abortable abort) throws IOException Master based locks on namespaces/tables/regions.- Throws:
IOException
-
unassign
Unassign the given region from the current regionserver and assign it randomly. Could still be assigned to us. This is used to solve some tough problems for which you need to reset the state of a region. For example, if you hit FileNotFound exception and want to refresh the store file list.See HBASE-17712 for more details.
- Throws:
IOException
-
isClusterUp
boolean isClusterUp()Returns True if cluster is up; false if cluster is not up (we are shutting down). -
getTableDescriptors
Returns Return table descriptors implementation. -
getBlockCache
Returns The block cache instance. -
getMobFileCache
Returns The cache for mob files. -
getAccessChecker
Returns theAccessChecker -
getZKPermissionWatcher
ReturnsZKPermissionWatcher -
reportRegionSizesForQuotas
Reports the provided Region sizes hosted by this RegionServer to the active Master.- Parameters:
sizeStore- The sizes for Regions locally hosted.- Returns:
falseif reporting should be temporarily paused,trueotherwise.
-
reportFileArchivalForQuotas
boolean reportFileArchivalForQuotas(TableName tableName, Collection<Map.Entry<String, Long>> archivedFiles) Reports a collection of files, and their sizes, that belonged to the giventablewere just moved to the archive directory.- Parameters:
tableName- The name of the table that files previously belonged toarchivedFiles- Files and their sizes that were moved to archive- Returns:
trueif the files were successfully reported,falseotherwise.
-