From 6ae185134da3eb138be0892d360940df2d859b67 Mon Sep 17 00:00:00 2001 From: Ashish Singhi Date: Mon, 10 Aug 2015 18:40:09 +0530 Subject: [PATCH] HBASE-5878 Use getVisibleLength public api from HdfsDataInputStream from Hadoop-2. --- .../regionserver/wal/SequenceFileLogReader.java | 48 ++++++++++++++-------- 1 file changed, 32 insertions(+), 16 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..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(); -- 1.9.2.msysgit.0