HBASE-16219 Move meta bootstrap out of HMaster

This commit is contained in:
Matteo Bertozzi 2016-07-14 06:14:53 -07:00
parent 4b7933bd65
commit a55af38689
3 changed files with 310 additions and 204 deletions

View File

@ -0,0 +1,257 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.HashSet;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
* Used by the HMaster on startup to split meta logs and assign the meta table.
*/
@InterfaceAudience.Private
public class MasterMetaBootstrap {
private static final Log LOG = LogFactory.getLog(MasterMetaBootstrap.class);
private final MonitoredTask status;
private final HMaster master;
private Set<ServerName> previouslyFailedServers;
private Set<ServerName> previouslyFailedMetaRSs;
public MasterMetaBootstrap(final HMaster master, final MonitoredTask status) {
this.master = master;
this.status = status;
}
public void splitMetaLogsBeforeAssignment() throws IOException, KeeperException {
// get a list for previously failed RS which need log splitting work
// we recover hbase:meta region servers inside master initialization and
// handle other failed servers in SSH in order to start up master node ASAP
previouslyFailedServers = master.getMasterWalManager().getFailedServersFromLogFolders();
// log splitting for hbase:meta server
ServerName oldMetaServerLocation = master.getMetaTableLocator()
.getMetaRegionLocation(master.getZooKeeper());
if (oldMetaServerLocation != null && previouslyFailedServers.contains(oldMetaServerLocation)) {
splitMetaLogBeforeAssignment(oldMetaServerLocation);
// Note: we can't remove oldMetaServerLocation from previousFailedServers list because it
// may also host user regions
}
previouslyFailedMetaRSs = getPreviouselyFailedMetaServersFromZK();
// need to use union of previouslyFailedMetaRSs recorded in ZK and previouslyFailedServers
// instead of previouslyFailedMetaRSs alone to address the following two situations:
// 1) the chained failure situation(recovery failed multiple times in a row).
// 2) master get killed right before it could delete the recovering hbase:meta from ZK while the
// same server still has non-meta wals to be replayed so that
// removeStaleRecoveringRegionsFromZK can't delete the stale hbase:meta region
// Passing more servers into splitMetaLog is all right. If a server doesn't have hbase:meta wal,
// there is no op for the server.
previouslyFailedMetaRSs.addAll(previouslyFailedServers);
}
public void assignMeta() throws InterruptedException, IOException, KeeperException {
assignMeta(previouslyFailedMetaRSs, HRegionInfo.DEFAULT_REPLICA_ID);
}
public void processDeadServers() throws IOException {
// Master has recovered hbase:meta region server and we put
// other failed region servers in a queue to be handled later by SSH
for (ServerName tmpServer : previouslyFailedServers) {
master.getServerManager().processDeadServer(tmpServer, true);
}
}
public void assignMetaReplicas()
throws IOException, InterruptedException, KeeperException {
int numReplicas = master.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
HConstants.DEFAULT_META_REPLICA_NUM);
final Set<ServerName> EMPTY_SET = new HashSet<ServerName>();
for (int i = 1; i < numReplicas; i++) {
assignMeta(EMPTY_SET, i);
}
unassignExcessMetaReplica(numReplicas);
}
private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
if (RecoveryMode.LOG_REPLAY == master.getMasterWalManager().getLogRecoveryMode()) {
// In log replay mode, we mark hbase:meta region as recovering in ZK
master.getMasterWalManager().prepareLogReplay(currentMetaServer,
Collections.<HRegionInfo>singleton(HRegionInfo.FIRST_META_REGIONINFO));
} else {
// In recovered.edits mode: create recovered edits file for hbase:meta server
master.getMasterWalManager().splitMetaLog(currentMetaServer);
}
}
private void unassignExcessMetaReplica(int numMetaReplicasConfigured) {
final ZooKeeperWatcher zooKeeper = master.getZooKeeper();
// unassign the unneeded replicas (for e.g., if the previous master was configured
// with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
try {
List<String> metaReplicaZnodes = zooKeeper.getMetaReplicaNodes();
for (String metaReplicaZnode : metaReplicaZnodes) {
int replicaId = zooKeeper.getMetaReplicaIdFromZnode(metaReplicaZnode);
if (replicaId >= numMetaReplicasConfigured) {
RegionState r = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);
LOG.info("Closing excess replica of meta region " + r.getRegion());
// send a close and wait for a max of 30 seconds
ServerManager.closeRegionSilentlyAndWait(master.getClusterConnection(),
r.getServerName(), r.getRegion(), 30000);
ZKUtil.deleteNode(zooKeeper, zooKeeper.getZNodeForReplica(replicaId));
}
}
} catch (Exception ex) {
// ignore the exception since we don't want the master to be wedged due to potential
// issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
LOG.warn("Ignoring exception " + ex);
}
}
/**
* Check <code>hbase:meta</code> is assigned. If not, assign it.
*/
protected void assignMeta(Set<ServerName> previouslyFailedMetaRSs, int replicaId)
throws InterruptedException, IOException, KeeperException {
final AssignmentManager assignmentManager = master.getAssignmentManager();
// Work on meta region
int assigned = 0;
long timeout = master.getConfiguration().getLong("hbase.catalog.verification.timeout", 1000);
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
status.setStatus("Assigning hbase:meta region");
} else {
status.setStatus("Assigning hbase:meta region, replicaId " + replicaId);
}
// Get current meta state from zk.
RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), replicaId);
HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO,
replicaId);
RegionStates regionStates = assignmentManager.getRegionStates();
regionStates.createRegionState(hri, metaState.getState(),
metaState.getServerName(), null);
if (!metaState.isOpened() || !master.getMetaTableLocator().verifyMetaRegionLocation(
master.getClusterConnection(), master.getZooKeeper(), timeout, replicaId)) {
ServerName currentMetaServer = metaState.getServerName();
if (master.getServerManager().isServerOnline(currentMetaServer)) {
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
LOG.info("Meta was in transition on " + currentMetaServer);
} else {
LOG.info("Meta with replicaId " + replicaId + " was in transition on " +
currentMetaServer);
}
assignmentManager.processRegionsInTransition(Collections.singletonList(metaState));
} else {
if (currentMetaServer != null) {
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
splitMetaLogBeforeAssignment(currentMetaServer);
regionStates.logSplit(HRegionInfo.FIRST_META_REGIONINFO);
previouslyFailedMetaRSs.add(currentMetaServer);
}
}
LOG.info("Re-assigning hbase:meta with replicaId, " + replicaId +
" it was on " + currentMetaServer);
assignmentManager.assignMeta(hri);
}
assigned++;
}
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
// TODO: should we prevent from using state manager before meta was initialized?
// tableStateManager.start();
master.getTableStateManager()
.setTableState(TableName.META_TABLE_NAME, TableState.State.ENABLED);
}
if ((RecoveryMode.LOG_REPLAY == master.getMasterWalManager().getLogRecoveryMode())
&& (!previouslyFailedMetaRSs.isEmpty())) {
// replay WAL edits mode need new hbase:meta RS is assigned firstly
status.setStatus("replaying log for Meta Region");
master.getMasterWalManager().splitMetaLog(previouslyFailedMetaRSs);
}
assignmentManager.setEnabledTable(TableName.META_TABLE_NAME);
master.getTableStateManager().start();
// Make sure a hbase:meta location is set. We need to enable SSH here since
// if the meta region server is died at this time, we need it to be re-assigned
// by SSH so that system tables can be assigned.
// No need to wait for meta is assigned = 0 when meta is just verified.
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableCrashedServerProcessing(assigned != 0);
LOG.info("hbase:meta with replicaId " + replicaId + " assigned=" + assigned + ", location="
+ master.getMetaTableLocator().getMetaRegionLocation(master.getZooKeeper(), replicaId));
status.setStatus("META assigned.");
}
private void enableCrashedServerProcessing(final boolean waitForMeta)
throws IOException, InterruptedException {
// If crashed server processing is disabled, we enable it and expire those dead but not expired
// servers. This is required so that if meta is assigning to a server which dies after
// assignMeta starts assignment, ServerCrashProcedure can re-assign it. Otherwise, we will be
// stuck here waiting forever if waitForMeta is specified.
if (!master.isServerCrashProcessingEnabled()) {
master.setServerCrashProcessingEnabled(true);
master.getServerManager().processQueuedDeadServers();
}
if (waitForMeta) {
master.getMetaTableLocator().waitMetaRegionLocation(master.getZooKeeper());
}
}
/**
* This function returns a set of region server names under hbase:meta recovering region ZK node
* @return Set of meta server names which were recorded in ZK
*/
private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
final ZooKeeperWatcher zooKeeper = master.getZooKeeper();
Set<ServerName> result = new HashSet<ServerName>();
String metaRecoveringZNode = ZKUtil.joinZNode(zooKeeper.recoveringRegionsZNode,
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(zooKeeper, metaRecoveringZNode);
if (regionFailedServers == null) return result;
for (String failedServer : regionFailedServers) {
ServerName server = ServerName.parseServerName(failedServer);
result.add(server);
}
return result;
}
}

View File

@ -33,11 +33,9 @@ import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -84,7 +82,6 @@ import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
@ -132,7 +129,6 @@ import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
@ -162,7 +158,6 @@ import org.apache.hadoop.hbase.util.ZKDataMigrator;
import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTracker;
@ -717,39 +712,14 @@ public class HMaster extends HRegionServer implements MasterServices {
sleeper.skipSleepCycle();
// Wait for region servers to report in
this.serverManager.waitForRegionServers(status);
// Check zk for region servers that are up but didn't register
for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
// The isServerOnline check is opportunistic, correctness is handled inside
if (!this.serverManager.isServerOnline(sn)
&& serverManager.checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
LOG.info("Registered server found up in zk but who has not yet reported in: " + sn);
}
}
status.setStatus("Wait for region servers to report in");
waitForRegionServers(status);
// get a list for previously failed RS which need log splitting work
// we recover hbase:meta region servers inside master initialization and
// handle other failed servers in SSH in order to start up master node ASAP
Set<ServerName> previouslyFailedServers =
this.walManager.getFailedServersFromLogFolders();
// log splitting for hbase:meta server
ServerName oldMetaServerLocation = metaTableLocator.getMetaRegionLocation(this.getZooKeeper());
if (oldMetaServerLocation != null && previouslyFailedServers.contains(oldMetaServerLocation)) {
splitMetaLogBeforeAssignment(oldMetaServerLocation);
// Note: we can't remove oldMetaServerLocation from previousFailedServers list because it
// may also host user regions
}
Set<ServerName> previouslyFailedMetaRSs = getPreviouselyFailedMetaServersFromZK();
// need to use union of previouslyFailedMetaRSs recorded in ZK and previouslyFailedServers
// instead of previouslyFailedMetaRSs alone to address the following two situations:
// 1) the chained failure situation(recovery failed multiple times in a row).
// 2) master get killed right before it could delete the recovering hbase:meta from ZK while the
// same server still has non-meta wals to be replayed so that
// removeStaleRecoveringRegionsFromZK can't delete the stale hbase:meta region
// Passing more servers into splitMetaLog is all right. If a server doesn't have hbase:meta wal,
// there is no op for the server.
previouslyFailedMetaRSs.addAll(previouslyFailedServers);
MasterMetaBootstrap metaBootstrap = createMetaBootstrap(this, status);
metaBootstrap.splitMetaLogsBeforeAssignment();
this.initializationBeforeMetaAssignment = true;
@ -769,7 +739,8 @@ public class HMaster extends HRegionServer implements MasterServices {
// Make sure meta assigned before proceeding.
status.setStatus("Assigning Meta Region");
assignMeta(status, previouslyFailedMetaRSs, HRegionInfo.DEFAULT_REPLICA_ID);
metaBootstrap.assignMeta();
// check if master is shutting down because above assignMeta could return even hbase:meta isn't
// assigned when master is shutting down
if (isStopped()) return;
@ -784,11 +755,7 @@ public class HMaster extends HRegionServer implements MasterServices {
ZKUtil.deleteChildrenRecursively(getZooKeeper(), getZooKeeper().tableZNode);
status.setStatus("Submitting log splitting work for previously failed region servers");
// Master has recovered hbase:meta region server and we put
// other failed region servers in a queue to be handled later by SSH
for (ServerName tmpServer : previouslyFailedServers) {
this.serverManager.processDeadServer(tmpServer, true);
}
metaBootstrap.processDeadServers();
// Fix up assignment manager status
status.setStatus("Starting assignment manager");
@ -830,14 +797,8 @@ public class HMaster extends HRegionServer implements MasterServices {
// Set master as 'initialized'.
setInitialized(true);
// assign the meta replicas
Set<ServerName> EMPTY_SET = new HashSet<ServerName>();
int numReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
HConstants.DEFAULT_META_REPLICA_NUM);
for (int i = 1; i < numReplicas; i++) {
assignMeta(status, EMPTY_SET, i);
}
unassignExcessMetaReplica(zooKeeper, numReplicas);
status.setStatus("Assign meta replicas");
metaBootstrap.assignMetaReplicas();
status.setStatus("Starting quota manager");
initQuotaManager();
@ -851,22 +812,10 @@ public class HMaster extends HRegionServer implements MasterServices {
status.setStatus("Checking ZNode ACLs");
zooKeeper.checkAndSetZNodeAcls();
status.setStatus("Initializing MOB Cleaner");
initMobCleaner();
status.setStatus("Calling postStartMaster coprocessors");
this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this);
getChoreService().scheduleChore(expiredMobFileCleanerChore);
int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD);
if (mobCompactionPeriod > 0) {
this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod);
getChoreService().scheduleChore(mobCompactChore);
} else {
LOG
.info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled");
}
this.mobCompactThread = new MasterMobCompactionThread(this);
if (this.cpHost != null) {
// don't let cp initialization errors kill the master
try {
@ -878,6 +827,32 @@ public class HMaster extends HRegionServer implements MasterServices {
zombieDetector.interrupt();
}
private void initMobCleaner() {
this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this);
getChoreService().scheduleChore(expiredMobFileCleanerChore);
int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD);
if (mobCompactionPeriod > 0) {
this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod);
getChoreService().scheduleChore(mobCompactChore);
} else {
LOG
.info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled");
}
this.mobCompactThread = new MasterMobCompactionThread(this);
}
/**
* Create a {@link MasterMetaBootstrap} instance.
*/
MasterMetaBootstrap createMetaBootstrap(final HMaster master, final MonitoredTask status) {
// We put this out here in a method so can do a Mockito.spy and stub it out
// w/ a mocked up MasterMetaBootstrap.
return new MasterMetaBootstrap(master, status);
}
/**
* Create a {@link ServerManager} instance.
*/
@ -888,103 +863,19 @@ public class HMaster extends HRegionServer implements MasterServices {
return new ServerManager(master);
}
private void unassignExcessMetaReplica(ZooKeeperWatcher zkw, int numMetaReplicasConfigured) {
// unassign the unneeded replicas (for e.g., if the previous master was configured
// with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
try {
List<String> metaReplicaZnodes = zooKeeper.getMetaReplicaNodes();
for (String metaReplicaZnode : metaReplicaZnodes) {
int replicaId = zooKeeper.getMetaReplicaIdFromZnode(metaReplicaZnode);
if (replicaId >= numMetaReplicasConfigured) {
RegionState r = MetaTableLocator.getMetaRegionState(zkw, replicaId);
LOG.info("Closing excess replica of meta region " + r.getRegion());
// send a close and wait for a max of 30 seconds
ServerManager.closeRegionSilentlyAndWait(getClusterConnection(), r.getServerName(),
r.getRegion(), 30000);
ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(replicaId));
}
private void waitForRegionServers(final MonitoredTask status)
throws IOException, InterruptedException {
this.serverManager.waitForRegionServers(status);
// Check zk for region servers that are up but didn't register
for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
// The isServerOnline check is opportunistic, correctness is handled inside
if (!this.serverManager.isServerOnline(sn)
&& serverManager.checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
LOG.info("Registered server found up in zk but who has not yet reported in: " + sn);
}
} catch (Exception ex) {
// ignore the exception since we don't want the master to be wedged due to potential
// issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
LOG.warn("Ignoring exception " + ex);
}
}
/**
* Check <code>hbase:meta</code> is assigned. If not, assign it.
*/
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
throws InterruptedException, IOException, KeeperException {
// Work on meta region
int assigned = 0;
long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
status.setStatus("Assigning hbase:meta region");
} else {
status.setStatus("Assigning hbase:meta region, replicaId " + replicaId);
}
// Get current meta state from zk.
RegionState metaState = MetaTableLocator.getMetaRegionState(getZooKeeper(), replicaId);
HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO,
replicaId);
RegionStates regionStates = assignmentManager.getRegionStates();
regionStates.createRegionState(hri, metaState.getState(),
metaState.getServerName(), null);
if (!metaState.isOpened() || !metaTableLocator.verifyMetaRegionLocation(
this.getClusterConnection(), this.getZooKeeper(), timeout, replicaId)) {
ServerName currentMetaServer = metaState.getServerName();
if (serverManager.isServerOnline(currentMetaServer)) {
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
LOG.info("Meta was in transition on " + currentMetaServer);
} else {
LOG.info("Meta with replicaId " + replicaId + " was in transition on " +
currentMetaServer);
}
assignmentManager.processRegionsInTransition(Arrays.asList(metaState));
} else {
if (currentMetaServer != null) {
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
splitMetaLogBeforeAssignment(currentMetaServer);
regionStates.logSplit(HRegionInfo.FIRST_META_REGIONINFO);
previouslyFailedMetaRSs.add(currentMetaServer);
}
}
LOG.info("Re-assigning hbase:meta with replicaId, " + replicaId +
" it was on " + currentMetaServer);
assignmentManager.assignMeta(hri);
}
assigned++;
}
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
// TODO: should we prevent from using state manager before meta was initialized?
// tableStateManager.start();
getTableStateManager().setTableState(TableName.META_TABLE_NAME, TableState.State.ENABLED);
}
if ((RecoveryMode.LOG_REPLAY == this.getMasterWalManager().getLogRecoveryMode())
&& (!previouslyFailedMetaRSs.isEmpty())) {
// replay WAL edits mode need new hbase:meta RS is assigned firstly
status.setStatus("replaying log for Meta Region");
this.walManager.splitMetaLog(previouslyFailedMetaRSs);
}
this.assignmentManager.setEnabledTable(TableName.META_TABLE_NAME);
tableStateManager.start();
// Make sure a hbase:meta location is set. We need to enable SSH here since
// if the meta region server is died at this time, we need it to be re-assigned
// by SSH so that system tables can be assigned.
// No need to wait for meta is assigned = 0 when meta is just verified.
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableCrashedServerProcessing(assigned != 0);
LOG.info("hbase:meta with replicaId " + replicaId + " assigned=" + assigned + ", location="
+ metaTableLocator.getMetaRegionLocation(this.getZooKeeper(), replicaId));
status.setStatus("META assigned.");
}
void initClusterSchemaService() throws IOException, InterruptedException {
this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
this.clusterSchemaService.startAndWait();
@ -1003,52 +894,6 @@ public class HMaster extends HRegionServer implements MasterServices {
catalogJanitorChore.getEnabled() : false;
}
private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
if (RecoveryMode.LOG_REPLAY == this.getMasterWalManager().getLogRecoveryMode()) {
// In log replay mode, we mark hbase:meta region as recovering in ZK
Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
regions.add(HRegionInfo.FIRST_META_REGIONINFO);
this.walManager.prepareLogReplay(currentMetaServer, regions);
} else {
// In recovered.edits mode: create recovered edits file for hbase:meta server
this.walManager.splitMetaLog(currentMetaServer);
}
}
private void enableCrashedServerProcessing(final boolean waitForMeta)
throws IOException, InterruptedException {
// If crashed server processing is disabled, we enable it and expire those dead but not expired
// servers. This is required so that if meta is assigning to a server which dies after
// assignMeta starts assignment, ServerCrashProcedure can re-assign it. Otherwise, we will be
// stuck here waiting forever if waitForMeta is specified.
if (!isServerCrashProcessingEnabled()) {
setServerCrashProcessingEnabled(true);
this.serverManager.processQueuedDeadServers();
}
if (waitForMeta) {
metaTableLocator.waitMetaRegionLocation(this.getZooKeeper());
}
}
/**
* This function returns a set of region server names under hbase:meta recovering region ZK node
* @return Set of meta server names which were recorded in ZK
*/
private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
Set<ServerName> result = new HashSet<ServerName>();
String metaRecoveringZNode = ZKUtil.joinZNode(zooKeeper.recoveringRegionsZNode,
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(zooKeeper, metaRecoveringZNode);
if (regionFailedServers == null) return result;
for(String failedServer : regionFailedServers) {
ServerName server = ServerName.parseServerName(failedServer);
result.add(server);
}
return result;
}
@Override
public TableDescriptors getTableDescriptors() {
return this.tableDescriptors;

View File

@ -267,8 +267,12 @@ public class TestMasterNoCluster {
TESTUTIL.getConfiguration());
HMaster master = new HMaster(conf, cp) {
@Override
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMeatRSs, int replicaId)
{ }
MasterMetaBootstrap createMetaBootstrap(final HMaster master, final MonitoredTask status) {
return new MasterMetaBootstrap(this, status) {
@Override
protected void assignMeta(Set<ServerName> previouslyFailedMeatRSs, int replicaId) { }
};
}
@Override
void initClusterSchemaService() throws IOException, InterruptedException {}