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

View File

@ -71,7 +71,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
}
public void updateTimestampToNow() {
this.stamp.set(System.currentTimeMillis());
setTimestamp(System.currentTimeMillis());
}
public State getState() {
@ -134,6 +134,10 @@ public class RegionState implements org.apache.hadoop.io.Writable {
return isOnServer(sn) && (isPendingClose() || isClosing());
}
public boolean isOnServer(final ServerName sn) {
return serverName != null && serverName.equals(sn);
}
@Override
public String toString() {
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);
}
private boolean isOnServer(final ServerName sn) {
return serverName != null && serverName.equals(sn);
protected void setTimestamp(final long timestamp) {
stamp.set(timestamp);
}
/**

View File

@ -329,9 +329,8 @@ public class RegionStates {
// of this server from online map of regions.
List<RegionState> rits = new ArrayList<RegionState>();
Set<HRegionInfo> assignedRegions = serverHoldings.get(sn);
if (assignedRegions == null || assignedRegions.isEmpty()) {
// No regions on this server, we are done, return empty list of RITs
return rits;
if (assignedRegions == null) {
assignedRegions = new HashSet<HRegionInfo>();
}
for (HRegionInfo region : assignedRegions) {
@ -344,6 +343,17 @@ public class RegionStates {
for (RegionState state : regionsInTransition.values()) {
if (assignedRegions.contains(state.getRegion())) {
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();
@ -367,11 +377,8 @@ public class RegionStates {
// before all table's regions.
HRegionInfo boundary = new HRegionInfo(tableName, null, null, false, 0L);
for (HRegionInfo hri: regionAssignments.tailMap(boundary).keySet()) {
if(Bytes.equals(hri.getTableName(), tableName)) {
tableRegions.add(hri);
} else {
break;
}
if(!Bytes.equals(hri.getTableName(), tableName)) break;
tableRegions.add(hri);
}
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.MasterServices;
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.util.Bytes;
import org.apache.hadoop.hbase.util.PairOfSameType;
@ -232,14 +233,6 @@ public class ServerShutdownHandler extends EventHandler {
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.
// TODO: Best way to hold strictly here? We should build this retry logic
// into the MetaReader operations themselves.
@ -274,87 +267,77 @@ public class ServerShutdownHandler extends EventHandler {
throw new IOException("Server is stopped");
}
// Skip regions that were in transition unless CLOSING or PENDING_CLOSE
for (RegionState rit : regionsInTransition) {
if (!rit.isClosing() && !rit.isPendingClose() && !rit.isSplitting()) {
LOG.debug("Removed " + rit.getRegion().getRegionNameAsString() +
" from list of regions to assign because in RIT; region state: " +
rit.getState());
if (hris != null) hris.remove(rit.getRegion());
}
}
assert regionsInTransition != null;
// 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.
AssignmentManager am = services.getAssignmentManager();
List<RegionState> regionsInTransition = am.processServerShutdown(serverName);
LOG.info("Reassigning " + ((hris == null)? 0: hris.size()) +
" region(s) that " + (serverName == null? "null": serverName) +
" was carrying (skipping " +
regionsInTransition.size() +
" was carrying (skipping " + regionsInTransition.size() +
" regions(s) that are already in transition)");
// Iterate regions that were on this server and assign them
if (hris != null) {
RegionStates regionStates = am.getRegionStates();
List<HRegionInfo> toAssignRegions = new ArrayList<HRegionInfo>();
for (Map.Entry<HRegionInfo, Result> e: hris.entrySet()) {
RegionState rit = services.getAssignmentManager()
.getRegionStates().getRegionTransitionState(e.getKey());
if (processDeadRegion(e.getKey(), e.getValue(),
this.services.getAssignmentManager(),
this.server.getCatalogTracker())) {
ServerName addressFromAM = this.services.getAssignmentManager()
.getRegionStates().getRegionServerOfRegion(e.getKey());
if (rit != null && !rit.isClosing() && !rit.isPendingClose() && !rit.isSplitting()) {
// Skip regions that were in transition unless CLOSING or
// PENDING_CLOSE
LOG.info("Skip assigning region " + rit.toString());
} else if (addressFromAM != null
&& !addressFromAM.equals(this.serverName)) {
LOG.debug("Skip assigning region "
+ e.getKey().getRegionNameAsString()
+ " because it has been opened in "
+ addressFromAM.getServerName());
} else {
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());
HRegionInfo hri = e.getKey();
RegionState rit = regionStates.getRegionTransitionState(hri);
if (processDeadRegion(hri, e.getValue(), am, server.getCatalogTracker())) {
ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
if (addressFromAM != null && !addressFromAM.equals(this.serverName)) {
// If this region is in transition on the dead server, it must be
// opening or pending_open, which is covered by AM#processServerShutdown
LOG.debug("Skip assigning region " + hri.getRegionNameAsString()
+ " because it has been opened in " + addressFromAM.getServerName());
continue;
}
if (rit != null) {
if (!rit.isOnServer(serverName)
|| rit.isClosed() || rit.isOpened() || rit.isSplit()) {
// Skip regions that are in transition on other server,
// or in state closed/opened/split
LOG.info("Skip assigning region " + rit);
continue;
}
} else if (rit != null && (rit.isSplitting() || rit.isSplit())) {
// This will happen when the RS went down and the call back for the SPLIITING or SPLIT
// has not yet happened for node Deleted event. In that case if the region was actually
// split
// but the RS had gone down before completing the split process then will not try to
// assign the parent region again. In that case we should make the region offline and
// also delete the region from RIT.
HRegionInfo region = rit.getRegion();
AssignmentManager am = this.services.getAssignmentManager();
am.regionOffline(region);
}
// If the table was partially disabled and the RS went down, we should clear the RIT
// and remove the node for the region.
// The rit that we use may be stale in case the table was in DISABLING state
// but though we did assign we will not be clearing the znode in CLOSING state.
// Doing this will have no harm. See HBASE-5927
if (rit != null
&& (rit.isClosing() || rit.isPendingClose())
&& this.services.getAssignmentManager().getZKTable()
.isDisablingOrDisabledTable(rit.getRegion().getTableNameAsString())) {
HRegionInfo hri = rit.getRegion();
AssignmentManager am = this.services.getAssignmentManager();
am.deleteClosingOrClosedNode(hri);
am.regionOffline(hri);
// To avoid region assignment if table is in disabling or disabled state.
toAssignRegions.remove(hri);
try{
//clean zk node
LOG.info("Reassigning region with rs = " + rit + " and deleting zk node if exists");
ZKAssign.deleteNodeFailSilent(services.getZooKeeper(), hri);
} catch (KeeperException ke) {
this.server.abort("Unexpected ZK exception deleting unassigned node " + hri, ke);
return;
}
}
toAssignRegions.add(hri);
} else if (rit != null) {
if (rit.isSplitting() || rit.isSplit()) {
// This will happen when the RS went down and the call back for the SPLIITING or SPLIT
// has not yet happened for node Deleted event. In that case if the region was actually
// split
// but the RS had gone down before completing the split process then will not try to
// assign the parent region again. In that case we should make the region offline and
// also delete the region from RIT.
am.regionOffline(hri);
} else if ((rit.isClosing() || rit.isPendingClose())
&& am.getZKTable().isDisablingOrDisabledTable(hri.getTableNameAsString())) {
// If the table was partially disabled and the RS went down, we should clear the RIT
// and remove the node for the region.
// The rit that we use may be stale in case the table was in DISABLING state
// but though we did assign we will not be clearing the znode in CLOSING state.
// 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 {
this.services.getAssignmentManager().assign(toAssignRegions);
am.assign(toAssignRegions);
} catch (InterruptedException ie) {
LOG.error("Caught " + ie + " during round-robin assignment");
throw new IOException(ie);

View File

@ -573,14 +573,14 @@ public class TestAssignmentManager {
// Make an RS Interface implementation. Make it so a scanner can go against it.
ClientProtocol implementation = Mockito.mock(ClientProtocol.class);
// Get a meta row result that has region up on SERVERNAME_A
Result r = null;
if (splitRegion) {
r = MetaMockingUtil.getMetaTableRowResultAsSplitRegion(REGIONINFO, SERVERNAME_A);
} else {
r = MetaMockingUtil.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
}
ScanResponse.Builder builder = ScanResponse.newBuilder();
builder.setMoreResults(true);
builder.addResult(ProtobufUtil.toResult(r));
@ -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.
* Create it ephemeral in case regionserver dies mid-split.