From 34e9ed46ff561450bd6c1ac9fded673ce954f797 Mon Sep 17 00:00:00 2001 From: Ashish Singhi Date: Fri, 3 Jul 2015 15:45:47 +0530 Subject: [PATCH] HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop-2. --- .../regionserver/wal/SequenceFileLogReader.java | 35 +++++++--------------- 1 file changed, 11 insertions(+), 24 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java index 128274a..446e262 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java @@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; +import org.apache.hadoop.hdfs.client.HdfsDataInputStream; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile.Metadata; import org.apache.hadoop.io.Text; @@ -111,31 +112,17 @@ public class SequenceFileLogReader extends ReaderBase { if (this.firstGetPosInvocation) { this.firstGetPosInvocation = false; long adjust = 0; - - try { - Field fIn = FilterInputStream.class.getDeclaredField("in"); - fIn.setAccessible(true); - Object realIn = fIn.get(this.in); - // In hadoop 0.22, DFSInputStream is a standalone class. Before this, - // it was an inner class of DFSClient. - if (realIn.getClass().getName().endsWith("DFSInputStream")) { - 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. " + - "Truncation / data loss may occur if RegionServers die.", e); + HdfsDataInputStream hdfsDataInputStream = null; + if (this.in.getClass().getName().endsWith("HdfsDataInputStream") + || this.in.getClass().getName().endsWith("DFSInputStream")) { + hdfsDataInputStream = (HdfsDataInputStream) this.getWrappedStream(); + long realLength = hdfsDataInputStream.getVisibleLength(); + assert(realLength >= this.length); + adjust = realLength - this.length; + } else { + LOG.info( + "Input stream class: " + this.in.getClass().getName() + ", not adjusting length"); } - return adjust + super.getPos(); } return super.getPos(); -- 1.9.5 (Apple Git-50.3)