HBASE-3674 Treat ChecksumException as we would a ParseException splitting logs; else we replay split on every restart -- fixup for distributed log splitting

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1096839 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-04-26 18:32:54 +00:00
parent 6164a0cb85
commit 7437b11bec
1 changed files with 6 additions and 3 deletions

View File

@ -333,7 +333,7 @@ public class HLogSplitter {
boolean progress_failed = false;
boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors", false);
boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors", true);
int interval = conf.getInt("hbase.splitlog.report.interval.loglines", 1024);
// How often to send a progress report (default 1/2 master timeout)
int period = conf.getInt("hbase.splitlog.report.period",
@ -685,8 +685,11 @@ public class HLogSplitter {
} catch (IOException e) {
// If the IOE resulted from bad file format,
// then this problem is idempotent and retrying won't help
if (e.getCause() instanceof ParseException) {
LOG.warn("ParseException from hlog " + path + ". continuing");
if (e.getCause() != null &&
(e.getCause() instanceof ParseException ||
e.getCause() instanceof org.apache.hadoop.fs.ChecksumException) {
LOG.warn("Parse exception " + e.getCause().toString() + " from hlog "
+ path + ". continuing");
return null;
}
if (!skipErrors) {