Index: CHANGES.txt =================================================================== --- CHANGES.txt (revision 1214532) +++ CHANGES.txt (revision 1215230) @@ -487,6 +487,7 @@ (Chunhui Shen) HBASE-4997 SplitLogManager can have a race on batch.installed (Prakash Khemani) HBASE-5026 Add coprocessor hook to HRegionServer.ScannerListener.leaseExpired() + HBASE-4935 hbase 0.92.0 doesn't work going against 0.20.205.0, its packaged hadoop TESTS HBASE-4492 TestRollingRestart fails intermittently Index: src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java (revision 1214532) +++ src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java (revision 1215230) @@ -26,13 +26,15 @@ import java.lang.reflect.Constructor; import java.lang.reflect.Field; import java.lang.reflect.Method; - +import java.util.Arrays; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSClient.DFSInputStream; import org.apache.hadoop.io.SequenceFile; public class SequenceFileLogReader implements HLog.Reader { @@ -82,8 +84,8 @@ // 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, + // 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) @@ -91,7 +93,7 @@ // 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 + // SequenceFile.Reader constructor (aka: first invocation) comes out // with the correct end of the file: // this.end = in.getPos() + length; @Override @@ -104,13 +106,18 @@ Field fIn = FilterInputStream.class.getDeclaredField("in"); fIn.setAccessible(true); Object realIn = fIn.get(this.in); - Method getFileLength = realIn.getClass(). - getMethod("getFileLength", new Class []{}); - getFileLength.setAccessible(true); - long realLength = ((Long)getFileLength. - invoke(realIn, new Object []{})).longValue(); - assert(realLength >= this.length); - adjust = realLength - this.length; + if (realIn.getClass() == DFSInputStream.class) { + Method getFileLength = realIn.getClass(). + getDeclaredMethod("getFileLength", new Class []{}); + getFileLength.setAccessible(true); + long realLength = ((Long)getFileLength. + invoke(realIn, new Object []{})).longValue(); + assert(realLength >= this.length); + adjust = realLength - this.length; + } else { + LOG.info("Input stream class: " + realIn.getClass().getName() + + ", not adjusting length"); + } } catch(Exception e) { SequenceFileLogReader.LOG.warn( "Error while trying to get accurate file length. " + @@ -142,7 +149,7 @@ /** * This constructor allows a specific HLogKey implementation to override that * which would otherwise be chosen via configuration property. - * + * * @param keyClass */ public SequenceFileLogReader(Class keyClass) { @@ -189,7 +196,7 @@ throw new IOException(iae); } } - + WALEdit val = new WALEdit(); e = new HLog.Entry(key, val); } @@ -235,8 +242,8 @@ } catch(Exception e) { /* reflection fail. keep going */ } String msg = (this.path == null? "": this.path.toString()) + - ", entryStart=" + entryStart + ", pos=" + pos + - ((end == Long.MAX_VALUE) ? "" : ", end=" + end) + + ", 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 @@ -246,7 +253,7 @@ .newInstance(msg) .initCause(ioe); } catch(Exception e) { /* reflection fail. keep going */ } - + return ioe; } }