HBASE-6060 Regions's in OPENING state from failed regionservers takes a long time to recover

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1404759 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
jxiang 2012-11-01 20:50:22 +00:00
parent 5dfdc621f1
commit ddb9e96654
5 changed files with 149 additions and 116 deletions

View File

@ -521,8 +521,8 @@ public class AssignmentManager extends ZooKeeperListener {
break; break;
case RS_ZK_REGION_OPENING: case RS_ZK_REGION_OPENING:
regionStates.updateRegionState(rt, RegionState.State.OPENING);
if (regionInfo.isMetaTable() || !serverManager.isServerOnline(sn)) { if (regionInfo.isMetaTable() || !serverManager.isServerOnline(sn)) {
regionStates.updateRegionState(rt, RegionState.State.OPENING);
// If ROOT or .META. table is waiting for timeout monitor to assign // If ROOT or .META. table is waiting for timeout monitor to assign
// it may take lot of time when the assignment.timeout.period is // it may take lot of time when the assignment.timeout.period is
// the default value which may be very long. We will not be able // the default value which may be very long. We will not be able
@ -532,11 +532,10 @@ public class AssignmentManager extends ZooKeeperListener {
// some time for timeout monitor to kick in. We know the region // some time for timeout monitor to kick in. We know the region
// won't open. So we will assign the opening // won't open. So we will assign the opening
// region immediately too. // region immediately too.
//
// Otherwise, just insert region into RIT. If the state never
// updates, the timeout will trigger new assignment
processOpeningState(regionInfo); processOpeningState(regionInfo);
} else {
// Just insert region into RIT.
// If this never updates the timeout will trigger new assignment
regionStates.updateRegionState(rt, RegionState.State.OPENING);
} }
break; break;
@ -940,19 +939,24 @@ public class AssignmentManager extends ZooKeeperListener {
"clearing from RIT; rs=" + rs); "clearing from RIT; rs=" + rs);
regionOffline(rs.getRegion()); regionOffline(rs.getRegion());
} else { } else {
LOG.debug("The znode of region " + regionInfo.getRegionNameAsString() String regionNameStr = regionInfo.getRegionNameAsString();
+ " has been deleted."); LOG.debug("The znode of region " + regionNameStr
+ " has been deleted.");
if (rs.isOpened()) { if (rs.isOpened()) {
ServerName serverName = rs.getServerName(); ServerName serverName = rs.getServerName();
regionOnline(regionInfo, serverName); regionOnline(regionInfo, serverName);
LOG.info("The master has opened the region " LOG.info("The master has opened the region "
+ regionInfo.getRegionNameAsString() + " that was online on " + regionNameStr + " that was online on " + serverName);
+ serverName); boolean disabled = getZKTable().isDisablingOrDisabledTable(
if (getZKTable().isDisablingOrDisabledTable( regionInfo.getTableNameAsString());
regionInfo.getTableNameAsString())) { if (!serverManager.isServerOnline(serverName) && !disabled) {
LOG.debug("Opened region " LOG.info("Opened region " + regionNameStr
+ regionInfo.getRegionNameAsString() + " but " + "but the region server is offline, reassign the region");
+ "this table is disabled, triggering close of region"); assign(regionInfo, true);
} else if (disabled) {
// if server is offline, no hurt to unassign again
LOG.info("Opened region " + regionNameStr
+ "but this table is disabled, triggering close of region");
unassign(regionInfo); unassign(regionInfo);
} }
} }
@ -1369,8 +1373,10 @@ public class AssignmentManager extends ZooKeeperListener {
ServerName server = state.getServerName(); ServerName server = state.getServerName();
// ClosedRegionhandler can remove the server from this.regions // ClosedRegionhandler can remove the server from this.regions
if (!serverManager.isServerOnline(server)) { if (!serverManager.isServerOnline(server)) {
// delete the node. if no node exists need not bother. if (transitionInZK) {
deleteClosingOrClosedNode(region); // delete the node. if no node exists need not bother.
deleteClosingOrClosedNode(region);
}
regionOffline(region); regionOffline(region);
return; return;
} }
@ -1391,7 +1397,9 @@ public class AssignmentManager extends ZooKeeperListener {
t = ((RemoteException)t).unwrapRemoteException(); t = ((RemoteException)t).unwrapRemoteException();
} }
if (t instanceof NotServingRegionException) { if (t instanceof NotServingRegionException) {
deleteClosingOrClosedNode(region); if (transitionInZK) {
deleteClosingOrClosedNode(region);
}
regionOffline(region); regionOffline(region);
return; return;
} else if (t instanceof RegionAlreadyInTransitionException) { } else if (t instanceof RegionAlreadyInTransitionException) {
@ -1899,29 +1907,29 @@ public class AssignmentManager extends ZooKeeperListener {
} }
/** /**
*
* @param region regioninfo of znode to be deleted. * @param region regioninfo of znode to be deleted.
*/ */
public void deleteClosingOrClosedNode(HRegionInfo region) { public void deleteClosingOrClosedNode(HRegionInfo region) {
String encodedName = region.getEncodedName();
try { try {
if (!ZKAssign.deleteNode(watcher, region.getEncodedName(), if (!ZKAssign.deleteNode(watcher, encodedName,
EventHandler.EventType.M_ZK_REGION_CLOSING)) { EventHandler.EventType.M_ZK_REGION_CLOSING)) {
boolean deleteNode = ZKAssign.deleteNode(watcher, region boolean deleteNode = ZKAssign.deleteNode(watcher,
.getEncodedName(), EventHandler.EventType.RS_ZK_REGION_CLOSED); encodedName, EventHandler.EventType.RS_ZK_REGION_CLOSED);
// TODO : We don't abort if the delete node returns false. Is there any // TODO : We don't abort if the delete node returns false. Is there any
// such corner case? // such corner case?
if (!deleteNode) { if (!deleteNode) {
LOG.error("The deletion of the CLOSED node for the region " LOG.error("The deletion of the CLOSED node for the region "
+ region.getEncodedName() + " returned " + deleteNode); + encodedName + " returned " + deleteNode);
} }
} }
} catch (NoNodeException e) { } catch (NoNodeException e) {
LOG.debug("CLOSING/CLOSED node for the region " + region.getEncodedName() LOG.debug("CLOSING/CLOSED node for the region " + encodedName
+ " already deleted"); + " already deleted");
} catch (KeeperException ke) { } catch (KeeperException ke) {
server.abort( server.abort(
"Unexpected ZK exception deleting node CLOSING/CLOSED for the region " "Unexpected ZK exception deleting node CLOSING/CLOSED for the region "
+ region.getEncodedName(), ke); + encodedName, ke);
return; return;
} }
} }

View File

@ -71,7 +71,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
} }
public void updateTimestampToNow() { public void updateTimestampToNow() {
this.stamp.set(System.currentTimeMillis()); setTimestamp(System.currentTimeMillis());
} }
public State getState() { public State getState() {
@ -134,6 +134,10 @@ public class RegionState implements org.apache.hadoop.io.Writable {
return isOnServer(sn) && (isPendingClose() || isClosing()); return isOnServer(sn) && (isPendingClose() || isClosing());
} }
public boolean isOnServer(final ServerName sn) {
return serverName != null && serverName.equals(sn);
}
@Override @Override
public String toString() { public String toString() {
return "{" + region.getRegionNameAsString() return "{" + region.getRegionNameAsString()
@ -242,8 +246,8 @@ public class RegionState implements org.apache.hadoop.io.Writable {
return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),state,proto.getStamp(),null); return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),state,proto.getStamp(),null);
} }
private boolean isOnServer(final ServerName sn) { protected void setTimestamp(final long timestamp) {
return serverName != null && serverName.equals(sn); stamp.set(timestamp);
} }
/** /**

View File

@ -329,9 +329,8 @@ public class RegionStates {
// of this server from online map of regions. // of this server from online map of regions.
List<RegionState> rits = new ArrayList<RegionState>(); List<RegionState> rits = new ArrayList<RegionState>();
Set<HRegionInfo> assignedRegions = serverHoldings.get(sn); Set<HRegionInfo> assignedRegions = serverHoldings.get(sn);
if (assignedRegions == null || assignedRegions.isEmpty()) { if (assignedRegions == null) {
// No regions on this server, we are done, return empty list of RITs assignedRegions = new HashSet<HRegionInfo>();
return rits;
} }
for (HRegionInfo region : assignedRegions) { for (HRegionInfo region : assignedRegions) {
@ -344,6 +343,17 @@ public class RegionStates {
for (RegionState state : regionsInTransition.values()) { for (RegionState state : regionsInTransition.values()) {
if (assignedRegions.contains(state.getRegion())) { if (assignedRegions.contains(state.getRegion())) {
rits.add(state); rits.add(state);
} else if (sn.equals(state.getServerName())) {
// Region is in transition on this region server, and this
// region is not open on this server. So the region must be
// moving to this server from another one (i.e. opening or
// pending open on this server, was open on another one
if (state.isPendingOpen() || state.isOpening()) {
state.setTimestamp(0); // timeout it, let timeout monitor reassign
} else {
LOG.warn("THIS SHOULD NOT HAPPEN: unexpected state "
+ state + " of region in transition on server " + sn);
}
} }
} }
assignedRegions.clear(); assignedRegions.clear();
@ -367,11 +377,8 @@ public class RegionStates {
// before all table's regions. // before all table's regions.
HRegionInfo boundary = new HRegionInfo(tableName, null, null, false, 0L); HRegionInfo boundary = new HRegionInfo(tableName, null, null, false, 0L);
for (HRegionInfo hri: regionAssignments.tailMap(boundary).keySet()) { for (HRegionInfo hri: regionAssignments.tailMap(boundary).keySet()) {
if(Bytes.equals(hri.getTableName(), tableName)) { if(!Bytes.equals(hri.getTableName(), tableName)) break;
tableRegions.add(hri); tableRegions.add(hri);
} else {
break;
}
} }
return tableRegions; return tableRegions;
} }

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.DeadServer; import org.apache.hadoop.hbase.master.DeadServer;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PairOfSameType; import org.apache.hadoop.hbase.util.PairOfSameType;
@ -232,14 +233,6 @@ public class ServerShutdownHandler extends EventHandler {
return; return;
} }
// Clean out anything in regions in transition. Being conservative and
// doing after log splitting. Could do some states before -- OPENING?
// OFFLINE? -- and then others after like CLOSING that depend on log
// splitting.
List<RegionState> regionsInTransition =
this.services.getAssignmentManager().
processServerShutdown(this.serverName);
// Wait on meta to come online; we need it to progress. // Wait on meta to come online; we need it to progress.
// TODO: Best way to hold strictly here? We should build this retry logic // TODO: Best way to hold strictly here? We should build this retry logic
// into the MetaReader operations themselves. // into the MetaReader operations themselves.
@ -274,87 +267,77 @@ public class ServerShutdownHandler extends EventHandler {
throw new IOException("Server is stopped"); throw new IOException("Server is stopped");
} }
// Skip regions that were in transition unless CLOSING or PENDING_CLOSE // Clean out anything in regions in transition. Being conservative and
for (RegionState rit : regionsInTransition) { // doing after log splitting. Could do some states before -- OPENING?
if (!rit.isClosing() && !rit.isPendingClose() && !rit.isSplitting()) { // OFFLINE? -- and then others after like CLOSING that depend on log
LOG.debug("Removed " + rit.getRegion().getRegionNameAsString() + // splitting.
" from list of regions to assign because in RIT; region state: " + AssignmentManager am = services.getAssignmentManager();
rit.getState()); List<RegionState> regionsInTransition = am.processServerShutdown(serverName);
if (hris != null) hris.remove(rit.getRegion());
}
}
assert regionsInTransition != null;
LOG.info("Reassigning " + ((hris == null)? 0: hris.size()) + LOG.info("Reassigning " + ((hris == null)? 0: hris.size()) +
" region(s) that " + (serverName == null? "null": serverName) + " region(s) that " + (serverName == null? "null": serverName) +
" was carrying (skipping " + " was carrying (skipping " + regionsInTransition.size() +
regionsInTransition.size() +
" regions(s) that are already in transition)"); " regions(s) that are already in transition)");
// Iterate regions that were on this server and assign them // Iterate regions that were on this server and assign them
if (hris != null) { if (hris != null) {
RegionStates regionStates = am.getRegionStates();
List<HRegionInfo> toAssignRegions = new ArrayList<HRegionInfo>(); List<HRegionInfo> toAssignRegions = new ArrayList<HRegionInfo>();
for (Map.Entry<HRegionInfo, Result> e: hris.entrySet()) { for (Map.Entry<HRegionInfo, Result> e: hris.entrySet()) {
RegionState rit = services.getAssignmentManager() HRegionInfo hri = e.getKey();
.getRegionStates().getRegionTransitionState(e.getKey()); RegionState rit = regionStates.getRegionTransitionState(hri);
if (processDeadRegion(e.getKey(), e.getValue(), if (processDeadRegion(hri, e.getValue(), am, server.getCatalogTracker())) {
this.services.getAssignmentManager(), ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
this.server.getCatalogTracker())) { if (addressFromAM != null && !addressFromAM.equals(this.serverName)) {
ServerName addressFromAM = this.services.getAssignmentManager() // If this region is in transition on the dead server, it must be
.getRegionStates().getRegionServerOfRegion(e.getKey()); // opening or pending_open, which is covered by AM#processServerShutdown
if (rit != null && !rit.isClosing() && !rit.isPendingClose() && !rit.isSplitting()) { LOG.debug("Skip assigning region " + hri.getRegionNameAsString()
// Skip regions that were in transition unless CLOSING or + " because it has been opened in " + addressFromAM.getServerName());
// PENDING_CLOSE continue;
LOG.info("Skip assigning region " + rit.toString()); }
} else if (addressFromAM != null if (rit != null) {
&& !addressFromAM.equals(this.serverName)) { if (!rit.isOnServer(serverName)
LOG.debug("Skip assigning region " || rit.isClosed() || rit.isOpened() || rit.isSplit()) {
+ e.getKey().getRegionNameAsString() // Skip regions that are in transition on other server,
+ " because it has been opened in " // or in state closed/opened/split
+ addressFromAM.getServerName()); LOG.info("Skip assigning region " + rit);
} else { continue;
if (rit != null) {
//clean zk node
try{
LOG.info("Reassigning region with rs =" + rit + " and deleting zk node if exists");
ZKAssign.deleteNodeFailSilent(services.getZooKeeper(), e.getKey());
}catch (KeeperException ke) {
this.server.abort("Unexpected ZK exception deleting unassigned node " + e.getKey(), ke);
return;
}
}
toAssignRegions.add(e.getKey());
} }
} else if (rit != null && (rit.isSplitting() || rit.isSplit())) { try{
// This will happen when the RS went down and the call back for the SPLIITING or SPLIT //clean zk node
// has not yet happened for node Deleted event. In that case if the region was actually LOG.info("Reassigning region with rs = " + rit + " and deleting zk node if exists");
// split ZKAssign.deleteNodeFailSilent(services.getZooKeeper(), hri);
// but the RS had gone down before completing the split process then will not try to } catch (KeeperException ke) {
// assign the parent region again. In that case we should make the region offline and this.server.abort("Unexpected ZK exception deleting unassigned node " + hri, ke);
// also delete the region from RIT. return;
HRegionInfo region = rit.getRegion(); }
AssignmentManager am = this.services.getAssignmentManager(); }
am.regionOffline(region); toAssignRegions.add(hri);
} } else if (rit != null) {
// If the table was partially disabled and the RS went down, we should clear the RIT if (rit.isSplitting() || rit.isSplit()) {
// and remove the node for the region. // This will happen when the RS went down and the call back for the SPLIITING or SPLIT
// The rit that we use may be stale in case the table was in DISABLING state // has not yet happened for node Deleted event. In that case if the region was actually
// but though we did assign we will not be clearing the znode in CLOSING state. // split
// Doing this will have no harm. See HBASE-5927 // but the RS had gone down before completing the split process then will not try to
if (rit != null // assign the parent region again. In that case we should make the region offline and
&& (rit.isClosing() || rit.isPendingClose()) // also delete the region from RIT.
&& this.services.getAssignmentManager().getZKTable() am.regionOffline(hri);
.isDisablingOrDisabledTable(rit.getRegion().getTableNameAsString())) { } else if ((rit.isClosing() || rit.isPendingClose())
HRegionInfo hri = rit.getRegion(); && am.getZKTable().isDisablingOrDisabledTable(hri.getTableNameAsString())) {
AssignmentManager am = this.services.getAssignmentManager(); // If the table was partially disabled and the RS went down, we should clear the RIT
am.deleteClosingOrClosedNode(hri); // and remove the node for the region.
am.regionOffline(hri); // The rit that we use may be stale in case the table was in DISABLING state
// To avoid region assignment if table is in disabling or disabled state. // but though we did assign we will not be clearing the znode in CLOSING state.
toAssignRegions.remove(hri); // Doing this will have no harm. See HBASE-5927
am.deleteClosingOrClosedNode(hri);
am.regionOffline(hri);
} else {
LOG.warn("THIS SHOULD NOT HAPPEN: unexpected region in transition "
+ rit + " not to be assigned by SSH of server " + serverName);
}
} }
} }
try { try {
this.services.getAssignmentManager().assign(toAssignRegions); am.assign(toAssignRegions);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
LOG.error("Caught " + ie + " during round-robin assignment"); LOG.error("Caught " + ie + " during round-robin assignment");
throw new IOException(ie); throw new IOException(ie);

View File

@ -906,6 +906,37 @@ public class TestAssignmentManager {
} }
} }
/**
* When a region is in transition, if the region server opening the region goes down,
* the region assignment takes a long time normally (waiting for timeout monitor to trigger assign).
* This test is to make sure SSH times out the transition right away.
*/
@Test
public void testSSHTimesOutOpeningRegionTransition()
throws KeeperException, IOException, ServiceException {
// We need a mocked catalog tracker.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
// Create an AM.
AssignmentManagerWithExtrasForTesting am =
setUpMockedAssignmentManager(this.server, this.serverManager);
// adding region in pending open.
RegionState state = new RegionState(REGIONINFO,
State.OPENING, System.currentTimeMillis(), SERVERNAME_A);
am.getRegionStates().regionsInTransition.put(REGIONINFO.getEncodedName(), state);
// adding region plan
am.regionPlans.put(REGIONINFO.getEncodedName(),
new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A));
am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
try {
processServerShutdownHandler(ct, am, false);
assertTrue("Transtion is timed out", state.getStamp() == 0);
} finally {
am.getRegionStates().regionsInTransition.remove(REGIONINFO.getEncodedName());
am.regionPlans.remove(REGIONINFO.getEncodedName());
}
}
/** /**
* Creates a new ephemeral node in the SPLITTING state for the specified region. * Creates a new ephemeral node in the SPLITTING state for the specified region.
* Create it ephemeral in case regionserver dies mid-split. * Create it ephemeral in case regionserver dies mid-split.