You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2015/08/11 02:14:42 UTC

hbase git commit: HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop-2.

Repository: hbase
Updated Branches:
  refs/heads/master e4106b4c4 -> 6e8cdec24


HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop-2.

Signed-off-by: Andrew Purtell <ap...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6e8cdec2
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6e8cdec2
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6e8cdec2

Branch: refs/heads/master
Commit: 6e8cdec242b6c40c09601982bad0a79a569e66c4
Parents: e4106b4
Author: Ashish Singhi <as...@huawei.com>
Authored: Mon Aug 10 16:20:07 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Aug 10 17:14:30 2015 -0700

----------------------------------------------------------------------
 .../regionserver/wal/SequenceFileLogReader.java | 36 ++++++++------------
 1 file changed, 14 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6e8cdec2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java
----------------------------------------------------------------------
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 11312b1..be39873 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
@@ -19,21 +19,20 @@
 
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import java.io.FilterInputStream;
 import java.io.IOException;
 import java.lang.reflect.Field;
-import java.lang.reflect.Method;
 import java.util.NavigableMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 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.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.wal.WAL.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 +110,24 @@ public class SequenceFileLogReader extends ReaderBase {
         if (this.firstGetPosInvocation) {
           this.firstGetPosInvocation = false;
           long adjust = 0;
-
+          HdfsDataInputStream hdfsDataInputStream = null;
           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();
+            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: " + realIn.getClass().getName() +
-                  ", not adjusting length");
+              LOG.info(
+                "Input stream class: " + this.in.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);
+          } catch (Exception e) {
+            LOG.warn("Error while trying to get accurate file length.  "
+                + "Truncation / data loss may occur if RegionServers die.",
+              e);
+            throw new IOException(e);
           }
-
           return adjust + super.getPos();
         }
         return super.getPos();