Author: stack
Date: Fri Sep 24 04:00:48 2010
New Revision: 1000697

URL: http://svn.apache.org/viewvc?rev=1000697&view=rev
Log:
HBASE-2889 Tool to look at HLogs -- parse and tail -f; fix for suppression of 
EOFEs by Nicolas

Modified:
    
hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java

Modified: 
hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
URL: 
http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java?rev=1000697&r1=1000696&r2=1000697&view=diff
==============================================================================
--- 
hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
 (original)
+++ 
hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
 Fri Sep 24 04:00:48 2010
@@ -22,7 +22,10 @@ package org.apache.hadoop.hbase.regionse
 
 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;
 import org.apache.hadoop.fs.FileSystem;
@@ -98,6 +101,7 @@ public class SequenceFileLogReader imple
   // Needed logging exceptions
   Path path;
   int edit = 0;
+  long entryStart = 0;
 
   public SequenceFileLogReader() { }
 
@@ -125,6 +129,7 @@ public class SequenceFileLogReader imple
 
   @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 imple
     } 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;
   }
-}
\ No newline at end of file
+}


Reply via email to