HBASE-3147 Regions stuck in transition after rolling restart, perpetual timeout handling but nothing happens

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1027646 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-10-26 16:52:31 +00:00
parent a702d2253c
commit b22ba3afd3
13 changed files with 246 additions and 77 deletions

View File

@ -613,6 +613,8 @@ Release 0.21.0 - Unreleased
HBASE-3136 Stale reads from ZK can break the atomic CAS operations we
have in ZKAssign
HBASE-2753 Remove sorted() methods from Result now that Gets are Scans
HBASE-3147 Regions stuck in transition after rolling restart, perpetual
timeout handling but nothing happens
IMPROVEMENTS
HBASE-1760 Cleanup TODOs in HTable

View File

@ -29,19 +29,16 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaNodeTracker;
import org.apache.hadoop.hbase.zookeeper.RootRegionTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException;
/**
* Tracks the availability of the catalog tables <code>-ROOT-</code> and
@ -63,6 +60,12 @@ public class CatalogTracker {
private final RootRegionTracker rootRegionTracker;
private final MetaNodeTracker metaNodeTracker;
private final AtomicBoolean metaAvailable = new AtomicBoolean(false);
/**
* Do not clear this address once set. Let it be cleared by
* {@link #setMetaLocation(HServerAddress)} only. Its needed when we do
* server shutdown processing -- we need to know who had .META. last. If you
* want to know if the address is good, rely on {@link #metaAvailable} value.
*/
private HServerAddress metaLocation;
private final int defaultTimeout;
private boolean stopped = false;
@ -365,7 +368,6 @@ public class CatalogTracker {
private void resetMetaLocation() {
LOG.info("Current cached META location is not valid, resetting");
this.metaAvailable.set(false);
this.metaLocation = null;
}
private void setMetaLocation(HServerAddress metaLocation) {
@ -471,37 +473,6 @@ public class CatalogTracker {
return getMetaServerConnection(true) != null;
}
/**
* Check if <code>hsi</code> was carrying <code>-ROOT-</code> or
* <code>.META.</code> and if so, clear out old locations.
* @param hsi Server that has crashed/shutdown.
* @throws InterruptedException
* @throws KeeperException
* @return Pair of booleans; if this server was carrying root, then first
* boolean is set, if server was carrying meta, then second boolean set.
*/
public Pair<Boolean, Boolean> processServerShutdown(final HServerInfo hsi)
throws InterruptedException, KeeperException {
Pair<Boolean, Boolean> result = new Pair<Boolean, Boolean>(false, false);
HServerAddress rootHsa = getRootLocation();
if (rootHsa == null) {
LOG.info("-ROOT- is not assigned; continuing");
} else if (hsi.getServerAddress().equals(rootHsa)) {
result.setFirst(true);
LOG.info(hsi.getServerName() + " carrying -ROOT-; unsetting");
}
HServerAddress metaHsa = getMetaLocation();
if (metaHsa == null) {
LOG.info(".META. is not assigned; continuing");
} else if (hsi.getServerAddress().equals(metaHsa)) {
LOG.info(hsi.getServerName() + " carrying .META.; unsetting " +
".META. location");
result.setSecond(true);
resetMetaLocation();
}
return result;
}
MetaNodeTracker getMetaNodeTracker() {
return this.metaNodeTracker;
}

View File

@ -246,9 +246,12 @@ public class MetaReader {
throw e;
}
} catch (RemoteException re) {
if (re.unwrapRemoteException() instanceof NotServingRegionException) {
IOException ioe = re.unwrapRemoteException();
if (ioe instanceof NotServingRegionException) {
// Treat this NSRE as unavailable table. Catch and fall through to
// return null below
} else if (ioe.getMessage().contains("Server not running")) {
// Treat as unavailable table.
} else {
throw re;
}

View File

@ -127,7 +127,8 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
M_ZK_REGION_OFFLINE (50), // Master adds this region as offline in ZK
// Master controlled events to be executed on the master
M_SERVER_SHUTDOWN (70); // Master is processing shutdown of a RS
M_SERVER_SHUTDOWN (70), // Master is processing shutdown of a RS
M_META_SERVER_SHUTDOWN (72); // Master is processing shutdown of RS hosting a meta region (-ROOT- or .META.).
/**
* Constructor

View File

@ -27,7 +27,6 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -77,6 +76,7 @@ public class ExecutorService {
MASTER_SERVER_OPERATIONS (3),
MASTER_TABLE_OPERATIONS (4),
MASTER_RS_SHUTDOWN (5),
MASTER_META_SERVER_OPERATIONS (6),
// RegionServer executor services
RS_OPEN_REGION (20),
@ -115,6 +115,9 @@ public class ExecutorService {
case M_SERVER_SHUTDOWN:
return ExecutorType.MASTER_SERVER_OPERATIONS;
case M_META_SERVER_SHUTDOWN:
return ExecutorType.MASTER_META_SERVER_OPERATIONS;
case C_M_DELETE_TABLE:
case C_M_DISABLE_TABLE:
case C_M_ENABLE_TABLE:

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
@ -96,13 +97,13 @@ public class AssignmentManager extends ZooKeeperListener {
private TimeoutMonitor timeoutMonitor;
/** Regions currently in transition. */
private final ConcurrentSkipListMap<String, RegionState> regionsInTransition =
final ConcurrentSkipListMap<String, RegionState> regionsInTransition =
new ConcurrentSkipListMap<String, RegionState>();
/** Plans for region movement. Key is the encoded version of a region name*/
// TODO: When do plans get cleaned out? Ever? In server open and in server
// shutdown processing -- St.Ack
protected final ConcurrentNavigableMap<String, RegionPlan> regionPlans =
final ConcurrentNavigableMap<String, RegionPlan> regionPlans =
new ConcurrentSkipListMap<String, RegionPlan>();
/** Set of tables that have been disabled. */
@ -315,7 +316,7 @@ public class AssignmentManager extends ZooKeeperListener {
if (!serverManager.isServerOnline(data.getServerName()) &&
!this.master.getServerName().equals(data.getServerName())) {
LOG.warn("Attempted to handle region transition for server but " +
"server is not online: " + data);
"server is not online: " + data.getRegionName());
return;
}
String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
@ -597,9 +598,8 @@ public class AssignmentManager extends ZooKeeperListener {
ZKAssign.deleteOfflineNode(watcher, regionInfo.getEncodedName());
}
} catch (KeeperException.NoNodeException nne) {
LOG.warn("Tried to delete closed node for " + regionInfo + " but it " +
"does not exist");
return;
LOG.debug("Tried to delete closed node for " + regionInfo + " but it " +
"does not exist so just offlining");
} catch (KeeperException e) {
this.master.abort("Error deleting CLOSED node in ZK", e);
}
@ -976,15 +976,29 @@ public class AssignmentManager extends ZooKeeperListener {
}
// Send CLOSE RPC
try {
serverManager.sendRegionClose(regions.get(region), state.getRegion());
if(!serverManager.sendRegionClose(regions.get(region),
state.getRegion())) {
throw new NotServingRegionException("Server failed to close region");
}
} catch (NotServingRegionException nsre) {
// Did not CLOSE, so set region offline and assign it
LOG.debug("Attempted to send CLOSE for region " +
region.getRegionNameAsString() + " but failed, setting region as " +
"OFFLINE and reassigning");
synchronized (regionsInTransition) {
forceRegionStateToOffline(region);
assign(region);
}
} catch (IOException e) {
// For now call abort if unexpected exception -- radical, but will get fellas attention.
// St.Ack 20101012
// I don't think IOE can happen anymore, only NSRE IOE is used here
// should be able to remove this at least. jgray 20101024
this.master.abort("Remote unexpected exception", e);
} catch (Throwable t) {
// For now call abort if unexpected exception -- radical, but will get fellas attention.
// St.Ack 20101012
this.master.abort("Unexpected exception", t);
this.master.abort("Remote unexpected exception", t);
}
}
@ -1435,14 +1449,40 @@ public class AssignmentManager extends ZooKeeperListener {
assign(regionState.getRegion());
break;
case PENDING_OPEN:
LOG.info("Region has been PENDING_OPEN for too " +
"long, reassigning region=" +
regionInfo.getRegionNameAsString());
// Should have a ZK node in OFFLINE state or no node at all
try {
if (ZKUtil.watchAndCheckExists(watcher,
ZKAssign.getNodeName(watcher,
regionInfo.getEncodedName())) &&
!ZKAssign.verifyRegionState(watcher, regionInfo,
EventType.M_ZK_REGION_OFFLINE)) {
LOG.info("Region exists and not in expected OFFLINE " +
"state so skipping timeout, region=" +
regionInfo.getRegionNameAsString());
break;
}
} catch (KeeperException ke) {
LOG.error("Unexpected ZK exception timing out " +
"PENDING_CLOSE region",
ke);
break;
}
AssignmentManager.this.setOffline(regionState.getRegion());
regionState.update(RegionState.State.OFFLINE);
assign(regionState.getRegion());
break;
case OPENING:
LOG.info("Region has been PENDING_OPEN or OPENING for too " +
LOG.info("Region has been OPENING for too " +
"long, reassigning region=" +
regionInfo.getRegionNameAsString());
// There could be two cases. No ZK node or ZK in CLOSING.
// Should have a ZK node in OPENING state
try {
if (ZKUtil.checkExists(watcher, watcher.assignmentZNode)
!= -1 &&
if (ZKUtil.watchAndCheckExists(watcher,
ZKAssign.getNodeName(watcher,
regionInfo.getEncodedName())) &&
ZKAssign.transitionNode(watcher, regionInfo,
HMaster.MASTER, EventType.RS_ZK_REGION_OPENING,
EventType.M_ZK_REGION_OFFLINE, -1) == -1) {
@ -1465,8 +1505,27 @@ public class AssignmentManager extends ZooKeeperListener {
"not happen; region=" + regionInfo.getRegionNameAsString());
break;
case PENDING_CLOSE:
LOG.info("Region has been PENDING_CLOSE for too " +
"long, running forced unassign again on region=" +
regionInfo.getRegionNameAsString());
try {
// If the server got the RPC, it will transition the node
// to CLOSING, so only do something here if no node exists
if (!ZKUtil.watchAndCheckExists(watcher,
ZKAssign.getNodeName(watcher,
regionInfo.getEncodedName()))) {
unassign(regionInfo, true);
}
} catch (NoNodeException e) {
LOG.debug("Node no longer existed so not forcing another " +
"unassignment");
} catch (KeeperException e) {
LOG.warn("Unexpected ZK exception timing out a region " +
"close", e);
}
break;
case CLOSING:
LOG.info("Region has been PENDING_CLOSE or CLOSING for too " +
LOG.info("Region has been CLOSING for too " +
"long, running forced unassign again on region=" +
regionInfo.getRegionNameAsString());
try {
@ -1500,6 +1559,7 @@ public class AssignmentManager extends ZooKeeperListener {
Map.Entry<String, RegionPlan> e = i.next();
if (e.getValue().getDestination().equals(hsi)) {
// Use iterator's remove else we'll get CME
LOG.info("REMOVING PLAN " + e.getValue());
i.remove();
}
}

View File

@ -413,7 +413,8 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
this.catalogTracker.waitForRoot();
assigned++;
}
LOG.info("-ROOT- assigned=" + assigned + ", rit=" + rit);
LOG.info("-ROOT- assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getRootLocation());
// Work on meta region
rit = this.assignmentManager.
@ -426,7 +427,8 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
assigned++;
}
LOG.info(".META. assigned=" + assigned + ", rit=" + rit);
LOG.info(".META. assigned=" + assigned + ", rit=" + rit +
", location=" + catalogTracker.getMetaLocation());
return assigned;
}
@ -502,6 +504,8 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
conf.getInt("hbase.master.executor.closeregion.threads", 5));
this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
conf.getInt("hbase.master.executor.serverops.threads", 3));
this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
conf.getInt("hbase.master.executor.serverops.threads", 2));
this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS,
conf.getInt("hbase.master.executor.tableops.threads", 3));

View File

@ -41,15 +41,19 @@ import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler;
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
/**
* The ServerManager class manages info about region servers - HServerInfo,
@ -490,10 +494,36 @@ public class ServerManager {
}
return;
}
this.services.getExecutorService().submit(new ServerShutdownHandler(this.master,
CatalogTracker ct = this.master.getCatalogTracker();
// Was this server carrying root?
boolean carryingRoot;
try {
HServerAddress address = ct.getRootLocation();
carryingRoot = address != null &&
hsi.getServerAddress().equals(address);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
LOG.info("Interrupted");
return;
}
// Was this server carrying meta? Can't ask CatalogTracker because it
// may have reset the meta location as null already (it may have already
// run into fact that meta is dead). I can ask assignment manager. It
// has an inmemory list of who has what. This list will be cleared as we
// process the dead server but should be find asking it now.
HServerAddress address = ct.getMetaLocation();
boolean carryingMeta =
address != null && hsi.getServerAddress().equals(address);
if (carryingRoot || carryingMeta) {
this.services.getExecutorService().submit(new MetaServerShutdownHandler(this.master,
this.services, this.deadservers, info, carryingRoot, carryingMeta));
} else {
this.services.getExecutorService().submit(new ServerShutdownHandler(this.master,
this.services, this.deadservers, info));
}
LOG.debug("Added=" + serverName +
" to dead servers, submitted shutdown handler to be executed");
" to dead servers, submitted shutdown handler to be executed, root=" +
carryingRoot + ", meta=" + carryingMeta);
}
// RPC methods to region servers
@ -546,16 +576,17 @@ public class ServerManager {
* @return true if server acknowledged close, false if not
* @throws IOException
*/
public void sendRegionClose(HServerInfo server, HRegionInfo region)
public boolean sendRegionClose(HServerInfo server, HRegionInfo region)
throws IOException {
if (server == null) return false;
HRegionInterface hri = getServerConnection(server);
if(hri == null) {
LOG.warn("Attempting to send CLOSE RPC to server " +
server.getServerName() + " failed because no RPC connection found " +
"to this server");
return;
return false;
}
hri.closeRegion(region);
return hri.closeRegion(region);
}
/**

View File

@ -57,7 +57,12 @@ public class ServerShutdownHandler extends EventHandler {
public ServerShutdownHandler(final Server server, final MasterServices services,
final DeadServer deadServers, final HServerInfo hsi) {
super(server, EventType.M_SERVER_SHUTDOWN);
this(server, services, deadServers, hsi, EventType.M_SERVER_SHUTDOWN);
}
ServerShutdownHandler(final Server server, final MasterServices services,
final DeadServer deadServers, final HServerInfo hsi, EventType type) {
super(server, type);
this.hsi = hsi;
this.server = server;
this.services = services;
@ -67,19 +72,22 @@ public class ServerShutdownHandler extends EventHandler {
}
}
/**
* @return True if the server we are processing was carrying <code>-ROOT-</code>
*/
boolean isCarryingRoot() {
return false;
}
/**
* @return True if the server we are processing was carrying <code>.META.</code>
*/
boolean isCarryingMeta() {
return false;
}
@Override
public void process() throws IOException {
Pair<Boolean, Boolean> carryingCatalog = null;
try {
carryingCatalog =
this.server.getCatalogTracker().processServerShutdown(this.hsi);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new IOException("Interrupted", e);
} catch (KeeperException e) {
this.server.abort("In server shutdown processing", e);
throw new IOException("Aborting", e);
}
final String serverName = this.hsi.getServerName();
LOG.info("Splitting logs for " + serverName);
@ -92,7 +100,7 @@ public class ServerShutdownHandler extends EventHandler {
this.services.getAssignmentManager().processServerShutdown(this.hsi);
// Assign root and meta if we were carrying them.
if (carryingCatalog.getFirst()) { // -ROOT-
if (isCarryingRoot()) { // -ROOT-
try {
this.services.getAssignmentManager().assignRoot();
} catch (KeeperException e) {
@ -100,9 +108,9 @@ public class ServerShutdownHandler extends EventHandler {
throw new IOException("Aborting", e);
}
}
if (carryingCatalog.getSecond()) { // .META.
this.services.getAssignmentManager().assignMeta();
}
// Carrying meta?
if (isCarryingMeta()) this.services.getAssignmentManager().assignMeta();
// Wait on meta to come online; we need it to progress.
try {

View File

@ -57,6 +57,7 @@ public class MetaNodeTracker extends ZooKeeperNodeTracker {
@Override
public void nodeDeleted(String path) {
super.nodeDeleted(path);
if (!path.equals(node)) return;
LOG.info("Detected completed assignment of META, notifying catalog tracker");
try {

View File

@ -97,7 +97,7 @@ public class ZKAssign {
* @param regionName region name
* @return full path node name
*/
private static String getNodeName(ZooKeeperWatcher zkw, String regionName) {
public static String getNodeName(ZooKeeperWatcher zkw, String regionName) {
return ZKUtil.joinZNode(zkw.assignmentZNode, regionName);
}
@ -762,4 +762,44 @@ public class ZKAssign {
Thread.sleep(200);
}
}
/**
* Verifies that the specified region is in the specified state in ZooKeeper.
* <p>
* Returns true if region is in transition and in the specified state in
* ZooKeeper. Returns false if the region does not exist in ZK or is in
* a different state.
* <p>
* Method synchronizes() with ZK so will yield an up-to-date result but is
* a slow read.
* @param watcher
* @param region
* @param expectedState
* @return true if region exists and is in expected state
*/
public static boolean verifyRegionState(ZooKeeperWatcher zkw,
HRegionInfo region, EventType expectedState)
throws KeeperException {
String encoded = region.getEncodedName();
String node = getNodeName(zkw, encoded);
zkw.sync(node);
// Read existing data of the node
byte [] existingBytes = null;
try {
existingBytes = ZKUtil.getDataAndWatch(zkw, node);
} catch (KeeperException.NoNodeException nne) {
return false;
} catch (KeeperException e) {
throw e;
}
if (existingBytes == null) return false;
RegionTransitionData existingData =
RegionTransitionData.fromBytes(existingBytes);
if (existingData.getEventType() == expectedState){
return true;
}
return false;
}
}

View File

@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.KeyValue;
@ -104,6 +103,26 @@ public class TestCatalogTracker {
return ct;
}
/**
* Test that we get notification if .META. moves.
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
*/
@Test public void testThatIfMETAMovesWeAreNotified()
throws IOException, InterruptedException, KeeperException {
HConnection connection = Mockito.mock(HConnection.class);
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
try {
RootLocationEditor.setRootLocation(this.watcher,
new HServerAddress("example.com:1234"));
} finally {
// Clean out root location or later tests will be confused... they presume
// start fresh in zk.
RootLocationEditor.deleteRootLocation(this.watcher);
}
}
/**
* Test interruptable while blocking wait on root and meta.
* @throws IOException

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
@ -794,10 +795,35 @@ public class TestMasterFailover {
cluster.waitForActiveAndReadyMaster();
log("Master is ready");
// Let's add some weird states to master in-memory state
// PENDING_OPEN and enabled
region = enabledRegions.remove(0);
regionsThatShouldBeOnline.add(region);
master.assignmentManager.regionsInTransition.put(region.getEncodedName(),
new RegionState(region, RegionState.State.PENDING_OPEN));
// PENDING_OPEN and disabled
region = disabledRegions.remove(0);
regionsThatShouldBeOffline.add(region);
master.assignmentManager.regionsInTransition.put(region.getEncodedName(),
new RegionState(region, RegionState.State.PENDING_OPEN));
// PENDING_CLOSE and enabled
region = enabledRegions.remove(0);
regionsThatShouldBeOnline.add(region);
master.assignmentManager.regionsInTransition.put(region.getEncodedName(),
new RegionState(region, RegionState.State.PENDING_CLOSE));
// PENDING_CLOSE and disabled
region = disabledRegions.remove(0);
regionsThatShouldBeOffline.add(region);
master.assignmentManager.regionsInTransition.put(region.getEncodedName(),
new RegionState(region, RegionState.State.PENDING_CLOSE));
// Failover should be completed, now wait for no RIT
log("Waiting for no more RIT");
ZKAssign.blockUntilNoRIT(zkw);
log("No more RIT in ZK, now doing final test verification");
log("No more RIT in ZK");
master.assignmentManager.waitUntilNoRegionsInTransition(120000);
log("No more RIT in RIT map, doing final test verification");
// Grab all the regions that are online across RSs
Set<HRegionInfo> onlineRegions = new TreeSet<HRegionInfo>();