[jira] [HBASE-5763] Fix random failures in TestFSErrorsExposed

Summary:
TestFSErrorsExposed frequently fails due to unclean mini-cluster shutdown.
Bringing datanodes back up, waiting for some time, and preemptively killing all
regionservers and the master before shutdown.

This is the trunk fix. The 89-fb patch is at D2739.

Test Plan: Run TestFSErrorsExposed 100 times

Reviewers: stack, tedyu, jdcryans, lhofhansl, jmhsieh, JIRA

Reviewed By: tedyu

Differential Revision: https://reviews.facebook.net/D2793

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1327337 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
mbautin 2012-04-18 00:22:54 +00:00
parent 2a3ca44ea4
commit 421400c256
2 changed files with 24 additions and 4 deletions

View File

@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.io.MapWritable;
@ -529,4 +531,17 @@ public class MiniHBaseCluster {
}
return count;
}
/**
* Do a simulated kill all masters and regionservers. Useful when it is
* impossible to bring the mini-cluster back for clean shutdown.
*/
public void killAll() {
for (RegionServerThread rst : getRegionServerThreads()) {
rst.getRegionServer().abort("killAll");
}
for (MasterThread masterThread : getMasterThreads()) {
masterThread.getMaster().abort("killAll", new Throwable());
}
}
}

View File

@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test cases that ensure that file system level errors are bubbled up
* appropriately to clients, rather than swallowed.
@ -163,13 +162,16 @@ public class TestFSErrorsExposed {
* removes the data from HDFS underneath it, and ensures that
* errors are bubbled to the client.
*/
@Test
@Test(timeout=5 * 60 * 1000)
public void testFullSystemBubblesFSErrors() throws Exception {
try {
// We set it not to run or it will trigger server shutdown while sync'ing
// because all the datanodes are bad
util.getConfiguration().setInt(
"hbase.regionserver.optionallogflushinterval", Integer.MAX_VALUE);
util.getConfiguration().setInt("hbase.client.retries.number", 3);
util.startMiniCluster(1);
byte[] tableName = Bytes.toBytes("table");
byte[] fam = Bytes.toBytes("fam");
@ -204,7 +206,11 @@ public class TestFSErrorsExposed {
assertTrue(e.getMessage().contains("Could not seek"));
}
// Restart data nodes so that HBase can shut down cleanly.
util.getDFSCluster().restartDataNodes();
} finally {
util.getMiniHBaseCluster().killAll();
util.shutdownMiniCluster();
}
}
@ -265,4 +271,3 @@ public class TestFSErrorsExposed {
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
}