HBASE-4935 hbase 0.92.0 doesn't work going against 0.20.205.0, its packaged hadoop
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1221033 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
1222383f1d
commit
77e16e8027
|
@ -457,6 +457,7 @@ Release 0.92.0 - Unreleased
|
|||
HBASE-4946 HTable.coprocessorExec (and possibly coprocessorProxy) does not work with
|
||||
dynamically loaded coprocessors (Andrei Dragomir)
|
||||
HBASE-5026 Add coprocessor hook to HRegionServer.ScannerListener.leaseExpired()
|
||||
HBASE-4935 hbase 0.92.0 doesn't work going against 0.20.205.0, its packaged hadoop
|
||||
|
||||
TESTS
|
||||
HBASE-4450 test for number of blocks read: to serve as baseline for expected
|
||||
|
|
|
@ -26,7 +26,8 @@ import java.lang.Class;
|
|||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.Field;
|
||||
import java.lang.reflect.Method;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -82,8 +83,8 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
// This section can be confusing. It is specific to how HDFS works.
|
||||
// Let me try to break it down. This is the problem:
|
||||
//
|
||||
// 1. HDFS DataNodes update the NameNode about a filename's length
|
||||
// on block boundaries or when a file is closed. Therefore,
|
||||
// 1. HDFS DataNodes update the NameNode about a filename's length
|
||||
// on block boundaries or when a file is closed. Therefore,
|
||||
// if an RS dies, then the NN's fs.getLength() can be out of date
|
||||
// 2. this.in.available() would work, but it returns int &
|
||||
// therefore breaks for files > 2GB (happens on big clusters)
|
||||
|
@ -91,7 +92,7 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
// 4. DFSInputStream is wrapped 2 levels deep : this.in.in
|
||||
//
|
||||
// So, here we adjust getPos() using getFileLength() so the
|
||||
// SequenceFile.Reader constructor (aka: first invocation) comes out
|
||||
// SequenceFile.Reader constructor (aka: first invocation) comes out
|
||||
// with the correct end of the file:
|
||||
// this.end = in.getPos() + length;
|
||||
@Override
|
||||
|
@ -104,13 +105,20 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
Field fIn = FilterInputStream.class.getDeclaredField("in");
|
||||
fIn.setAccessible(true);
|
||||
Object realIn = fIn.get(this.in);
|
||||
Method getFileLength = realIn.getClass().
|
||||
getMethod("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");
|
||||
}
|
||||
} catch(Exception e) {
|
||||
SequenceFileLogReader.LOG.warn(
|
||||
"Error while trying to get accurate file length. " +
|
||||
|
@ -142,7 +150,7 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
/**
|
||||
* This constructor allows a specific HLogKey implementation to override that
|
||||
* which would otherwise be chosen via configuration property.
|
||||
*
|
||||
*
|
||||
* @param keyClass
|
||||
*/
|
||||
public SequenceFileLogReader(Class<? extends HLogKey> keyClass) {
|
||||
|
@ -189,7 +197,7 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
throw new IOException(iae);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
WALEdit val = new WALEdit();
|
||||
e = new HLog.Entry(key, val);
|
||||
}
|
||||
|
@ -235,8 +243,8 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
} catch(Exception e) { /* reflection fail. keep going */ }
|
||||
|
||||
String msg = (this.path == null? "": this.path.toString()) +
|
||||
", entryStart=" + entryStart + ", pos=" + pos +
|
||||
((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
|
||||
", entryStart=" + entryStart + ", pos=" + pos +
|
||||
((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
|
||||
", edit=" + this.edit;
|
||||
|
||||
// Enhance via reflection so we don't change the original class type
|
||||
|
@ -246,7 +254,7 @@ public class SequenceFileLogReader implements HLog.Reader {
|
|||
.newInstance(msg)
|
||||
.initCause(ioe);
|
||||
} catch(Exception e) { /* reflection fail. keep going */ }
|
||||
|
||||
|
||||
return ioe;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue