hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From apurt...@apache.org
Subject hbase git commit: Revert "HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop-2."
Date Tue, 11 Aug 2015 00:20:06 GMT
Repository: hbase
Updated Branches:
  refs/heads/0.98 b69569f51 -> fabfb423f


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

This reverts commit b69569f512068d795199310ce662ab381bb6b6b7.


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

Branch: refs/heads/0.98
Commit: fabfb423f9cf48ddd52e9583ca66600004f42349
Parents: b69569f
Author: Andrew Purtell <apurtell@apache.org>
Authored: Mon Aug 10 17:19:54 2015 -0700
Committer: Andrew Purtell <apurtell@apache.org>
Committed: Mon Aug 10 17:19:54 2015 -0700

----------------------------------------------------------------------
 .../regionserver/wal/SequenceFileLogReader.java | 48 +++++++-------------
 1 file changed, 16 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fabfb423/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 b9c92f6..128274a 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
@@ -116,40 +116,24 @@ public class SequenceFileLogReader extends ReaderBase {
             Field fIn = FilterInputStream.class.getDeclaredField("in");
             fIn.setAccessible(true);
             Object realIn = fIn.get(this.in);
-            if (this.in.getClass().getName().endsWith("HdfsDataInputStream")
-                || realIn.getClass().getName().endsWith("DFSInputStream")) {
-              // Here we try to use reflection because HdfsDataInputStream is not available
in
-              // hadoop 1.1. HBASE-5878
-              try {
-                Class<?> hdfsDataInputStream =
-                    Class.forName("org.apache.hadoop.hdfs.client.HdfsDataInputStream");
-                Method getVisibleLength = hdfsDataInputStream.getDeclaredMethod("getVisibleLength");
-                getVisibleLength.setAccessible(true);
-                long realLength =
-                    ((Long) getVisibleLength.invoke(realIn, new Object[] {})).longValue();
-                assert (realLength >= this.length);
-                adjust = realLength - this.length;
-              } catch (ClassNotFoundException e) {
-                // Failed to found the class HdfsDataInputStream, may be it is deployed on
hadoop
-                // 1.1
-                // In hadoop 0.22, DFSInputStream is a standalone class. Before this,
-                // it was an inner class of DFSClient.
-                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;
-              }
+            // 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");
+              LOG.info("Input stream class: " + realIn.getClass().getName() +
+                  ", not adjusting length");
             }
-          } 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);
+          } 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();


Mime
View raw message