HBASE-8485 Retry to open a HLog on more exceptions

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1478880 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
jxiang 2013-05-03 17:19:27 +00:00
parent 6784e894e2
commit 3ca2bd7339
2 changed files with 10 additions and 5 deletions

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -129,7 +128,9 @@ public class HLogFactory {
}
} catch (IOException e) {
String msg = e.getMessage();
if (msg != null && msg.contains("Cannot obtain block length")) {
if (msg != null && (msg.contains("Cannot obtain block length")
|| msg.contains("Could not obtain the last block")
|| msg.matches("Blocklist for [^ ]* has changed.*"))) {
if (++nbAttempt == 1) {
LOG.warn("Lease should have recovered. This is not expected. Will retry", e);
}

View File

@ -889,7 +889,8 @@ public class TestHLogSplit {
fs.initialize(fs.getUri(), conf);
FileSystem spiedFs = Mockito.spy(fs);
// The "Cannot obtain block length" part is very important,
// The "Cannot obtain block length", "Could not obtain the last block",
// and "Blocklist for [^ ]* has changed.*" part is very important,
// that's how it comes out of HDFS. If HDFS changes the exception
// message, this test needs to be adjusted accordingly.
//
@ -898,11 +899,14 @@ public class TestHLogSplit {
// last block is under recovery, HDFS may have problem to obtain
// the block length, in which case, retry may help.
Mockito.doAnswer(new Answer<FSDataInputStream>() {
private final String[] errors = new String[] {
"Cannot obtain block length", "Could not obtain the last block",
"Blocklist for " + OLDLOGDIR + " has changed"};
private int count = 0;
public FSDataInputStream answer(InvocationOnMock invocation) throws Throwable {
if (count++ < 3) {
throw new IOException("Cannot obtain block length");
if (count < 3) {
throw new IOException(errors[count++]);
}
return (FSDataInputStream)invocation.callRealMethod();
}