@InterfaceAudience.LimitedPrivate(value="Tools") public class HMaster extends HRegionServer implements MasterServices, Server
The Master can be asked shutdown the cluster. See shutdown()
. In
this case it will tell all regionservers to go down and then wait on them
all reporting in that they are down. This master will then shut itself down.
You can also shutdown just this master. Call stopMaster()
.
Watcher
Modifier and Type | Class and Description |
---|---|
static class |
HMaster.RedirectServlet |
HRegionServer.MovedRegionsCleaner
RegionServerServices.PostOpenDeployContext, RegionServerServices.RegionStateTransitionContext
Modifier and Type | Field and Description |
---|---|
static String |
MASTER |
cacheConfig, cacheFlusher, CLOSE, clusterConnection, clusterStatusTracker, compactSplitThread, conf, configurationManager, csm, fs, fsOk, hMemManager, infoServer, leases, lock, metaTableLocator, movedRegions, msgInterval, numRegionsToReport, onlineRegions, OPEN, recoveringRegions, regionFavoredNodesMap, REGIONSERVER, regionsInTransitionInRS, replicationSinkHandler, replicationSourceHandler, rpcServices, serverName, service, sleeper, startcode, tableDescriptors, tableLockManager, threadWakeFrequency, walFactory, zooKeeper
Constructor and Description |
---|
HMaster(org.apache.hadoop.conf.Configuration conf,
CoordinatedStateManager csm)
Initializes the HMaster.
|
Modifier and Type | Method and Description |
---|---|
void |
abort(String msg,
Throwable t)
Cause the server to exit without closing the regions it is serving, the log
it is using and without notifying the master.
|
boolean |
abortProcedure(long procId,
boolean mayInterruptIfRunning)
Abort a procedure.
|
void |
addColumn(TableName tableName,
HColumnDescriptor columnDescriptor,
long nonceGroup,
long nonce)
Add a new column to an existing table
|
void |
assignRegion(HRegionInfo hri) |
boolean |
balance() |
boolean |
balanceSwitch(boolean b) |
protected boolean |
canCreateBaseZNode() |
protected boolean |
canUpdateTableDescriptor() |
void |
checkTableModifiable(TableName tableName)
Check table is modifiable; i.e.
|
protected void |
configureInfoServer() |
static HMaster |
constructMaster(Class<? extends HMaster> masterClass,
org.apache.hadoop.conf.Configuration conf,
CoordinatedStateManager cp)
Utility for constructing an instance of the passed HMaster class.
|
void |
createNamespace(NamespaceDescriptor descriptor)
Create a new namespace
|
protected RSRpcServices |
createRpcServices() |
long |
createTable(HTableDescriptor hTableDescriptor,
byte[][] splitKeys,
long nonceGroup,
long nonce)
Create a table using the given table definition.
|
void |
deleteColumn(TableName tableName,
byte[] columnName,
long nonceGroup,
long nonce)
Delete a column from an existing table
|
void |
deleteNamespace(String name)
Delete an existing namespace.
|
long |
deleteTable(TableName tableName,
long nonceGroup,
long nonce)
Delete a table
|
long |
disableTable(TableName tableName,
long nonceGroup,
long nonce)
Disable an existing table
|
void |
dispatchMergingRegions(HRegionInfo region_a,
HRegionInfo region_b,
boolean forcible)
Merge two regions.
|
protected void |
doMetrics()
Emit the HMaster metrics, such as region in transition metrics.
|
long |
enableTable(TableName tableName,
long nonceGroup,
long nonce)
Enable an existing table
|
protected void |
ensureNamespaceExists(String name)
Ensure that the specified namespace exists, otherwise throws a NamespaceNotFoundException
|
AssignmentManager |
getAssignmentManager() |
double |
getAverageLoad()
Compute the average load across all region servers.
|
ClusterStatus |
getClusterStatus() |
protected Class<? extends javax.servlet.http.HttpServlet> |
getDumpServlet() |
HFileCleaner |
getHFileCleaner() |
long |
getLastMajorCompactionTimestamp(TableName table) |
long |
getLastMajorCompactionTimestampForRegion(byte[] regionName) |
String |
getLoadBalancerClassName()
Fetch the configured
LoadBalancer class name. |
static String |
getLoadedCoprocessors()
The set of loaded coprocessors is stored in a static set.
|
long |
getMasterActiveTime() |
MasterCoprocessorHost |
getMasterCoprocessorHost() |
String[] |
getMasterCoprocessors() |
MasterFileSystem |
getMasterFileSystem() |
ProcedureExecutor<MasterProcedureEnv> |
getMasterProcedureExecutor() |
MasterQuotaManager |
getMasterQuotaManager() |
MasterRpcServices |
getMasterRpcServices() |
long |
getMasterStartTime() |
NamespaceDescriptor |
getNamespaceDescriptor(String name)
Get a namespace descriptor by name
|
protected String |
getProcessName() |
MemoryBoundedLogMessageBuffer |
getRegionServerFatalLogBuffer() |
int |
getRegionServerInfoPort(ServerName sn) |
ServerManager |
getServerManager() |
ServerName |
getServerName() |
SnapshotManager |
getSnapshotManagerForTesting()
Exposed for TESTING!
|
TableDescriptors |
getTableDescriptors() |
ZooKeeperWatcher |
getZooKeeper()
Gets the ZooKeeper instance for this server.
|
boolean |
isActiveMaster()
Report whether this master is currently the active master or not.
|
boolean |
isBalancerOn()
Queries the state of the
LoadBalancerTracker . |
boolean |
isInitializationStartsMetaRegionAssignment()
Report whether this master has started initialization and is about to do meta region assignment
|
boolean |
isInitialized()
Report whether this master has completed with its initialization and is
ready.
|
boolean |
isMasterProcedureExecutorEnabled()
Check whether the procedure executor is enabled
|
boolean |
isServerShutdownHandlerEnabled()
ServerShutdownHandlerEnabled is set false before completing
assignMeta to prevent processing of ServerShutdownHandler.
|
List<NamespaceDescriptor> |
listNamespaceDescriptors()
List available namespace descriptors
|
List<ProcedureInfo> |
listProcedures()
List procedures
|
List<HTableDescriptor> |
listTableDescriptors(String namespace,
String regex,
List<TableName> tableNameList,
boolean includeSysTables)
Returns the list of table descriptors that match the specified request
|
List<HTableDescriptor> |
listTableDescriptorsByNamespace(String name)
Get list of table descriptors by namespace
|
List<TableName> |
listTableNames(String namespace,
String regex,
boolean includeSysTables)
Returns the list of table names that match the specified request
|
List<TableName> |
listTableNamesByNamespace(String name)
Get list of table names by namespace
|
protected void |
login(UserProvider user,
String host)
For compatibility, if failed with regionserver credentials, try the master one
|
static void |
main(String[] args) |
void |
modifyColumn(TableName tableName,
HColumnDescriptor descriptor,
long nonceGroup,
long nonce)
Modify the column descriptor of an existing column in an existing table
|
void |
modifyNamespace(NamespaceDescriptor descriptor)
Modify an existing namespace
|
void |
modifyTable(TableName tableName,
HTableDescriptor descriptor,
long nonceGroup,
long nonce)
Modify the descriptor of an existing table
|
boolean |
registerService(com.google.protobuf.Service instance)
Registers a new protocol buffer
Service subclass as a coprocessor endpoint to be
available for handling |
protected void |
sendShutdownInterrupt()
Called on stop/abort before closing the cluster connection and meta locator.
|
void |
setCatalogJanitorEnabled(boolean b)
Switch for the background CatalogJanitor thread.
|
void |
shutdown() |
void |
stopMaster() |
protected void |
stopServiceThreads()
Wait on all threads to finish.
|
void |
truncateTable(TableName tableName,
boolean preserveSplits,
long nonceGroup,
long nonce)
Truncate a table
|
protected void |
waitForMasterActive()
If configured to put regions on active master,
wait till a backup master becomes active.
|
abort, addToMovedRegions, addToOnlineRegions, checkFileSystem, cleanMovedRegions, closeAllRegions, closeRegion, constructRegionServer, convertThrowableToIOE, createClusterConnection, createRegionLoad, createRegionServerStatusStub, ensureMetaWALRoller, execRegionServerService, getCacheConfig, getChoreService, getClusterId, getCompactionPressure, getCompactionRequester, getCompactSplitThread, getConfiguration, getConfigurationManager, getConnection, getCoordinatedStateManager, getExecutorService, getFavoredNodesForRegion, getFileSystem, getFlushRequester, getFromOnlineRegions, getHeapMemoryManager, getInfoServer, getLastSequenceId, getLeases, getMasterAddressTracker, getMetaTableLocator, getMostLoadedRegions, getNonceManager, getNumberOfOnlineRegions, getOnlineRegion, getOnlineRegions, getOnlineRegionsLocalContext, getOnlineTables, getRecoveringRegions, getRegion, getRegionBlockLocations, getRegionByEncodedName, getRegionByEncodedName, getRegionServerAccounting, getRegionServerCoprocessorHost, getRegionServerCoprocessors, getRegionServerMetrics, getRegionServerQuotaManager, getRegionsInTransitionInRS, getRootDir, getRpcServer, getRSRpcServices, getStartcode, getTableLockManager, getThreadWakeFrequency, getWAL, handleReportForDutyResponse, isAborted, isOnline, isStopped, isStopping, kill, movedRegionCleanerPeriod, postOpenDeployTasks, postOpenDeployTasks, removeFromOnlineRegions, reportRegionStateTransition, reportRegionStateTransition, reportRegionStateTransition, run, setupClusterConnection, shouldUseThisHostnameInstead, stop, toString, tryRegionServerReport, updateConfiguration, updateRegionFavoredNodesMapping, waitForServerOnline
getName, getThread, interrupt, isAlive, isInterrupted, join, join, join, setDaemon, setName, setPriority, setUncaughtExceptionHandler, start
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
getExecutorService, getTableLockManager
getChoreService, getConfiguration, getConnection, getCoordinatedStateManager, getMetaTableLocator
public static final String MASTER
public HMaster(org.apache.hadoop.conf.Configuration conf, CoordinatedStateManager csm) throws IOException, org.apache.zookeeper.KeeperException, InterruptedException
Remaining steps of initialization occur in #finishActiveMasterInitialization(MonitoredTask) after the master becomes the active one.
InterruptedException
org.apache.zookeeper.KeeperException
IOException
protected void login(UserProvider user, String host) throws IOException
login
in class HRegionServer
IOException
protected void waitForMasterActive()
waitForMasterActive
in class HRegionServer
public MasterRpcServices getMasterRpcServices()
public boolean balanceSwitch(boolean b) throws IOException
IOException
protected String getProcessName()
getProcessName
in class HRegionServer
protected boolean canCreateBaseZNode()
canCreateBaseZNode
in class HRegionServer
protected boolean canUpdateTableDescriptor()
canUpdateTableDescriptor
in class HRegionServer
protected RSRpcServices createRpcServices() throws IOException
createRpcServices
in class HRegionServer
IOException
protected void configureInfoServer()
configureInfoServer
in class HRegionServer
protected Class<? extends javax.servlet.http.HttpServlet> getDumpServlet()
getDumpServlet
in class HRegionServer
protected void doMetrics()
doMetrics
in class HRegionServer
public TableDescriptors getTableDescriptors()
getTableDescriptors
in interface MasterServices
getTableDescriptors
in class HRegionServer
public ServerManager getServerManager()
getServerManager
in interface MasterServices
ServerManager
instance.public MasterFileSystem getMasterFileSystem()
getMasterFileSystem
in interface MasterServices
MasterFileSystem
utility class.protected void sendShutdownInterrupt()
HRegionServer
sendShutdownInterrupt
in class HRegionServer
protected void stopServiceThreads()
HRegionServer
stopServiceThreads
in class HRegionServer
public boolean isMasterProcedureExecutorEnabled()
isMasterProcedureExecutorEnabled
in interface MasterServices
public boolean balance() throws IOException
IOException
public void setCatalogJanitorEnabled(boolean b)
b
- If false, the catalog janitor won't do anything.public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b, boolean forcible) throws IOException
MasterServices
dispatchMergingRegions
in interface MasterServices
region_a
- region to mergeregion_b
- region to mergeforcible
- true if do a compulsory merge, otherwise we will only merge
two adjacent regionsIOException
public long createTable(HTableDescriptor hTableDescriptor, byte[][] splitKeys, long nonceGroup, long nonce) throws IOException
MasterServices
createTable
in interface MasterServices
hTableDescriptor
- The table definitionsplitKeys
- Starting row keys for the initial table regions. If nullnonce
- a single region is created.IOException
public long deleteTable(TableName tableName, long nonceGroup, long nonce) throws IOException
MasterServices
deleteTable
in interface MasterServices
tableName
- The table nameIOException
public void truncateTable(TableName tableName, boolean preserveSplits, long nonceGroup, long nonce) throws IOException
MasterServices
truncateTable
in interface MasterServices
tableName
- The table namepreserveSplits
- True if the splits should be preservedIOException
public void addColumn(TableName tableName, HColumnDescriptor columnDescriptor, long nonceGroup, long nonce) throws IOException
MasterServices
addColumn
in interface MasterServices
tableName
- The table namecolumnDescriptor
- The column definitionIOException
public void modifyColumn(TableName tableName, HColumnDescriptor descriptor, long nonceGroup, long nonce) throws IOException
MasterServices
modifyColumn
in interface MasterServices
tableName
- The table namedescriptor
- The updated column definitionIOException
public void deleteColumn(TableName tableName, byte[] columnName, long nonceGroup, long nonce) throws IOException
MasterServices
deleteColumn
in interface MasterServices
tableName
- The table namecolumnName
- The column nameIOException
public long enableTable(TableName tableName, long nonceGroup, long nonce) throws IOException
MasterServices
enableTable
in interface MasterServices
tableName
- The table nameIOException
public long disableTable(TableName tableName, long nonceGroup, long nonce) throws IOException
MasterServices
disableTable
in interface MasterServices
tableName
- The table nameIOException
public void modifyTable(TableName tableName, HTableDescriptor descriptor, long nonceGroup, long nonce) throws IOException
MasterServices
modifyTable
in interface MasterServices
tableName
- The table namedescriptor
- The updated table descriptorIOException
public void checkTableModifiable(TableName tableName) throws IOException, TableNotFoundException, TableNotDisabledException
MasterServices
checkTableModifiable
in interface MasterServices
tableName
- Name of table to check.TableNotDisabledException
TableNotFoundException
IOException
public ClusterStatus getClusterStatus() throws InterruptedIOException
InterruptedIOException
public static String getLoadedCoprocessors()
public long getMasterStartTime()
public long getMasterActiveTime()
public int getRegionServerInfoPort(ServerName sn)
public String[] getMasterCoprocessors()
public void abort(String msg, Throwable t)
HRegionServer
abort
in interface Abortable
abort
in class HRegionServer
msg
- the reason we are abortingt
- the exception that caused the abort, or nullpublic ZooKeeperWatcher getZooKeeper()
Server
getZooKeeper
in interface Server
getZooKeeper
in class HRegionServer
public MasterCoprocessorHost getMasterCoprocessorHost()
getMasterCoprocessorHost
in interface MasterServices
MasterCoprocessorHost
public MasterQuotaManager getMasterQuotaManager()
getMasterQuotaManager
in interface MasterServices
MasterQuotaManager
public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor()
getMasterProcedureExecutor
in interface MasterServices
ProcedureExecutor
public ServerName getServerName()
getServerName
in interface Server
getServerName
in class HRegionServer
public AssignmentManager getAssignmentManager()
getAssignmentManager
in interface MasterServices
AssignmentManager
public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer()
public void shutdown()
public void stopMaster()
public boolean isActiveMaster()
public boolean isInitialized()
isInitialized
in interface MasterServices
public boolean isServerShutdownHandlerEnabled()
isServerShutdownHandlerEnabled
in interface MasterServices
public boolean isInitializationStartsMetaRegionAssignment()
public void assignRegion(HRegionInfo hri)
public double getAverageLoad()
public boolean registerService(com.google.protobuf.Service instance)
RegionServerServices
Service
subclass as a coprocessor endpoint to be
available for handlingregisterService
in interface MasterServices
registerService
in interface RegionServerServices
registerService
in class HRegionServer
instance
- the Service
subclass instance to expose as a coprocessor endpointtrue
if the registration was successful, false
public static HMaster constructMaster(Class<? extends HMaster> masterClass, org.apache.hadoop.conf.Configuration conf, CoordinatedStateManager cp)
masterClass
- conf
- public static void main(String[] args)
HMasterCommandLine
public HFileCleaner getHFileCleaner()
public SnapshotManager getSnapshotManagerForTesting()
public void createNamespace(NamespaceDescriptor descriptor) throws IOException
MasterServices
createNamespace
in interface MasterServices
descriptor
- descriptor which describes the new namespaceIOException
public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException
MasterServices
modifyNamespace
in interface MasterServices
descriptor
- descriptor which updates the existing namespaceIOException
public void deleteNamespace(String name) throws IOException
MasterServices
deleteNamespace
in interface MasterServices
name
- namespace nameIOException
protected void ensureNamespaceExists(String name) throws IOException, NamespaceNotFoundException
name
- the namespace to checkIOException
- if the namespace manager is not ready yet.NamespaceNotFoundException
- if the namespace does not existspublic NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException
MasterServices
getNamespaceDescriptor
in interface MasterServices
name
- name of namespace descriptorIOException
public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException
MasterServices
listNamespaceDescriptors
in interface MasterServices
IOException
public boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException
MasterServices
abortProcedure
in interface MasterServices
procId
- ID of the proceduremayInterruptIfRunning
- if the proc completed at least one step, should it be aborted?IOException
public List<ProcedureInfo> listProcedures() throws IOException
MasterServices
listProcedures
in interface MasterServices
IOException
public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException
MasterServices
listTableDescriptorsByNamespace
in interface MasterServices
name
- namespace nameIOException
public List<TableName> listTableNamesByNamespace(String name) throws IOException
MasterServices
listTableNamesByNamespace
in interface MasterServices
name
- namespace nameIOException
public List<HTableDescriptor> listTableDescriptors(String namespace, String regex, List<TableName> tableNameList, boolean includeSysTables) throws IOException
namespace
- the namespace to query, or null if querying for allregex
- The regular expression to match against, or null if querying for alltableNameList
- the list of table names, or null if querying for allincludeSysTables
- False to match only against userspace tablesIOException
public List<TableName> listTableNames(String namespace, String regex, boolean includeSysTables) throws IOException
regex
- The regular expression to match against, or null if querying for allnamespace
- the namespace to query, or null if querying for allincludeSysTables
- False to match only against userspace tablesIOException
public long getLastMajorCompactionTimestamp(TableName table) throws IOException
getLastMajorCompactionTimestamp
in interface MasterServices
IOException
public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException
getLastMajorCompactionTimestampForRegion
in interface MasterServices
IOException
public boolean isBalancerOn()
LoadBalancerTracker
. If the balancer is not initialized,
false is returned.public String getLoadBalancerClassName()
LoadBalancer
class name. If none is set, a default is returned.LoadBalancer
in use.Copyright © 2007-2016 The Apache Software Foundation. All Rights Reserved.