@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 |
|---|---|
private static class |
HMaster.InitializationMonitor
Protection against zombie master.
|
private static class |
HMaster.PeriodicDoMetrics |
static class |
HMaster.RedirectServlet |
HRegionServer.MovedRegionsCleanerRegionServerServices.PostOpenDeployContext, RegionServerServices.RegionStateTransitionContextcacheConfig, cacheFlusher, CLOSE, clusterConnection, clusterStatusTracker, compactSplitThread, conf, configurationManager, csm, fs, fsOk, hMemManager, infoServer, 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, 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
|
(package private) void |
assignMeta(MonitoredTask status,
Set<ServerName> previouslyFailedMetaRSs,
int replicaId)
Check
hbase:meta is assigned. |
private void |
assignMetaZkLess(RegionStates regionStates,
RegionState regionState,
long timeout,
Set<ServerName> previouslyFailedRs) |
void |
assignRegion(HRegionInfo hri) |
boolean |
balance() |
boolean |
balanceSwitch(boolean b) |
protected boolean |
canCreateBaseZNode() |
protected boolean |
canUpdateTableDescriptor() |
private void |
checkClassLoading(org.apache.hadoop.conf.Configuration conf,
HTableDescriptor htd) |
private void |
checkCompactionPolicy(org.apache.hadoop.conf.Configuration conf,
HTableDescriptor htd) |
private void |
checkCompression(HColumnDescriptor hcd) |
private void |
checkCompression(HTableDescriptor htd) |
private void |
checkEncryption(org.apache.hadoop.conf.Configuration conf,
HColumnDescriptor hcd) |
private void |
checkEncryption(org.apache.hadoop.conf.Configuration conf,
HTableDescriptor htd) |
(package private) void |
checkInitialized() |
(package private) void |
checkNamespaceManagerReady() |
(package private) void |
checkServiceStarted() |
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.
|
protected CatalogJanitor |
createCatalogJanitor() |
void |
createNamespace(NamespaceDescriptor descriptor)
Create a new namespace
|
protected RSRpcServices |
createRpcServices() |
(package private) ServerManager |
createServerManager(Server master,
MasterServices services)
Create a
ServerManager instance. |
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.
|
private void |
doMetrics()
Emit the HMaster metrics, such as region in transition metrics.
|
private void |
enableCrashedServerProcessing(boolean waitForMeta) |
private void |
enableMeta(TableName metaTableName) |
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
|
private static void |
filterTablesByRegex(Collection<HTableDescriptor> descriptors,
Pattern pattern)
Removes the table descriptors that don't match the pattern.
|
private void |
finishActiveMasterInitialization(MonitoredTask status)
Finish initialization of HMaster after becoming the primary master.
|
AssignmentManager |
getAssignmentManager() |
double |
getAverageLoad()
Compute the average load across all region servers.
|
private int |
getBalancerCutoffTime() |
(package private) String |
getClientIdAuditPrefix() |
ClusterStatus |
getClusterStatus() |
protected Class<? extends javax.servlet.http.HttpServlet> |
getDumpServlet() |
HFileCleaner |
getHFileCleaner() |
MasterProcedureScheduler.ProcedureEvent |
getInitializedEvent() |
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() |
(package private) MetricsMaster |
getMasterMetrics() |
ProcedureExecutor<MasterProcedureEnv> |
getMasterProcedureExecutor() |
MasterQuotaManager |
getMasterQuotaManager() |
MasterRpcServices |
getMasterRpcServices() |
long |
getMasterStartTime() |
NamespaceDescriptor |
getNamespaceDescriptor(String name)
Get a namespace descriptor by name
|
private Set<ServerName> |
getPreviouselyFailedMetaServersFromZK()
This function returns a set of region server names under hbase:meta recovering region ZK node
|
protected String |
getProcessName() |
RegionNormalizerTracker |
getRegionNormalizerTracker() |
MemoryBoundedLogMessageBuffer |
getRegionServerFatalLogBuffer() |
int |
getRegionServerInfoPort(ServerName sn) |
String |
getRegionServerVersion(ServerName sn) |
(package private) InetAddress |
getRemoteInetAddress(int port,
long serverStartCode) |
MasterProcedureScheduler.ProcedureEvent |
getServerCrashProcessingEnabledEvent() |
ServerManager |
getServerManager() |
ServerName |
getServerName() |
SnapshotManager |
getSnapshotManager()
Exposed for TESTING!
|
TableDescriptors |
getTableDescriptors() |
(package private) Pair<HRegionInfo,ServerName> |
getTableRegionForRow(TableName tableName,
byte[] rowKey)
Return the region and current deployment for the region containing
the given row.
|
ZooKeeperWatcher |
getZooKeeper()
Gets the ZooKeeper instance for this server.
|
(package private) void |
initializeZKBasedSystemTrackers()
Initialize all ZK based system trackers.
|
(package private) void |
initNamespace() |
private void |
initQuotaManager() |
boolean |
isActiveMaster()
Report whether this master is currently the active master or not.
|
boolean |
isBalancerOn()
Queries the state of the
LoadBalancerTracker. |
(package private) boolean |
isCatalogJanitorEnabled() |
private static boolean |
isCatalogTable(TableName tableName) |
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 |
isNormalizerOn()
Queries the state of the
RegionNormalizerTracker. |
boolean |
isServerCrashProcessingEnabled()
ServerCrashProcessingEnabled is set false before completing assignMeta to prevent processing
of crashed servers.
|
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
|
(package private) void |
move(byte[] encodedRegionName,
byte[] destServerName) |
boolean |
normalizeRegions()
Perform normalization of cluster (invoked by
RegionNormalizerChore). |
private int |
putUpJettyServer() |
boolean |
registerService(com.google.protobuf.Service instance)
Registers a new protocol buffer
Service subclass as a coprocessor endpoint to be
available for handling |
private void |
sanityCheckTableDescriptor(HTableDescriptor htd)
Checks whether the table conforms to some sane limits, and configured
values (compression, etc) work.
|
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() |
private void |
splitMetaLogBeforeAssignment(ServerName currentMetaServer) |
private void |
startActiveMasterManager(int infoPort) |
private void |
startProcedureExecutor() |
private void |
startServiceThreads() |
private void |
stopChores() |
void |
stopMaster() |
private void |
stopProcedureExecutor() |
protected void |
stopServiceThreads()
Wait on all threads to finish.
|
void |
truncateTable(TableName tableName,
boolean preserveSplits,
long nonceGroup,
long nonce)
Truncate a table
|
private void |
unassignExcessMetaReplica(ZooKeeperWatcher zkw,
int numMetaReplicasConfigured) |
protected void |
waitForMasterActive()
If configured to put regions on active master,
wait till a backup master becomes active.
|
private static void |
warnOrThrowExceptionForFailure(boolean logWarn,
String confKey,
String message,
Exception cause) |
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, 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, getReplicationSourceService, getRootDir, getRpcServer, getRSRpcServices, getStartcode, getTableLockManager, getThreadWakeFrequency, getWAL, handleReportForDutyResponse, isAborted, isOnline, isStopped, isStopping, kill, movedRegionCleanerPeriod, postOpenDeployTasks, postOpenDeployTasks, removeFromOnlineRegions, reportRegionStateTransition, reportRegionStateTransition, reportRegionStateTransition, run, setupClusterConnection, shouldUseThisHostnameInstead, stop, stop, toString, tryRegionServerReport, 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, getMetaTableLocatorprivate static final org.apache.commons.logging.Log LOG
public static final String MASTER
private final ActiveMasterManager activeMasterManager
RegionServerTracker regionServerTracker
private DrainingServerTracker drainingServerTracker
LoadBalancerTracker loadBalancerTracker
private RegionNormalizerTracker regionNormalizerTracker
private TableNamespaceManager tableNamespaceManager
final MetricsMaster metricsMaster
private MasterFileSystem fileSystemManager
volatile ServerManager serverManager
AssignmentManager assignmentManager
MemoryBoundedLogMessageBuffer rsFatals
private volatile boolean isActiveMaster
private final MasterProcedureScheduler.ProcedureEvent initialized
volatile boolean serviceStarted
private final MasterProcedureScheduler.ProcedureEvent serverCrashProcessingEnabled
LoadBalancer balancer
private RegionNormalizer normalizer
private BalancerChore balancerChore
private RegionNormalizerChore normalizerChore
private ClusterStatusChore clusterStatusChore
private ClusterStatusPublisher clusterStatusPublisherChore
private HMaster.PeriodicDoMetrics periodicDoMetricsChore
CatalogJanitor catalogJanitorChore
private ReplicationZKLockCleanerChore replicationZKLockCleanerChore
private LogCleaner logCleaner
private HFileCleaner hfileCleaner
MasterCoprocessorHost cpHost
private final boolean preLoadTableDescriptors
private long masterActiveTime
private final boolean masterCheckCompression
private final boolean masterCheckEncryption
SnapshotManager snapshotManager
MasterProcedureManagerHost mpmHost
private volatile MasterQuotaManager quotaManager
private ProcedureExecutor<MasterProcedureEnv> procedureExecutor
private WALProcedureStore procedureStore
private volatile boolean initializationBeforeMetaAssignment
private org.mortbay.jetty.Server masterJettyServer
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.KeeperExceptionIOExceptionprivate int putUpJettyServer()
throws IOException
IOExceptionprotected 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 HRegionServerprivate void doMetrics()
MetricsMaster getMasterMetrics()
void initializeZKBasedSystemTrackers()
throws IOException,
InterruptedException,
org.apache.zookeeper.KeeperException,
CoordinatedStateException
IOExceptionInterruptedExceptionorg.apache.zookeeper.KeeperExceptionCoordinatedStateExceptionprivate void finishActiveMasterInitialization(MonitoredTask status) throws IOException, InterruptedException, org.apache.zookeeper.KeeperException, CoordinatedStateException
IOExceptionInterruptedExceptionorg.apache.zookeeper.KeeperExceptionCoordinatedStateExceptionprotected CatalogJanitor createCatalogJanitor()
private void initQuotaManager()
throws IOException
IOExceptionServerManager createServerManager(Server master, MasterServices services) throws IOException
ServerManager instance.master - services - ServerManagerZooKeeperConnectionExceptionIOExceptionprivate void unassignExcessMetaReplica(ZooKeeperWatcher zkw, int numMetaReplicasConfigured)
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId) throws InterruptedException, IOException, org.apache.zookeeper.KeeperException
hbase:meta is assigned. If not, assign it.status - MonitoredTaskpreviouslyFailedMetaRSs - replicaId - InterruptedExceptionIOExceptionorg.apache.zookeeper.KeeperExceptionprivate void assignMetaZkLess(RegionStates regionStates, RegionState regionState, long timeout, Set<ServerName> previouslyFailedRs) throws IOException, org.apache.zookeeper.KeeperException
IOExceptionorg.apache.zookeeper.KeeperExceptionvoid initNamespace()
throws IOException
IOExceptionboolean isCatalogJanitorEnabled()
private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException
IOExceptionprivate void enableCrashedServerProcessing(boolean waitForMeta)
throws IOException,
InterruptedException
IOExceptionInterruptedExceptionprivate void enableMeta(TableName metaTableName)
private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperExceptionpublic 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.private void startServiceThreads()
throws IOException
IOExceptionprotected void sendShutdownInterrupt()
HRegionServersendShutdownInterrupt in class HRegionServerprotected void stopServiceThreads()
HRegionServerstopServiceThreads in class HRegionServerprivate void startProcedureExecutor()
throws IOException
IOExceptionprivate void stopProcedureExecutor()
private void stopChores()
InetAddress getRemoteInetAddress(int port, long serverStartCode) throws UnknownHostException
UnknownHostExceptionprivate int getBalancerCutoffTime()
public boolean balance()
throws IOException
IOExceptionpublic boolean normalizeRegions()
throws IOException,
CoordinatedStateException
RegionNormalizerChore).IOExceptionCoordinatedStateExceptionString getClientIdAuditPrefix()
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
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 regionsIOExceptionvoid 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.IOExceptionprivate void sanityCheckTableDescriptor(HTableDescriptor htd) throws IOException
IOExceptionprivate void checkCompactionPolicy(org.apache.hadoop.conf.Configuration conf,
HTableDescriptor htd)
throws IOException
IOExceptionprivate static void warnOrThrowExceptionForFailure(boolean logWarn,
String confKey,
String message,
Exception cause)
throws IOException
IOExceptionprivate void startActiveMasterManager(int infoPort)
throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperExceptionprivate void checkCompression(HTableDescriptor htd) throws IOException
IOExceptionprivate void checkCompression(HColumnDescriptor hcd) throws IOException
IOExceptionprivate void checkEncryption(org.apache.hadoop.conf.Configuration conf,
HTableDescriptor htd)
throws IOException
IOExceptionprivate void checkEncryption(org.apache.hadoop.conf.Configuration conf,
HColumnDescriptor hcd)
throws IOException
IOExceptionprivate void checkClassLoading(org.apache.hadoop.conf.Configuration conf,
HTableDescriptor htd)
throws IOException
IOExceptionprivate static boolean isCatalogTable(TableName tableName)
public 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 nameIOExceptionPair<HRegionInfo,ServerName> getTableRegionForRow(TableName tableName, byte[] rowKey) throws IOException
IOExceptionpublic 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 InterruptedIOException
InterruptedIOExceptionpublic static String getLoadedCoprocessors()
public long getMasterStartTime()
public long getMasterActiveTime()
public int getRegionServerInfoPort(ServerName sn)
public String getRegionServerVersion(ServerName sn)
public 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
IOExceptionvoid checkServiceStarted()
throws ServerNotRunningYetException
ServerNotRunningYetExceptionvoid checkInitialized()
throws PleaseHoldException,
ServerNotRunningYetException
void checkNamespaceManagerReady()
throws IOException
IOExceptionpublic boolean isActiveMaster()
public boolean isInitialized()
isInitialized in interface MasterServicespublic 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 void assignRegion(HRegionInfo hri)
public double getAverageLoad()
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 SnapshotManager getSnapshotManager()
public void createNamespace(NamespaceDescriptor descriptor) throws IOException
MasterServicescreateNamespace in interface MasterServicesdescriptor - descriptor which describes the new namespaceIOExceptionpublic void modifyNamespace(NamespaceDescriptor descriptor) throws IOException
MasterServicesmodifyNamespace in interface MasterServicesdescriptor - descriptor which updates the existing namespaceIOExceptionpublic void deleteNamespace(String name) 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 tablesIOExceptionprivate static void filterTablesByRegex(Collection<HTableDescriptor> descriptors, Pattern pattern)
descriptors - list of table descriptors to filterpattern - the regex to usepublic long getLastMajorCompactionTimestamp(TableName table) throws IOException
getLastMajorCompactionTimestamp in interface MasterServicesIOExceptionpublic 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 String getLoadBalancerClassName()
LoadBalancer class name. If none is set, a default is returned.LoadBalancer in use.public RegionNormalizerTracker getRegionNormalizerTracker()
Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.