@InterfaceAudience.LimitedPrivate(value="Tools") public class HMaster extends HRegionServer implements MasterServices
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 |
---|---|
protected static interface |
HMaster.TableDescriptorGetter
Implement to return TableDescriptor after pre-checks
|
Thread.State, Thread.UncaughtExceptionHandler
RegionServerServices.PostOpenDeployContext, RegionServerServices.RegionStateTransitionContext
clusterConnection, clusterId, clusterStatusTracker, conf, configurationManager, executorService, infoServer, REGIONSERVER, rpcServices, serverName, sleeper, startcode, tableDescriptors, TEST_SKIP_REPORTING_TRANSITION, useThisHostnameInstead, zooKeeper
MAX_PRIORITY, MIN_PRIORITY, NORM_PRIORITY
Constructor and Description |
---|
HMaster(org.apache.hadoop.conf.Configuration conf)
Initializes the HMaster.
|
Modifier and Type | Method and Description |
---|---|
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.
|
boolean |
abortProcedure(long procId,
boolean mayInterruptIfRunning)
Abort a procedure.
|
long |
addColumn(TableName tableName,
ColumnFamilyDescriptor column,
long nonceGroup,
long nonce)
Add a new column to an existing table
|
long |
addReplicationPeer(String peerId,
ReplicationPeerConfig peerConfig,
boolean enabled)
Add a new replication peer for replicating data to slave cluster
|
BalanceResponse |
balance() |
BalanceResponse |
balance(BalanceRequest request) |
BalanceResponse |
balanceOrUpdateMetrics()
Trigger a normal balance, see
balance() . |
boolean |
balanceSwitch(boolean b) |
private void |
balanceThrottling(long nextBalanceStartTime,
int maxRegionsInTransition,
long cutoffTime)
It first sleep to the next balance plan start time.
|
private org.apache.hbase.thirdparty.org.eclipse.jetty.servlet.ServletHolder |
buildApiV1Servlet() |
protected boolean |
cacheTableDescriptor() |
protected boolean |
canCreateBaseZNode() |
protected boolean |
canUpdateTableDescriptor() |
void |
checkIfShouldMoveSystemRegionAsync()
Called when a new RegionServer is added to the cluster.
|
(package private) void |
checkInitialized() |
protected void |
checkServiceStarted() |
private void |
checkTableExists(TableName tableName) |
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)
Utility for constructing an instance of the passed HMaster class.
|
protected ActiveMasterManager |
createActiveMasterManager(ZKWatcher zk,
ServerName sn,
Server server)
Protected to have custom implementations in tests override the default ActiveMaster
implementation.
|
protected AssignmentManager |
createAssignmentManager(MasterServices master,
MasterRegion masterRegion) |
private void |
createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor) |
(package private) long |
createNamespace(NamespaceDescriptor namespaceDescriptor,
long nonceGroup,
long nonce)
Create a new Namespace.
|
private void |
createProcedureExecutor() |
private SpaceQuotaSnapshotNotifier |
createQuotaSnapshotNotifier() |
protected RSRpcServices |
createRpcServices() |
protected ServerManager |
createServerManager(MasterServices master,
RegionServerList storage)
Create a
ServerManager instance. |
long |
createSystemTable(TableDescriptor tableDescriptor)
Create a system table using the given table definition.
|
long |
createTable(TableDescriptor tableDescriptor,
byte[][] splitKeys,
long nonceGroup,
long nonce)
Create a table using the given table definition.
|
void |
decommissionRegionServers(List<ServerName> servers,
boolean offload)
Mark region server(s) as decommissioned (previously called 'draining') to prevent additional
regions from getting assigned to them.
|
static void |
decorateMasterConfiguration(org.apache.hadoop.conf.Configuration conf)
This method modifies the master's configuration in order to inject replication-related features
|
long |
deleteColumn(TableName tableName,
byte[] columnName,
long nonceGroup,
long nonce)
Delete a column from an existing table
|
(package private) long |
deleteNamespace(String name,
long nonceGroup,
long nonce)
Delete an existing Namespace.
|
long |
deleteTable(TableName tableName,
long nonceGroup,
long nonce)
Delete a table
|
long |
disableReplicationPeer(String peerId)
Stop the replication stream to the specified peer
|
long |
disableTable(TableName tableName,
long nonceGroup,
long nonce)
Disable an existing table
|
long |
enableReplicationPeer(String peerId)
Restart the replication stream to the specified peer
|
long |
enableTable(TableName tableName,
long nonceGroup,
long nonce)
Enable an existing table
|
private long |
executePeerProcedure(ModifyPeerProcedure procedure) |
List<RegionPlan> |
executeRegionPlansWithThrottling(List<RegionPlan> plans)
Execute region plans with throttling
|
private static void |
filterTablesByRegex(Collection<TableDescriptor> 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.
|
void |
flushMasterStore()
Flush master local region
|
Optional<ServerName> |
getActiveMaster() |
int |
getActiveMasterInfoPort()
Returns info port of active master or 0 if any exception occurs.
|
AssignmentManager |
getAssignmentManager()
Returns Master's instance of the
AssignmentManager |
double |
getAverageLoad()
Compute the average load across all region servers.
|
int |
getBackupMasterInfoPort(ServerName sn) |
List<ServerName> |
getBackupMasters() |
CatalogJanitor |
getCatalogJanitor()
Returns Master's instance of
CatalogJanitor |
String |
getClientIdAuditPrefix()
Returns Client info for use as prefix on an audit log string; who did an action
|
String |
getClusterId() |
ClusterMetrics |
getClusterMetrics()
Returns cluster status
|
ClusterMetrics |
getClusterMetrics(EnumSet<ClusterMetrics.Option> options) |
ClusterMetrics |
getClusterMetricsWithoutCoprocessor() |
ClusterMetrics |
getClusterMetricsWithoutCoprocessor(EnumSet<ClusterMetrics.Option> options) |
ClusterSchema |
getClusterSchema()
Returns Master's instance of
ClusterSchema |
CompactionState |
getCompactionState(TableName tableName)
Get the compaction state of the table
|
ConfigurationManager |
getConfigurationManager() |
protected Class<? extends javax.servlet.http.HttpServlet> |
getDumpServlet() |
FavoredNodesManager |
getFavoredNodesManager()
Returns Favored Nodes Manager
|
HbckChore |
getHbckChore() |
HFileCleaner |
getHFileCleaner() |
List<HFileCleaner> |
getHFileCleaners() |
ProcedureEvent<?> |
getInitializedEvent()
Returns Tripped when Master has finished initialization.
|
long |
getLastMajorCompactionTimestamp(TableName table) |
long |
getLastMajorCompactionTimestampForRegion(byte[] regionName)
n * @return the timestamp of the last successful major compaction for the passed region or 0 if
no HFile resulting from a major compaction exists n
|
Collection<ServerName> |
getLiveRegionServers() |
LoadBalancer |
getLoadBalancer()
Returns load balancer
|
String |
getLoadBalancerClassName()
Fetch the configured
LoadBalancer class name. |
static String |
getLoadedCoprocessors()
The set of loaded coprocessors is stored in a static set.
|
LockManager |
getLockManager()
Returns
LockManager to lock namespaces/tables/regions. |
List<LockedResource> |
getLocks()
Get locks
|
LogCleaner |
getLogCleaner() |
long |
getMasterActiveTime()
Returns timestamp in millis when HMaster became the active master.
|
MasterCoprocessorHost |
getMasterCoprocessorHost()
Returns Master's instance of
MasterCoprocessorHost |
String[] |
getMasterCoprocessors()
Returns array of coprocessor SimpleNames.
|
MasterFileSystem |
getMasterFileSystem()
Returns Master's filesystem
MasterFileSystem utility class. |
long |
getMasterFinishedInitializationTime()
Returns timestamp in millis when HMaster finished becoming the active master
|
MetricsMaster |
getMasterMetrics()
Returns Master's instance of
MetricsMaster |
ProcedureExecutor<MasterProcedureEnv> |
getMasterProcedureExecutor()
Returns Master's instance of
ProcedureExecutor |
MasterProcedureManagerHost |
getMasterProcedureManagerHost()
Returns the underlying MasterProcedureManagerHost
|
MasterQuotaManager |
getMasterQuotaManager()
Returns Master's instance of
MasterQuotaManager |
MasterRegion |
getMasterRegion() |
MasterRpcServices |
getMasterRpcServices() |
long |
getMasterStartTime()
Returns timestamp in millis when HMaster was started.
|
MasterWalManager |
getMasterWalManager()
Returns Master's WALs
MasterWalManager utility class. |
private int |
getMaxBalancingTime()
Returns Maximum time we should run balancer for
|
private int |
getMaxRegionsInTransition()
Returns Maximum number of regions in transition
|
MetaLocationSyncer |
getMetaLocationSyncer()
Get the meta location syncer.
|
org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState |
getMobCompactionState(TableName tableName)
Gets the mob file compaction state for a specific table.
|
(package private) NamespaceDescriptor |
getNamespace(String name)
Get a Namespace
|
(package private) List<NamespaceDescriptor> |
getNamespaces()
Get all Namespaces
|
int |
getNumWALFiles() |
private Map<ServerName,ServerMetrics> |
getOnlineServers() |
List<Procedure<?>> |
getProcedures()
Get procedures
|
ProcedureStore |
getProcedureStore() |
protected String |
getProcessName() |
QuotaObserverChore |
getQuotaObserverChore() |
RegionNormalizerManager |
getRegionNormalizerManager()
Returns Master's instance of
RegionNormalizerManager |
MemoryBoundedLogMessageBuffer |
getRegionServerFatalLogBuffer() |
int |
getRegionServerInfoPort(ServerName sn) |
Iterator<ServerName> |
getRegionServers() |
String |
getRegionServerVersion(ServerName sn) |
(package private) InetAddress |
getRemoteInetAddress(int port,
long serverStartCode)
Returns Get remote side's InetAddress
|
private RemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv,?> |
getRemoteProcedure(long procId) |
HashMap<String,List<Pair<ServerName,ReplicationLoadSource>>> |
getReplicationLoad(ServerName[] serverNames) |
ReplicationPeerConfig |
getReplicationPeerConfig(String peerId)
Returns the configured ReplicationPeerConfig for the specified peer
|
ReplicationPeerManager |
getReplicationPeerManager()
Returns the
ReplicationPeerManager . |
ServerManager |
getServerManager()
Returns Master's
ServerManager instance. |
ServerName |
getServerName()
Returns The unique server name for this server.
|
SnapshotManager |
getSnapshotManager()
Returns the underlying snapshot manager
|
SnapshotQuotaObserverChore |
getSnapshotQuotaObserverChore() |
SpaceQuotaSnapshotNotifier |
getSpaceQuotaSnapshotNotifier() |
SplitOrMergeTracker |
getSplitOrMergeTracker() |
SplitWALManager |
getSplitWALManager()
Returns return null if current is zk-based WAL splitting
|
private List<TableDescriptor> |
getTableDescriptors(List<TableDescriptor> htds,
String namespace,
String regex,
List<TableName> tableNameList,
boolean includeSysTables)
Return a list of table table descriptors after applying any provided filter parameters.
|
TableStateManager |
getTableStateManager()
Returns Master's instance of
TableStateManager |
private List<ServerName> |
getUnknownServers() |
protected String |
getUseThisHostnameInstead(org.apache.hadoop.conf.Configuration conf) |
Map<String,ReplicationStatus> |
getWalGroupsReplicationStatus() |
ZKWatcher |
getZooKeeper()
Gets the ZooKeeper instance for this server.
|
protected void |
initClusterSchemaService() |
private void |
initializeCoprocessorHost(org.apache.hadoop.conf.Configuration conf) |
private void |
initializeZKBasedSystemTrackers()
Initialize all ZK based system trackers.
|
private void |
initMobCleaner() |
private void |
initQuotaManager() |
boolean |
isActiveMaster()
Report whether this master is currently the active master or not.
|
boolean |
isBalancerOn()
Queries the state of the
LoadBalancerTracker . |
boolean |
isCatalogJanitorEnabled() |
private static boolean |
isCatalogTable(TableName tableName) |
(package private) boolean |
isCleanerChoreEnabled() |
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 |
isOnline()
Report whether this master is started This method is used for testing.
|
private boolean |
isRegionOnline(RegionInfo ri) |
boolean |
isSplitOrMergeEnabled(MasterSwitchType switchType)
Queries the state of the
SplitOrMergeTracker . |
List<ServerName> |
listDecommissionedRegionServers()
List region servers marked as decommissioned (previously called 'draining') to not get regions
assigned to them.
|
List<String> |
listNamespaces()
List namespace names
|
List<ReplicationPeerDescription> |
listReplicationPeers(String regex)
Return a list of replication peers.
|
List<TableDescriptor> |
listTableDescriptors(String namespace,
String regex,
List<TableName> tableNameList,
boolean includeSysTables)
Returns the list of table descriptors that match the specified request
|
List<TableDescriptor> |
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) |
long |
mergeRegions(RegionInfo[] regionsToMerge,
boolean forcible,
long ng,
long nonce)
Merge regions in a table.
|
long |
modifyColumn(TableName tableName,
ColumnFamilyDescriptor descriptor,
long nonceGroup,
long nonce)
Modify the column descriptor of an existing column in an existing table
|
long |
modifyColumnStoreFileTracker(TableName tableName,
byte[] family,
String dstSFT,
long nonceGroup,
long nonce)
Modify the store file tracker of an existing column in an existing table
|
(package private) long |
modifyNamespace(NamespaceDescriptor newNsDescriptor,
long nonceGroup,
long nonce)
Modify an existing Namespace.
|
private long |
modifyTable(TableName tableName,
HMaster.TableDescriptorGetter newDescriptorGetter,
long nonceGroup,
long nonce,
boolean shouldCheckDescriptor) |
long |
modifyTable(TableName tableName,
TableDescriptor newDescriptor,
long nonceGroup,
long nonce)
Modify the descriptor of an existing table
|
long |
modifyTableStoreFileTracker(TableName tableName,
String dstSFT,
long nonceGroup,
long nonce)
Modify the store file tracker of an existing table
|
void |
move(byte[] encodedRegionName,
byte[] destServerName) |
boolean |
normalizeRegions(NormalizeTableFilterParams ntfp,
boolean isHighPriority)
Perform normalization of cluster.
|
void |
onConfigurationChange(org.apache.hadoop.conf.Configuration newConf)
This method would be called by the
ConfigurationManager object when the
Configuration object is reloaded from disk. |
private int |
putUpJettyServer() |
void |
recommissionRegionServer(ServerName server,
List<byte[]> encodedRegionNames)
Remove decommission marker (previously called 'draining') from a region server to allow regions
assignments.
|
private void |
registerConfigurationObservers() |
boolean |
registerService(com.google.protobuf.Service instance)
Registers a new protocol buffer
Service subclass as a coprocessor endpoint to be
available for handling |
void |
remoteProcedureCompleted(long procId) |
void |
remoteProcedureFailed(long procId,
RemoteProcedureException error) |
long |
removeReplicationPeer(String peerId)
Removes a peer and stops the replication
|
(package private) long |
reopenRegions(TableName tableName,
List<byte[]> regionNames,
long nonceGroup,
long nonce)
Reopen regions provided in the argument
|
void |
reportMobCompactionEnd(TableName tableName) |
void |
reportMobCompactionStart(TableName tableName) |
long |
restoreSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshotDesc,
long nonceGroup,
long nonce,
boolean restoreAcl,
String customSFT) |
void |
run()
The HRegionServer sticks in this loop until closed.
|
void |
runReplicationBarrierCleaner()
Run the ReplicationBarrierChore.
|
(package private) void |
setAssignmentManager(AssignmentManager assignmentManager) |
void |
setCatalogJanitorEnabled(boolean b)
Switch for the background CatalogJanitor thread.
|
(package private) static void |
setDisableBalancerChoreForTest(boolean disable) |
void |
setInitialized(boolean isInitialized) |
(package private) void |
setLoadBalancer(LoadBalancer loadBalancer) |
private void |
setQuotasObserver(org.apache.hadoop.conf.Configuration conf) |
void |
shutdown()
Shutdown the cluster.
|
boolean |
skipRegionManagementAction(String action)
Checks master state before initiating action over region topology.
|
long |
splitRegion(RegionInfo regionInfo,
byte[] splitRow,
long nonceGroup,
long nonce)
Split a region.
|
private void |
startActiveMasterManager(int infoPort) |
protected void |
startProcedureExecutor() |
private void |
startServiceThreads() |
void |
stop(String msg)
Stop this service.
|
private void |
stopChores() |
void |
stopMaster() |
private void |
stopProcedureExecutor() |
protected void |
stopServiceThreads()
Wait on all threads to finish.
|
private void |
switchSnapshotCleanup(boolean on) |
(package private) void |
switchSnapshotCleanup(boolean on,
boolean synchronous)
Turn on/off Snapshot Cleanup Chore
|
long |
truncateTable(TableName tableName,
boolean preserveSplits,
long nonceGroup,
long nonce)
Truncate a table
|
private void |
tryMigrateMetaLocationsFromZooKeeper() |
void |
updateConfigurationForQuotasObserver(org.apache.hadoop.conf.Configuration conf)
Adds the
MasterQuotasObserver to the list of configured Master observers to
automatically remove quotas for a table when that table is deleted. |
long |
updateReplicationPeerConfig(String peerId,
ReplicationPeerConfig peerConfig)
Update the peerConfig for the specified peer
|
protected void |
waitForMasterActive()
If configured to put regions on active master, wait till a backup master becomes active.
|
boolean |
waitForMetaOnline()
Check hbase:meta is up and ready for reading.
|
boolean |
waitForNamespaceOnline()
Check hbase:namespace table is assigned.
|
private void |
waitForRegionServers(MonitoredTask status) |
addRegion, closeRegion, createClusterConnection, createConnection, createRegionLoad, createRegionServerStatusStub, dumpRowLocks, finishRegionProcedure, getAccessChecker, getBlockCache, getBrokenStoreFileCleaner, getChoreService, getClusterConnection, getCompactedHFilesDischarger, getCompactionPressure, getCompactionRequestor, getCompactSplitThread, getConfiguration, getConnection, getCoordinatedStateManager, getDataRootDir, getEventLoopGroupConfig, getExecutorService, getFavoredNodesForRegion, getFileSystem, getFlushPressure, getFlushRequester, getFlushThroughputController, getHeapMemoryManager, getInfoServer, getLastSequenceId, getLeaseManager, getMasterAddressTracker, getMemStoreFlusher, getMetaRegionLocationCache, getMetrics, getMobFileCache, getMovedRegion, getNamedQueueRecorder, getNonceManager, getNumberOfOnlineRegions, getOnlineRegion, getOnlineRegions, getOnlineRegionsLocalContext, getOnlineTables, getRegion, getRegion, getRegionByEncodedName, getRegions, getRegions, getRegionServerAccounting, getRegionServerCoprocessorHost, getRegionServerCoprocessors, getRegionServerRpcQuotaManager, getRegionServerSpaceQuotaManager, getRegionsInTransitionInRS, getReplicationSourceService, getRetryPauseTime, getRpcServer, getRSMobFileCleanerChore, getRSRpcServices, getSecureBulkLoadManager, getStartcode, getTableDescriptors, getWAL, getWALFileSystem, getWalRoller, getWALRootDir, getWALs, getZKPermissionWatcher, handleReportForDutyResponse, initializeMemStoreChunkCreator, isAborted, isClusterUp, isShutDown, isStopped, isStopping, kill, movedRegionCacheExpiredTime, postOpenDeployTasks, regionLock, remoteProcedureComplete, removeRegion, reportFileArchivalForQuotas, reportRegionSizesForQuotas, reportRegionStateTransition, setAbortRequested, setupClusterConnection, shutdownChore, stop, toString, tryRegionServerReport, unassign, updateRegionFavoredNodesMapping, waitForServerOnline, walRollRequestFinished
activeCount, checkAccess, clone, countStackFrames, currentThread, destroy, dumpStack, enumerate, getAllStackTraces, getContextClassLoader, getDefaultUncaughtExceptionHandler, getId, getName, getPriority, getStackTrace, getState, getThreadGroup, getUncaughtExceptionHandler, holdsLock, interrupt, interrupted, isAlive, isDaemon, isInterrupted, join, join, join, resume, setContextClassLoader, setDaemon, setDefaultUncaughtExceptionHandler, setName, setPriority, setUncaughtExceptionHandler, sleep, sleep, start, stop, stop, suspend, yield
equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
getAccessChecker, getExecutorService, getTableDescriptors, getZKPermissionWatcher, isClusterUp
createConnection, getChoreService, getClusterConnection, getConfiguration, getConnection, getCoordinatedStateManager, getFileSystem, isStopping
private static final org.slf4j.Logger LOG
public static final String MASTER
private final ActiveMasterManager activeMasterManager
private final RegionServerTracker regionServerTracker
private DrainingServerTracker drainingServerTracker
LoadBalancerTracker loadBalancerTracker
private MetaLocationSyncer metaLocationSyncer
@InterfaceAudience.Private MasterAddressSyncer masterAddressSyncer
SnapshotCleanupTracker snapshotCleanupTracker
private SplitOrMergeTracker splitOrMergeTracker
private ClusterSchemaService clusterSchemaService
public static final String HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
public static final int DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS
public static final String HBASE_MASTER_CLEANER_INTERVAL
public static final int DEFAULT_HBASE_MASTER_CLEANER_INTERVAL
final MetricsMaster metricsMaster
private MasterFileSystem fileSystemManager
private MasterWalManager walManager
private SplitWALManager splitWALManager
private volatile ServerManager serverManager
private AssignmentManager assignmentManager
private ReplicationPeerManager replicationPeerManager
MemoryBoundedLogMessageBuffer rsFatals
private volatile boolean activeMaster
private final ProcedureEvent<?> initialized
volatile boolean serviceStarted
private final int maxBalancingTime
private final double maxRitPercent
private final LockManager lockManager
private LoadBalancer balancer
private BalancerChore balancerChore
private static boolean disableBalancerChoreForTest
private RegionNormalizerManager regionNormalizerManager
private ClusterStatusChore clusterStatusChore
private ClusterStatusPublisher clusterStatusPublisherChore
private SnapshotCleanerChore snapshotCleanerChore
CatalogJanitor catalogJanitorChore
private DirScanPool logCleanerPool
private LogCleaner logCleaner
private List<HFileCleaner> hfileCleaners
private List<org.apache.hadoop.fs.Path> hfileCleanerPaths
private DirScanPool sharedHFileCleanerPool
private DirScanPool exclusiveHFileCleanerPool
private ReplicationBarrierCleaner replicationBarrierCleaner
private MobFileCleanerChore mobFileCleanerChore
private MobFileCompactionChore mobFileCompactionChore
private RollingUpgradeChore rollingUpgradeChore
private final IdLock mobCompactionLock
private Map<TableName,AtomicInteger> mobCompactionStates
volatile MasterCoprocessorHost cpHost
private final boolean preLoadTableDescriptors
private long masterActiveTime
private long masterFinishedInitializationTime
Map<String,com.google.protobuf.Service> coprocessorServiceHandlers
SnapshotManager snapshotManager
private MasterProcedureManagerHost mpmHost
private RegionsRecoveryChore regionsRecoveryChore
private RegionsRecoveryConfigManager regionsRecoveryConfigManager
private volatile MasterQuotaManager quotaManager
private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier
private QuotaObserverChore quotaObserverChore
private SnapshotQuotaObserverChore snapshotQuotaChore
private ProcedureExecutor<MasterProcedureEnv> procedureExecutor
private ProcedureStore procedureStore
private MasterRegion masterRegion
private RegionServerList rsListStorage
private TableStateManager tableStateManager
private FavoredNodesManager favoredNodesManager
private org.apache.hbase.thirdparty.org.eclipse.jetty.server.Server masterJettyServer
private final boolean maintenanceMode
static final String MAINTENANCE_MODE
private final CachedClusterId cachedClusterId
public static final String WARMUP_BEFORE_MOVE
private static final boolean DEFAULT_WARMUP_BEFORE_MOVE
public HMaster(org.apache.hadoop.conf.Configuration conf) throws IOException
Remaining steps of initialization occur in
finishActiveMasterInitialization(MonitoredTask)
after the master becomes the active
one.
IOException
protected ActiveMasterManager createActiveMasterManager(ZKWatcher zk, ServerName sn, Server server) throws InterruptedIOException
InterruptedIOException
protected String getUseThisHostnameInstead(org.apache.hadoop.conf.Configuration conf)
getUseThisHostnameInstead
in class HRegionServer
private void registerConfigurationObservers()
public void run()
HRegionServer
run
in interface Runnable
run
in class HRegionServer
private int putUpJettyServer() throws IOException
IOException
protected void login(UserProvider user, String host) throws IOException
login
in class HRegionServer
IOException
protected void waitForMasterActive()
waitForMasterActive
in class HRegionServer
@InterfaceAudience.Private 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 boolean cacheTableDescriptor()
cacheTableDescriptor
in class HRegionServer
protected RSRpcServices createRpcServices() throws IOException
createRpcServices
in class HRegionServer
IOException
protected void configureInfoServer()
configureInfoServer
in class HRegionServer
private org.apache.hbase.thirdparty.org.eclipse.jetty.servlet.ServletHolder buildApiV1Servlet()
protected Class<? extends javax.servlet.http.HttpServlet> getDumpServlet()
getDumpServlet
in class HRegionServer
public MetricsMaster getMasterMetrics()
MasterServices
MetricsMaster
getMasterMetrics
in interface MasterServices
private void initializeZKBasedSystemTrackers() throws IOException, org.apache.zookeeper.KeeperException, ReplicationException
RegionServerTracker
, it
should have already been initialized along with ServerManager
.IOException
org.apache.zookeeper.KeeperException
ReplicationException
@InterfaceAudience.Private protected AssignmentManager createAssignmentManager(MasterServices master, MasterRegion masterRegion)
private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, org.apache.zookeeper.KeeperException
IOException
org.apache.zookeeper.KeeperException
private void finishActiveMasterInitialization(MonitoredTask status) throws IOException, InterruptedException, org.apache.zookeeper.KeeperException, ReplicationException
IOException
InterruptedException
org.apache.zookeeper.KeeperException
ReplicationException
private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor) throws IOException
IOException
@InterfaceAudience.Private public boolean waitForMetaOnline()
private boolean isRegionOnline(RegionInfo ri)
@InterfaceAudience.Private public boolean waitForNamespaceOnline()
@InterfaceAudience.Private public void updateConfigurationForQuotasObserver(org.apache.hadoop.conf.Configuration conf)
MasterQuotasObserver
to the list of configured Master observers to
automatically remove quotas for a table when that table is deleted.private void initMobCleaner()
@InterfaceAudience.Private protected ServerManager createServerManager(MasterServices master, RegionServerList storage) throws IOException
Create a ServerManager
instance.
Will be overridden in tests.
IOException
private void waitForRegionServers(MonitoredTask status) throws IOException, InterruptedException
IOException
InterruptedException
@InterfaceAudience.Private protected void initClusterSchemaService() throws IOException, InterruptedException
IOException
InterruptedException
private void initQuotaManager() throws IOException
IOException
private SpaceQuotaSnapshotNotifier createQuotaSnapshotNotifier()
public boolean isCatalogJanitorEnabled()
boolean isCleanerChoreEnabled()
public ServerManager getServerManager()
MasterServices
ServerManager
instance.getServerManager
in interface MasterServices
public MasterFileSystem getMasterFileSystem()
MasterServices
MasterFileSystem
utility class.getMasterFileSystem
in interface MasterServices
public MasterWalManager getMasterWalManager()
MasterServices
MasterWalManager
utility class.getMasterWalManager
in interface MasterServices
public SplitWALManager getSplitWALManager()
MasterServices
getSplitWALManager
in interface MasterServices
public TableStateManager getTableStateManager()
MasterServices
TableStateManager
getTableStateManager
in interface MasterServices
private void startServiceThreads() throws IOException
IOException
protected void stopServiceThreads()
HRegionServer
stopServiceThreads
in class HRegionServer
private void createProcedureExecutor() throws IOException
IOException
protected void startProcedureExecutor() throws IOException
IOException
void switchSnapshotCleanup(boolean on, boolean synchronous)
on
- indicates whether Snapshot Cleanup Chore is to be runprivate void switchSnapshotCleanup(boolean on)
private void stopProcedureExecutor()
private void stopChores()
InetAddress getRemoteInetAddress(int port, long serverStartCode) throws UnknownHostException
UnknownHostException
private int getMaxBalancingTime()
private int getMaxRegionsInTransition()
private void balanceThrottling(long nextBalanceStartTime, int maxRegionsInTransition, long cutoffTime)
nextBalanceStartTime
- The next balance plan start timemaxRegionsInTransition
- max number of regions in transitioncutoffTime
- when to exit balancerpublic BalanceResponse balance() throws IOException
IOException
public BalanceResponse balanceOrUpdateMetrics() throws IOException
balance()
. If the balance is not executed this
time, the metrics related to the balance will be updated. When balance is running, related
metrics will be updated at the same time. But if some checking logic failed and cause the
balancer exit early, we lost the chance to update balancer metrics. This will lead to user
missing the latest balancer info.IOException
public boolean skipRegionManagementAction(String action)
skipRegionManagementAction
in interface MasterServices
action
- the name of the action under consideration, for logging.true
when the caller should exit early, false
otherwise.public BalanceResponse balance(BalanceRequest request) throws IOException
IOException
public List<RegionPlan> executeRegionPlansWithThrottling(List<RegionPlan> plans)
executeRegionPlansWithThrottling
in interface MasterServices
plans
- to executepublic RegionNormalizerManager getRegionNormalizerManager()
MasterServices
RegionNormalizerManager
getRegionNormalizerManager
in interface MasterServices
public boolean normalizeRegions(NormalizeTableFilterParams ntfp, boolean isHighPriority) throws IOException
MasterServices
normalizeRegions
in interface MasterServices
ntfp
- Selection criteria for identifying which tables to normalize.isHighPriority
- true
when these requested tables should skip to the front of the
queue.true
when the request was submitted, false
otherwise.IOException
public String getClientIdAuditPrefix()
getClientIdAuditPrefix
in interface MasterServices
public void setCatalogJanitorEnabled(boolean b)
b
- If false, the catalog janitor won't do anything.public long mergeRegions(RegionInfo[] regionsToMerge, boolean forcible, long ng, long nonce) throws IOException
MasterServices
mergeRegions
in interface MasterServices
regionsToMerge
- daughter regions to mergeforcible
- whether to force to merge even two regions are not adjacentng
- used to detect duplicatenonce
- used to detect duplicateIOException
public long splitRegion(RegionInfo regionInfo, byte[] splitRow, long nonceGroup, long nonce) throws IOException
MasterServices
splitRegion
in interface MasterServices
regionInfo
- region to splitsplitRow
- split pointnonceGroup
- used to detect duplicatenonce
- used to detect duplicateIOException
@InterfaceAudience.Private public void move(byte[] encodedRegionName, byte[] destServerName) throws HBaseIOException
HBaseIOException
public long createTable(TableDescriptor tableDescriptor, byte[][] splitKeys, long nonceGroup, long nonce) throws IOException
MasterServices
createTable
in interface MasterServices
tableDescriptor
- The table definitionsplitKeys
- Starting row keys for the initial table regions. If null nn * a single region
is created.IOException
public long createSystemTable(TableDescriptor tableDescriptor) throws IOException
MasterServices
createSystemTable
in interface MasterServices
tableDescriptor
- The system table definition a single region is created.IOException
private void startActiveMasterManager(int infoPort) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperException
private static boolean isCatalogTable(TableName tableName)
public long deleteTable(TableName tableName, long nonceGroup, long nonce) throws IOException
MasterServices
deleteTable
in interface MasterServices
tableName
- The table name nnnIOException
public long 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 preserved nnnIOException
public long addColumn(TableName tableName, ColumnFamilyDescriptor column, long nonceGroup, long nonce) throws IOException
MasterServices
addColumn
in interface MasterServices
tableName
- The table namecolumn
- The column definition nnnIOException
public long modifyColumn(TableName tableName, ColumnFamilyDescriptor descriptor, long nonceGroup, long nonce) throws IOException
MasterServices
modifyColumn
in interface MasterServices
tableName
- The table namedescriptor
- The updated column definition nnnIOException
public long modifyColumnStoreFileTracker(TableName tableName, byte[] family, String dstSFT, long nonceGroup, long nonce) throws IOException
MasterServices
modifyColumnStoreFileTracker
in interface MasterServices
IOException
public long deleteColumn(TableName tableName, byte[] columnName, long nonceGroup, long nonce) throws IOException
MasterServices
deleteColumn
in interface MasterServices
tableName
- The table namecolumnName
- The column name nnnIOException
public long enableTable(TableName tableName, long nonceGroup, long nonce) throws IOException
MasterServices
enableTable
in interface MasterServices
tableName
- The table name nnnIOException
public long disableTable(TableName tableName, long nonceGroup, long nonce) throws IOException
MasterServices
disableTable
in interface MasterServices
tableName
- The table name nnnIOException
private long modifyTable(TableName tableName, HMaster.TableDescriptorGetter newDescriptorGetter, long nonceGroup, long nonce, boolean shouldCheckDescriptor) throws IOException
IOException
public long modifyTable(TableName tableName, TableDescriptor newDescriptor, long nonceGroup, long nonce) throws IOException
MasterServices
modifyTable
in interface MasterServices
tableName
- The table namenewDescriptor
- The updated table descriptor nnnIOException
public long modifyTableStoreFileTracker(TableName tableName, String dstSFT, long nonceGroup, long nonce) throws IOException
MasterServices
modifyTableStoreFileTracker
in interface MasterServices
IOException
public long restoreSnapshot(org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription snapshotDesc, long nonceGroup, long nonce, boolean restoreAcl, String customSFT) throws IOException
IOException
private void checkTableExists(TableName tableName) throws IOException, TableNotFoundException
IOException
TableNotFoundException
public void checkTableModifiable(TableName tableName) throws IOException, TableNotFoundException, TableNotDisabledException
MasterServices
checkTableModifiable
in interface MasterServices
tableName
- Name of table to check. nnnIOException
TableNotFoundException
TableNotDisabledException
public ClusterMetrics getClusterMetricsWithoutCoprocessor() throws InterruptedIOException
InterruptedIOException
public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet<ClusterMetrics.Option> options) throws InterruptedIOException
InterruptedIOException
private List<ServerName> getUnknownServers()
private Map<ServerName,ServerMetrics> getOnlineServers()
public ClusterMetrics getClusterMetrics() throws IOException
IOException
public ClusterMetrics getClusterMetrics(EnumSet<ClusterMetrics.Option> options) throws IOException
IOException
public Optional<ServerName> getActiveMaster()
getActiveMaster
in class HRegionServer
public List<ServerName> getBackupMasters()
getBackupMasters
in class HRegionServer
public int getActiveMasterInfoPort()
public int getBackupMasterInfoPort(ServerName sn)
sn
- is ServerName of the backup masterpublic Iterator<ServerName> getRegionServers()
getRegionServers
in class HRegionServer
public static String getLoadedCoprocessors()
public long getMasterStartTime()
public long getMasterActiveTime()
public long getMasterFinishedInitializationTime()
public int getNumWALFiles()
public ProcedureStore getProcedureStore()
public int getRegionServerInfoPort(ServerName sn)
public String getRegionServerVersion(ServerName sn)
getRegionServerVersion
in interface MasterServices
public void checkIfShouldMoveSystemRegionAsync()
MasterServices
checkIfShouldMoveSystemRegionAsync
in interface MasterServices
public String[] getMasterCoprocessors()
public void abort(String reason, Throwable cause)
HRegionServer
abort
in interface Abortable
abort
in class HRegionServer
reason
- Why we're aborting.cause
- Throwable that caused abort. Can be null.public ZKWatcher getZooKeeper()
Server
getZooKeeper
in interface Server
getZooKeeper
in class HRegionServer
public MasterCoprocessorHost getMasterCoprocessorHost()
MasterServices
MasterCoprocessorHost
getMasterCoprocessorHost
in interface MasterServices
public MasterQuotaManager getMasterQuotaManager()
MasterServices
MasterQuotaManager
getMasterQuotaManager
in interface MasterServices
public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor()
MasterServices
ProcedureExecutor
getMasterProcedureExecutor
in interface MasterServices
public ServerName getServerName()
Server
getServerName
in interface Server
getServerName
in class HRegionServer
public AssignmentManager getAssignmentManager()
MasterServices
AssignmentManager
getAssignmentManager
in interface MasterServices
public CatalogJanitor getCatalogJanitor()
MasterServices
CatalogJanitor
getCatalogJanitor
in interface MasterServices
public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer()
public void shutdown() throws IOException
IOException
public void stopMaster() throws IOException
IOException
public void stop(String msg)
Stoppable
stop
in interface Stoppable
stop
in class HRegionServer
msg
- Why we're stopping.@InterfaceAudience.Private protected void checkServiceStarted() throws ServerNotRunningYetException
ServerNotRunningYetException
void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException, MasterNotRunningException, MasterStoppedException
public boolean isActiveMaster()
isActiveMaster
in interface MasterServices
public boolean isInitialized()
isInitialized
in interface MasterServices
public boolean isOnline()
isOnline
in class HRegionServer
public boolean isInMaintenanceMode()
isInMaintenanceMode
in interface MasterServices
@InterfaceAudience.Private public void setInitialized(boolean isInitialized)
public ProcedureEvent<?> getInitializedEvent()
MasterServices
getInitializedEvent
in interface MasterServices
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)
public static void main(String[] args)
HMasterCommandLine
public HFileCleaner getHFileCleaner()
public List<HFileCleaner> getHFileCleaners()
public LogCleaner getLogCleaner()
public SnapshotManager getSnapshotManager()
getSnapshotManager
in interface MasterServices
public MasterProcedureManagerHost getMasterProcedureManagerHost()
getMasterProcedureManagerHost
in interface MasterServices
public ClusterSchema getClusterSchema()
MasterServices
ClusterSchema
getClusterSchema
in interface MasterServices
long createNamespace(NamespaceDescriptor namespaceDescriptor, long nonceGroup, long nonce) throws IOException
namespaceDescriptor
- descriptor for new NamespacenonceGroup
- Identifier for the source of the request, a client or process.nonce
- A unique identifier for this operation from the client or process
identified by nonceGroup
(the source must ensure each
operation gets a unique id).IOException
long modifyNamespace(NamespaceDescriptor newNsDescriptor, long nonceGroup, long nonce) throws IOException
nonceGroup
- Identifier for the source of the request, a client or process.nonce
- A unique identifier for this operation from the client or process identified
by nonceGroup
(the source must ensure each operation gets a
unique id).IOException
long deleteNamespace(String name, long nonceGroup, long nonce) throws IOException
nonceGroup
- Identifier for the source of the request, a client or process.nonce
- A unique identifier for this operation from the client or process identified
by nonceGroup
(the source must ensure each operation gets a
unique id).IOException
NamespaceDescriptor getNamespace(String name) throws IOException
name
- Name of the Namespacename
IOException
List<NamespaceDescriptor> getNamespaces() throws IOException
IOException
public List<String> listNamespaces() throws IOException
IOException
public List<TableName> listTableNamesByNamespace(String name) throws IOException
MasterServices
listTableNamesByNamespace
in interface MasterServices
name
- namespace nameIOException
public List<TableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException
MasterServices
listTableDescriptorsByNamespace
in interface MasterServices
name
- namespace name nnIOException
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<Procedure<?>> getProcedures() throws IOException
MasterServices
getProcedures
in interface MasterServices
IOException
public List<LockedResource> getLocks() throws IOException
MasterServices
getLocks
in interface MasterServices
IOException
public List<TableDescriptor> 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
private List<TableDescriptor> getTableDescriptors(List<TableDescriptor> htds, String namespace, String regex, List<TableName> tableNameList, boolean includeSysTables) throws IOException
NormalizeTableFilterParams
.IOException
private static void filterTablesByRegex(Collection<TableDescriptor> descriptors, Pattern pattern)
descriptors
- list of table descriptors to filterpattern
- the regex to usepublic long getLastMajorCompactionTimestamp(TableName table) throws IOException
getLastMajorCompactionTimestamp
in interface MasterServices
table
- the table for which last successful major compaction time is queriedIOException
public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException
MasterServices
getLastMajorCompactionTimestampForRegion
in interface MasterServices
IOException
public org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState getMobCompactionState(TableName tableName)
tableName
- The current table name.public void reportMobCompactionStart(TableName tableName) throws IOException
IOException
public void reportMobCompactionEnd(TableName tableName) throws IOException
IOException
public 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(MasterSwitchType switchType)
SplitOrMergeTracker
. If it is not initialized, false is
returned. If switchType is illegal, false will return.isSplitOrMergeEnabled
in interface MasterServices
switchType
- see MasterSwitchType
public String getLoadBalancerClassName()
LoadBalancer
class name. If none is set, a default is returned.LoadBalancer
in use.public SplitOrMergeTracker getSplitOrMergeTracker()
public LoadBalancer getLoadBalancer()
MasterServices
getLoadBalancer
in interface MasterServices
public FavoredNodesManager getFavoredNodesManager()
MasterServices
getFavoredNodesManager
in interface MasterServices
private long executePeerProcedure(ModifyPeerProcedure procedure) throws IOException
IOException
public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) throws ReplicationException, IOException
MasterServices
addReplicationPeer
in interface MasterServices
peerId
- a short name that identifies the peerpeerConfig
- configuration for the replication slave clusterenabled
- peer state, true if ENABLED and false if DISABLEDReplicationException
IOException
public long removeReplicationPeer(String peerId) throws ReplicationException, IOException
MasterServices
removeReplicationPeer
in interface MasterServices
peerId
- a short name that identifies the peerReplicationException
IOException
public long enableReplicationPeer(String peerId) throws ReplicationException, IOException
MasterServices
enableReplicationPeer
in interface MasterServices
peerId
- a short name that identifies the peerReplicationException
IOException
public long disableReplicationPeer(String peerId) throws ReplicationException, IOException
MasterServices
disableReplicationPeer
in interface MasterServices
peerId
- a short name that identifies the peerReplicationException
IOException
public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws ReplicationException, IOException
MasterServices
getReplicationPeerConfig
in interface MasterServices
peerId
- a short name that identifies the peerReplicationException
IOException
public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) throws ReplicationException, IOException
MasterServices
updateReplicationPeerConfig
in interface MasterServices
peerId
- a short name that identifies the peerpeerConfig
- new config for the peerReplicationException
IOException
public List<ReplicationPeerDescription> listReplicationPeers(String regex) throws ReplicationException, IOException
MasterServices
listReplicationPeers
in interface MasterServices
regex
- The regular expression to match peer idReplicationException
IOException
public void decommissionRegionServers(List<ServerName> servers, boolean offload) throws HBaseIOException
servers
- Region servers to decommission.HBaseIOException
public List<ServerName> listDecommissionedRegionServers()
public void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) throws IOException
server
- Region server to remove decommission marker from.IOException
public LockManager getLockManager()
MasterServices
LockManager
to lock namespaces/tables/regions.getLockManager
in interface MasterServices
public QuotaObserverChore getQuotaObserverChore()
public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier()
private RemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv,?> getRemoteProcedure(long procId)
public void remoteProcedureCompleted(long procId)
public void remoteProcedureFailed(long procId, RemoteProcedureException error)
long reopenRegions(TableName tableName, List<byte[]> regionNames, long nonceGroup, long nonce) throws IOException
tableName
- The current table nameregionNames
- The region names of the regions to reopennonceGroup
- Identifier for the source of the request, a client or processnonce
- A unique identifier for this operation from the client or process identified
by nonceGroup
(the source must ensure each operation gets a
unique id).IOException
- if reopening region fails while running procedurepublic ReplicationPeerManager getReplicationPeerManager()
MasterServices
ReplicationPeerManager
.getReplicationPeerManager
in interface MasterServices
public HashMap<String,List<Pair<ServerName,ReplicationLoadSource>>> getReplicationLoad(ServerName[] serverNames)
@InterfaceAudience.Private public static void decorateMasterConfiguration(org.apache.hadoop.conf.Configuration conf)
public Map<String,ReplicationStatus> getWalGroupsReplicationStatus()
getWalGroupsReplicationStatus
in class HRegionServer
public HbckChore getHbckChore()
public String getClusterId()
getClusterId
in class HRegionServer
public void runReplicationBarrierCleaner()
MasterServices
runReplicationBarrierCleaner
in interface MasterServices
public SnapshotQuotaObserverChore getSnapshotQuotaObserverChore()
public CompactionState getCompactionState(TableName tableName)
tableName
- The table namepublic MetaLocationSyncer getMetaLocationSyncer()
MasterServices
getMetaLocationSyncer
in interface MasterServices
public void flushMasterStore() throws IOException
MasterServices
flushMasterStore
in interface MasterServices
IOException
public MasterRegion getMasterRegion()
public Collection<ServerName> getLiveRegionServers()
void setLoadBalancer(LoadBalancer loadBalancer)
void setAssignmentManager(AssignmentManager assignmentManager)
static void setDisableBalancerChoreForTest(boolean disable)
public void onConfigurationChange(org.apache.hadoop.conf.Configuration newConf)
ConfigurationObserver
ConfigurationManager
object when the
Configuration
object is reloaded from disk.onConfigurationChange
in interface ConfigurationObserver
onConfigurationChange
in class HRegionServer
public ConfigurationManager getConfigurationManager()
private void setQuotasObserver(org.apache.hadoop.conf.Configuration conf)
private void initializeCoprocessorHost(org.apache.hadoop.conf.Configuration conf)
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.