hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r1002027 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
Date Tue, 28 Sep 2010 05:55:51 GMT
Author: stack
Date: Tue Sep 28 05:55:51 2010
New Revision: 1002027

URL: http://svn.apache.org/viewvc?rev=1002027&view=rev
Log:
HBASE-3038 WALReaderFSDataInputStream.getPos() fails if Filesize > MAX_INT

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

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1002027&r1=1002026&r2=1002027&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Tue Sep 28 05:55:51 2010
@@ -550,6 +550,8 @@ Release 0.21.0 - Unreleased
    HBASE-3042  Use LO4J in SequenceFileLogReader
                (Nicolas Spiegelberg via Stack)
    HBASE-2995  Incorrect dependency on Log class from Jetty
+   HBASE-3038  WALReaderFSDataInputStream.getPos() fails if Filesize > MAX_INT
+               (Nicolas Spiegelberg via Stack)
 
   IMPROVEMENTS
    HBASE-1760  Cleanup TODOs in HTable

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=1002027&r1=1002026&r2=1002027&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
Tue Sep 28 05:55:51 2010
@@ -20,7 +20,7 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import java.io.EOFException;
+import java.io.FilterInputStream;
 import java.io.IOException;
 import java.lang.Class;
 import java.lang.reflect.Constructor;
@@ -78,18 +78,43 @@ public class SequenceFileLogReader imple
         this.length = l;
       }
 
+      // This section can be confusing.  It is specific to how HDFS works.
+      // Let me try to break it down.  This is the problem:
+      //
+      //  1. HDFS DataNodes update the NameNode about a filename's length 
+      //     on block boundaries or when a file is closed. Therefore, 
+      //     if an RS dies, then the NN's fs.getLength() can be out of date
+      //  2. this.in.available() would work, but it returns int &
+      //     therefore breaks for files > 2GB (happens on big clusters)
+      //  3. DFSInputStream.getFileLength() gets the actual length from the DNs
+      //  4. DFSInputStream is wrapped 2 levels deep : this.in.in
+      //
+      // So, here we adjust getPos() using getFileLength() so the
+      // SequenceFile.Reader constructor (aka: first invocation) comes out 
+      // with the correct end of the file:
+      //         this.end = in.getPos() + length;
       @Override
       public long getPos() throws IOException {
         if (this.firstGetPosInvocation) {
           this.firstGetPosInvocation = false;
-          // Tell a lie.  We're doing this just so that this line up in
-          // SequenceFile.Reader constructor comes out with the correct length
-          // on the file:
-          //         this.end = in.getPos() + length;
-          long available = this.in.available();
-          // Length gets added up in the SF.Reader constructor so subtract the
-          // difference.  If available < this.length, then return this.length.
-          return available >= this.length? available - this.length: this.length;
+          long adjust = 0;
+
+          try {
+            Field fIn = FilterInputStream.class.getDeclaredField("in");
+            fIn.setAccessible(true);
+            Object realIn = fIn.get(this.in);
+            long realLength = ((Long)realIn.getClass().
+                getMethod("getFileLength", new Class<?> []{}).
+                invoke(realIn, new Object []{})).longValue();
+            assert(realLength >= this.length);
+            adjust = realLength - this.length;
+          } catch(Exception e) {
+            SequenceFileLogReader.LOG.warn(
+              "Error while trying to get accurate file length.  " +
+              "Truncation / data loss may occur if RegionServers die.", e);
+          }
+
+          return adjust + super.getPos();
         }
         return super.getPos();
       }



Mime
View raw message