@InterfaceAudience.LimitedPrivate(value="Tools") public class HRegionServer extends HasThread implements RegionServerServices, LastSequenceId
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 static String |
CLOSE |
protected ClusterConnection |
clusterConnection
Cluster connection to be shared by services.
|
protected ClusterStatusTracker |
clusterStatusTracker |
CompactSplitThread |
compactSplitThread |
protected org.apache.hadoop.conf.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 BaseCoordinatedStateManager |
csm |
protected HFileSystem |
fs |
protected boolean |
fsOk |
protected HeapMemoryManager |
hMemManager |
protected InfoServer |
infoServer |
protected Leases |
leases |
protected ReentrantReadWriteLock |
lock |
static org.apache.commons.logging.Log |
LOG |
protected MetaTableLocator |
metaTableLocator |
protected Map<String,org.apache.hadoop.hbase.regionserver.HRegionServer.MovedRegionInfo> |
movedRegions |
protected int |
msgInterval |
protected int |
numRegionsToReport |
protected Map<String,Region> |
onlineRegions
Map of regions currently being served by this region server.
|
protected static String |
OPEN |
protected Map<String,Region> |
recoveringRegions
Set of regions currently being in recovering state which means it can accept writes(edits from
previous failed region server) but not reads.
|
protected Map<String,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 String |
REGIONSERVER
region server process name
|
protected ConcurrentMap<byte[],Boolean> |
regionsInTransitionInRS |
protected ReplicationSinkService |
replicationSinkHandler |
protected ReplicationSourceService |
replicationSourceHandler |
protected RSRpcServices |
rpcServices |
protected ServerName |
serverName
The server name the Master sees us as.
|
protected ExecutorService |
service |
protected Sleeper |
sleeper |
protected long |
startcode
This servers startcode.
|
protected TableDescriptors |
tableDescriptors
Go here to get table descriptors.
|
protected TableLockManager |
tableLockManager |
protected int |
threadWakeFrequency |
protected WALFactory |
walFactory |
protected ZooKeeperWatcher |
zooKeeper |
Constructor and Description |
---|
HRegionServer(org.apache.hadoop.conf.Configuration conf)
Starts a HRegionServer at the default location.
|
HRegionServer(org.apache.hadoop.conf.Configuration conf,
CoordinatedStateManager csm)
Starts a HRegionServer at the default location
|
Modifier and Type | Method and Description |
---|---|
void |
abort(String reason) |
void |
abort(String reason,
Throwable cause)
Cause the server to exit without closing the regions it is serving, the log
it is using and without notifying the master.
|
protected void |
addToMovedRegions(String encodedName,
ServerName destination,
long closeSeqNum) |
void |
addToOnlineRegions(Region region)
Add to online regions.
|
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.
|
protected void |
closeAllRegions(boolean abort)
Closes all regions.
|
protected boolean |
closeRegion(String encodedName,
boolean abort,
CloseRegionCoordination.CloseRegionDetails crd,
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(Class<? extends HRegionServer> regionServerClass,
org.apache.hadoop.conf.Configuration conf2,
CoordinatedStateManager cp)
Utility for constructing an instance of the passed HRegionServer class.
|
protected IOException |
convertThrowableToIOE(Throwable t,
String msg) |
protected ClusterConnection |
createClusterConnection()
Create a 'smarter' HConnection, one that is capable of by-passing RPC if the request is to
the local server.
|
ClusterStatusProtos.RegionLoad |
createRegionLoad(String encodedRegionName) |
protected ServerName |
createRegionServerStatusStub()
Get the current master from ZooKeeper and open the RPC connection to it.
|
protected RSRpcServices |
createRpcServices() |
protected void |
doMetrics() |
protected LogRoller |
ensureMetaWALRoller()
We initialize the roller for the wal that handles meta lazily
since we don't know if this regionserver will handle it.
|
ClientProtos.CoprocessorServiceResponse |
execRegionServerService(com.google.protobuf.RpcController controller,
ClientProtos.CoprocessorServiceRequest serviceRequest) |
CacheConfig |
getCacheConfig() |
ChoreService |
getChoreService() |
String |
getClusterId() |
double |
getCompactionPressure() |
CompactionRequestor |
getCompactionRequester() |
CompactSplitThread |
getCompactSplitThread() |
org.apache.hadoop.conf.Configuration |
getConfiguration()
Gets the configuration object for this server.
|
protected ConfigurationManager |
getConfigurationManager() |
ClusterConnection |
getConnection()
Returns a reference to the servers' cluster connection.
|
BaseCoordinatedStateManager |
getCoordinatedStateManager()
Get CoordinatedStateManager instance for this server.
|
protected Class<? extends javax.servlet.http.HttpServlet> |
getDumpServlet() |
ExecutorService |
getExecutorService() |
InetSocketAddress[] |
getFavoredNodesForRegion(String encodedRegionName)
Return the favored nodes for a region given its encoded name.
|
org.apache.hadoop.fs.FileSystem |
getFileSystem() |
FlushRequester |
getFlushRequester() |
Region |
getFromOnlineRegions(String encodedRegionName)
Return
Region instance. |
HeapMemoryManager |
getHeapMemoryManager() |
InfoServer |
getInfoServer() |
ClusterStatusProtos.RegionStoreSequenceIds |
getLastSequenceId(byte[] encodedRegionName) |
Leases |
getLeases() |
MasterAddressTracker |
getMasterAddressTracker() |
MetaTableLocator |
getMetaTableLocator()
Returns instance of
MetaTableLocator
running inside this server. |
protected HRegionInfo[] |
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() |
Region |
getOnlineRegion(byte[] regionName) |
List<Region> |
getOnlineRegions(TableName tableName)
Gets the online regions of the specified table.
|
Collection<Region> |
getOnlineRegionsLocalContext()
For tests, web ui and metrics.
|
Set<TableName> |
getOnlineTables()
Gets the online tables in this RS.
|
protected String |
getProcessName() |
Map<String,Region> |
getRecoveringRegions() |
protected Region |
getRegion(byte[] regionName)
Protected utility method for safely obtaining an HRegion handle.
|
InetSocketAddress[] |
getRegionBlockLocations(String encodedRegionName) |
protected Region |
getRegionByEncodedName(byte[] regionName,
String encodedRegionName) |
Region |
getRegionByEncodedName(String encodedRegionName) |
RegionServerAccounting |
getRegionServerAccounting() |
RegionServerCoprocessorHost |
getRegionServerCoprocessorHost() |
String[] |
getRegionServerCoprocessors() |
MetricsRegionServer |
getRegionServerMetrics() |
RegionServerQuotaManager |
getRegionServerQuotaManager() |
ConcurrentMap<byte[],Boolean> |
getRegionsInTransitionInRS()
Get the regions that are currently being opened or closed in the RS
|
protected org.apache.hadoop.fs.Path |
getRootDir() |
RpcServerInterface |
getRpcServer()
Returns a reference to the region server's RPC server
|
RSRpcServices |
getRSRpcServices() |
ServerName |
getServerName() |
long |
getStartcode() |
TableDescriptors |
getTableDescriptors() |
TableLockManager |
getTableLockManager() |
int |
getThreadWakeFrequency()
Interval at which threads should run
|
WAL |
getWAL(HRegionInfo regionInfo) |
ZooKeeperWatcher |
getZooKeeper()
Gets the ZooKeeper instance for this server.
|
protected void |
handleReportForDutyResponse(RegionServerStatusProtos.RegionServerStartupResponse c) |
boolean |
isAborted()
Check if the server or client was aborted.
|
boolean |
isOnline()
Report the status of the server.
|
boolean |
isStopped() |
boolean |
isStopping() |
protected void |
kill() |
protected void |
login(UserProvider user,
String host) |
static void |
main(String[] args) |
protected int |
movedRegionCleanerPeriod() |
void |
postOpenDeployTasks(Region r)
Tasks to perform after region open to complete deploy of region on
regionserver
|
void |
postOpenDeployTasks(RegionServerServices.PostOpenDeployContext context)
Tasks to perform after region open to complete deploy of region on
regionserver
|
boolean |
registerService(com.google.protobuf.Service instance)
Registers a new protocol buffer
Service subclass as a coprocessor endpoint to be
available for handling |
boolean |
removeFromOnlineRegions(Region r,
ServerName destination)
This method removes Region corresponding to hri from the Map of onlineRegions.
|
boolean |
reportRegionStateTransition(RegionServerServices.RegionStateTransitionContext context)
Notify master that a handler requests to change a region state
|
boolean |
reportRegionStateTransition(RegionServerStatusProtos.RegionStateTransition.TransitionCode code,
HRegionInfo... hris)
Notify master that a handler requests to change a region state
|
boolean |
reportRegionStateTransition(RegionServerStatusProtos.RegionStateTransition.TransitionCode code,
long openSeqNum,
HRegionInfo... hris)
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.
|
protected boolean |
shouldUseThisHostnameInstead() |
void |
stop(String msg)
Stop this service.
|
protected void |
stopServiceThreads()
Wait on all threads to finish.
|
String |
toString() |
protected void |
tryRegionServerReport(long reportStartTime,
long reportEndTime) |
void |
updateConfiguration()
Reload the configuration from disk.
|
void |
updateRegionFavoredNodesMapping(String encodedRegionName,
List<HBaseProtos.ServerName> favoredNodes)
Used to update the favored nodes mapping when required.
|
protected void |
waitForMasterActive() |
void |
waitForServerOnline() |
getName, getThread, interrupt, isAlive, isInterrupted, join, join, join, setDaemon, setName, setPriority, setUncaughtExceptionHandler, start
public static final org.apache.commons.logging.Log LOG
protected static final String OPEN
protected static final String CLOSE
protected final ConcurrentMap<byte[],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 CompactSplitThread compactSplitThread
protected final Map<String,Region> onlineRegions
protected final Map<String,InetSocketAddress[]> regionFavoredNodesMap
protected final Map<String,Region> recoveringRegions
protected Leases leases
protected ExecutorService service
protected volatile boolean fsOk
protected HFileSystem fs
protected final org.apache.hadoop.conf.Configuration conf
protected final ReentrantReadWriteLock lock
protected final int threadWakeFrequency
protected final int msgInterval
protected final int numRegionsToReport
protected InfoServer infoServer
public static final String REGIONSERVER
protected volatile WALFactory walFactory
protected ZooKeeperWatcher zooKeeper
protected ClusterStatusTracker clusterStatusTracker
protected final Sleeper sleeper
protected CacheConfig cacheConfig
protected ServerName serverName
protected final long startcode
protected TableLockManager tableLockManager
protected final RSRpcServices rpcServices
protected BaseCoordinatedStateManager csm
protected final ConfigurationManager configurationManager
public HRegionServer(org.apache.hadoop.conf.Configuration conf) throws IOException, InterruptedException
conf
- IOException
InterruptedException
public HRegionServer(org.apache.hadoop.conf.Configuration conf, CoordinatedStateManager csm) throws IOException, InterruptedException
conf
- csm
- implementation of CoordinatedStateManager to be usedIOException
InterruptedException
protected boolean shouldUseThisHostnameInstead()
protected void login(UserProvider user, String host) throws IOException
IOException
protected void waitForMasterActive()
protected String getProcessName()
protected boolean canCreateBaseZNode()
protected boolean canUpdateTableDescriptor()
protected RSRpcServices createRpcServices() throws IOException
IOException
protected void configureInfoServer()
protected Class<? extends javax.servlet.http.HttpServlet> getDumpServlet()
protected void doMetrics()
public boolean registerService(com.google.protobuf.Service instance)
RegionServerServices
Service
subclass as a coprocessor endpoint to be
available for handlingregisterService
in interface RegionServerServices
instance
- the Service
subclass instance to expose as a coprocessor endpointtrue
if the registration was successful, false
protected ClusterConnection createClusterConnection() throws IOException
IOException
public String getClusterId()
protected void setupClusterConnection() throws IOException
IOException
public void run()
protected void tryRegionServerReport(long reportStartTime, long reportEndTime) throws IOException
IOException
protected void handleReportForDutyResponse(RegionServerStatusProtos.RegionServerStartupResponse c) throws IOException
IOException
public RegionServerAccounting getRegionServerAccounting()
getRegionServerAccounting
in interface RegionServerServices
public TableLockManager getTableLockManager()
getTableLockManager
in interface RegionServerServices
TableLockManager
public ClusterStatusProtos.RegionLoad createRegionLoad(String encodedRegionName) throws IOException
encodedRegionName
- IOException
public boolean isOnline()
protected LogRoller ensureMetaWALRoller()
public MetricsRegionServer getRegionServerMetrics()
public MasterAddressTracker getMasterAddressTracker()
public WAL getWAL(HRegionInfo regionInfo) throws IOException
getWAL
in interface RegionServerServices
IOException
public ClusterConnection getConnection()
Server
getConnection
in interface Server
public MetaTableLocator getMetaTableLocator()
Server
MetaTableLocator
running inside this server. This MetaServerLocator is started and stopped by server, clients
shouldn't manage it's lifecycle.getMetaTableLocator
in interface Server
MetaTableLocator
associated with this server.public void waitForServerOnline()
public void postOpenDeployTasks(Region r) throws org.apache.zookeeper.KeeperException, IOException
RegionServerServices
postOpenDeployTasks
in interface RegionServerServices
r
- Region to open.org.apache.zookeeper.KeeperException
IOException
public void postOpenDeployTasks(RegionServerServices.PostOpenDeployContext context) throws org.apache.zookeeper.KeeperException, IOException
RegionServerServices
postOpenDeployTasks
in interface RegionServerServices
context
- the contextorg.apache.zookeeper.KeeperException
IOException
public boolean reportRegionStateTransition(RegionServerStatusProtos.RegionStateTransition.TransitionCode code, HRegionInfo... hris)
RegionServerServices
reportRegionStateTransition
in interface RegionServerServices
public boolean reportRegionStateTransition(RegionServerStatusProtos.RegionStateTransition.TransitionCode code, long openSeqNum, HRegionInfo... hris)
RegionServerServices
reportRegionStateTransition
in interface RegionServerServices
public boolean reportRegionStateTransition(RegionServerServices.RegionStateTransitionContext context)
RegionServerServices
reportRegionStateTransition
in interface RegionServerServices
public RpcServerInterface getRpcServer()
RegionServerServices
getRpcServer
in interface RegionServerServices
public RSRpcServices getRSRpcServices()
public void abort(String reason, Throwable cause)
public void abort(String reason)
abort(String, Throwable)
public boolean isAborted()
Abortable
protected void kill()
protected void sendShutdownInterrupt()
protected void stopServiceThreads()
protected ServerName createRegionServerStatusStub()
public ClusterStatusProtos.RegionStoreSequenceIds getLastSequenceId(byte[] encodedRegionName)
getLastSequenceId
in interface LastSequenceId
encodedRegionName
- Encoded region nameprotected void closeAllRegions(boolean abort)
public InfoServer getInfoServer()
public boolean isStopped()
public boolean isStopping()
isStopping
in interface RegionServerServices
public Map<String,Region> getRecoveringRegions()
getRecoveringRegions
in interface RegionServerServices
public org.apache.hadoop.conf.Configuration getConfiguration()
Server
getConfiguration
in interface Server
public int getNumberOfOnlineRegions()
public Collection<Region> getOnlineRegionsLocalContext()
public void addToOnlineRegions(Region region)
OnlineRegions
addToOnlineRegions
in interface OnlineRegions
public long getStartcode()
public FlushRequester getFlushRequester()
getFlushRequester
in interface RegionServerServices
protected HRegionInfo[] getMostLoadedRegions()
public Leases getLeases()
getLeases
in interface RegionServerServices
protected org.apache.hadoop.fs.Path getRootDir()
public org.apache.hadoop.fs.FileSystem getFileSystem()
getFileSystem
in interface RegionServerServices
public int getThreadWakeFrequency()
public ZooKeeperWatcher getZooKeeper()
Server
getZooKeeper
in interface Server
public BaseCoordinatedStateManager getCoordinatedStateManager()
Server
getCoordinatedStateManager
in interface Server
public ServerName getServerName()
getServerName
in interface Server
public CompactionRequestor getCompactionRequester()
getCompactionRequester
in interface RegionServerServices
CompactionRequestor
or null.public RegionServerCoprocessorHost getRegionServerCoprocessorHost()
public ConcurrentMap<byte[],Boolean> getRegionsInTransitionInRS()
RegionServerServices
getRegionsInTransitionInRS
in interface RegionServerServices
public ExecutorService getExecutorService()
getExecutorService
in interface RegionServerServices
public ChoreService getChoreService()
getChoreService
in interface Server
ChoreService
instance for this serverpublic RegionServerQuotaManager getRegionServerQuotaManager()
getRegionServerQuotaManager
in interface RegionServerServices
RegionServerQuotaManager
public static HRegionServer constructRegionServer(Class<? extends HRegionServer> regionServerClass, org.apache.hadoop.conf.Configuration conf2, CoordinatedStateManager cp)
regionServerClass
- conf2
- public static void main(String[] args) throws Exception
Exception
HRegionServerCommandLine
public List<Region> getOnlineRegions(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.getOnlineRegions
in interface OnlineRegions
tableName
- tableName
public Set<TableName> getOnlineTables()
getOnlineTables
in interface RegionServerServices
public String[] getRegionServerCoprocessors()
protected boolean closeRegion(String encodedName, boolean abort, CloseRegionCoordination.CloseRegionDetails crd, ServerName sn) throws NotServingRegionException, RegionAlreadyInTransitionException
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 abortingcrd
- details about closing region coordination-coordinated taskNotServingRegionException
- if the region is not onlineRegionAlreadyInTransitionException
- if the region is already closingpublic Region getOnlineRegion(byte[] regionName)
regionName
- regionName
or null if
named region is not member of the online regions.public InetSocketAddress[] getRegionBlockLocations(String encodedRegionName)
public Region getFromOnlineRegions(String encodedRegionName)
OnlineRegions
Region
instance.
Only works if caller is in same context, in same JVM. Region is not
serializable.getFromOnlineRegions
in interface OnlineRegions
encodedRegionName
or
null if named region is not member of the online regions.public boolean removeFromOnlineRegions(Region r, ServerName destination)
OnlineRegions
removeFromOnlineRegions
in interface OnlineRegions
r
- Region to remove.destination
- Destination, if any, null otherwise.protected Region getRegion(byte[] regionName) throws NotServingRegionException
regionName
- Name of online Region
to returnRegion
for regionName
NotServingRegionException
public Region getRegionByEncodedName(String encodedRegionName) throws NotServingRegionException
NotServingRegionException
protected Region getRegionByEncodedName(byte[] regionName, String encodedRegionName) throws NotServingRegionException
NotServingRegionException
protected IOException convertThrowableToIOE(Throwable t, String msg)
public boolean checkFileSystem()
public void updateRegionFavoredNodesMapping(String encodedRegionName, List<HBaseProtos.ServerName> favoredNodes)
FavoredNodesForRegion
updateRegionFavoredNodesMapping
in interface FavoredNodesForRegion
public InetSocketAddress[] getFavoredNodesForRegion(String encodedRegionName)
regionFavoredNodesMap
on why it is InetSocketAddress[]getFavoredNodesForRegion
in interface FavoredNodesForRegion
encodedRegionName
- public ServerNonceManager getNonceManager()
RegionServerServices
getNonceManager
in interface RegionServerServices
protected void addToMovedRegions(String encodedName, ServerName destination, long closeSeqNum)
protected void cleanMovedRegions()
protected int movedRegionCleanerPeriod()
public CompactSplitThread getCompactSplitThread()
CompactSplitThread
for the serverspublic ClientProtos.CoprocessorServiceResponse execRegionServerService(com.google.protobuf.RpcController controller, ClientProtos.CoprocessorServiceRequest serviceRequest) throws com.google.protobuf.ServiceException
com.google.protobuf.ServiceException
public CacheConfig getCacheConfig()
protected ConfigurationManager getConfigurationManager()
public TableDescriptors getTableDescriptors()
public void updateConfiguration()
public HeapMemoryManager getHeapMemoryManager()
getHeapMemoryManager
in interface RegionServerServices
public double getCompactionPressure()
getCompactionPressure
in interface RegionServerServices
Store.getCompactionPressure()
Copyright © 2007-2016 The Apache Software Foundation. All Rights Reserved.