public class HRegionServer extends HasThread implements RegionServerServices, LastSequenceId, ConfigurationObserver
| Modifier and Type | Class and Description |
|---|---|
protected static class |
HRegionServer.MovedRegionsCleaner
Creates a Chore thread to clean the moved region cache.
|
RegionServerServices.PostOpenDeployContext, RegionServerServices.RegionStateTransitionContext| Modifier and Type | Field and Description |
|---|---|
protected CacheConfig |
cacheConfig |
protected org.apache.hadoop.hbase.regionserver.MemStoreFlusher |
cacheFlusher |
protected ClusterConnection |
clusterConnection
Cluster connection to be shared by services.
|
protected java.lang.String |
clusterId
Unique identifier for the cluster we are a part of.
|
protected ClusterStatusTracker |
clusterStatusTracker |
CompactSplit |
compactSplitThread |
protected Configuration |
conf |
protected ConfigurationManager |
configurationManager
Configuration manager is used to register/deregister and notify the configuration observers
when the regionserver is notified that there was a change in the on disk configs.
|
protected CoordinatedStateManager |
csm |
protected ExecutorService |
executorService |
protected HFileSystem |
fs |
protected boolean |
fsOk |
protected FileSystemUtilizationChore |
fsUtilizationChore |
protected HeapMemoryManager |
hMemManager |
protected InfoServer |
infoServer |
protected Leases |
leases |
protected java.util.concurrent.locks.ReentrantReadWriteLock |
lock |
protected static java.lang.String |
MASTER_HOSTNAME_KEY |
protected MetaTableLocator |
metaTableLocator |
protected java.util.Map<java.lang.String,org.apache.hadoop.hbase.regionserver.HRegionServer.MovedRegionInfo> |
movedRegions |
protected int |
msgInterval |
protected int |
numRegionsToReport |
protected java.util.Map<java.lang.String,HRegion> |
onlineRegions
Map of regions currently being served by this region server.
|
protected java.util.Map<java.lang.String,java.net.InetSocketAddress[]> |
regionFavoredNodesMap
Map of encoded region names to the DataNode locations they should be hosted on
We store the value as InetSocketAddress since this is used only in HDFS
API (create() that takes favored nodes as hints for placing file blocks).
|
static java.lang.String |
REGIONSERVER
region server process name
|
protected java.util.concurrent.ConcurrentMap<byte[],java.lang.Boolean> |
regionsInTransitionInRS |
protected ReplicationSinkService |
replicationSinkHandler |
protected ReplicationSourceService |
replicationSourceHandler |
protected RSRpcServices |
rpcServices |
protected SecureBulkLoadManager |
secureBulkLoadManager |
protected ServerName |
serverName
The server name the Master sees us as.
|
protected Sleeper |
sleeper |
protected long |
startcode
This servers startcode.
|
protected TableDescriptors |
tableDescriptors
Go here to get table descriptors.
|
static boolean |
TEST_SKIP_REPORTING_TRANSITION
For testing only! Set to true to skip notifying region assignment to master .
|
protected int |
threadWakeFrequency |
protected java.lang.String |
useThisHostnameInstead |
protected WALFactory |
walFactory |
protected HFileSystem |
walFs |
protected LogRoller |
walRoller |
protected ZKWatcher |
zooKeeper |
| Constructor and Description |
|---|
HRegionServer(Configuration conf)
Starts a HRegionServer at the default location
|
| Modifier and Type | Method and Description |
|---|---|
void |
abort(java.lang.String reason) |
void |
abort(java.lang.String reason,
java.lang.Throwable cause)
Cause the server to exit without closing the regions it is serving, the log
it is using and without notifying the master.
|
void |
addRegion(HRegion region)
Add to online regions.
|
protected void |
addToMovedRegions(java.lang.String encodedName,
ServerName destination,
long closeSeqNum) |
protected boolean |
canCreateBaseZNode() |
protected boolean |
canUpdateTableDescriptor() |
boolean |
checkFileSystem()
Checks to see if the file system is still accessible.
|
protected void |
cleanMovedRegions()
Remove the expired entries from the moved regions list.
|
CacheEvictionStats |
clearRegionBlockCache(Region region) |
protected void |
closeAllRegions(boolean abort)
Closes all regions.
|
protected boolean |
closeAndOfflineRegionForSplitOrMerge(java.util.List<java.lang.String> regionEncodedName)
Close and offline the region for split or merge
|
protected boolean |
closeRegion(java.lang.String encodedName,
boolean abort,
ServerName sn)
Close asynchronously a region, can be called from the master or internally by the regionserver
when stopping.
|
protected void |
configureInfoServer() |
static HRegionServer |
constructRegionServer(java.lang.Class<? extends HRegionServer> regionServerClass,
Configuration conf2)
Utility for constructing an instance of the passed HRegionServer class.
|
protected java.io.IOException |
convertThrowableToIOE(java.lang.Throwable t,
java.lang.String msg) |
protected ClusterConnection |
createClusterConnection()
Create a 'smarter' Connection, one that is capable of by-passing RPC if the request is to
the local server; i.e.
|
Connection |
createConnection(Configuration conf) |
RegionLoad |
createRegionLoad(java.lang.String encodedRegionName) |
protected ServerName |
createRegionServerStatusStub()
Get the current master from ZooKeeper and open the RPC connection to it.
|
protected ServerName |
createRegionServerStatusStub(boolean refresh)
Get the current master from ZooKeeper and open the RPC connection to it.
|
protected RSRpcServices |
createRpcServices() |
CoprocessorServiceResponse |
execRegionServerService(RpcController controller,
CoprocessorServiceRequest serviceRequest) |
CacheConfig |
getCacheConfig() |
ChoreService |
getChoreService() |
ClusterConnection |
getClusterConnection()
Returns a reference to the servers' cluster connection.
|
java.lang.String |
getClusterId() |
double |
getCompactionPressure() |
CompactionRequester |
getCompactionRequestor() |
CompactSplit |
getCompactSplitThread() |
Configuration |
getConfiguration()
Gets the configuration object for this server.
|
protected ConfigurationManager |
getConfigurationManager() |
Connection |
getConnection()
Returns a reference to the servers' connection.
|
CoordinatedStateManager |
getCoordinatedStateManager()
Get CoordinatedStateManager instance for this server.
|
protected java.lang.Class<? extends HttpServlet> |
getDumpServlet() |
NettyEventLoopGroupConfig |
getEventLoopGroupConfig() |
ExecutorService |
getExecutorService() |
java.net.InetSocketAddress[] |
getFavoredNodesForRegion(java.lang.String encodedRegionName)
Return the favored nodes for a region given its encoded name.
|
FileSystem |
getFileSystem() |
double |
getFlushPressure() |
FlushRequester |
getFlushRequester() |
ThroughputController |
getFlushThroughputController() |
protected TableDescriptors |
getFsTableDescriptors() |
HeapMemoryManager |
getHeapMemoryManager() |
InfoServer |
getInfoServer() |
RegionStoreSequenceIds |
getLastSequenceId(byte[] encodedRegionName) |
Leases |
getLeases() |
MasterAddressTracker |
getMasterAddressTracker() |
MetaTableLocator |
getMetaTableLocator()
Returns instance of
org.apache.hadoop.hbase.zookeeper.MetaTableLocator
running inside this server. |
protected java.util.function.Function<TableDescriptorBuilder,TableDescriptorBuilder> |
getMetaTableObserver() |
MetricsRegionServer |
getMetrics() |
protected RegionInfo[] |
getMostLoadedRegions()
Get the top N most loaded regions this server is serving so we can tell the
master which regions it can reallocate if we're overloaded.
|
ServerNonceManager |
getNonceManager()
Only required for "old" log replay; if it's removed, remove this.
|
int |
getNumberOfOnlineRegions() |
HRegion |
getOnlineRegion(byte[] regionName) |
java.util.Collection<HRegion> |
getOnlineRegionsLocalContext()
For tests, web ui and metrics.
|
java.util.Set<TableName> |
getOnlineTables()
Gets the online tables in this RS.
|
protected java.lang.String |
getProcessName() |
protected HRegion |
getRegion(byte[] regionName)
Protected Utility method for safely obtaining an HRegion handle.
|
HRegion |
getRegion(java.lang.String encodedRegionName)
Return
Region instance. |
java.net.InetSocketAddress[] |
getRegionBlockLocations(java.lang.String encodedRegionName) |
protected HRegion |
getRegionByEncodedName(byte[] regionName,
java.lang.String encodedRegionName) |
HRegion |
getRegionByEncodedName(java.lang.String encodedRegionName) |
java.util.List<HRegion> |
getRegions()
Get all online regions in this RS.
|
java.util.List<HRegion> |
getRegions(TableName tableName)
Gets the online regions of the specified table.
|
RegionServerAccounting |
getRegionServerAccounting() |
RegionServerCoprocessorHost |
getRegionServerCoprocessorHost() |
java.lang.String[] |
getRegionServerCoprocessors() |
MetricsRegionServer |
getRegionServerMetrics() |
RegionServerRpcQuotaManager |
getRegionServerRpcQuotaManager() |
RegionServerSpaceQuotaManager |
getRegionServerSpaceQuotaManager() |
java.util.concurrent.ConcurrentMap<byte[],java.lang.Boolean> |
getRegionsInTransitionInRS()
Get the regions that are currently being opened or closed in the RS
|
ReplicationSourceService |
getReplicationSourceService() |
protected Path |
getRootDir() |
RpcServerInterface |
getRpcServer()
Returns a reference to the region server's RPC server
|
RSRpcServices |
getRSRpcServices() |
SecureBulkLoadManager |
getSecureBulkLoadManager() |
ServerName |
getServerName() |
long |
getStartcode() |
TableDescriptors |
getTableDescriptors() |
int |
getThreadWakeFrequency()
Interval at which threads should run
|
protected java.lang.String |
getUseThisHostnameInstead(Configuration conf) |
WAL |
getWAL(RegionInfo regionInfo) |
protected FileSystem |
getWALFileSystem() |
LogRoller |
getWalRoller() |
protected Path |
getWALRootDir() |
java.util.List<WAL> |
getWALs() |
ZKWatcher |
getZooKeeper()
Gets the ZooKeeper instance for this server.
|
protected void |
handleReportForDutyResponse(RegionServerStartupResponse c) |
protected void |
initializeMemStoreChunkCreator() |
boolean |
isAborted() |
boolean |
isClusterUp() |
boolean |
isOnline()
Report the status of the server.
|
boolean |
isStopped() |
boolean |
isStopping() |
protected void |
kill() |
protected void |
login(UserProvider user,
java.lang.String host) |
static void |
main(java.lang.String[] args) |
protected int |
movedRegionCleanerPeriod() |
void |
onConfigurationChange(Configuration newConf)
This method would be called by the
ConfigurationManager
object when the Configuration object is reloaded from disk. |
void |
postOpenDeployTasks(RegionServerServices.PostOpenDeployContext context)
Tasks to perform after region open to complete deploy of region on
regionserver
|
EntityLock |
regionLock(java.util.List<RegionInfo> regionInfos,
java.lang.String description,
Abortable abort)
Master based locks on namespaces/tables/regions.
|
boolean |
registerService(com.google.protobuf.Service instance)
Registers a new protocol buffer
Service subclass as a coprocessor endpoint to be
available for handling |
boolean |
removeRegion(HRegion r,
ServerName destination)
Removes the given Region from the list of onlineRegions.
|
boolean |
reportFileArchivalForQuotas(TableName tableName,
java.util.Collection<java.util.Map.Entry<java.lang.String,java.lang.Long>> archivedFiles)
Reports a collection of files, and their sizes, that belonged to the given
table were
just moved to the archive directory. |
boolean |
reportRegionSizesForQuotas(RegionSizeStore regionSizeStore)
Reports the given map of Regions and their size on the filesystem to the active Master.
|
boolean |
reportRegionStateTransition(RegionServerServices.RegionStateTransitionContext context)
Notify master that a handler requests to change a region state
|
void |
run()
The HRegionServer sticks in this loop until closed.
|
protected void |
sendShutdownInterrupt()
Called on stop/abort before closing the cluster connection and meta locator.
|
protected void |
setupClusterConnection()
Setup our cluster connection if not already initialized.
|
void |
stop(java.lang.String msg) |
void |
stop(java.lang.String msg,
boolean force,
User user)
Stops the regionserver.
|
protected void |
stopServiceThreads()
Wait on all threads to finish.
|
java.lang.String |
toString() |
protected void |
tryRegionServerReport(long reportStartTime,
long reportEndTime) |
void |
unassign(byte[] regionName)
Unassign the given region from the current regionserver and assign it randomly.
|
void |
updateConfiguration()
Reload the configuration from disk.
|
void |
updateRegionFavoredNodesMapping(java.lang.String encodedRegionName,
java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName> favoredNodes)
Used to update the favored nodes mapping when required.
|
protected void |
waitForMasterActive() |
void |
waitForServerOnline() |
boolean |
walRollRequestFinished()
For testing
|
public static boolean TEST_SKIP_REPORTING_TRANSITION
protected final java.util.concurrent.ConcurrentMap<byte[],java.lang.Boolean> regionsInTransitionInRS
protected org.apache.hadoop.hbase.regionserver.MemStoreFlusher cacheFlusher
protected HeapMemoryManager hMemManager
protected ClusterConnection clusterConnection
protected MetaTableLocator metaTableLocator
protected TableDescriptors tableDescriptors
protected ReplicationSourceService replicationSourceHandler
protected ReplicationSinkService replicationSinkHandler
public CompactSplit compactSplitThread
protected final java.util.Map<java.lang.String,HRegion> onlineRegions
protected final java.util.Map<java.lang.String,java.net.InetSocketAddress[]> regionFavoredNodesMap
protected Leases leases
protected ExecutorService executorService
protected volatile boolean fsOk
protected HFileSystem fs
protected HFileSystem walFs
protected final Configuration conf
protected final java.util.concurrent.locks.ReentrantReadWriteLock lock
protected final int threadWakeFrequency
protected final int msgInterval
protected final int numRegionsToReport
protected InfoServer infoServer
public static final java.lang.String REGIONSERVER
protected volatile WALFactory walFactory
protected LogRoller walRoller
protected final ZKWatcher zooKeeper
protected final ClusterStatusTracker clusterStatusTracker
protected final Sleeper sleeper
protected CacheConfig cacheConfig
protected ServerName serverName
protected java.lang.String useThisHostnameInstead
protected static final java.lang.String MASTER_HOSTNAME_KEY
protected final long startcode
protected java.lang.String clusterId
protected final RSRpcServices rpcServices
protected CoordinatedStateManager csm
protected final ConfigurationManager configurationManager
protected SecureBulkLoadManager secureBulkLoadManager
protected FileSystemUtilizationChore fsUtilizationChore
protected java.util.Map<java.lang.String,org.apache.hadoop.hbase.regionserver.HRegionServer.MovedRegionInfo> movedRegions
public HRegionServer(Configuration conf)
throws java.io.IOException
java.io.IOExceptionprotected java.lang.String getUseThisHostnameInstead(Configuration conf)
throws java.io.IOException
java.io.IOExceptionprotected TableDescriptors getFsTableDescriptors()
throws java.io.IOException
java.io.IOExceptionprotected java.util.function.Function<TableDescriptorBuilder,TableDescriptorBuilder> getMetaTableObserver()
protected void login(UserProvider user,
java.lang.String host)
throws java.io.IOException
java.io.IOExceptionprotected void waitForMasterActive()
protected java.lang.String getProcessName()
protected boolean canCreateBaseZNode()
protected boolean canUpdateTableDescriptor()
protected RSRpcServices createRpcServices() throws java.io.IOException
java.io.IOExceptionprotected void configureInfoServer()
protected java.lang.Class<? extends HttpServlet> getDumpServlet()
public boolean registerService(com.google.protobuf.Service instance)
RegionServerServicesService subclass as a coprocessor endpoint to be
available for handlingregisterService in interface RegionServerServicesinstance - the Service subclass instance to expose as a coprocessor endpointtrue if the registration was successful, falseprotected ClusterConnection createClusterConnection()
throws java.io.IOException
java.io.IOExceptionpublic java.lang.String getClusterId()
protected void setupClusterConnection()
throws java.io.IOException
java.io.IOExceptionpublic boolean isClusterUp()
isClusterUp in interface RegionServerServicespublic void run()
protected void tryRegionServerReport(long reportStartTime,
long reportEndTime)
throws java.io.IOException
java.io.IOExceptionpublic boolean reportRegionSizesForQuotas(RegionSizeStore regionSizeStore)
reportRegionSizesForQuotas in interface RegionServerServicesregionSizeStore - The store containing region sizesprotected void handleReportForDutyResponse(RegionServerStartupResponse c)
throws java.io.IOException
java.io.IOExceptionprotected void initializeMemStoreChunkCreator()
public RegionServerAccounting getRegionServerAccounting()
getRegionServerAccounting in interface RegionServerServicespublic RegionLoad createRegionLoad(java.lang.String encodedRegionName)
throws java.io.IOException
encodedRegionName - java.io.IOExceptionpublic boolean isOnline()
public MetricsRegionServer getRegionServerMetrics()
public MasterAddressTracker getMasterAddressTracker()
public java.util.List<WAL> getWALs()
throws java.io.IOException
getWALs in interface RegionServerServicesjava.io.IOExceptionpublic WAL getWAL(RegionInfo regionInfo)
throws java.io.IOException
getWAL in interface RegionServerServicesjava.io.IOExceptionpublic LogRoller getWalRoller()
public Connection getConnection()
ServergetConnection in interface Serverpublic ClusterConnection getClusterConnection()
ServerServer.getConnection().
Important note: this method returns a reference to Connection which is managed
by Server itself, so callers must NOT attempt to close connection obtained.getClusterConnection in interface Serverpublic MetaTableLocator getMetaTableLocator()
Serverorg.apache.hadoop.hbase.zookeeper.MetaTableLocator
running inside this server. This MetaServerLocator is started and stopped by server, clients
shouldn't manage it's lifecycle.getMetaTableLocator in interface ServerMetaTableLocator associated with this server.public void stop(java.lang.String msg)
public void stop(java.lang.String msg,
boolean force,
User user)
msg - Status messageforce - True if this is a regionserver abortuser - The user executing the stop request, or null if no user is associatedpublic void waitForServerOnline()
public void postOpenDeployTasks(RegionServerServices.PostOpenDeployContext context) throws KeeperException, java.io.IOException
RegionServerServicespostOpenDeployTasks in interface RegionServerServicescontext - the contextKeeperExceptionjava.io.IOExceptionpublic boolean reportRegionStateTransition(RegionServerServices.RegionStateTransitionContext context)
RegionServerServicesreportRegionStateTransition in interface RegionServerServicespublic RpcServerInterface getRpcServer()
RegionServerServicesgetRpcServer in interface RegionServerServicespublic RSRpcServices getRSRpcServices()
public void abort(java.lang.String reason,
java.lang.Throwable cause)
reason - the reason we are abortingcause - the exception that caused the abort, or nullpublic void abort(java.lang.String reason)
abort(String, Throwable)public boolean isAborted()
protected void kill()
protected void sendShutdownInterrupt()
protected void stopServiceThreads()
public ReplicationSourceService getReplicationSourceService()
protected ServerName createRegionServerStatusStub()
protected ServerName createRegionServerStatusStub(boolean refresh)
refresh - If true then master address will be read from ZK, otherwise use cached datapublic RegionStoreSequenceIds getLastSequenceId(byte[] encodedRegionName)
getLastSequenceId in interface LastSequenceIdencodedRegionName - Encoded region nameprotected void closeAllRegions(boolean abort)
public InfoServer getInfoServer()
public boolean isStopped()
public boolean isStopping()
isStopping in interface Serverpublic Configuration getConfiguration()
ServergetConfiguration in interface Serverpublic int getNumberOfOnlineRegions()
public java.util.Collection<HRegion> getOnlineRegionsLocalContext()
public void addRegion(HRegion region)
MutableOnlineRegionsaddRegion in interface MutableOnlineRegionspublic long getStartcode()
public FlushRequester getFlushRequester()
getFlushRequester in interface RegionServerServicespublic CompactionRequester getCompactionRequestor()
getCompactionRequestor in interface RegionServerServicesCompactionRequester or null. Usually it will not be null
unless during intialization.protected RegionInfo[] getMostLoadedRegions()
public Leases getLeases()
getLeases in interface RegionServerServicesprotected Path getRootDir()
public FileSystem getFileSystem()
getFileSystem in interface Serverprotected Path getWALRootDir()
protected FileSystem getWALFileSystem()
public java.lang.String toString()
public int getThreadWakeFrequency()
public ZKWatcher getZooKeeper()
ServergetZooKeeper in interface Serverpublic CoordinatedStateManager getCoordinatedStateManager()
ServergetCoordinatedStateManager in interface Serverpublic ServerName getServerName()
getServerName in interface Serverpublic RegionServerCoprocessorHost getRegionServerCoprocessorHost()
public java.util.concurrent.ConcurrentMap<byte[],java.lang.Boolean> getRegionsInTransitionInRS()
RegionServerServicesgetRegionsInTransitionInRS in interface RegionServerServicespublic ExecutorService getExecutorService()
getExecutorService in interface RegionServerServicespublic ChoreService getChoreService()
getChoreService in interface ServerChoreService instance for this serverpublic RegionServerRpcQuotaManager getRegionServerRpcQuotaManager()
getRegionServerRpcQuotaManager in interface RegionServerServicesRegionServerRpcQuotaManagerpublic static HRegionServer constructRegionServer(java.lang.Class<? extends HRegionServer> regionServerClass, Configuration conf2)
regionServerClass - conf2 - public static void main(java.lang.String[] args)
throws java.lang.Exception
java.lang.ExceptionHRegionServerCommandLinepublic java.util.List<HRegion> getRegions(TableName tableName)
hbase:meta.
Only returns online regions. If a region on this table has been
closed during a disable, etc., it will not be included in the returned list.
So, the returned list may not necessarily be ALL regions in this table, its
all the ONLINE regions in the table.getRegions in interface OnlineRegionstableName - tableNamepublic java.util.List<HRegion> getRegions()
OnlineRegionsgetRegions in interface OnlineRegionspublic java.util.Set<TableName> getOnlineTables()
public java.lang.String[] getRegionServerCoprocessors()
protected boolean closeRegion(java.lang.String encodedName,
boolean abort,
ServerName sn)
throws NotServingRegionException
If an opening was in progress, this method will cancel it, but will not start a new close. The coprocessors are not called in this case. A NotServingRegionException exception is thrown.
If a close was in progress, this new request will be ignored, and an exception thrown.
encodedName - Region to closeabort - True if we are abortingNotServingRegionException - if the region is not onlineprotected boolean closeAndOfflineRegionForSplitOrMerge(java.util.List<java.lang.String> regionEncodedName)
throws java.io.IOException
regionEncodedName - the name of the region(s) to closejava.io.IOExceptionpublic HRegion getOnlineRegion(byte[] regionName)
regionName - regionName or null if
named region is not member of the online regions.public java.net.InetSocketAddress[] getRegionBlockLocations(java.lang.String encodedRegionName)
public HRegion getRegion(java.lang.String encodedRegionName)
OnlineRegionsRegion instance.
Only works if caller is in same context, in same JVM. Region is not
serializable.getRegion in interface OnlineRegionsencodedRegionName or
null if named region is not member of the online regions.public boolean removeRegion(HRegion r, ServerName destination)
MutableOnlineRegionsremoveRegion in interface MutableOnlineRegionsr - Region to remove.destination - Destination, if any, null otherwise.protected HRegion getRegion(byte[] regionName) throws NotServingRegionException
public HRegion getRegionByEncodedName(java.lang.String encodedRegionName) throws NotServingRegionException
NotServingRegionExceptionprotected HRegion getRegionByEncodedName(byte[] regionName, java.lang.String encodedRegionName) throws NotServingRegionException
NotServingRegionExceptionprotected java.io.IOException convertThrowableToIOE(java.lang.Throwable t,
java.lang.String msg)
public boolean checkFileSystem()
public void updateRegionFavoredNodesMapping(java.lang.String encodedRegionName,
java.util.List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName> favoredNodes)
FavoredNodesForRegionupdateRegionFavoredNodesMapping in interface FavoredNodesForRegionpublic java.net.InetSocketAddress[] getFavoredNodesForRegion(java.lang.String encodedRegionName)
regionFavoredNodesMap on why it is InetSocketAddress[]getFavoredNodesForRegion in interface FavoredNodesForRegionencodedRegionName - public ServerNonceManager getNonceManager()
RegionServerServicesgetNonceManager in interface RegionServerServicesprotected void addToMovedRegions(java.lang.String encodedName,
ServerName destination,
long closeSeqNum)
protected void cleanMovedRegions()
protected int movedRegionCleanerPeriod()
public CompactSplit getCompactSplitThread()
CompactSplit for the serverspublic CoprocessorServiceResponse execRegionServerService(RpcController controller,
CoprocessorServiceRequest serviceRequest)
throws ServiceException
ServiceExceptionpublic CacheConfig getCacheConfig()
protected ConfigurationManager getConfigurationManager()
public TableDescriptors getTableDescriptors()
public void updateConfiguration()
public CacheEvictionStats clearRegionBlockCache(Region region)
public double getCompactionPressure()
getCompactionPressure in interface RegionServerServicesStore.getCompactionPressure()public HeapMemoryManager getHeapMemoryManager()
getHeapMemoryManager in interface RegionServerServicespublic boolean walRollRequestFinished()
public ThroughputController getFlushThroughputController()
getFlushThroughputController in interface RegionServerServicespublic double getFlushPressure()
getFlushPressure in interface RegionServerServicespublic void onConfigurationChange(Configuration newConf)
ConfigurationObserverConfigurationManager
object when the Configuration object is reloaded from disk.onConfigurationChange in interface ConfigurationObserverpublic MetricsRegionServer getMetrics()
getMetrics in interface RegionServerServicespublic SecureBulkLoadManager getSecureBulkLoadManager()
getSecureBulkLoadManager in interface RegionServerServicesSecureBulkLoadManagerpublic EntityLock regionLock(java.util.List<RegionInfo> regionInfos, java.lang.String description, Abortable abort) throws java.io.IOException
RegionServerServicesregionLock in interface RegionServerServicesjava.io.IOExceptionpublic void unassign(byte[] regionName)
throws java.io.IOException
RegionServerServicesSee HBASE-17712 for more details.
unassign in interface RegionServerServicesjava.io.IOExceptionpublic RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager()
getRegionServerSpaceQuotaManager in interface RegionServerServicesRegionServerSpaceQuotaManagerpublic boolean reportFileArchivalForQuotas(TableName tableName,
java.util.Collection<java.util.Map.Entry<java.lang.String,java.lang.Long>> archivedFiles)
RegionServerServicestable were
just moved to the archive directory.reportFileArchivalForQuotas in interface RegionServerServicestableName - The name of the table that files previously belonged toarchivedFiles - Files and their sizes that were moved to archivetrue if the files were successfully reported, false otherwise.public NettyEventLoopGroupConfig getEventLoopGroupConfig()
public Connection createConnection(Configuration conf)
throws java.io.IOException
createConnection in interface Serverjava.io.IOException