Provide configurable per-column bloom filters - part 2.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@547468 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jim Kellerman 2007-06-15 00:39:34 +00:00
parent e78644ed10
commit f503143412
7 changed files with 422 additions and 55 deletions

View File

@ -33,4 +33,5 @@ Trunk (unreleased changes)
18. HADOOP-1469 Asychronous table creation
19. HADOOP-1415 Integrate BSD licensed bloom filter implementation.
20. HADOOP-1465 Add cluster stop/start scripts for hbase
21. HADOOP-1415 Provide configurable per-column bloom filters - part 2.

View File

@ -0,0 +1,151 @@
/**
* Copyright 2007 The Apache Software Foundation
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.WritableComparable;
/**
* Supplied as a parameter to HColumnDescriptor to specify what kind of
* bloom filter to use for a column, and its configuration parameters
*/
public class BloomFilterDescriptor implements WritableComparable {
/*
* Specify the kind of bloom filter that will be instantiated
*/
/**
* <i>Bloom filter</i>, as defined by Bloom in 1970.
*/
public static final int BLOOMFILTER = 1;
/**
* <i>counting Bloom filter</i>, as defined by Fan et al. in a ToN 2000 paper.
*/
public static final int COUNTING_BLOOMFILTER = 2;
/**
* <i>retouched Bloom filter</i>, as defined in the CoNEXT 2006 paper.
*/
public static final int RETOUCHED_BLOOMFILTER = 3;
/** Default constructor - used in conjunction with Writable */
public BloomFilterDescriptor() {}
/**
* @param type The kind of bloom filter to use.
* @param vectorSize The vector size of <i>this</i> filter.
* @param nbHash The number of hash functions to consider.
*/
public BloomFilterDescriptor(int type, int vectorSize, int nbHash) {
switch(type) {
case BLOOMFILTER:
case COUNTING_BLOOMFILTER:
case RETOUCHED_BLOOMFILTER:
this.filterType = type;
break;
default:
throw new IllegalArgumentException("Invalid bloom filter type: " + type);
}
this.vectorSize = vectorSize;
this.nbHash = nbHash;
}
int filterType;
int vectorSize;
int nbHash;
@Override
public String toString() {
StringBuilder value = new StringBuilder();
switch(filterType) {
case BLOOMFILTER:
value.append("standard");
break;
case COUNTING_BLOOMFILTER:
value.append("counting");
break;
case RETOUCHED_BLOOMFILTER:
value.append("retouched");
}
value.append("(vector size=");
value.append(vectorSize);
value.append(", number hashes=");
value.append(nbHash);
value.append(")");
return value.toString();
}
@Override
public boolean equals(Object obj) {
return compareTo(obj) == 0;
}
@Override
public int hashCode() {
int result = Integer.valueOf(this.filterType).hashCode();
result ^= Integer.valueOf(this.vectorSize).hashCode();
result ^= Integer.valueOf(this.nbHash).hashCode();
return result;
}
// Writable
/* (non-Javadoc)
* @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
*/
public void readFields(DataInput in) throws IOException {
filterType = in.readInt();
vectorSize = in.readInt();
nbHash = in.readInt();
}
/* (non-Javadoc)
* @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
*/
public void write(DataOutput out) throws IOException {
out.writeInt(filterType);
out.writeInt(vectorSize);
out.writeInt(nbHash);
}
// Comparable
/* (non-Javadoc)
* @see java.lang.Comparable#compareTo(java.lang.Object)
*/
public int compareTo(Object o) {
BloomFilterDescriptor other = (BloomFilterDescriptor)o;
int result = this.filterType - other.filterType;
if(result == 0) {
result = this.vectorSize - other.vectorSize;
}
if(result == 0) {
result = this.nbHash - other.nbHash;
}
return result;
}
}

View File

@ -65,7 +65,8 @@ public class HColumnDescriptor implements WritableComparable {
byte compressionType; // Compression setting if any
boolean inMemory; // Serve reads from in-memory cache
int maxValueLength; // Maximum value size
boolean bloomFilterEnabled; // True if column has a bloom filter
private boolean bloomFilterSpecified; // True if bloom filter was specified
BloomFilterDescriptor bloomFilter; // Descriptor of bloom filter
byte versionNumber; // Version number of this class
/**
@ -77,7 +78,8 @@ public class HColumnDescriptor implements WritableComparable {
this.compressionType = COMPRESSION_NONE;
this.inMemory = false;
this.maxValueLength = Integer.MAX_VALUE;
this.bloomFilterEnabled = false;
this.bloomFilterSpecified = false;
this.bloomFilter = null;
this.versionNumber = COLUMN_DESCRIPTOR_VERSION;
}
@ -100,7 +102,7 @@ public class HColumnDescriptor implements WritableComparable {
* @param inMemory - If true, column data should be kept in a
* HRegionServer's cache
* @param maxValueLength - Restrict values to &lt;= this value
* @param bloomFilter - Enable a bloom filter for this column
* @param bloomFilter - Enable the specified bloom filter for this column
*
* @throws IllegalArgumentException if passed a family name that is made of
* other than 'word' characters: i.e. <code>[a-zA-Z_0-9]</code> and does not
@ -108,7 +110,7 @@ public class HColumnDescriptor implements WritableComparable {
* @throws IllegalArgumentException if the number of versions is &lt;= 0
*/
public HColumnDescriptor(Text name, int maxVersions, CompressionType compression,
boolean inMemory, int maxValueLength, boolean bloomFilter) {
boolean inMemory, int maxValueLength, BloomFilterDescriptor bloomFilter) {
String familyStr = name.toString();
Matcher m = LEGAL_FAMILY_NAME.matcher(familyStr);
if(m == null || !m.matches()) {
@ -138,7 +140,8 @@ public class HColumnDescriptor implements WritableComparable {
}
this.inMemory = inMemory;
this.maxValueLength = maxValueLength;
this.bloomFilterEnabled = bloomFilter;
this.bloomFilter = bloomFilter;
this.bloomFilterSpecified = this.bloomFilter == null ? false : true;
this.versionNumber = COLUMN_DESCRIPTOR_VERSION;
}
@ -195,7 +198,8 @@ public class HColumnDescriptor implements WritableComparable {
return "(" + name + ", max versions: " + maxVersions + ", compression: "
+ compression + ", in memory: " + inMemory + ", max value length: "
+ maxValueLength + ", bloom filter:" + bloomFilterEnabled + ")";
+ maxValueLength + ", bloom filter: "
+ (bloomFilterSpecified ? bloomFilter.toString() : "none") + ")";
}
@Override
@ -210,8 +214,11 @@ public class HColumnDescriptor implements WritableComparable {
result ^= Byte.valueOf(this.compressionType).hashCode();
result ^= Boolean.valueOf(this.inMemory).hashCode();
result ^= Integer.valueOf(this.maxValueLength).hashCode();
result ^= Boolean.valueOf(this.bloomFilterEnabled).hashCode();
result ^= Boolean.valueOf(this.bloomFilterSpecified).hashCode();
result ^= Byte.valueOf(this.versionNumber).hashCode();
if(this.bloomFilterSpecified) {
result ^= this.bloomFilter.hashCode();
}
return result;
}
@ -226,7 +233,12 @@ public class HColumnDescriptor implements WritableComparable {
this.compressionType = in.readByte();
this.inMemory = in.readBoolean();
this.maxValueLength = in.readInt();
this.bloomFilterEnabled = in.readBoolean();
this.bloomFilterSpecified = in.readBoolean();
if(bloomFilterSpecified) {
bloomFilter = new BloomFilterDescriptor();
bloomFilter.readFields(in);
}
}
public void write(DataOutput out) throws IOException {
@ -236,7 +248,11 @@ public class HColumnDescriptor implements WritableComparable {
out.writeByte(this.compressionType);
out.writeBoolean(this.inMemory);
out.writeInt(this.maxValueLength);
out.writeBoolean(this.bloomFilterEnabled);
out.writeBoolean(this.bloomFilterSpecified);
if(bloomFilterSpecified) {
bloomFilter.write(out);
}
}
//////////////////////////////////////////////////////////////////////////////
@ -279,10 +295,10 @@ public class HColumnDescriptor implements WritableComparable {
}
if(result == 0) {
if(this.bloomFilterEnabled == other.bloomFilterEnabled) {
if(this.bloomFilterSpecified == other.bloomFilterSpecified) {
result = 0;
} else if(this.bloomFilterEnabled) {
} else if(this.bloomFilterSpecified) {
result = -1;
} else {
@ -290,6 +306,10 @@ public class HColumnDescriptor implements WritableComparable {
}
}
if(result == 0 && this.bloomFilterSpecified) {
result = this.bloomFilter.compareTo(other.bloomFilter);
}
return result;
}

View File

@ -27,12 +27,12 @@ public class HGlobals implements HConstants {
static {
rootTableDesc = new HTableDescriptor(ROOT_TABLE_NAME.toString());
rootTableDesc.addFamily(new HColumnDescriptor(COLUMN_FAMILY, 1,
HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE, false));
HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE, null));
rootRegionInfo = new HRegionInfo(0L, rootTableDesc, null, null);
metaTableDesc = new HTableDescriptor(META_TABLE_NAME.toString());
metaTableDesc.addFamily(new HColumnDescriptor(COLUMN_FAMILY, 1,
HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE, false));
HColumnDescriptor.CompressionType.NONE, false, Integer.MAX_VALUE, null));
}
}

View File

@ -28,11 +28,17 @@ 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;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.MapFile;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.onelab.filter.*;
/*******************************************************************************
* HStore maintains a bunch of data files. It is responsible for maintaining
@ -49,6 +55,8 @@ class HStore implements HConstants {
static final String WORKING_COMPACTION = "compaction.inprogress";
static final String COMPACTION_TO_REPLACE = "toreplace";
static final String COMPACTION_DONE = "done";
private static final String BLOOMFILTER_FILE_NAME = "filter";
Path dir;
Text regionName;
@ -60,6 +68,8 @@ class HStore implements HConstants {
Path mapdir;
Path compactdir;
Path loginfodir;
Path filterDir;
Filter bloomFilter;
Integer compactLock = 0;
Integer flushLock = 0;
@ -135,6 +145,16 @@ class HStore implements HConstants {
fs.mkdirs(mapdir);
this.loginfodir = HStoreFile.getInfoDir(dir, regionName, familyName);
fs.mkdirs(loginfodir);
if(family.bloomFilter == null) {
this.filterDir = null;
this.bloomFilter = null;
} else {
this.filterDir = HStoreFile.getFilterDir(dir, regionName, familyName);
fs.mkdirs(filterDir);
loadOrCreateBloomFilter();
}
if(LOG.isDebugEnabled()) {
LOG.debug("starting HStore for " + regionName + "/"+ familyName);
@ -182,6 +202,9 @@ class HStore implements HConstants {
}
}
// If a bloom filter is enabled, try to read it in.
// If it doesn't exist, create it.
// Read the reconstructionLog to see whether we need to build a brand-new
// MapFile out of non-flushed log entries.
//
@ -257,11 +280,163 @@ class HStore implements HConstants {
for(Map.Entry<Long, HStoreFile> e: mapFiles.entrySet()) {
// TODO - is this really necessary? Don't I do this inside compact()?
maps.put(e.getKey(),
new MapFile.Reader(fs, e.getValue().getMapFilePath().toString(), conf));
getMapFileReader(e.getValue().getMapFilePath().toString()));
}
LOG.info("HStore online for " + this.regionName + "/" + this.familyName);
}
//////////////////////////////////////////////////////////////////////////////
// Bloom filters
//////////////////////////////////////////////////////////////////////////////
/**
* Called by constructor if a bloom filter is enabled for this column family.
* If the HStore already exists, it will read in the bloom filter saved
* previously. Otherwise, it will create a new bloom filter.
*/
private void loadOrCreateBloomFilter() throws IOException {
Path filterFile = new Path(filterDir, BLOOMFILTER_FILE_NAME);
if(fs.exists(filterFile)) {
switch(family.bloomFilter.filterType) {
case BloomFilterDescriptor.BLOOMFILTER:
bloomFilter = new BloomFilter();
break;
case BloomFilterDescriptor.COUNTING_BLOOMFILTER:
bloomFilter = new CountingBloomFilter();
break;
case BloomFilterDescriptor.RETOUCHED_BLOOMFILTER:
bloomFilter = new RetouchedBloomFilter();
}
FSDataInputStream in = fs.open(filterFile);
bloomFilter.readFields(in);
fs.close();
} else {
switch(family.bloomFilter.filterType) {
case BloomFilterDescriptor.BLOOMFILTER:
bloomFilter = new BloomFilter(family.bloomFilter.vectorSize,
family.bloomFilter.nbHash);
break;
case BloomFilterDescriptor.COUNTING_BLOOMFILTER:
bloomFilter = new CountingBloomFilter(family.bloomFilter.vectorSize,
family.bloomFilter.nbHash);
break;
case BloomFilterDescriptor.RETOUCHED_BLOOMFILTER:
bloomFilter = new RetouchedBloomFilter(family.bloomFilter.vectorSize,
family.bloomFilter.nbHash);
}
}
}
/**
* Flushes bloom filter to disk
*
* @throws IOException
*/
private void flushBloomFilter() throws IOException {
FSDataOutputStream out =
fs.create(new Path(filterDir, BLOOMFILTER_FILE_NAME));
bloomFilter.write(out);
out.close();
}
/** 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());
}
/**
* Extends MapFile.Reader and overrides get and getClosest to consult the
* bloom filter before attempting to read from disk.
*/
private class BloomFilterReader extends MapFile.Reader {
BloomFilterReader(FileSystem fs, String dirName, Configuration conf)
throws IOException {
super(fs, dirName, conf);
}
@Override
public Writable get(WritableComparable key, Writable val) throws IOException {
// Note - the key being passed to us is always a HStoreKey
if(bloomFilter.membershipTest(getBloomFilterKey((HStoreKey)key))) {
return super.get(key, val);
}
return null;
}
@Override
public WritableComparable getClosest(WritableComparable key, Writable val)
throws IOException {
// Note - the key being passed to us is always a HStoreKey
if(bloomFilter.membershipTest(getBloomFilterKey((HStoreKey)key))) {
return super.getClosest(key, val);
}
return null;
}
}
/**
* Extends MapFile.Writer and overrides append, so that whenever a MapFile
* is written to, the key is added to the bloom filter.
*/
private class BloomFilterWriter extends MapFile.Writer {
@SuppressWarnings("unchecked")
BloomFilterWriter(Configuration conf, FileSystem fs, String dirName,
Class keyClass, Class valClass, SequenceFile.CompressionType compression)
throws IOException {
super(conf, fs, dirName, keyClass, valClass, compression);
}
@Override
public void append(WritableComparable key, Writable val) throws IOException {
// Note - the key being passed to us is always a HStoreKey
bloomFilter.add(getBloomFilterKey((HStoreKey)key));
super.append(key, val);
}
}
/**
* Get a MapFile reader
* This allows us to substitute a BloomFilterReader if a bloom filter is enabled
*/
MapFile.Reader getMapFileReader(String dirName) throws IOException {
if(bloomFilter != null) {
return new BloomFilterReader(fs, dirName, conf);
}
return new MapFile.Reader(fs, dirName, conf);
}
/**
* Get a MapFile writer
* This allows us to substitute a BloomFilterWriter if a bloom filter is enabled
*/
MapFile.Writer getMapFileWriter(String dirName) throws IOException {
if(bloomFilter != null) {
return new BloomFilterWriter(conf, fs, dirName, HStoreKey.class,
BytesWritable.class, compression);
}
return new MapFile.Writer(conf, fs, dirName, HStoreKey.class,
BytesWritable.class, compression);
}
//////////////////////////////////////////////////////////////////////////////
// End bloom filters
//////////////////////////////////////////////////////////////////////////////
/**
* Turn off all the MapFile readers
@ -327,8 +502,7 @@ class HStore implements HConstants {
LOG.debug("map file is: " + mapfile.toString());
}
MapFile.Writer out = new MapFile.Writer(conf, fs, mapfile.toString(),
HStoreKey.class, BytesWritable.class, compression);
MapFile.Writer out = getMapFileWriter(mapfile.toString());
try {
for (Map.Entry<HStoreKey, BytesWritable> es: inputCache.entrySet()) {
@ -352,14 +526,20 @@ class HStore implements HConstants {
LOG.debug("writing log cache flush id");
}
flushedFile.writeInfo(fs, logCacheFlushId);
// C. Flush the bloom filter if any
if(bloomFilter != null) {
flushBloomFilter();
}
// C. Finally, make the new MapFile available.
// D. Finally, make the new MapFile available.
if(addToAvailableMaps) {
this.lock.obtainWriteLock();
try {
maps.put(logCacheFlushId, new MapFile.Reader(fs, mapfile.toString(), conf));
maps.put(logCacheFlushId, getMapFileReader(mapfile.toString()));
mapFiles.put(logCacheFlushId, flushedFile);
if(LOG.isDebugEnabled()) {
LOG.debug("HStore available for " + this.regionName + "/"
@ -466,9 +646,8 @@ class HStore implements HConstants {
// Step through them, writing to the brand-new TreeMap
MapFile.Writer compactedOut = new MapFile.Writer(conf, fs,
compactedOutputFile.getMapFilePath().toString(), HStoreKey.class,
BytesWritable.class, compression);
MapFile.Writer compactedOut =
getMapFileWriter(compactedOutputFile.getMapFilePath().toString());
try {
@ -491,7 +670,7 @@ class HStore implements HConstants {
int pos = 0;
for(Iterator<HStoreFile> it = toCompactFiles.iterator(); it.hasNext(); ) {
HStoreFile hsf = it.next();
readers[pos] = new MapFile.Reader(fs, hsf.getMapFilePath().toString(), conf);
readers[pos] = getMapFileReader(hsf.getMapFilePath().toString());
keys[pos] = new HStoreKey();
vals[pos] = new BytesWritable();
done[pos] = false;
@ -772,8 +951,8 @@ class HStore implements HConstants {
}
mapFiles.put(orderVal, finalCompactedFile);
maps.put(orderVal, new MapFile.Reader(fs,
finalCompactedFile.getMapFilePath().toString(), conf));
maps.put(orderVal, getMapFileReader(
finalCompactedFile.getMapFilePath().toString()));
} finally {
@ -988,7 +1167,7 @@ class HStore implements HConstants {
int i = readers.length - 1;
for(Iterator<HStoreFile> it = mapFiles.values().iterator(); it.hasNext(); ) {
HStoreFile curHSF = it.next();
readers[i--] = new MapFile.Reader(fs, curHSF.getMapFilePath().toString(), conf);
readers[i--] = getMapFileReader(curHSF.getMapFilePath().toString());
}
this.keys = new HStoreKey[readers.length];

View File

@ -31,12 +31,13 @@ import java.util.*;
* This class handles all that path-building stuff for you.
******************************************************************************/
public class HStoreFile implements HConstants, WritableComparable {
public static final byte INFO_SEQ_NUM = 0;
public static final String HSTORE_DATFILE_PREFIX = "mapfile.dat.";
public static final String HSTORE_INFOFILE_PREFIX = "mapfile.info.";
public static final String HSTORE_DATFILE_DIR = "mapfiles";
public static final String HSTORE_INFO_DIR = "info";
static Random rand = new Random();
static final byte INFO_SEQ_NUM = 0;
static final String HSTORE_DATFILE_PREFIX = "mapfile.dat.";
static final String HSTORE_INFOFILE_PREFIX = "mapfile.info.";
static final String HSTORE_DATFILE_DIR = "mapfiles";
static final String HSTORE_INFO_DIR = "info";
static final String HSTORE_FILTER_DIR = "filter";
private static Random rand = new Random();
Path dir;
Text regionName;
@ -52,7 +53,7 @@ public class HStoreFile implements HConstants, WritableComparable {
* When merging or splitting HRegions, we might want to modify one of the
* params for an HStoreFile (effectively moving it elsewhere).
*/
public HStoreFile(Configuration conf) {
HStoreFile(Configuration conf) {
this.conf = conf;
this.dir = new Path(Path.CUR_DIR);
this.regionName = new Text();
@ -60,7 +61,7 @@ public class HStoreFile implements HConstants, WritableComparable {
this.fileId = 0;
}
public HStoreFile(Configuration conf, Path dir, Text regionName,
HStoreFile(Configuration conf, Path dir, Text regionName,
Text colFamily, long fileId) {
this.conf = conf;
@ -72,30 +73,30 @@ public class HStoreFile implements HConstants, WritableComparable {
// Get the individual components
public Path getDir() {
Path getDir() {
return dir;
}
public Text getRegionName() {
Text getRegionName() {
return regionName;
}
public Text getColFamily() {
Text getColFamily() {
return colFamily;
}
public long fileId() {
long fileId() {
return fileId;
}
// Build full filenames from those components
public Path getMapFilePath() {
Path getMapFilePath() {
return new Path(HStoreFile.getMapDir(dir, regionName, colFamily),
HSTORE_DATFILE_PREFIX + fileId);
}
public Path getInfoFilePath() {
Path getInfoFilePath() {
return new Path(HStoreFile.getInfoDir(dir, regionName, colFamily),
HSTORE_INFOFILE_PREFIX + fileId);
}
@ -103,22 +104,27 @@ public class HStoreFile implements HConstants, WritableComparable {
// Static methods to build partial paths to internal directories. Useful for
// HStore construction and log-rebuilding.
public static Path getMapDir(Path dir, Text regionName, Text colFamily) {
static Path getMapDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
}
public static Path getInfoDir(Path dir, Text regionName, Text colFamily) {
static Path getInfoDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
new Path(colFamily.toString(), HSTORE_INFO_DIR)));
}
static Path getFilterDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
new Path(colFamily.toString(), HSTORE_FILTER_DIR)));
}
public static Path getHStoreDir(Path dir, Text regionName, Text colFamily) {
static Path getHStoreDir(Path dir, Text regionName, Text colFamily) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
colFamily.toString()));
}
public static Path getHRegionDir(Path dir, Text regionName) {
static Path getHRegionDir(Path dir, Text regionName) {
return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName));
}
@ -199,7 +205,7 @@ public class HStoreFile implements HConstants, WritableComparable {
* Break this HStoreFile file into two new parts, which live in different
* brand-new HRegions.
*/
public void splitStoreFile(Text midKey, HStoreFile dstA, HStoreFile dstB,
void splitStoreFile(Text midKey, HStoreFile dstA, HStoreFile dstB,
FileSystem fs, Configuration conf) throws IOException {
// Copy the appropriate tuples to one MapFile or the other.
@ -251,7 +257,7 @@ public class HStoreFile implements HConstants, WritableComparable {
* Write to this HStoreFile with all the contents of the given source HStoreFiles.
* We are merging multiple regions into a single new one.
*/
public void mergeStoreFiles(Vector<HStoreFile> srcFiles, FileSystem fs,
void mergeStoreFiles(Vector<HStoreFile> srcFiles, FileSystem fs,
Configuration conf) throws IOException {
// Copy all the source MapFile tuples into this HSF's MapFile
@ -295,7 +301,7 @@ public class HStoreFile implements HConstants, WritableComparable {
}
/** Read in an info file, give it a unique ID. */
public long loadInfo(FileSystem fs) throws IOException {
long loadInfo(FileSystem fs) throws IOException {
Path p = getInfoFilePath();
DataInputStream in = new DataInputStream(fs.open(p));
@ -304,9 +310,8 @@ public class HStoreFile implements HConstants, WritableComparable {
if(flag == INFO_SEQ_NUM) {
return in.readLong();
} else {
throw new IOException("Cannot process log file: " + p);
}
throw new IOException("Cannot process log file: " + p);
} finally {
in.close();
@ -314,7 +319,7 @@ public class HStoreFile implements HConstants, WritableComparable {
}
/** Write the file-identifier to disk */
public void writeInfo(FileSystem fs, long infonum) throws IOException {
void writeInfo(FileSystem fs, long infonum) throws IOException {
Path p = getInfoFilePath();
DataOutputStream out = new DataOutputStream(fs.create(p));
@ -326,7 +331,8 @@ public class HStoreFile implements HConstants, WritableComparable {
out.close();
}
}
@Override
public boolean equals(Object o) {
return this.compareTo(o) == 0;
}
@ -344,6 +350,9 @@ public class HStoreFile implements HConstants, WritableComparable {
// Writable
//////////////////////////////////////////////////////////////////////////////
/* (non-Javadoc)
* @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
*/
public void write(DataOutput out) throws IOException {
out.writeUTF(dir.toString());
regionName.write(out);
@ -351,6 +360,9 @@ public class HStoreFile implements HConstants, WritableComparable {
out.writeLong(fileId);
}
/* (non-Javadoc)
* @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
*/
public void readFields(DataInput in) throws IOException {
this.dir = new Path(in.readUTF());
this.regionName.readFields(in);
@ -362,6 +374,9 @@ public class HStoreFile implements HConstants, WritableComparable {
// Comparable
//////////////////////////////////////////////////////////////////////////////
/* (non-Javadoc)
* @see java.lang.Comparable#compareTo(java.lang.Object)
*/
public int compareTo(Object o) {
HStoreFile other = (HStoreFile) o;
int result = this.dir.compareTo(other.dir);

View File

@ -18,21 +18,22 @@ public class TestToString extends TestCase {
HTableDescriptor htd = new HTableDescriptor("hank");
htd.addFamily(new HColumnDescriptor("hankfamily:"));
htd.addFamily(new HColumnDescriptor(new Text("hankotherfamily:"), 10,
HColumnDescriptor.CompressionType.BLOCK, true, 1000, false));
HColumnDescriptor.CompressionType.BLOCK, true, 1000, null));
assertEquals("Table descriptor", "name: hank, families: "
+ "{hankfamily:=(hankfamily:, max versions: 3, compression: none, "
+ "in memory: false, max value length: 2147483647, bloom filter:false), "
+ "in memory: false, max value length: 2147483647, bloom filter: none), "
+ "hankotherfamily:=(hankotherfamily:, max versions: 10, "
+ "compression: block, in memory: true, max value length: 1000, "
+ "bloom filter:false)}", htd.toString());
+ "bloom filter: none)}", htd.toString());
HRegionInfo hri = new HRegionInfo(-1, htd, new Text(), new Text("10"));
System.out.println(hri.toString());
assertEquals("HRegionInfo",
"regionname: hank__-1, startKey: <>, tableDesc: {" + "name: hank, "
+ "families: {hankfamily:=(hankfamily:, max versions: 3, "
+ "compression: none, in memory: false, max value length: 2147483647, "
+ "bloom filter:false), hankotherfamily:=(hankotherfamily:, "
+ "bloom filter: none), hankotherfamily:=(hankotherfamily:, "
+ "max versions: 10, compression: block, in memory: true, max value "
+ "length: 1000, bloom filter:false)}}",
+ "length: 1000, bloom filter: none)}}",
hri.toString());
}
}