HBASE-2889 Tool to look at HLogs -- parse and tail -f; fix for suppression of EOFEs by Nicolas
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1000697 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0f3c62e19b
commit
a85ee11f3b
|
@ -22,6 +22,9 @@ package org.apache.hadoop.hbase.regionserver.wal;
|
|||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.lang.Class;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.Field;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
|
@ -98,6 +101,7 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
// Needed logging exceptions
|
||||
Path path;
|
||||
int edit = 0;
|
||||
long entryStart = 0;
|
||||
|
||||
public SequenceFileLogReader() { }
|
||||
|
||||
|
@ -125,6 +129,7 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
|
||||
@Override
|
||||
public HLog.Entry next(HLog.Entry reuse) throws IOException {
|
||||
this.entryStart = this.reader.getPosition();
|
||||
HLog.Entry e = reuse;
|
||||
if (e == null) {
|
||||
HLogKey key = HLog.newKey(conf);
|
||||
|
@ -163,15 +168,28 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
} catch (IOException e) {
|
||||
Log.warn("Failed getting position to add to throw", e);
|
||||
}
|
||||
// Preserve EOFE because these are treated differently if it comes up during
|
||||
// a split of logs
|
||||
|
||||
// See what SequenceFile.Reader thinks is the end of the file
|
||||
long end = Long.MAX_VALUE;
|
||||
try {
|
||||
Field fEnd = SequenceFile.Reader.class.getDeclaredField("end");
|
||||
fEnd.setAccessible(true);
|
||||
end = fEnd.getLong(this.reader);
|
||||
} catch(Exception e) { /* reflection fail. keep going */ }
|
||||
|
||||
String msg = (this.path == null? "": this.path.toString()) +
|
||||
", pos=" + pos + ", edit=" + this.edit;
|
||||
if (ioe instanceof EOFException) {
|
||||
EOFException eof = new EOFException(msg);
|
||||
eof.initCause(ioe);
|
||||
return eof;
|
||||
}
|
||||
return new IOException(msg, ioe);
|
||||
", entryStart=" + entryStart + ", pos=" + pos +
|
||||
((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
|
||||
", edit=" + this.edit;
|
||||
|
||||
// Enhance via reflection so we don't change the original class type
|
||||
try {
|
||||
return (IOException) ioe.getClass()
|
||||
.getConstructor(String.class)
|
||||
.newInstance(msg)
|
||||
.initCause(ioe);
|
||||
} catch(Exception e) { /* reflection fail. keep going */ }
|
||||
|
||||
return ioe;
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue