HBASE-980 Undo core of HBASE-975, caching of start and end row
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@711207 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d6997a949f
commit
062e954a03
|
@ -61,6 +61,7 @@ Release 0.19.0 - Unreleased
|
|||
HBASE-977 Arcane HStoreKey comparator bug
|
||||
HBASE-979 REST web app is not started automatically
|
||||
HBASE-964 Startup stuck "waiting for root region"
|
||||
HBASE-980 Undo core of HBASE-975, caching of start and end row
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-901 Add a limit to key length, check key and value length on client side
|
||||
|
|
|
@ -30,11 +30,13 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HStoreKey;
|
||||
import org.apache.hadoop.hbase.util.Hash;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
import org.onelab.filter.BloomFilter;
|
||||
import org.onelab.filter.Key;
|
||||
|
||||
/**
|
||||
* On write, all keys are added to a bloom filter. On read, all keys are
|
||||
|
@ -228,4 +230,13 @@ public class BloomFilterMapFile extends HBaseMapFile {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Custom bloom filter key maker.
|
||||
* @param key
|
||||
* @return Key made of bytes of row only.
|
||||
*/
|
||||
protected static Key getBloomFilterKey(WritableComparable key) {
|
||||
return new Key(((HStoreKey) key).getRow());
|
||||
}
|
||||
}
|
|
@ -41,26 +41,18 @@ import org.onelab.filter.Key;
|
|||
*/
|
||||
public class HBaseMapFile extends MapFile {
|
||||
private static final Log LOG = LogFactory.getLog(HBaseMapFile.class);
|
||||
|
||||
/**
|
||||
* Values are instances of this class.
|
||||
*/
|
||||
public static final Class<? extends Writable> VALUE_CLASS =
|
||||
ImmutableBytesWritable.class;
|
||||
|
||||
/**
|
||||
* Custom bloom filter key maker.
|
||||
* @param key
|
||||
* @return Key made of bytes of row only.
|
||||
*/
|
||||
protected static Key getBloomFilterKey(WritableComparable key) {
|
||||
return new Key(((HStoreKey) key).getRow());
|
||||
}
|
||||
|
||||
/**
|
||||
* A reader capable of reading and caching blocks of the data file.
|
||||
*/
|
||||
public static class HBaseReader extends MapFile.Reader {
|
||||
private final boolean blockCacheEnabled;
|
||||
private final HStoreKey firstKey;
|
||||
private final HStoreKey finalKey;
|
||||
private final String dirName;
|
||||
|
||||
/**
|
||||
* @param fs
|
||||
|
@ -88,37 +80,17 @@ public class HBaseMapFile extends MapFile {
|
|||
throws IOException {
|
||||
super(fs, dirName, new HStoreKey.HStoreKeyWritableComparator(hri),
|
||||
conf, false); // defer opening streams
|
||||
this.dirName = dirName;
|
||||
this.blockCacheEnabled = blockCacheEnabled;
|
||||
open(fs, dirName, new HStoreKey.HStoreKeyWritableComparator(hri), conf);
|
||||
|
||||
// Force reading of the mapfile index by calling midKey.
|
||||
// Reading the index will bring the index into memory over
|
||||
// here on the client and then close the index file freeing
|
||||
// up socket connection and resources in the datanode.
|
||||
// Usually, the first access on a MapFile.Reader will load the
|
||||
// index force the issue in HStoreFile MapFiles because an
|
||||
// access may not happen for some time; meantime we're
|
||||
// using up datanode resources. See HADOOP-2341.
|
||||
// Of note, midKey just goes to index. Does not seek.
|
||||
// Force reading of the mapfile index by calling midKey. Reading the
|
||||
// index will bring the index into memory over here on the client and
|
||||
// then close the index file freeing up socket connection and resources
|
||||
// in the datanode. Usually, the first access on a MapFile.Reader will
|
||||
// load the index force the issue in HStoreFile MapFiles because an
|
||||
// access may not happen for some time; meantime we're using up datanode
|
||||
// resources (See HADOOP-2341). midKey() goes to index. Does not seek.
|
||||
midKey();
|
||||
|
||||
// Read in the first and last key. Cache them. Make sure we are at start
|
||||
// of the file.
|
||||
reset();
|
||||
HStoreKey key = new HStoreKey();
|
||||
super.next(key, new ImmutableBytesWritable());
|
||||
key.setHRegionInfo(hri);
|
||||
this.firstKey = key;
|
||||
// Set us back to start of file. Call to finalKey restores whatever
|
||||
// the previous position.
|
||||
reset();
|
||||
|
||||
// Get final key.
|
||||
key = new HStoreKey();
|
||||
super.finalKey(key);
|
||||
key.setHRegionInfo(hri);
|
||||
this.finalKey = key;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -129,71 +101,18 @@ public class HBaseMapFile extends MapFile {
|
|||
return super.createDataFileReader(fs, dataFile, conf);
|
||||
}
|
||||
final int blockSize = conf.getInt("hbase.hstore.blockCache.blockSize",
|
||||
64 * 1024);
|
||||
64 * 1024);
|
||||
return new SequenceFile.Reader(fs, dataFile, conf) {
|
||||
@Override
|
||||
protected FSDataInputStream openFile(FileSystem fs, Path file,
|
||||
int bufferSize, long length) throws IOException {
|
||||
|
||||
int bufferSize, long length)
|
||||
throws IOException {
|
||||
return new FSDataInputStream(new BlockFSInputStream(
|
||||
super.openFile(fs, file, bufferSize, length), length,
|
||||
blockSize));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void finalKey(final WritableComparable fk)
|
||||
throws IOException {
|
||||
Writables.copyWritable(this.finalKey, fk);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param hsk
|
||||
* @return True if the file *could* contain <code>hsk</code> and false if
|
||||
* outside the bounds of this files' start and end keys.
|
||||
*/
|
||||
public boolean containsKey(final HStoreKey hsk) {
|
||||
return this.firstKey.compareTo(hsk) <= 0 &&
|
||||
this.finalKey.compareTo(hsk) >= 0;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
HStoreKey midkey = null;
|
||||
try {
|
||||
midkey = (HStoreKey)midKey();
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn("Failed get of midkey", ioe);
|
||||
}
|
||||
return "dirName=" + this.dirName + ", firstKey=" +
|
||||
this.firstKey.toString() + ", midKey=" + midkey +
|
||||
", finalKey=" + this.finalKey;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return First key in this file. Can be null around construction time.
|
||||
*/
|
||||
public HStoreKey getFirstKey() {
|
||||
return this.firstKey;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Final key in file. Can be null around construction time.
|
||||
*/
|
||||
public HStoreKey getFinalKey() {
|
||||
return this.finalKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized WritableComparable getClosest(WritableComparable key,
|
||||
Writable value, boolean before)
|
||||
throws IOException {
|
||||
if ((!before && ((HStoreKey)key).compareTo(this.finalKey) > 0) ||
|
||||
(before && ((HStoreKey)key).compareTo(this.firstKey) < 0)) {
|
||||
return null;
|
||||
}
|
||||
return super.getClosest(key, value, before);
|
||||
}
|
||||
}
|
||||
|
||||
public static class HBaseWriter extends MapFile.Writer {
|
||||
|
@ -213,7 +132,7 @@ public class HBaseMapFile extends MapFile {
|
|||
// Default for mapfiles is 128. Makes random reads faster if we
|
||||
// have more keys indexed and we're not 'next'-ing around in the
|
||||
// mapfile.
|
||||
setIndexInterval(conf.getInt("hbase.io.index.interval", 128));
|
||||
setIndexInterval(conf.getInt("hbase.io.index.interval", 32));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,8 +48,6 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
|
|||
private final boolean top;
|
||||
private final HStoreKey midkey;
|
||||
private boolean firstNextCall = true;
|
||||
private final WritableComparable<HStoreKey> firstKey;
|
||||
private final WritableComparable<HStoreKey> finalKey;
|
||||
|
||||
public HalfMapFileReader(final FileSystem fs, final String dirName,
|
||||
final Configuration conf, final Range r,
|
||||
|
@ -76,18 +74,6 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
|
|||
this.midkey.setHRegionInfo(hri);
|
||||
// Is it top or bottom half?
|
||||
this.top = Reference.isTopFileRegion(r);
|
||||
// Firstkey comes back null if passed midkey is lower than first key in file
|
||||
// and its a bottom half HalfMapFileReader, OR, if midkey is higher than
|
||||
// the final key in the backing file. In either case, it means this half
|
||||
// file is empty.
|
||||
this.firstKey = this.top?
|
||||
super.getClosest(this.midkey, new ImmutableBytesWritable()):
|
||||
super.getFirstKey().compareTo(this.midkey) > 0?
|
||||
null: super.getFirstKey();
|
||||
this.finalKey = this.top?
|
||||
super.getFinalKey():
|
||||
super.getClosest(new HStoreKey.BeforeThisStoreKey(this.midkey),
|
||||
new ImmutableBytesWritable(), true);
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -108,12 +94,16 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public synchronized void finalKey(WritableComparable key)
|
||||
throws IOException {
|
||||
Writables.copyWritable(this.finalKey, key);
|
||||
return;
|
||||
if (top) {
|
||||
super.finalKey(key);
|
||||
} else {
|
||||
Writable value = new ImmutableBytesWritable();
|
||||
WritableComparable found = super.getClosest(midkey, value, true);
|
||||
Writables.copyWritable(found, key);
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
|
@ -134,8 +124,8 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
|
|||
// If top, the lowest possible key is first key. Do not have to check
|
||||
// what comes back from super getClosest. Will return exact match or
|
||||
// greater.
|
||||
closest = (key.compareTo(getFirstKey()) < 0)?
|
||||
getClosest(getFirstKey(), val): super.getClosest(key, val);
|
||||
closest = (key.compareTo(this.midkey) < 0)?
|
||||
this.midkey: super.getClosest(key, val);
|
||||
} else {
|
||||
// We're serving bottom of the file.
|
||||
if (key.compareTo(this.midkey) < 0) {
|
||||
|
@ -143,8 +133,7 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
|
|||
closest = super.getClosest(key, val);
|
||||
// midkey was made against largest store file at time of split. Smaller
|
||||
// store files could have anything in them. Check return value is
|
||||
// not beyond the midkey (getClosest returns exact match or next
|
||||
// after).
|
||||
// not beyond the midkey (getClosest returns exact match or next after)
|
||||
if (closest != null && closest.compareTo(this.midkey) >= 0) {
|
||||
// Don't let this value out.
|
||||
closest = null;
|
||||
|
@ -168,19 +157,17 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
|
|||
throws IOException {
|
||||
if (firstNextCall) {
|
||||
firstNextCall = false;
|
||||
if (isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
// Seek and fill by calling getClosest on first key.
|
||||
if (this.firstKey != null) {
|
||||
Writables.copyWritable(this.firstKey, key);
|
||||
WritableComparable nearest = getClosest(key, val);
|
||||
if (!key.equals(nearest)) {
|
||||
throw new IOException("Keys don't match and should: " +
|
||||
key.toString() + ", " + nearest.toString());
|
||||
if (this.top) {
|
||||
// Seek to midkey. Midkey may not exist in this file. That should be
|
||||
// fine. Then we'll either be positioned at end or start of file.
|
||||
WritableComparable nearest = getClosest(this.midkey, val);
|
||||
// Now copy the midkey into the passed key.
|
||||
if (nearest != null) {
|
||||
Writables.copyWritable(nearest, key);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
boolean result = super.next(key, val);
|
||||
if (!top && key.compareTo(midkey) >= 0) {
|
||||
|
@ -189,15 +176,10 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
|
|||
return result;
|
||||
}
|
||||
|
||||
private boolean isEmpty() {
|
||||
return this.firstKey == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void reset() throws IOException {
|
||||
if (top) {
|
||||
firstNextCall = true;
|
||||
// I don't think this is needed. seek(this.firstKey);
|
||||
return;
|
||||
}
|
||||
super.reset();
|
||||
|
@ -210,19 +192,4 @@ public class HalfMapFileReader extends BloomFilterMapFile.Reader {
|
|||
checkKey(key);
|
||||
return super.seek(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HStoreKey getFirstKey() {
|
||||
return (HStoreKey)this.firstKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HStoreKey getFinalKey() {
|
||||
return (HStoreKey)this.finalKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return super.toString() + ", half=" + (top? "top": "bottom");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1371,7 +1371,6 @@ public class HStore implements HConstants {
|
|||
i >= 0 && !hasEnoughVersions(versions, results); i--) {
|
||||
MapFile.Reader map = maparray[i];
|
||||
synchronized(map) {
|
||||
map.reset();
|
||||
// Do the priming read
|
||||
ImmutableBytesWritable readval = new ImmutableBytesWritable();
|
||||
HStoreKey readkey = (HStoreKey)map.getClosest(key, readval);
|
||||
|
@ -1379,7 +1378,9 @@ public class HStore implements HConstants {
|
|||
// map.getClosest returns null if the passed key is > than the
|
||||
// last key in the map file. getClosest is a bit of a misnomer
|
||||
// since it returns exact match or the next closest key AFTER not
|
||||
// BEFORE.
|
||||
// BEFORE. We use getClosest because we're usually passed a
|
||||
// key that has a timestamp of maximum long to indicate we want
|
||||
// most recent update.
|
||||
continue;
|
||||
}
|
||||
if (!readkey.matchesRowCol(key)) {
|
||||
|
@ -1868,9 +1869,6 @@ public class HStore implements HConstants {
|
|||
* @throws IOException
|
||||
*/
|
||||
private HStoreKey getFinalKey(final MapFile.Reader mf) throws IOException {
|
||||
if (mf instanceof HBaseMapFile.HBaseReader) {
|
||||
return ((HBaseMapFile.HBaseReader)mf).getFinalKey();
|
||||
}
|
||||
HStoreKey finalKey = new HStoreKey();
|
||||
mf.finalKey(finalKey);
|
||||
finalKey.setHRegionInfo(this.info);
|
||||
|
@ -1961,16 +1959,18 @@ public class HStore implements HConstants {
|
|||
// Cast to HbaseReader.
|
||||
HBaseMapFile.HBaseReader r =
|
||||
(HBaseMapFile.HBaseReader)this.readers.get(mapIndex);
|
||||
|
||||
// get the midkey
|
||||
// Get first, last, and mid keys.
|
||||
r.reset();
|
||||
HStoreKey firstKey = new HStoreKey();
|
||||
HStoreKey lastKey = new HStoreKey();
|
||||
r.next(firstKey, new ImmutableBytesWritable());
|
||||
r.finalKey(lastKey);
|
||||
HStoreKey mk = (HStoreKey)r.midKey();
|
||||
if (mk != null) {
|
||||
// if the midkey is the same as the first and last keys, then we cannot
|
||||
// (ever) split this region.
|
||||
if (HStoreKey.equalsTwoRowKeys(info, mk.getRow(),
|
||||
r.getFirstKey().getRow()) &&
|
||||
HStoreKey.equalsTwoRowKeys(info, mk.getRow(),
|
||||
r.getFinalKey().getRow())) {
|
||||
if (HStoreKey.equalsTwoRowKeys(info, mk.getRow(), firstKey.getRow()) &&
|
||||
HStoreKey.equalsTwoRowKeys(info, mk.getRow(), lastKey.getRow())) {
|
||||
return null;
|
||||
}
|
||||
return new StoreSize(maxSize, mk.getRow());
|
||||
|
@ -2070,4 +2070,4 @@ public class HStore implements HConstants {
|
|||
HRegionInfo getHRegionInfo() {
|
||||
return this.info;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,17 +23,16 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestCase;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HStoreKey;
|
||||
import org.apache.hadoop.hbase.io.HBaseMapFile;
|
||||
import org.apache.hadoop.hbase.io.HalfMapFileReader;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.io.Reference;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.io.MapFile;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
@ -117,25 +116,6 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testContains() throws Exception {
|
||||
HStoreFile hsf = new HStoreFile(this.conf, this.fs, this.dir,
|
||||
HRegionInfo.FIRST_META_REGIONINFO,
|
||||
Bytes.toBytes("colfamily"), 1234567890L, null);
|
||||
MapFile.Writer writer =
|
||||
hsf.getWriter(this.fs, SequenceFile.CompressionType.NONE, false, 0);
|
||||
writeStoreFile(writer);
|
||||
MapFile.Reader r = hsf.getReader(this.fs, false, false);
|
||||
HBaseMapFile.HBaseReader reader =
|
||||
(HBaseMapFile.HBaseReader)r;
|
||||
// Store file should contain 'aa' and 'bb' but not 'AA' nor 'ZZ'.
|
||||
assertTrue(reader.containsKey(new HStoreKey("aa", "bb")));
|
||||
assertTrue(reader.containsKey(new HStoreKey("bb")));
|
||||
assertTrue(reader.containsKey(new HStoreKey("zz")));
|
||||
assertFalse(reader.containsKey(new HStoreKey("AA")));
|
||||
assertFalse(reader.containsKey(new HStoreKey("{{")));
|
||||
assertFalse(reader.containsKey(new HStoreKey()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that our mechanism of writing store files in one region to reference
|
||||
* store files in other regions works.
|
||||
|
@ -192,13 +172,6 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
}
|
||||
}
|
||||
assertTrue(Bytes.equals(key.getRow(), finalKey));
|
||||
// Assert contains works properly.
|
||||
HBaseMapFile.HBaseReader hbaseMapfileHalfReader =
|
||||
(HBaseMapFile.HBaseReader)halfReader;
|
||||
assertTrue(hbaseMapfileHalfReader.containsKey(midkey));
|
||||
assertTrue(hbaseMapfileHalfReader.containsKey(new HStoreKey(finalKey)));
|
||||
assertFalse(hbaseMapfileHalfReader.containsKey(new HStoreKey("aa")));
|
||||
assertFalse(hbaseMapfileHalfReader.containsKey(new HStoreKey("{{")));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue