|
@@ -29,6 +29,7 @@ import java.util.Vector;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
+
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
@@ -52,7 +53,7 @@ import org.onelab.filter.*;
|
|
|
* be called directly by any writer, but rather by an HRegion manager.
|
|
|
*/
|
|
|
class HStore implements HConstants {
|
|
|
- private static final Log LOG = LogFactory.getLog(HStore.class);
|
|
|
+ static final Log LOG = LogFactory.getLog(HStore.class);
|
|
|
|
|
|
static final String COMPACTION_DIR = "compaction.tmp";
|
|
|
static final String WORKING_COMPACTION = "compaction.inprogress";
|
|
@@ -299,6 +300,10 @@ class HStore implements HConstants {
|
|
|
private void loadOrCreateBloomFilter() throws IOException {
|
|
|
Path filterFile = new Path(filterDir, BLOOMFILTER_FILE_NAME);
|
|
|
if(fs.exists(filterFile)) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("loading bloom filter for " + family.getName());
|
|
|
+ }
|
|
|
+
|
|
|
switch(family.bloomFilter.filterType) {
|
|
|
|
|
|
case BloomFilterDescriptor.BLOOMFILTER:
|
|
@@ -317,6 +322,10 @@ class HStore implements HConstants {
|
|
|
fs.close();
|
|
|
|
|
|
} else {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("creating bloom filter for " + family.getName());
|
|
|
+ }
|
|
|
+
|
|
|
switch(family.bloomFilter.filterType) {
|
|
|
|
|
|
case BloomFilterDescriptor.BLOOMFILTER:
|
|
@@ -342,18 +351,33 @@ class HStore implements HConstants {
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
private void flushBloomFilter() throws IOException {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("flushing bloom filter for " + family.getName());
|
|
|
+ }
|
|
|
FSDataOutputStream out =
|
|
|
fs.create(new Path(filterDir, BLOOMFILTER_FILE_NAME));
|
|
|
|
|
|
bloomFilter.write(out);
|
|
|
out.close();
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("flushed bloom filter for " + family.getName());
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/** Generates a bloom filter key from the row and column keys */
|
|
|
Key getBloomFilterKey(HStoreKey k) {
|
|
|
StringBuilder s = new StringBuilder(k.getRow().toString());
|
|
|
s.append(k.getColumn().toString());
|
|
|
- return new Key(s.toString().getBytes());
|
|
|
+
|
|
|
+ byte[] bytes = null;
|
|
|
+ try {
|
|
|
+ bytes = s.toString().getBytes(HConstants.UTF8_ENCODING);
|
|
|
+
|
|
|
+ } catch (UnsupportedEncodingException e) {
|
|
|
+ e.printStackTrace();
|
|
|
+ assert(false);
|
|
|
+ }
|
|
|
+ return new Key(bytes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -372,8 +396,14 @@ class HStore implements HConstants {
|
|
|
// Note - the key being passed to us is always a HStoreKey
|
|
|
|
|
|
if(bloomFilter.membershipTest(getBloomFilterKey((HStoreKey)key))) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("bloom filter reported that key exists");
|
|
|
+ }
|
|
|
return super.get(key, val);
|
|
|
}
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("bloom filter reported that key does not exist");
|
|
|
+ }
|
|
|
return null;
|
|
|
}
|
|
|
|
|
@@ -383,8 +413,14 @@ class HStore implements HConstants {
|
|
|
// Note - the key being passed to us is always a HStoreKey
|
|
|
|
|
|
if(bloomFilter.membershipTest(getBloomFilterKey((HStoreKey)key))) {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("bloom filter reported that key exists");
|
|
|
+ }
|
|
|
return super.getClosest(key, val);
|
|
|
}
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("bloom filter reported that key does not exist");
|
|
|
+ }
|
|
|
return null;
|
|
|
}
|
|
|
}
|
|
@@ -1083,8 +1119,8 @@ class HStore implements HConstants {
|
|
|
// Iterate through all the MapFiles
|
|
|
for(Map.Entry<Long, HStoreFile> e: mapFiles.entrySet()) {
|
|
|
HStoreFile curHSF = e.getValue();
|
|
|
- long size = fs.getLength(
|
|
|
- new Path(curHSF.getMapFilePath(), MapFile.DATA_FILE_NAME));
|
|
|
+ long size = fs.getFileStatus(
|
|
|
+ new Path(curHSF.getMapFilePath(), MapFile.DATA_FILE_NAME)).getLen();
|
|
|
if(size > maxSize) { // This is the largest one so far
|
|
|
maxSize = size;
|
|
|
mapIndex = e.getKey();
|