HBASE-8575 TestDistributedLogSplitting#testMarkRegionsRecoveringInZK fails intermittently due to lack of online region (Jeff Zhong)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1484899 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2013-05-21 18:17:12 +00:00
parent aa12c8ac72
commit e89712d29d
1 changed files with 20 additions and 7 deletions

View File

@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Delete;
@ -607,13 +608,24 @@ public class TestDistributedLogSplitting {
HTable ht = installTable(zkw, "table", "family", 40);
final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
final HRegionServer hrs = rsts.get(0).getRegionServer();
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
HRegionInfo region = regions.get(0);
Set<HRegionInfo> regionSet = new HashSet<HRegionInfo>();
HRegionInfo region = null;
HRegionServer hrs = null;
ServerName firstFailedServer = null;
ServerName secondFailedServer = null;
for (int i = 0; i < NUM_RS; i++) {
hrs = rsts.get(i).getRegionServer();
List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(hrs);
if (regions.isEmpty()) continue;
region = regions.get(0);
regionSet.add(region);
slm.markRegionsRecoveringInZK(rsts.get(0).getRegionServer().getServerName(), regionSet);
slm.markRegionsRecoveringInZK(rsts.get(1).getRegionServer().getServerName(), regionSet);
firstFailedServer = hrs.getServerName();
secondFailedServer = rsts.get((i + 1) % NUM_RS).getRegionServer().getServerName();
break;
}
slm.markRegionsRecoveringInZK(firstFailedServer, regionSet);
slm.markRegionsRecoveringInZK(secondFailedServer, regionSet);
List<String> recoveringRegions = ZKUtil.listChildrenNoWatch(zkw,
ZKUtil.joinZNode(zkw.recoveringRegionsZNode, region.getEncodedName()));
@ -621,10 +633,11 @@ public class TestDistributedLogSplitting {
assertEquals(recoveringRegions.size(), 2);
// wait for splitLogWorker to mark them up because there is no WAL files recorded in ZK
final HRegionServer tmphrs = hrs;
TEST_UTIL.waitFor(60000, 1000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return (hrs.getRecoveringRegions().size() == 0);
return (tmphrs.getRecoveringRegions().size() == 0);
}
});
ht.close();