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:21 UTC

[1/5] hbase git commit: HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop-2.

Repository: hbase
Updated Branches:
  refs/heads/0.98 3bf9957d9 -> b69569f51
  refs/heads/branch-1 afa6fa6ba -> 0862abd65
  refs/heads/branch-1.0 bc4498a26 -> e7c137aa0
  refs/heads/branch-1.1 e00caff3b -> 22b968315
  refs/heads/branch-1.2 a4c08f630 -> 7f33e6330


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/0862abd6
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0862abd6
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0862abd6

Branch: refs/heads/branch-1
Commit: 0862abd6599a6936fb8079f4c70afc660175ba11
Parents: afa6fa6
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 16:20:23 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/0862abd6/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();


[3/5] hbase git commit: HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop-2.

Posted by ap...@apache.org.
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/b69569f5
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b69569f5
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b69569f5

Branch: refs/heads/0.98
Commit: b69569f512068d795199310ce662ab381bb6b6b7
Parents: 3bf9957
Author: Ashish Singhi <as...@huawei.com>
Authored: Mon Aug 10 16:20:53 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Aug 10 16:20:53 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/b69569f5/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 128274a..b9c92f6 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,24 +116,40 @@ public class SequenceFileLogReader extends ReaderBase {
             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;
+            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;
+              }
             } 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) {
-            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();


[4/5] hbase git commit: HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop 2

Posted by ap...@apache.org.
HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop 2

Amended to make only the change in SequenceFileLogReader that rethrows
exceptions caught while trying to get the file length as IOException.

Signed-off-by: Andrew Purtell <ap...@apache.org>
Amending-Author: 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/e7c137aa
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e7c137aa
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e7c137aa

Branch: refs/heads/branch-1.0
Commit: e7c137aa061645c48011a1ca8626c9a82b9cf4c3
Parents: bc4498a
Author: Ashish Singhi <as...@huawei.com>
Authored: Mon Aug 10 17:01:52 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Aug 10 17:01:52 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e7c137aa/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..5f6b377 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
@@ -134,6 +134,7 @@ public class SequenceFileLogReader extends ReaderBase {
             SequenceFileLogReader.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();


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

Posted by ap...@apache.org.
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/7f33e633
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/7f33e633
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/7f33e633

Branch: refs/heads/branch-1.2
Commit: 7f33e6330a37b0401c2f9143ddbea67361217453
Parents: a4c08f6
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 16:20:26 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/7f33e633/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();


[5/5] hbase git commit: HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop 2

Posted by ap...@apache.org.
HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop 2

Amended to make only the change in SequenceFileLogReader that rethrows
exceptions caught while trying to get the file length as IOException.

Signed-off-by: Andrew Purtell <ap...@apache.org>
Amending-Author: 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/22b96831
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/22b96831
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/22b96831

Branch: refs/heads/branch-1.1
Commit: 22b96831584d44a63c71c5366f372622d13fd9b4
Parents: e00caff
Author: Ashish Singhi <as...@huawei.com>
Authored: Mon Aug 10 17:02:00 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Mon Aug 10 17:02:00 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/regionserver/wal/SequenceFileLogReader.java | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/22b96831/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..5f6b377 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
@@ -134,6 +134,7 @@ public class SequenceFileLogReader extends ReaderBase {
             SequenceFileLogReader.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();