@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.MovedRegionsCleanerRegionServerServices.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, initLatch, leases, lock, MASTER_HOSTNAME_KEY, metaTableLocator, movedRegions, msgInterval, numRegionsToReport, onlineRegions, OPEN, recoveringRegions, regionFavoredNodesMap, REGIONSERVER, regionsInTransitionInRS, replicationSinkHandler, replicationSourceHandler, rpcServices, serverName, service, sleeper, startcode, tableDescriptors, tableLockManager, threadWakeFrequency, useThisHostnameInstead, walFactory, walFs, 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
|
boolean |
balance() |
boolean |
balance(boolean force) |
boolean |
balanceSwitch(boolean b) |
protected boolean |
canCreateBaseZNode() |
protected boolean |
canUpdateTableDescriptor() |
void |
checkIfShouldMoveSystemRegionAsync() |
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,
long nonceGroup,
long nonce)
Create a new namespace
|
void |
createNamespaceSync(NamespaceDescriptor descriptor,
long nonceGroup,
long nonce,
boolean executeCoprocessor)
Create a new namespace synchronously.
|
protected RSRpcServices |
createRpcServices() |
long |
createSystemTable(HTableDescriptor hTableDescriptor)
Create a system table using the given table definition.
|
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,
long nonceGroup,
long nonce)
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,
User user)
Merge two regions.
|
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() |
ClusterStatus |
getClusterStatusWithoutCoprocessor() |
protected Class<? extends javax.servlet.http.HttpServlet> |
getDumpServlet() |
HFileCleaner |
getHFileCleaner() |
MasterProcedureScheduler.ProcedureEvent |
getInitializedEvent() |
long |
getLastMajorCompactionTimestamp(TableName table) |
long |
getLastMajorCompactionTimestampForRegion(byte[] regionName) |
LoadBalancer |
getLoadBalancer() |
String |
getLoadBalancerClassName()
Fetch the configured
LoadBalancer class name. |
static String |
getLoadedCoprocessors()
The set of loaded coprocessors is stored in a static set.
|
LogCleaner |
getLogCleaner() |
long |
getMasterActiveTime() |
MasterCoprocessorHost |
getMasterCoprocessorHost() |
String[] |
getMasterCoprocessors() |
MasterFileSystem |
getMasterFileSystem() |
long |
getMasterFinishedInitializationTime() |
ProcedureExecutor<MasterProcedureEnv> |
getMasterProcedureExecutor() |
MasterProcedureManagerHost |
getMasterProcedureManagerHost() |
MasterQuotaManager |
getMasterQuotaManager() |
MasterRpcServices |
getMasterRpcServices() |
long |
getMasterStartTime() |
long |
getMergePlanCount() |
NamespaceDescriptor |
getNamespaceDescriptor(String name)
Get a namespace descriptor by name
|
int |
getNumWALFiles() |
protected String |
getProcessName() |
RegionNormalizerTracker |
getRegionNormalizerTracker() |
MemoryBoundedLogMessageBuffer |
getRegionServerFatalLogBuffer() |
int |
getRegionServerInfoPort(ServerName sn) |
String |
getRegionServerVersion(ServerName sn) |
MasterProcedureScheduler.ProcedureEvent |
getServerCrashProcessingEnabledEvent() |
ServerManager |
getServerManager() |
ServerName |
getServerName() |
SnapshotManager |
getSnapshotManager() |
SplitOrMergeTracker |
getSplitOrMergeTracker() |
long |
getSplitPlanCount() |
TableDescriptors |
getTableDescriptors() |
TableNamespaceManager |
getTableNamespaceManager() |
WALProcedureStore |
getWalProcedureStore() |
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 |
isInMaintenanceMode()
Report whether this master is in maintenance mode.
|
boolean |
isNormalizerOn()
Queries the state of the
RegionNormalizerTracker. |
boolean |
isServerCrashProcessingEnabled()
ServerCrashProcessingEnabled is set false before completing assignMeta to prevent processing
of crashed servers.
|
boolean |
isSplitOrMergeEnabled(Admin.MasterSwitchType switchType)
Queries the state of the
SplitOrMergeTracker. |
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,
long nonceGroup,
long nonce)
Modify an existing namespace
|
void |
modifyTable(TableName tableName,
HTableDescriptor descriptor,
long nonceGroup,
long nonce)
Modify the descriptor of an existing table
|
void |
move(byte[] encodedRegionName,
byte[] destServerName) |
boolean |
normalizeRegions()
Perform normalization of cluster (invoked by
RegionNormalizerChore). |
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 |
setInitialized(boolean isInitialized) |
void |
setServerCrashProcessingEnabled(boolean b) |
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, createRegionServerStatusStub, ensureMetaWALRoller, execRegionServerService, getCacheConfig, getChoreService, getClusterId, getCompactionPressure, getCompactionRequester, getCompactSplitThread, getConfiguration, getConfigurationManager, getConnection, getCoordinatedStateManager, getExecutorService, getFavoredNodesForRegion, getFileSystem, getFlushPressure, getFlushRequester, getFlushThroughputController, getFromOnlineRegions, getHeapMemoryManager, getInfoServer, getLastSequenceId, getLeases, getMasterAddressTracker, getMetaTableLocator, getMetrics, getNonceManager, getNumberOfOnlineRegions, getOnlineRegion, getOnlineRegions, getOnlineRegions, getOnlineRegionsLocalContext, getOnlineTables, getRecoveringRegions, getRegion, getRegionBlockLocations, getRegionByEncodedName, getRegionByEncodedName, getRegionServerAccounting, getRegionServerCoprocessorHost, getRegionServerCoprocessors, getRegionServerMetrics, getRegionServerQuotaManager, getRegionsInTransitionInRS, getReplicationSourceService, getRootDir, getRpcServer, getRSRpcServices, getStartcode, getTableLockManager, getThreadWakeFrequency, getWAL, getWALFileSystem, getWALRootDir, getWALs, handleReportForDutyResponse, isAborted, isOnline, isStopped, isStopping, kill, movedRegionCleanerPeriod, onConfigurationChange, postOpenDeployTasks, postOpenDeployTasks, removeFromOnlineRegions, reportRegionStateTransition, reportRegionStateTransition, reportRegionStateTransition, run, setInitLatch, setupClusterConnection, shouldUseThisHostnameInstead, stop, stop, toString, tryRegionServerReport, unassign, updateConfiguration, updateRegionFavoredNodesMapping, waitForServerOnline, walRollRequestFinishedgetName, getThread, interrupt, isAlive, isInterrupted, join, join, join, setDaemon, setName, setPriority, setUncaughtExceptionHandler, startclone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, waitgetExecutorService, getTableLockManagergetChoreService, getConfiguration, getConnection, getCoordinatedStateManager, getMetaTableLocatorpublic 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.
InterruptedExceptionorg.apache.zookeeper.KeeperExceptionIOExceptionprotected void login(UserProvider user, String host) throws IOException
login in class HRegionServerIOExceptionprotected void waitForMasterActive()
waitForMasterActive in class HRegionServerpublic MasterRpcServices getMasterRpcServices()
public boolean balanceSwitch(boolean b)
throws IOException
IOExceptionprotected String getProcessName()
getProcessName in class HRegionServerprotected boolean canCreateBaseZNode()
canCreateBaseZNode in class HRegionServerprotected boolean canUpdateTableDescriptor()
canUpdateTableDescriptor in class HRegionServerprotected RSRpcServices createRpcServices() throws IOException
createRpcServices in class HRegionServerIOExceptionprotected void configureInfoServer()
configureInfoServer in class HRegionServerprotected Class<? extends javax.servlet.http.HttpServlet> getDumpServlet()
getDumpServlet in class HRegionServerpublic TableDescriptors getTableDescriptors()
getTableDescriptors in interface MasterServicesgetTableDescriptors in class HRegionServerpublic ServerManager getServerManager()
getServerManager in interface MasterServicesServerManager instance.public MasterFileSystem getMasterFileSystem()
getMasterFileSystem in interface MasterServicesMasterFileSystem utility class.public TableNamespaceManager getTableNamespaceManager()
getTableNamespaceManager in interface MasterServicesTableNamespaceManagerprotected void sendShutdownInterrupt()
HRegionServersendShutdownInterrupt in class HRegionServerprotected void stopServiceThreads()
HRegionServerstopServiceThreads in class HRegionServerpublic boolean balance()
throws IOException
IOExceptionpublic boolean balance(boolean force)
throws IOException
IOExceptionpublic boolean normalizeRegions()
throws IOException,
CoordinatedStateException
RegionNormalizerChore).IOExceptionCoordinatedStateExceptionpublic 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, User user) throws IOException
MasterServicesdispatchMergingRegions in interface MasterServicesregion_a - region to mergeregion_b - region to mergeforcible - true if do a compulsory merge, otherwise we will only merge
two adjacent regionsuser - effective userIOExceptionpublic void move(byte[] encodedRegionName,
byte[] destServerName)
throws HBaseIOException
HBaseIOExceptionpublic long createTable(HTableDescriptor hTableDescriptor, byte[][] splitKeys, long nonceGroup, long nonce) throws IOException
MasterServicescreateTable in interface MasterServiceshTableDescriptor - The table definitionsplitKeys - Starting row keys for the initial table regions. If nullnonce - a single region is created.IOExceptionpublic long createSystemTable(HTableDescriptor hTableDescriptor) throws IOException
MasterServicescreateSystemTable in interface MasterServiceshTableDescriptor - The system table definition
a single region is created.IOExceptionpublic long deleteTable(TableName tableName, long nonceGroup, long nonce) throws IOException
MasterServicesdeleteTable in interface MasterServicestableName - The table nameIOExceptionpublic void truncateTable(TableName tableName, boolean preserveSplits, long nonceGroup, long nonce) throws IOException
MasterServicestruncateTable in interface MasterServicestableName - The table namepreserveSplits - True if the splits should be preservedIOExceptionpublic void addColumn(TableName tableName, HColumnDescriptor columnDescriptor, long nonceGroup, long nonce) throws IOException
MasterServicesaddColumn in interface MasterServicestableName - The table namecolumnDescriptor - The column definitionIOExceptionpublic void modifyColumn(TableName tableName, HColumnDescriptor descriptor, long nonceGroup, long nonce) throws IOException
MasterServicesmodifyColumn in interface MasterServicestableName - The table namedescriptor - The updated column definitionIOExceptionpublic void deleteColumn(TableName tableName, byte[] columnName, long nonceGroup, long nonce) throws IOException
MasterServicesdeleteColumn in interface MasterServicestableName - The table namecolumnName - The column nameIOExceptionpublic long enableTable(TableName tableName, long nonceGroup, long nonce) throws IOException
MasterServicesenableTable in interface MasterServicestableName - The table nameIOExceptionpublic long disableTable(TableName tableName, long nonceGroup, long nonce) throws IOException
MasterServicesdisableTable in interface MasterServicestableName - The table nameIOExceptionpublic void modifyTable(TableName tableName, HTableDescriptor descriptor, long nonceGroup, long nonce) throws IOException
MasterServicesmodifyTable in interface MasterServicestableName - The table namedescriptor - The updated table descriptorIOExceptionpublic void checkTableModifiable(TableName tableName) throws IOException, TableNotFoundException, TableNotDisabledException
MasterServicescheckTableModifiable in interface MasterServicestableName - Name of table to check.TableNotDisabledExceptionTableNotFoundExceptionIOExceptionpublic ClusterStatus getClusterStatus() throws IOException
IOExceptionpublic ClusterStatus getClusterStatusWithoutCoprocessor() throws InterruptedIOException
InterruptedIOExceptionpublic static String getLoadedCoprocessors()
public long getMasterStartTime()
public long getMasterActiveTime()
public long getMasterFinishedInitializationTime()
public int getNumWALFiles()
public WALProcedureStore getWalProcedureStore()
public int getRegionServerInfoPort(ServerName sn)
public String getRegionServerVersion(ServerName sn)
getRegionServerVersion in interface MasterServicespublic void checkIfShouldMoveSystemRegionAsync()
checkIfShouldMoveSystemRegionAsync in interface MasterServicespublic String[] getMasterCoprocessors()
public void abort(String msg, Throwable t)
HRegionServerabort in interface Abortableabort in class HRegionServermsg - the reason we are abortingt - the exception that caused the abort, or nullpublic ZooKeeperWatcher getZooKeeper()
ServergetZooKeeper in interface ServergetZooKeeper in class HRegionServerpublic MasterCoprocessorHost getMasterCoprocessorHost()
getMasterCoprocessorHost in interface MasterServicesMasterCoprocessorHostpublic MasterQuotaManager getMasterQuotaManager()
getMasterQuotaManager in interface MasterServicesMasterQuotaManagerpublic ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor()
getMasterProcedureExecutor in interface MasterServicesProcedureExecutorpublic ServerName getServerName()
getServerName in interface ServergetServerName in class HRegionServerpublic AssignmentManager getAssignmentManager()
getAssignmentManager in interface MasterServicesAssignmentManagerpublic MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer()
public void shutdown()
throws IOException
IOExceptionpublic void stopMaster()
throws IOException
IOExceptionpublic boolean isActiveMaster()
public boolean isInitialized()
isInitialized in interface MasterServicespublic boolean isInMaintenanceMode()
throws IOException
isInMaintenanceMode in interface MasterServicesIOExceptionpublic void setInitialized(boolean isInitialized)
public MasterProcedureScheduler.ProcedureEvent getInitializedEvent()
public boolean isServerCrashProcessingEnabled()
isServerCrashProcessingEnabled in interface MasterServicespublic void setServerCrashProcessingEnabled(boolean b)
public MasterProcedureScheduler.ProcedureEvent getServerCrashProcessingEnabledEvent()
public boolean isInitializationStartsMetaRegionAssignment()
public double getAverageLoad()
public long getSplitPlanCount()
public long getMergePlanCount()
public boolean registerService(com.google.protobuf.Service instance)
RegionServerServicesService subclass as a coprocessor endpoint to be
available for handlingregisterService in interface MasterServicesregisterService in interface RegionServerServicesregisterService in class HRegionServerinstance - the Service subclass instance to expose as a coprocessor endpointtrue if the registration was successful, falsepublic static HMaster constructMaster(Class<? extends HMaster> masterClass, org.apache.hadoop.conf.Configuration conf, CoordinatedStateManager cp)
masterClass - conf - public static void main(String[] args)
HMasterCommandLinepublic HFileCleaner getHFileCleaner()
public LogCleaner getLogCleaner()
public SnapshotManager getSnapshotManager()
getSnapshotManager in interface MasterServicespublic MasterProcedureManagerHost getMasterProcedureManagerHost()
getMasterProcedureManagerHost in interface MasterServicespublic void createNamespace(NamespaceDescriptor descriptor, long nonceGroup, long nonce) throws IOException
MasterServicescreateNamespace in interface MasterServicesdescriptor - descriptor which describes the new namespaceIOExceptionpublic void createNamespaceSync(NamespaceDescriptor descriptor, long nonceGroup, long nonce, boolean executeCoprocessor) throws IOException
MasterServicescreateNamespaceSync in interface MasterServicesdescriptor - descriptor which describes the new namespaceIOExceptionpublic void modifyNamespace(NamespaceDescriptor descriptor, long nonceGroup, long nonce) throws IOException
MasterServicesmodifyNamespace in interface MasterServicesdescriptor - descriptor which updates the existing namespaceIOExceptionpublic void deleteNamespace(String name, long nonceGroup, long nonce) throws IOException
MasterServicesdeleteNamespace in interface MasterServicesname - namespace nameIOExceptionprotected 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
MasterServicesgetNamespaceDescriptor in interface MasterServicesname - name of namespace descriptorIOExceptionpublic List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException
MasterServiceslistNamespaceDescriptors in interface MasterServicesIOExceptionpublic boolean abortProcedure(long procId,
boolean mayInterruptIfRunning)
throws IOException
MasterServicesabortProcedure in interface MasterServicesprocId - ID of the proceduremayInterruptIfRunning - if the proc completed at least one step, should it be aborted?IOExceptionpublic List<ProcedureInfo> listProcedures() throws IOException
MasterServiceslistProcedures in interface MasterServicesIOExceptionpublic List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException
MasterServiceslistTableDescriptorsByNamespace in interface MasterServicesname - namespace nameIOExceptionpublic List<TableName> listTableNamesByNamespace(String name) throws IOException
MasterServiceslistTableNamesByNamespace in interface MasterServicesname - namespace nameIOExceptionpublic 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 tablesIOExceptionpublic 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 tablesIOExceptionpublic long getLastMajorCompactionTimestamp(TableName table) throws IOException
getLastMajorCompactionTimestamp in interface MasterServicestable - the table for which last successful major compaction time is queriedIOExceptionpublic long getLastMajorCompactionTimestampForRegion(byte[] regionName)
throws IOException
getLastMajorCompactionTimestampForRegion in interface MasterServicesIOExceptionpublic boolean isBalancerOn()
LoadBalancerTracker. If the balancer is not initialized,
false is returned.public boolean isNormalizerOn()
RegionNormalizerTracker. If it's not initialized,
false is returned.public boolean isSplitOrMergeEnabled(Admin.MasterSwitchType switchType)
SplitOrMergeTracker. If it is not initialized,
false is returned. If switchType is illegal, false will return.switchType - see Admin.MasterSwitchTypepublic String getLoadBalancerClassName()
LoadBalancer class name. If none is set, a default is returned.LoadBalancer in use.public RegionNormalizerTracker getRegionNormalizerTracker()
public SplitOrMergeTracker getSplitOrMergeTracker()
public LoadBalancer getLoadBalancer()
getLoadBalancer in interface MasterServicesCopyright © 2007–2019 The Apache Software Foundation. All rights reserved.