Permalink
Browse files

HBASE-3674 Treat ChecksumException as we would a ParseException split…

…ting 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
  • Loading branch information...
1 parent 5453365 commit dded2af3e01fe6389af343d04e3c31716563ee7a @saintstack saintstack committed Apr 26, 2011
Showing with 6 additions and 3 deletions.
  1. +6 −3 src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
@@ -333,7 +333,7 @@ public boolean splitLogFileToTemp(FileStatus logfile, String tmpname,
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 @@ static private Entry getNextLogLine(Reader in, Path path, boolean skipErrors)
} 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) {

0 comments on commit dded2af

Please sign in to comment.