diff --git a/CHANGES.txt b/CHANGES.txt index 8a2fc992e27..39b2d53f1d5 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -186,6 +186,13 @@ Trunk (unreleased changes) HBASE-744 BloomFilter serialization/deserialization broken HBASE-742 Column length limit is not enforced (Jean-Daniel Cryans via Stack) HBASE-737 Scanner: every cell in a row has the same timestamp + HBASE-700 hbase.io.index.interval need be configuratable in column family + (Andrew Purtell via Stack) + HBASE-62 Allow user add arbitrary key/value pairs to table and column + descriptors (Andrew Purtell via Stack) + HBASE-34 Set memcache flush size per column (Andrew Purtell via Stack) + HBASE-42 Set region split size on table creation (Andrew Purtell via Stack) + HBASE-43 Add a read-only attribute to columns (Andrew Purtell via Stack) IMPROVEMENTS HBASE-559 MR example job to count table rows diff --git a/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java b/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java index 53848bf314e..0fcd19f3700 100644 --- a/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java @@ -73,7 +73,19 @@ public class BloomFilterDescriptor implements WritableComparable { public BloomFilterDescriptor() { super(); } - + + /* + * Constructor. + *

+ * Creates a deep copy of the supplied BloomFilterDescriptor. + */ + public BloomFilterDescriptor(BloomFilterDescriptor desc) { + super(); + this.filterType = desc.filterType; + this.nbHash = desc.nbHash; + this.vectorSize = desc.vectorSize; + } + /** * Creates a BloomFilterDescriptor for the specified type of filter, fixes * the number of hash functions to 4 and computes a vector size using: diff --git a/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java index 9ccde381bf8..cb8754f7aa4 100644 --- a/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java @@ -22,7 +22,10 @@ package org.apache.hadoop.hbase; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.WritableComparable; @@ -39,10 +42,11 @@ import org.apache.hadoop.io.WritableComparable; public class HColumnDescriptor implements WritableComparable { // For future backward compatibility - // Version 3 was when column names becaome byte arrays and when we picked up + // Version 3 was when column names become byte arrays and when we picked up // Time-to-live feature. Version 4 was when we moved to byte arrays, HBASE-82. // Version 5 was when bloom filter descriptors were removed. - private static final byte COLUMN_DESCRIPTOR_VERSION = (byte)5; + // Version 6 adds metadata as a map where keys and values are byte[]. + private static final byte COLUMN_DESCRIPTOR_VERSION = (byte)6; /** * The type of compression. @@ -57,14 +61,17 @@ public class HColumnDescriptor implements WritableComparable { BLOCK } - // Defines for jruby/shell public static final String COMPRESSION = "COMPRESSION"; public static final String IN_MEMORY = "IN_MEMORY"; public static final String BLOCKCACHE = "BLOCKCACHE"; public static final String LENGTH = "LENGTH"; public static final String TTL = "TTL"; + public static final String VERSIONS = "VERSIONS"; public static final String BLOOMFILTER = "BLOOMFILTER"; public static final String FOREVER = "FOREVER"; + public static final String MAPFILE_INDEX_INTERVAL = + "MAPFILE_INDEX_INTERVAL"; + public static final String MEMCACHE_FLUSHSIZE = "MEMCACHE_FLUSHSIZE"; /** * Default compression type. @@ -104,20 +111,16 @@ public class HColumnDescriptor implements WritableComparable { // Column family name private byte [] name; - // Number of versions to keep - private int maxVersions = DEFAULT_VERSIONS; - // Compression setting if any - private CompressionType compressionType = DEFAULT_COMPRESSION; - // Serve reads from in-memory cache - private boolean inMemory = DEFAULT_IN_MEMORY; - // Serve reads from in-memory block cache - private boolean blockCacheEnabled = DEFAULT_BLOCKCACHE; - // Maximum value size - private int maxValueLength = DEFAULT_LENGTH; - // Time to live of cell contents, in seconds from last timestamp - private int timeToLive = DEFAULT_TTL; - // True if bloom filter was specified - private boolean bloomFilter = false; + + /** + * Default mapfile index interval. + */ + public static final int DEFAULT_MAPFILE_INDEX_INTERVAL = 128; + + // Column metadata + protected Map values = + new HashMap(); + /** * Default constructor. Must be present for Writable. @@ -159,6 +162,21 @@ public class HColumnDescriptor implements WritableComparable { Integer.MAX_VALUE, DEFAULT_TTL, false); } + /** + * Constructor. + * Makes a deep copy of the supplied descriptor. + * Can make a modifiable descriptor from an UnmodifyableHColumnDescriptor. + * @param desc The descriptor. + */ + public HColumnDescriptor(HColumnDescriptor desc) { + super(); + this.name = desc.name.clone(); + for (Map.Entry e: + desc.values.entrySet()) { + this.values.put(e.getKey(), e.getValue()); + } + } + /** * Constructor * @param columnName Column family name. Must have the ':' ending. @@ -188,13 +206,13 @@ public class HColumnDescriptor implements WritableComparable { // Until there is support, consider 0 or < 0 -- a configuration error. throw new IllegalArgumentException("Maximum versions must be positive"); } - this.maxVersions = maxVersions; - this.inMemory = inMemory; - this.blockCacheEnabled = blockCacheEnabled; - this.maxValueLength = maxValueLength; - this.timeToLive = timeToLive; - this.bloomFilter = bloomFilter; - this.compressionType = compression; + setMaxVersions(maxVersions); + setInMemory(inMemory); + setBlockCacheEnabled(blockCacheEnabled); + setMaxValueLength(maxValueLength); + setTimeToLive(timeToLive); + setCompressionType(compression); + setBloomfilter(bloomFilter); } private static byte [] stripColon(final byte [] n) { @@ -203,7 +221,7 @@ public class HColumnDescriptor implements WritableComparable { System.arraycopy(n, 0, result, 0, n.length - 1); return result; } - + /** * @param b Family name. * @return b @@ -237,6 +255,13 @@ public class HColumnDescriptor implements WritableComparable { return name; } + /** + * @return Name of this column family with colon as required by client API + */ + public byte [] getNameWithColon() { + return HStoreKey.addDelimiter(this.name); + } + /** * @return Name of this column family */ @@ -244,71 +269,216 @@ public class HColumnDescriptor implements WritableComparable { return Bytes.toString(this.name); } + /** + * @param key The key. + * @return The value. + */ + public byte[] getValue(byte[] key) { + ImmutableBytesWritable ibw = values.get(new ImmutableBytesWritable(key)); + if (ibw == null) + return null; + return ibw.get(); + } + + /** + * @param key The key. + * @return The value as a string. + */ + public String getValue(String key) { + byte[] value = getValue(Bytes.toBytes(key)); + if (value == null) + return null; + return Bytes.toString(value); + } + + /** + * @param key The key. + * @param value The value. + */ + public void setValue(byte[] key, byte[] value) { + values.put(new ImmutableBytesWritable(key), + new ImmutableBytesWritable(value)); + } + + /** + * @param key The key. + * @param value The value. + */ + public void setValue(String key, String value) { + setValue(Bytes.toBytes(key), Bytes.toBytes(value)); + } + /** @return compression type being used for the column family */ public CompressionType getCompression() { - return this.compressionType; + String value = getValue(COMPRESSION); + if (value != null) { + if (value.equalsIgnoreCase("BLOCK")) + return CompressionType.BLOCK; + else if (value.equalsIgnoreCase("RECORD")) + return CompressionType.RECORD; + } + return CompressionType.NONE; } /** @return maximum number of versions */ public int getMaxVersions() { - return this.maxVersions; + String value = getValue(VERSIONS); + if (value != null) + return Integer.valueOf(value); + return DEFAULT_VERSIONS; + } + + /** + * @param maxVersions maximum number of versions + */ + public void setMaxVersions(int maxVersions) { + setValue(VERSIONS, Integer.toString(maxVersions)); } /** * @return Compression type setting. */ public CompressionType getCompressionType() { - return this.compressionType; + return getCompression(); + } + + /** + * @param type Compression type setting. + */ + public void setCompressionType(CompressionType type) { + String compressionType; + switch (type) { + case BLOCK: compressionType = "BLOCK"; break; + case RECORD: compressionType = "RECORD"; break; + default: compressionType = "NONE"; break; + } + setValue(COMPRESSION, compressionType); } /** * @return True if we are to keep all in use HRegionServer cache. */ public boolean isInMemory() { - return this.inMemory; + String value = getValue(IN_MEMORY); + if (value != null) + return Boolean.valueOf(value); + return DEFAULT_IN_MEMORY; } + /** + * @param inMemory True if we are to keep all values in the HRegionServer + * cache + */ + public void setInMemory(boolean inMemory) { + setValue(IN_MEMORY, Boolean.toString(inMemory)); + } + /** * @return Maximum value length. */ public int getMaxValueLength() { - return this.maxValueLength; + String value = getValue(LENGTH); + if (value != null) + return Integer.valueOf(value); + return DEFAULT_LENGTH; + } + + /** + * @param maxLength Maximum value length. + */ + public void setMaxValueLength(int maxLength) { + setValue(LENGTH, Integer.toString(maxLength)); } /** * @return Time to live. */ public int getTimeToLive() { - return this.timeToLive; + String value = getValue(TTL); + if (value != null) + return Integer.valueOf(value); + return DEFAULT_TTL; + } + + /** + * @param timeToLive + */ + public void setTimeToLive(int timeToLive) { + setValue(TTL, Integer.toString(timeToLive)); } /** * @return True if MapFile blocks should be cached. */ public boolean isBlockCacheEnabled() { - return blockCacheEnabled; + String value = getValue(BLOCKCACHE); + if (value != null) + return Boolean.valueOf(value); + return DEFAULT_BLOCKCACHE; + } + + /** + * @param blockCacheEnabled True if MapFile blocks should be cached. + */ + public void setBlockCacheEnabled(boolean blockCacheEnabled) { + setValue(BLOCKCACHE, Boolean.toString(blockCacheEnabled)); } /** * @return true if a bloom filter is enabled */ - public boolean isBloomFilterEnabled() { - return this.bloomFilter; + public boolean isBloomfilter() { + String value = getValue(BLOOMFILTER); + if (value != null) + return Boolean.valueOf(value); + return DEFAULT_BLOOMFILTER; + } + + /** + * @param onOff Enable/Disable bloom filter + */ + public void setBloomfilter(final boolean onOff) { + setValue(BLOOMFILTER, Boolean.toString(onOff)); + } + + /** + * @return The number of entries that are added to the store MapFile before + * an index entry is added. + */ + public int getMapFileIndexInterval() { + String value = getValue(MAPFILE_INDEX_INTERVAL); + if (value != null) + return Integer.valueOf(value); + return DEFAULT_MAPFILE_INDEX_INTERVAL; + } + + /** + * @param interval The number of entries that are added to the store MapFile before + * an index entry is added. + */ + public void setMapFileIndexInterval(int interval) { + setValue(MAPFILE_INDEX_INTERVAL, Integer.toString(interval)); } /** {@inheritDoc} */ @Override public String toString() { - return "{" + HConstants.NAME + " => '" + Bytes.toString(name) + - "', " + HConstants.VERSIONS + " => " + maxVersions + - ", " + COMPRESSION + " => '" + this.compressionType + - "', " + IN_MEMORY + " => " + inMemory + - ", " + BLOCKCACHE + " => " + blockCacheEnabled + - ", " + LENGTH + " => " + maxValueLength + - ", " + TTL + " => " + - (timeToLive == HConstants.FOREVER ? "FOREVER" : - Integer.toString(timeToLive)) + - ", " + BLOOMFILTER + " => " + bloomFilter + "}"; + StringBuffer s = new StringBuffer(); + s.append('{'); + s.append(HConstants.NAME); + s.append(" => '"); + s.append(Bytes.toString(name)); + s.append("'"); + for (Map.Entry e: + values.entrySet()) { + s.append(", "); + s.append(Bytes.toString(e.getKey().get())); + s.append(" => '"); + s.append(Bytes.toString(e.getValue().get())); + s.append("'"); + } + s.append('}'); + return s.toString(); } /** {@inheritDoc} */ @@ -321,14 +491,8 @@ public class HColumnDescriptor implements WritableComparable { @Override public int hashCode() { int result = Bytes.hashCode(this.name); - result ^= Integer.valueOf(this.maxVersions).hashCode(); - result ^= this.compressionType.hashCode(); - result ^= Boolean.valueOf(this.inMemory).hashCode(); - result ^= Boolean.valueOf(this.blockCacheEnabled).hashCode(); - result ^= Integer.valueOf(this.maxValueLength).hashCode(); - result ^= Integer.valueOf(this.timeToLive).hashCode(); - result ^= Boolean.valueOf(this.bloomFilter).hashCode(); result ^= Byte.valueOf(COLUMN_DESCRIPTOR_VERSION).hashCode(); + result ^= values.hashCode(); return result; } @@ -336,37 +500,51 @@ public class HColumnDescriptor implements WritableComparable { /** {@inheritDoc} */ public void readFields(DataInput in) throws IOException { - int versionNumber = in.readByte(); - if (versionNumber <= 2) { - Text t = new Text(); - t.readFields(in); - this.name = t.getBytes(); - if (HStoreKey.getFamilyDelimiterIndex(this.name) > 0) { - this.name = stripColon(this.name); + int version = in.readByte(); + if (version < 6) { + if (version <= 2) { + Text t = new Text(); + t.readFields(in); + this.name = t.getBytes(); + if (HStoreKey.getFamilyDelimiterIndex(this.name) > 0) { + this.name = stripColon(this.name); + } + } else { + this.name = Bytes.readByteArray(in); + } + this.values.clear(); + setMaxVersions(in.readInt()); + int ordinal = in.readInt(); + setCompressionType(CompressionType.values()[ordinal]); + setInMemory(in.readBoolean()); + setMaxValueLength(in.readInt()); + setBloomfilter(in.readBoolean()); + if (isBloomfilter() && version < 5) { + // If a bloomFilter is enabled and the column descriptor is less than + // version 5, we need to skip over it to read the rest of the column + // descriptor. There are no BloomFilterDescriptors written to disk for + // column descriptors with a version number >= 5 + BloomFilterDescriptor junk = new BloomFilterDescriptor(); + junk.readFields(in); + } + if (version > 1) { + setBlockCacheEnabled(in.readBoolean()); + } + if (version > 2) { + setTimeToLive(in.readInt()); } } else { + // version 6+ this.name = Bytes.readByteArray(in); - } - this.maxVersions = in.readInt(); - int ordinal = in.readInt(); - this.compressionType = CompressionType.values()[ordinal]; - this.inMemory = in.readBoolean(); - this.maxValueLength = in.readInt(); - this.bloomFilter = in.readBoolean(); - if (this.bloomFilter && versionNumber < 5) { - // If a bloomFilter is enabled and the column descriptor is less than - // version 5, we need to skip over it to read the rest of the column - // descriptor. There are no BloomFilterDescriptors written to disk for - // column descriptors with a version number >= 5 - BloomFilterDescriptor junk = new BloomFilterDescriptor(); - junk.readFields(in); - } - if (versionNumber > 1) { - this.blockCacheEnabled = in.readBoolean(); - } - - if (versionNumber > 2) { - this.timeToLive = in.readInt(); + this.values.clear(); + int numValues = in.readInt(); + for (int i = 0; i < numValues; i++) { + ImmutableBytesWritable key = new ImmutableBytesWritable(); + ImmutableBytesWritable value = new ImmutableBytesWritable(); + key.readFields(in); + value.readFields(in); + values.put(key, value); + } } } @@ -374,13 +552,12 @@ public class HColumnDescriptor implements WritableComparable { public void write(DataOutput out) throws IOException { out.writeByte(COLUMN_DESCRIPTOR_VERSION); Bytes.writeByteArray(out, this.name); - out.writeInt(this.maxVersions); - out.writeInt(this.compressionType.ordinal()); - out.writeBoolean(this.inMemory); - out.writeInt(this.maxValueLength); - out.writeBoolean(this.bloomFilter); - out.writeBoolean(this.blockCacheEnabled); - out.writeInt(this.timeToLive); + out.writeInt(values.size()); + for (Map.Entry e: + values.entrySet()) { + e.getKey().write(out); + e.getValue().write(out); + } } // Comparable @@ -389,57 +566,13 @@ public class HColumnDescriptor implements WritableComparable { public int compareTo(Object o) { HColumnDescriptor other = (HColumnDescriptor)o; int result = Bytes.compareTo(this.name, other.getName()); - if(result == 0) { - result = Integer.valueOf(this.maxVersions).compareTo( - Integer.valueOf(other.maxVersions)); - } - - if(result == 0) { - result = this.compressionType.compareTo(other.compressionType); - } - - if(result == 0) { - if(this.inMemory == other.inMemory) { - result = 0; - - } else if(this.inMemory) { + if (result == 0) { + // punt on comparison for ordering, just calculate difference + result = this.values.hashCode() - other.values.hashCode(); + if (result < 0) result = -1; - - } else { + else if (result > 0) result = 1; - } - } - - if(result == 0) { - if(this.blockCacheEnabled == other.blockCacheEnabled) { - result = 0; - - } else if(this.blockCacheEnabled) { - result = -1; - - } else { - result = 1; - } - } - - if(result == 0) { - result = other.maxValueLength - this.maxValueLength; - } - - if(result == 0) { - result = other.timeToLive - this.timeToLive; - } - - if(result == 0) { - if(this.bloomFilter == other.bloomFilter) { - result = 0; - - } else if(this.bloomFilter) { - result = -1; - - } else { - result = 1; - } } return result; } diff --git a/src/java/org/apache/hadoop/hbase/HRegionInfo.java b/src/java/org/apache/hadoop/hbase/HRegionInfo.java index b730cee484a..285aff07257 100644 --- a/src/java/org/apache/hadoop/hbase/HRegionInfo.java +++ b/src/java/org/apache/hadoop/hbase/HRegionInfo.java @@ -27,6 +27,7 @@ import java.util.List; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JenkinsHash; +import org.apache.hadoop.io.VersionedWritable; import org.apache.hadoop.io.WritableComparable; /** @@ -34,7 +35,9 @@ import org.apache.hadoop.io.WritableComparable; * Contains HRegion id, start and end keys, a reference to this * HRegions' table descriptor, etc. */ -public class HRegionInfo implements WritableComparable { +public class HRegionInfo extends VersionedWritable implements WritableComparable { + private final byte VERSION = 0; + /** * @param regionName * @return the encodedName @@ -137,6 +140,7 @@ public class HRegionInfo implements WritableComparable { * first meta regions */ private HRegionInfo(long regionId, HTableDescriptor tableDesc) { + super(); this.regionId = regionId; this.tableDesc = tableDesc; this.regionName = createRegionName(tableDesc.getName(), null, regionId); @@ -146,6 +150,7 @@ public class HRegionInfo implements WritableComparable { /** Default constructor - creates empty object */ public HRegionInfo() { + super(); this.tableDesc = new HTableDescriptor(); } @@ -193,6 +198,7 @@ public class HRegionInfo implements WritableComparable { public HRegionInfo(HTableDescriptor tableDesc, final byte [] startKey, final byte [] endKey, final boolean split, final long regionid) throws IllegalArgumentException { + super(); if (tableDesc == null) { throw new IllegalArgumentException("tableDesc cannot be null"); } @@ -214,6 +220,7 @@ public class HRegionInfo implements WritableComparable { * @param other */ public HRegionInfo(HRegionInfo other) { + super(); this.endKey = other.getEndKey(); this.offLine = other.isOffline(); this.regionId = other.getRegionId(); @@ -307,7 +314,14 @@ public class HRegionInfo implements WritableComparable { public HTableDescriptor getTableDesc(){ return tableDesc; } - + + /** + * @param newDesc new table descriptor to use + */ + public void setTableDesc(HTableDescriptor newDesc) { + this.tableDesc = newDesc; + } + /** @return true if this is the root region */ public boolean isRootRegion() { return this.tableDesc.isRootRegion(); @@ -381,6 +395,10 @@ public class HRegionInfo implements WritableComparable { public int hashCode() { return this.hashCode; } + + public byte getVersion() { + return VERSION; + } // // Writable @@ -390,6 +408,7 @@ public class HRegionInfo implements WritableComparable { * {@inheritDoc} */ public void write(DataOutput out) throws IOException { + super.write(out); Bytes.writeByteArray(out, endKey); out.writeBoolean(offLine); out.writeLong(regionId); @@ -404,6 +423,7 @@ public class HRegionInfo implements WritableComparable { * {@inheritDoc} */ public void readFields(DataInput in) throws IOException { + super.readFields(in); this.endKey = Bytes.readByteArray(in); this.offLine = in.readBoolean(); this.regionId = in.readLong(); diff --git a/src/java/org/apache/hadoop/hbase/HTableDescriptor.java b/src/java/org/apache/hadoop/hbase/HTableDescriptor.java index 06dd9b98daf..df28183657a 100644 --- a/src/java/org/apache/hadoop/hbase/HTableDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/HTableDescriptor.java @@ -29,6 +29,7 @@ import java.util.Iterator; import java.util.Map; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.WritableComparable; @@ -53,13 +54,32 @@ public class HTableDescriptor implements WritableComparable { new HColumnDescriptor(HConstants.COLUMN_FAMILY_HISTORIAN, HConstants.ALL_VERSIONS, HColumnDescriptor.CompressionType.NONE, false, false, Integer.MAX_VALUE, HConstants.FOREVER, false) }); - - private boolean rootregion = false; - private boolean metaregion = false; + + // Changes prior to version 3 were not recorded here. + // Version 3 adds metadata as a map where keys and values are byte[]. + public static final byte TABLE_DESCRIPTOR_VERSION = 3; + private byte [] name = HConstants.EMPTY_BYTE_ARRAY; private String nameAsString = ""; + + // Table metadata + protected Map values = + new HashMap(); public static final String FAMILIES = "FAMILIES"; + + public static final String MAX_FILESIZE = "MAX_FILESIZE"; + public static final String IN_MEMORY = "IN_MEMORY"; + public static final String READONLY = "READONLY"; + public static final String MEMCACHE_FLUSHSIZE = "MEMCACHE_FLUSHSIZE"; + public static final String IS_ROOT = "IS_ROOT"; + public static final String IS_META = "IS_META"; + + public static final boolean DEFAULT_IN_MEMORY = false; + + public static final boolean DEFAULT_READONLY = false; + + public static final int DEFAULT_MEMCACHE_FLUSH_SIZE = 1024*1024*64; // Key is hash of the family name. private final Map families = @@ -107,9 +127,32 @@ public class HTableDescriptor implements WritableComparable { * @see HADOOP-1581 HBASE: Un-openable tablename bug */ public HTableDescriptor(final byte [] name) { - setMetaFlags(name); - this.name = this.metaregion? name: isLegalTableName(name); + super(); + this.name = this.isMetaRegion() ? name: isLegalTableName(name); this.nameAsString = Bytes.toString(this.name); + setMetaFlags(this.name); + } + + /** + * Constructor. + *

+ * Makes a deep copy of the supplied descriptor. + * Can make a modifiable descriptor from an UnmodifyableHTableDescriptor. + * @param desc The descriptor. + */ + public HTableDescriptor(final HTableDescriptor desc) + { + super(); + this.name = desc.name.clone(); + this.nameAsString = Bytes.toString(this.name); + setMetaFlags(this.name); + for (HColumnDescriptor c: desc.families.values()) { + this.families.put(Bytes.mapKey(c.getName()), new HColumnDescriptor(c)); + } + for (Map.Entry e: + desc.values.entrySet()) { + this.values.put(e.getKey(), e.getValue()); + } } /* @@ -118,11 +161,46 @@ public class HTableDescriptor implements WritableComparable { * @param name */ private void setMetaFlags(final byte [] name) { - this.rootregion = Bytes.equals(name, HConstants.ROOT_TABLE_NAME); - this.metaregion = - this.rootregion? true: Bytes.equals(name, HConstants.META_TABLE_NAME); + setRootRegion(Bytes.equals(name, HConstants.ROOT_TABLE_NAME)); + setMetaRegion(isRootRegion() || + Bytes.equals(name, HConstants.META_TABLE_NAME)); } - + + /** @return true if this is the root region */ + public boolean isRootRegion() { + String value = getValue(IS_ROOT); + if (value != null) + return Boolean.valueOf(value); + return false; + } + + /** @param isRoot true if this is the root region */ + protected void setRootRegion(boolean isRoot) { + values.put(new ImmutableBytesWritable(Bytes.toBytes(IS_ROOT)), + new ImmutableBytesWritable(Bytes.toBytes(Boolean.toString(isRoot)))); + } + + /** @return true if this is a meta region (part of the root or meta tables) */ + public boolean isMetaRegion() { + String value = getValue(IS_META); + if (value != null) + return Boolean.valueOf(value); + return false; + } + + /** + * @param isMeta true if this is a meta region (part of the root or meta + * tables) */ + protected void setMetaRegion(boolean isMeta) { + values.put(new ImmutableBytesWritable(Bytes.toBytes(IS_META)), + new ImmutableBytesWritable(Bytes.toBytes(Boolean.toString(isMeta)))); + } + + /** @return true if table is the meta table */ + public boolean isMetaTable() { + return isMetaRegion() && !isRootRegion(); + } + /** * Check passed buffer is legal user-space table name. * @param b Table name. @@ -147,19 +225,80 @@ public class HTableDescriptor implements WritableComparable { return b; } - /** @return true if this is the root region */ - public boolean isRootRegion() { - return rootregion; + /** + * @param key The key. + * @return The value. + */ + public byte[] getValue(byte[] key) { + ImmutableBytesWritable ibw = values.get(new ImmutableBytesWritable(key)); + if (ibw == null) + return null; + return ibw.get(); } - - /** @return true if table is the meta table */ - public boolean isMetaTable() { - return metaregion && !rootregion; + + /** + * @param key The key. + * @return The value as a string. + */ + public String getValue(String key) { + byte[] value = getValue(Bytes.toBytes(key)); + if (value == null) + return null; + return Bytes.toString(value); } - - /** @return true if this is a meta region (part of the root or meta tables) */ - public boolean isMetaRegion() { - return metaregion; + + /** + * @param key The key. + * @param value The value. + */ + public void setValue(byte[] key, byte[] value) { + values.put(new ImmutableBytesWritable(key), + new ImmutableBytesWritable(value)); + } + + /** + * @param key The key. + * @param value The value. + */ + public void setValue(String key, String value) { + setValue(Bytes.toBytes(key), Bytes.toBytes(value)); + } + + /** + * @return true if all columns in the table should be kept in the + * HRegionServer cache only + */ + public boolean isInMemory() { + String value = getValue(IN_MEMORY); + if (value != null) + return Boolean.valueOf(value); + return DEFAULT_IN_MEMORY; + } + + /** + * @param inMemory True if all of the columns in the table should be kept in + * the HRegionServer cache only. + */ + public void setInMemory(boolean inMemory) { + setValue(IN_MEMORY, Boolean.toString(inMemory)); + } + + /** + * @return true if all columns in the table should be read only + */ + public boolean isReadOnly() { + String value = getValue(READONLY); + if (value != null) + return Boolean.valueOf(value); + return DEFAULT_READONLY; + } + + /** + * @param readOnly True if all of the columns in the table should be read + * only. + */ + public void setReadOnly(boolean readOnly) { + setValue(READONLY, Boolean.toString(readOnly)); } /** @return name of table */ @@ -172,6 +311,39 @@ public class HTableDescriptor implements WritableComparable { return this.nameAsString; } + /** @return max hregion size for table */ + public long getMaxFileSize() { + String value = getValue(MAX_FILESIZE); + if (value != null) + return Long.valueOf(value); + return HConstants.DEFAULT_MAX_FILE_SIZE; + } + + /** + * @param maxFileSize The maximum file size that a store file can grow to + * before a split is triggered. + */ + public void setMaxFileSize(long maxFileSize) { + setValue(MAX_FILESIZE, Long.toString(maxFileSize)); + } + + /** + * @return memory cache flush size for each hregion + */ + public int getMemcacheFlushSize() { + String value = getValue(MEMCACHE_FLUSHSIZE); + if (value != null) + return Integer.valueOf(value); + return DEFAULT_MEMCACHE_FLUSH_SIZE; + } + + /** + * @param memcacheFlushSize memory cache flush size for each hregion + */ + public void setMemcacheFlushSize(int memcacheFlushSize) { + setValue(MEMCACHE_FLUSHSIZE, Integer.toString(memcacheFlushSize)); + } + /** * Adds a column family. * @param family HColumnDescriptor of familyto add. @@ -211,10 +383,28 @@ public class HTableDescriptor implements WritableComparable { */ @Override public String toString() { - return HConstants.NAME + " => '" + Bytes.toString(this.name) + - "', " + FAMILIES + " => " + this.families.values(); + StringBuffer s = new StringBuffer(); + s.append('{'); + s.append(HConstants.NAME); + s.append(" => '"); + s.append(Bytes.toString(name)); + s.append("'"); + for (Map.Entry e: + values.entrySet()) { + s.append(", "); + s.append(Bytes.toString(e.getKey().get())); + s.append(" => '"); + s.append(Bytes.toString(e.getValue().get())); + s.append("'"); + } + s.append(", "); + s.append(FAMILIES); + s.append(" => "); + s.append(families.values()); + s.append('}'); + return s.toString(); } - + /** {@inheritDoc} */ @Override public boolean equals(Object obj) { @@ -224,42 +414,64 @@ public class HTableDescriptor implements WritableComparable { /** {@inheritDoc} */ @Override public int hashCode() { - // TODO: Cache. int result = Bytes.hashCode(this.name); + result ^= Byte.valueOf(TABLE_DESCRIPTOR_VERSION).hashCode(); if (this.families != null && this.families.size() > 0) { for (HColumnDescriptor e: this.families.values()) { result ^= e.hashCode(); } } + result ^= values.hashCode(); return result; } - + // Writable /** {@inheritDoc} */ - public void write(DataOutput out) throws IOException { - out.writeBoolean(rootregion); - out.writeBoolean(metaregion); - Bytes.writeByteArray(out, name); - out.writeInt(families.size()); - for(Iterator it = families.values().iterator(); - it.hasNext(); ) { - it.next().write(out); + public void readFields(DataInput in) throws IOException { + int version = in.readInt(); + if (version < 3) + throw new IOException("versions < 3 are not supported (and never existed!?)"); + // version 3+ + name = Bytes.readByteArray(in); + nameAsString = Bytes.toString(this.name); + setRootRegion(in.readBoolean()); + setMetaRegion(in.readBoolean()); + values.clear(); + int numVals = in.readInt(); + for (int i = 0; i < numVals; i++) { + ImmutableBytesWritable key = new ImmutableBytesWritable(); + ImmutableBytesWritable value = new ImmutableBytesWritable(); + key.readFields(in); + value.readFields(in); + values.put(key, value); + } + families.clear(); + int numFamilies = in.readInt(); + for (int i = 0; i < numFamilies; i++) { + HColumnDescriptor c = new HColumnDescriptor(); + c.readFields(in); + families.put(Bytes.mapKey(c.getName()), c); } } /** {@inheritDoc} */ - public void readFields(DataInput in) throws IOException { - this.rootregion = in.readBoolean(); - this.metaregion = in.readBoolean(); - this.name = Bytes.readByteArray(in); - this.nameAsString = Bytes.toString(this.name); - int numCols = in.readInt(); - this.families.clear(); - for (int i = 0; i < numCols; i++) { - HColumnDescriptor c = new HColumnDescriptor(); - c.readFields(in); - this.families.put(Bytes.mapKey(c.getName()), c); + public void write(DataOutput out) throws IOException { + out.writeInt(TABLE_DESCRIPTOR_VERSION); + Bytes.writeByteArray(out, name); + out.writeBoolean(isRootRegion()); + out.writeBoolean(isMetaRegion()); + out.writeInt(values.size()); + for (Map.Entry e: + values.entrySet()) { + e.getKey().write(out); + e.getValue().write(out); + } + out.writeInt(families.size()); + for(Iterator it = families.values().iterator(); + it.hasNext(); ) { + HColumnDescriptor family = it.next(); + family.write(out); } } @@ -272,12 +484,10 @@ public class HTableDescriptor implements WritableComparable { if (result == 0) { result = families.size() - other.families.size(); } - if (result == 0 && families.size() != other.families.size()) { result = Integer.valueOf(families.size()).compareTo( Integer.valueOf(other.families.size())); } - if (result == 0) { for (Iterator it = families.values().iterator(), it2 = other.families.values().iterator(); it.hasNext(); ) { @@ -287,6 +497,14 @@ public class HTableDescriptor implements WritableComparable { } } } + if (result == 0) { + // punt on comparison for ordering, just calculate difference + result = this.values.hashCode() - other.values.hashCode(); + if (result < 0) + result = -1; + else if (result > 0) + result = 1; + } return result; } @@ -323,4 +541,4 @@ public class HTableDescriptor implements WritableComparable { public static Path getTableDir(Path rootdir, final byte [] tableName) { return new Path(rootdir, Bytes.toString(tableName)); } -} \ No newline at end of file +} diff --git a/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 9e55d851c15..e276b3116a8 100644 --- a/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/src/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -563,7 +563,26 @@ public class HBaseAdmin { } } - + /** + * Modify a table's HTableDescriptor + * + * @param tableName name of table + * @param desc the updated descriptor + * @throws IOException + */ + public void modifyTableMeta(final byte [] tableName, HTableDescriptor desc) + throws IOException { + if (this.master == null) { + throw new MasterNotRunningException("master has been shut down"); + } + HTableDescriptor.isLegalTableName(tableName); + try { + this.master.modifyTableMeta(tableName, desc); + } catch (RemoteException e) { + throw RemoteExceptionHandler.decodeRemoteException(e); + } + } + /** * Shuts down the HBase instance * @throws IOException diff --git a/src/java/org/apache/hadoop/hbase/client/HTable.java b/src/java/org/apache/hadoop/hbase/client/HTable.java index d3ea4474c23..18088a46368 100644 --- a/src/java/org/apache/hadoop/hbase/client/HTable.java +++ b/src/java/org/apache/hadoop/hbase/client/HTable.java @@ -243,7 +243,8 @@ public class HTable { * @throws IOException */ public HTableDescriptor getTableDescriptor() throws IOException { - return this.connection.getHTableDescriptor(this.tableName); + return new UnmodifyableHTableDescriptor( + this.connection.getHTableDescriptor(this.tableName)); } /** diff --git a/src/java/org/apache/hadoop/hbase/client/MetaScanner.java b/src/java/org/apache/hadoop/hbase/client/MetaScanner.java index 249dca23e0a..e5b996ecb51 100644 --- a/src/java/org/apache/hadoop/hbase/client/MetaScanner.java +++ b/src/java/org/apache/hadoop/hbase/client/MetaScanner.java @@ -82,8 +82,7 @@ class MetaScanner implements HConstants { /** * Visitor class called to process each row of the .META. table */ - protected interface MetaScannerVisitor { - + interface MetaScannerVisitor { /** * Visitor method that accepts a RowResult and the meta region location. * Implementations can return false to stop the region's loop if it becomes diff --git a/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java new file mode 100644 index 00000000000..eba8eb4ccf3 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHColumnDescriptor.java @@ -0,0 +1,55 @@ +package org.apache.hadoop.hbase.client; + +import org.apache.hadoop.hbase.HColumnDescriptor; + +public class UnmodifyableHColumnDescriptor extends HColumnDescriptor { + + public UnmodifyableHColumnDescriptor (final HColumnDescriptor desc) { + super(desc); + } + + @Override + public void setValue(byte[] key, byte[] value) { + throw new UnsupportedOperationException("HColumnDescriptor is read-only"); + } + + @Override + public void setValue(String key, String value) { + throw new UnsupportedOperationException("HColumnDescriptor is read-only"); + } + + @Override + public void setMaxVersions(int maxVersions) { + throw new UnsupportedOperationException("HColumnDescriptor is read-only"); + } + + @Override + public void setInMemory(boolean inMemory) { + throw new UnsupportedOperationException("HColumnDescriptor is read-only"); + } + + @Override + public void setBlockCacheEnabled(boolean blockCacheEnabled) { + throw new UnsupportedOperationException("HColumnDescriptor is read-only"); + } + + @Override + public void setMaxValueLength(int maxLength) { + throw new UnsupportedOperationException("HColumnDescriptor is read-only"); + } + + @Override + public void setTimeToLive(int timeToLive) { + throw new UnsupportedOperationException("HColumnDescriptor is read-only"); + } + + @Override + public void setCompressionType(CompressionType type) { + throw new UnsupportedOperationException("HColumnDescriptor is read-only"); + } + + @Override + public void setMapFileIndexInterval(int interval) { + throw new UnsupportedOperationException("HTableDescriptor is read-only"); + } +} diff --git a/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java index 64f95ad5359..78c4b1430a6 100644 --- a/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HTableDescriptor; * Read-only table descriptor. */ public class UnmodifyableHTableDescriptor extends HTableDescriptor { - public UnmodifyableHTableDescriptor() { super(); } @@ -39,7 +38,7 @@ public class UnmodifyableHTableDescriptor extends HTableDescriptor { UnmodifyableHTableDescriptor(final HTableDescriptor desc) { super(desc.getName()); for (HColumnDescriptor c: desc.getFamilies()) { - super.addFamily(c); + super.addFamily(new UnmodifyableHColumnDescriptor(c)); } } @@ -61,4 +60,34 @@ public class UnmodifyableHTableDescriptor extends HTableDescriptor { public HColumnDescriptor removeFamily(final byte [] column) { throw new UnsupportedOperationException("HTableDescriptor is read-only"); } + + @Override + public void setInMemory(boolean inMemory) { + throw new UnsupportedOperationException("HTableDescriptor is read-only"); + } + + @Override + public void setReadOnly(boolean readOnly) { + throw new UnsupportedOperationException("HTableDescriptor is read-only"); + } + + @Override + public void setValue(byte[] key, byte[] value) { + throw new UnsupportedOperationException("HTableDescriptor is read-only"); + } + + @Override + public void setValue(String key, String value) { + throw new UnsupportedOperationException("HTableDescriptor is read-only"); + } + + @Override + public void setMaxFileSize(long maxFileSize) { + throw new UnsupportedOperationException("HTableDescriptor is read-only"); + } + + @Override + public void setMemcacheFlushSize(int memcacheFlushSize) { + throw new UnsupportedOperationException("HTableDescriptor is read-only"); + } } diff --git a/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java b/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java index e959ce2c8dc..80e8eee06c8 100644 --- a/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java +++ b/src/java/org/apache/hadoop/hbase/ipc/HMasterInterface.java @@ -104,7 +104,16 @@ public interface HMasterInterface extends VersionedProtocol { * @throws IOException */ public void disableTable(final byte [] tableName) throws IOException; - + + /** + * Modify a table's metadata + * + * @param tableName + * @param desc + */ + public void modifyTableMeta(byte[] tableName, HTableDescriptor desc) + throws IOException; + /** * Shutdown an HBase cluster. * @throws IOException diff --git a/src/java/org/apache/hadoop/hbase/master/HMaster.java b/src/java/org/apache/hadoop/hbase/master/HMaster.java index 6c94123ed7c..0362e23a6d7 100644 --- a/src/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/src/java/org/apache/hadoop/hbase/master/HMaster.java @@ -688,6 +688,13 @@ public class HMaster extends Thread implements HConstants, HMasterInterface, new ChangeTableState(this, tableName, false).process(); } + /** {@inheritDoc} */ + public void modifyTableMeta(final byte[] tableName, HTableDescriptor desc) + throws IOException + { + new ModifyTableMeta(this, tableName, desc).process(); + } + /** {@inheritDoc} */ public HServerAddress findRootRegion() { return regionManager.getRootRegionLocation(); diff --git a/src/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java b/src/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java new file mode 100644 index 00000000000..c7a15999d49 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/master/ModifyTableMeta.java @@ -0,0 +1,79 @@ +/** + * Copyright 2008 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.master; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableNotDisabledException; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; + +/** Instantiated to modify table descriptor metadata */ +class ModifyTableMeta extends TableOperation { + + private static Log LOG = LogFactory.getLog(ModifyTableMeta.class); + + private HTableDescriptor desc; + + ModifyTableMeta(final HMaster master, final byte [] tableName, + HTableDescriptor desc) + throws IOException { + super(master, tableName); + this.desc = desc; + LOG.debug("modifying " + Bytes.toString(tableName) + ": " + + desc.toString()); + } + + protected void updateRegionInfo(HRegionInterface server, byte [] regionName, + HRegionInfo i) + throws IOException { + BatchUpdate b = new BatchUpdate(i.getRegionName()); + b.put(COL_REGIONINFO, Writables.getBytes(i)); + server.batchUpdate(regionName, b); + LOG.debug("updated HTableDescriptor for region " + i.getRegionNameAsString()); + } + + @Override + protected void processScanItem( + @SuppressWarnings("unused") String serverName, + @SuppressWarnings("unused") long startCode, final HRegionInfo info) + throws IOException { + if (isEnabled(info)) { + throw new TableNotDisabledException(tableName.toString()); + } + } + + @Override + protected void postProcessMeta(MetaRegion m, HRegionInterface server) + throws IOException { + for (HRegionInfo i: unservedRegions) { + i.setTableDesc(desc); + updateRegionInfo(server, m.getRegionName(), i); + } + // kick off a meta scan right away + master.regionManager.metaScannerThread.interrupt(); + } +} diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java index e1bbf6ae33c..6f9cc596633 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HAbstractScanner.java @@ -125,7 +125,7 @@ public abstract class HAbstractScanner implements InternalScanner { private boolean multipleMatchers; /** Constructor for abstract base class */ - HAbstractScanner(long timestamp, byte [][] targetCols) throws IOException { + protected HAbstractScanner(long timestamp, byte [][] targetCols) throws IOException { this.timestamp = timestamp; this.wildcardMatch = false; this.multipleMatchers = false; diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 1816806aefd..3f7e910fa0b 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -156,6 +156,7 @@ public class HRegion implements HConstants { b.getRegionInfo().getTableDesc().getNameAsString())) { throw new IOException("Regions do not belong to the same table"); } + FileSystem fs = a.getFilesystem(); // Make sure each region's cache is empty @@ -483,13 +484,19 @@ public class HRegion implements HConstants { fs.delete(merges, true); } - // By default, we flush the cache when 64M. - this.memcacheFlushSize = conf.getInt("hbase.hregion.memcache.flush.size", - 1024*1024*64); + int flushSize = regionInfo.getTableDesc().getMemcacheFlushSize(); + if (flushSize == HTableDescriptor.DEFAULT_MEMCACHE_FLUSH_SIZE) { + flushSize = conf.getInt("hbase.hregion.memcache.flush.size", + HTableDescriptor.DEFAULT_MEMCACHE_FLUSH_SIZE); + } + this.memcacheFlushSize = flushSize; this.blockingMemcacheSize = this.memcacheFlushSize * conf.getInt("hbase.hregion.memcache.block.multiplier", 1); + if (this.regionInfo.getTableDesc().isReadOnly()) + this.writestate.writesEnabled = false; + // HRegion is ready to go! this.writestate.compacting = false; this.lastFlushTime = System.currentTimeMillis(); @@ -1311,6 +1318,10 @@ public class HRegion implements HConstants { public void batchUpdate(BatchUpdate b) throws IOException { + if (!this.writestate.writesEnabled) { + throw new IOException("region is read only"); + } + // Do a rough check that we have resources to accept a write. The check is // 'rough' in that between the resource check and the call to obtain a // read lock, resources may run out. For now, the thought is that this @@ -1418,6 +1429,9 @@ public class HRegion implements HConstants { public void deleteAll(final byte [] row, final byte [] column, final long ts) throws IOException { checkColumn(column); + if (!this.writestate.writesEnabled) { + throw new IOException("region is read only"); + } Integer lid = obtainRowLock(row); try { deleteMultiple(row, column, ts, ALL_VERSIONS); @@ -1434,6 +1448,9 @@ public class HRegion implements HConstants { */ public void deleteAll(final byte [] row, final long ts) throws IOException { + if (!this.writestate.writesEnabled) { + throw new IOException("region is read only"); + } Integer lid = obtainRowLock(row); try { for (HStore store : stores.values()){ @@ -1461,6 +1478,9 @@ public class HRegion implements HConstants { */ public void deleteFamily(byte [] row, byte [] family, long timestamp) throws IOException{ + if (!this.writestate.writesEnabled) { + throw new IOException("region is read only"); + } Integer lid = obtainRowLock(row); try { // find the HStore for the column family @@ -1493,6 +1513,9 @@ public class HRegion implements HConstants { private void deleteMultiple(final byte [] row, final byte [] column, final long ts, final int versions) throws IOException { + if (!this.writestate.writesEnabled) { + throw new IOException("region is read only"); + } HStoreKey origin = new HStoreKey(row, column, ts); Set keys = getKeys(origin, versions); if (keys.size() > 0) { @@ -1520,6 +1543,9 @@ public class HRegion implements HConstants { final byte [] val) throws IOException { checkColumn(key.getColumn()); + if (!this.writestate.writesEnabled) { + throw new IOException("region is read only"); + } TreeMap targets = this.targetColumns.get(lockid); if (targets == null) { targets = new TreeMap(); @@ -1541,6 +1567,9 @@ public class HRegion implements HConstants { if (updatesByColumn == null || updatesByColumn.size() <= 0) { return; } + if (!this.writestate.writesEnabled) { + throw new IOException("region is read only"); + } boolean flush = false; this.updatesLock.readLock().lock(); try { diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HStore.java b/src/java/org/apache/hadoop/hbase/regionserver/HStore.java index f424d891d40..35c699dc3a4 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -172,15 +172,20 @@ public class HStore implements HConstants { this.storeName = Bytes.toBytes(this.info.getEncodedName() + "/" + Bytes.toString(this.family.getName())); this.storeNameStr = Bytes.toString(this.storeName); - + // By default, we compact if an HStore has more than // MIN_COMMITS_FOR_COMPACTION map files this.compactionThreshold = conf.getInt("hbase.hstore.compactionThreshold", 3); // By default we split region if a file > DEFAULT_MAX_FILE_SIZE. - this.desiredMaxFileSize = - conf.getLong("hbase.hregion.max.filesize", DEFAULT_MAX_FILE_SIZE); + long maxFileSize = info.getTableDesc().getMaxFileSize(); + if (maxFileSize == HConstants.DEFAULT_MAX_FILE_SIZE) { + maxFileSize = conf.getLong("hbase.hregion.max.filesize", + HConstants.DEFAULT_MAX_FILE_SIZE); + } + this.desiredMaxFileSize = maxFileSize; + this.storeSize = 0L; if (family.getCompression() == HColumnDescriptor.CompressionType.BLOCK) { @@ -242,11 +247,11 @@ public class HStore implements HConstants { if (first) { // Use a block cache (if configured) for the first reader only // so as to control memory usage. - r = e.getValue().getReader(this.fs, this.family.isBloomFilterEnabled(), + r = e.getValue().getReader(this.fs, this.family.isBloomfilter(), family.isBlockCacheEnabled()); first = false; } else { - r = e.getValue().getReader(this.fs, this.family.isBloomFilterEnabled(), + r = e.getValue().getReader(this.fs, this.family.isBloomfilter(), false); } this.readers.put(e.getKey(), r); @@ -582,7 +587,8 @@ public class HStore implements HConstants { HStoreFile flushedFile = new HStoreFile(conf, fs, basedir, info.getEncodedName(), family.getName(), -1L, null); MapFile.Writer out = flushedFile.getWriter(this.fs, this.compression, - this.family.isBloomFilterEnabled(), cache.size()); + this.family.isBloomfilter(), cache.size()); + out.setIndexInterval(family.getMapFileIndexInterval()); // Here we tried picking up an existing HStoreFile from disk and // interlacing the memcache flush compacting as we go. The notion was @@ -651,7 +657,7 @@ public class HStore implements HConstants { Long flushid = Long.valueOf(logCacheFlushId); // Open the map file reader. this.readers.put(flushid, - flushedFile.getReader(this.fs, this.family.isBloomFilterEnabled(), + flushedFile.getReader(this.fs, this.family.isBloomfilter(), this.family.isBlockCacheEnabled())); this.storefiles.put(flushid, flushedFile); // Tell listeners of the change in readers. @@ -737,9 +743,9 @@ public class HStore implements HConstants { return checkSplit(); } /* - * We create a new list of MapFile.Reader objects so we don't screw up the - * caching associated with the currently-loaded ones. Our iteration-based - * access pattern is practically designed to ruin the cache. + * We create a new list of MapFile.Reader objects so we don't screw up + * the caching associated with the currently-loaded ones. Our iteration- + * based access pattern is practically designed to ruin the cache. */ List readers = new ArrayList(); for (HStoreFile file: filesToCompact) { @@ -749,7 +755,7 @@ public class HStore implements HConstants { readers.add(reader); // Compute the size of the new bloomfilter if needed - if (this.family.isBloomFilterEnabled()) { + if (this.family.isBloomfilter()) { nrows += reader.getBloomFilterSize(); } } catch (IOException e) { @@ -775,7 +781,8 @@ public class HStore implements HConstants { FSUtils.getPath(compactedOutputFile.getMapFilePath())); } MapFile.Writer writer = compactedOutputFile.getWriter(this.fs, - this.compression, this.family.isBloomFilterEnabled(), nrows); + this.compression, this.family.isBloomfilter(), nrows); + writer.setIndexInterval(family.getMapFileIndexInterval()); try { compactHStoreFiles(writer, readers); } finally { @@ -1029,7 +1036,7 @@ public class HStore implements HConstants { // Use a block cache (if configured) for this reader since // it is the only one. finalCompactedFile.getReader(this.fs, - this.family.isBloomFilterEnabled(), + this.family.isBloomfilter(), this.family.isBlockCacheEnabled())); this.storefiles.put(orderVal, finalCompactedFile); // Tell observers that list of Readers has changed. @@ -1814,4 +1821,4 @@ public class HStore implements HConstants { return key; } } -} \ No newline at end of file +} diff --git a/src/java/org/apache/hadoop/hbase/rest/TableHandler.java b/src/java/org/apache/hadoop/hbase/rest/TableHandler.java index 2ea28a26231..4d519ceecbc 100644 --- a/src/java/org/apache/hadoop/hbase/rest/TableHandler.java +++ b/src/java/org/apache/hadoop/hbase/rest/TableHandler.java @@ -413,7 +413,7 @@ public class TableHandler extends GenericHandler { doElement(outputter, "name", Bytes.toString(e.getName())); doElement(outputter, "compression", e.getCompression().toString()); doElement(outputter, "bloomfilter", - Boolean.toString(e.isBloomFilterEnabled())); + Boolean.toString(e.isBloomfilter())); doElement(outputter, "max-versions", Integer.toString(e.getMaxVersions())); doElement(outputter, "maximum-cell-size", diff --git a/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java b/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java index 0ce6dbd6735..ae261235ea8 100644 --- a/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java +++ b/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java @@ -67,7 +67,7 @@ public class ThriftUtilities { col.inMemory = in.isInMemory(); col.blockCacheEnabled = in.isBlockCacheEnabled(); col.maxValueLength = in.getMaxValueLength(); - col.bloomFilterType = Boolean.toString(in.isBloomFilterEnabled()); + col.bloomFilterType = Boolean.toString(in.isBloomfilter()); return col; } diff --git a/src/java/org/apache/hadoop/hbase/util/Migrate.java b/src/java/org/apache/hadoop/hbase/util/Migrate.java index 4a7cefecb01..1ec5f93050c 100644 --- a/src/java/org/apache/hadoop/hbase/util/Migrate.java +++ b/src/java/org/apache/hadoop/hbase/util/Migrate.java @@ -37,9 +37,11 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HStoreKey; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStoreFile; @@ -188,7 +190,7 @@ public class Migrate extends Configured implements Tool { float version = Float.parseFloat(versionStr); if (version == 0.1f) { checkForUnrecoveredLogFiles(getRootDirFiles()); - migrate(); + migrateToV5(); } else { throw new IOException("Unrecognized or non-migratable version: " + version); @@ -209,10 +211,94 @@ public class Migrate extends Configured implements Tool { } } - private void migrate() throws IOException { + private void migrateToV5() throws IOException { + rewriteMetaHRegionInfo(); addHistorianFamilyToMeta(); updateBloomFilters(); } + + /** + * Rewrite the meta tables so that HRI is versioned and so we move to new + * HCD and HCD. + * @throws IOException + */ + private void rewriteMetaHRegionInfo() throws IOException { + if (this.readOnly && this.migrationNeeded) { + return; + } + // Read using old classes. + final org.apache.hadoop.hbase.util.migration.v5.MetaUtils utils = + new org.apache.hadoop.hbase.util.migration.v5.MetaUtils(this.conf); + try { + // Scan the root region + utils.scanRootRegion(new org.apache.hadoop.hbase.util.migration.v5.MetaUtils.ScannerListener() { + public boolean processRow(org.apache.hadoop.hbase.util.migration.v5.HRegionInfo info) + throws IOException { + // Scan every meta region + final org.apache.hadoop.hbase.util.migration.v5.HRegion metaRegion = + utils.getMetaRegion(info); + // If here, we were able to read with old classes. If readOnly, then + // needs migration. + if (readOnly && !migrationNeeded) { + migrationNeeded = true; + return false; + } + updateHRegionInfo(utils.getRootRegion(), info); + utils.scanMetaRegion(info, new org.apache.hadoop.hbase.util.migration.v5.MetaUtils.ScannerListener() { + public boolean processRow(org.apache.hadoop.hbase.util.migration.v5.HRegionInfo hri) + throws IOException { + updateHRegionInfo(metaRegion, hri); + return true; + } + }); + return true; + } + }); + } catch (Exception e) { + LOG.error("", e); + } finally { + utils.shutdown(); + } + } + + /* + * Move from old pre-v5 hregioninfo to current HRegionInfo + * Persist back into r + * @param mr + * @param oldHri + */ + void updateHRegionInfo(org.apache.hadoop.hbase.util.migration.v5.HRegion mr, + org.apache.hadoop.hbase.util.migration.v5.HRegionInfo oldHri) + throws IOException { + byte [] oldHriTableName = oldHri.getTableDesc().getName(); + HTableDescriptor newHtd = + Bytes.equals(HConstants.ROOT_TABLE_NAME, oldHriTableName)? + HTableDescriptor.ROOT_TABLEDESC: + Bytes.equals(HConstants.META_TABLE_NAME, oldHriTableName)? + HTableDescriptor.META_TABLEDESC: + new HTableDescriptor(oldHri.getTableDesc().getName()); + for (org.apache.hadoop.hbase.util.migration.v5.HColumnDescriptor oldHcd: + oldHri.getTableDesc().getFamilies()) { + HColumnDescriptor newHcd = new HColumnDescriptor( + HStoreKey.addDelimiter(oldHcd.getName()), + oldHcd.getMaxValueLength(), + HColumnDescriptor.CompressionType.valueOf(oldHcd.getCompressionType().toString()), + oldHcd.isInMemory(), oldHcd.isBlockCacheEnabled(), + oldHcd.getMaxValueLength(), oldHcd.getTimeToLive(), + oldHcd.isBloomFilterEnabled()); + newHtd.addFamily(newHcd); + } + HRegionInfo newHri = new HRegionInfo(newHtd, oldHri.getStartKey(), + oldHri.getEndKey(), oldHri.isSplit(), oldHri.getRegionId()); + BatchUpdate b = new BatchUpdate(newHri.getRegionName()); + b.put(HConstants.COL_REGIONINFO, Writables.getBytes(newHri)); + mr.batchUpdate(b); + if (LOG.isDebugEnabled()) { + LOG.debug("New " + Bytes.toString(HConstants.COL_REGIONINFO) + + " for " + oldHri.toString() + " in " + mr.toString() + " is: " + + newHri.toString()); + } + } private FileStatus[] getRootDirFiles() throws IOException { FileStatus[] stats = fs.listStatus(FSUtils.getRootDir(this.conf)); @@ -243,77 +329,6 @@ public class Migrate extends Configured implements Tool { } } - void migrateRegionDir(final byte [] tableName, String oldPath) - throws IOException { - // Create directory where table will live - Path rootdir = FSUtils.getRootDir(this.conf); - Path tableDir = new Path(rootdir, Bytes.toString(tableName)); - fs.mkdirs(tableDir); - - // Move the old region directory under the table directory - - Path newPath = new Path(tableDir, - oldPath.substring(OLD_PREFIX.length())); - fs.rename(new Path(rootdir, oldPath), newPath); - - processRegionSubDirs(fs, newPath); - } - - private void processRegionSubDirs(FileSystem fs, Path newPath) - throws IOException { - String newName = newPath.getName(); - FileStatus[] children = fs.listStatus(newPath); - for (int i = 0; i < children.length; i++) { - String child = children[i].getPath().getName(); - if (children[i].isDir()) { - processRegionSubDirs(fs, children[i].getPath()); - - // Rename old compaction directories - - if (child.startsWith(OLD_PREFIX)) { - fs.rename(children[i].getPath(), - new Path(newPath, child.substring(OLD_PREFIX.length()))); - } - } else { - if (newName.compareTo("mapfiles") == 0) { - // Check to see if this mapfile is a reference - - if (HStore.isReference(children[i].getPath())) { - // Keep track of references in case we come across a region - // that we can't otherwise account for. - references.add(child.substring(child.indexOf(".") + 1)); - } - } - } - } - } - - private void scanRootRegion() throws IOException { - final MetaUtils utils = new MetaUtils(this.conf); - try { - utils.scanRootRegion(new MetaUtils.ScannerListener() { - public boolean processRow(HRegionInfo info) throws IOException { - // First move the meta region to where it should be and rename - // subdirectories as necessary - migrateRegionDir(HConstants.META_TABLE_NAME, OLD_PREFIX - + info.getEncodedName()); - utils.scanMetaRegion(info, new MetaUtils.ScannerListener() { - public boolean processRow(HRegionInfo tableInfo) throws IOException { - // Move the region to where it should be and rename - // subdirectories as necessary - migrateRegionDir(tableInfo.getTableDesc().getName(), OLD_PREFIX - + tableInfo.getEncodedName()); - return true; - } - }); - return true; - } - }); - } finally { - utils.shutdown(); - } - } - private void addHistorianFamilyToMeta() throws IOException { if (this.migrationNeeded) { // Be careful. We cannot use MetAutils if current hbase in the @@ -359,17 +374,16 @@ public class Migrate extends Configured implements Tool { // Scan every meta region final HRegion metaRegion = utils.getMetaRegion(info); utils.scanMetaRegion(info, new MetaUtils.ScannerListener() { - public boolean processRow(HRegionInfo tableInfo) throws IOException { - HTableDescriptor desc = tableInfo.getTableDesc(); + public boolean processRow(HRegionInfo hri) throws IOException { + HTableDescriptor desc = hri.getTableDesc(); Path tableDir = HTableDescriptor.getTableDir(rootDir, desc.getName()); for (HColumnDescriptor column: desc.getFamilies()) { - if (column.isBloomFilterEnabled()) { + if (column.isBloomfilter()) { // Column has a bloom filter migrationNeeded = true; - Path filterDir = HStoreFile.getFilterDir(tableDir, - tableInfo.getEncodedName(), column.getName()); + hri.getEncodedName(), column.getName()); if (fs.exists(filterDir)) { // Filter dir exists if (readOnly) { @@ -379,8 +393,10 @@ public class Migrate extends Configured implements Tool { } // Delete the filter fs.delete(filterDir, true); - // Update the HRegionInfo in meta - utils.updateMETARegionInfo(metaRegion, tableInfo); + // Update the HRegionInfo in meta setting the bloomfilter + // to be disabled. + column.setBloomfilter(false); + utils.updateMETARegionInfo(metaRegion, hri); } } } diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/FlushRequester.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/FlushRequester.java new file mode 100644 index 00000000000..57d5d2f6595 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/FlushRequester.java @@ -0,0 +1,36 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +/** + * Implementors of this interface want to be notified when an HRegion + * determines that a cache flush is needed. A FlushRequester (or null) + * must be passed to the HRegion constructor so it knows who to call when it + * has a filled memcache. + */ +public interface FlushRequester { + /** + * Tell the listener the cache needs to be flushed. + * + * @param region the HRegion requesting the cache flush + */ + void request(HRegion region); +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HColumnDescriptor.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HColumnDescriptor.java new file mode 100644 index 00000000000..60390d8f60d --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HColumnDescriptor.java @@ -0,0 +1,449 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.hbase.BloomFilterDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.WritableComparable; + +/** + * An HColumnDescriptor contains information about a column family such as the + * number of versions, compression settings, etc. + * + * It is used as input when creating a table or adding a column. Once set, the + * parameters that specify a column cannot be changed without deleting the + * column and recreating it. If there is data stored in the column, it will be + * deleted when the column is deleted. + */ +public class HColumnDescriptor implements WritableComparable { + // For future backward compatibility + + // Version 3 was when column names becaome byte arrays and when we picked up + // Time-to-live feature. + // Version 4 was when bloom filter descriptors were removed. + private static final byte COLUMN_DESCRIPTOR_VERSION = (byte)4; + + /** + * The type of compression. + * @see org.apache.hadoop.io.SequenceFile.Writer + */ + public static enum CompressionType { + /** Do not compress records. */ + NONE, + /** Compress values only, each separately. */ + RECORD, + /** Compress sequences of records together in blocks. */ + BLOCK + } + + // Defines for jruby/shell + public static final String COMPRESSION = "COMPRESSION"; + public static final String IN_MEMORY = "IN_MEMORY"; + public static final String BLOCKCACHE = "BLOCKCACHE"; + public static final String LENGTH = "LENGTH"; + public static final String TTL = "TTL"; + public static final String BLOOMFILTER = "BLOOMFILTER"; + public static final String FOREVER = "FOREVER"; + + /** + * Default compression type. + */ + public static final CompressionType DEFAULT_COMPRESSION = + CompressionType.NONE; + + /** + * Default number of versions of a record to keep. + */ + public static final int DEFAULT_VERSIONS = 3; + + /** + * Default maximum cell length. + */ + public static final int DEFAULT_LENGTH = Integer.MAX_VALUE; + + /** + * Default setting for whether to serve from memory or not. + */ + public static final boolean DEFAULT_IN_MEMORY = false; + + /** + * Default setting for whether to use a block cache or not. + */ + public static final boolean DEFAULT_BLOCKCACHE = false; + + /** + * Default setting for whether or not to use bloomfilters. + */ + public static final boolean DEFAULT_BLOOMFILTER = false; + + /** + * Default time to live of cell contents. + */ + public static final int DEFAULT_TTL = HConstants.FOREVER; + + // Column family name + private byte [] name; + // Number of versions to keep + private int maxVersions = DEFAULT_VERSIONS; + // Compression setting if any + private CompressionType compressionType = DEFAULT_COMPRESSION; + // Serve reads from in-memory cache + private boolean inMemory = DEFAULT_IN_MEMORY; + // Serve reads from in-memory block cache + private boolean blockCacheEnabled = DEFAULT_BLOCKCACHE; + // Maximum value size + private int maxValueLength = DEFAULT_LENGTH; + // Time to live of cell contents, in seconds from last timestamp + private int timeToLive = DEFAULT_TTL; + // True if bloom filter was specified + private boolean bloomFilter = false; + + /** + * Default constructor. Must be present for Writable. + */ + public HColumnDescriptor() { + this.name = null; + } + + /** + * Construct a column descriptor specifying only the family name + * The other attributes are defaulted. + * + * @param columnName - column family name + */ + public HColumnDescriptor(final String columnName) { + this(Bytes.toBytes(columnName)); + } + + /** + * Construct a column descriptor specifying only the family name + * The other attributes are defaulted. + * + * @param columnName - column family name + */ + public HColumnDescriptor(final Text columnName) { + this(columnName.getBytes()); + } + + /** + * Construct a column descriptor specifying only the family name + * The other attributes are defaulted. + * + * @param columnName Column family name. Must have the ':' ending. + */ + public HColumnDescriptor(final byte [] columnName) { + this (columnName == null || columnName.length <= 0? + HConstants.EMPTY_BYTE_ARRAY: columnName, DEFAULT_VERSIONS, + DEFAULT_COMPRESSION, DEFAULT_IN_MEMORY, DEFAULT_BLOCKCACHE, + Integer.MAX_VALUE, DEFAULT_TTL, false); + } + + /** + * Constructor + * @param columnName Column family name. Must have the ':' ending. + * @param maxVersions Maximum number of versions to keep + * @param compression Compression type + * @param inMemory If true, column data should be kept in an HRegionServer's + * cache + * @param blockCacheEnabled If true, MapFile blocks should be cached + * @param maxValueLength Restrict values to <= this value + * @param timeToLive Time-to-live of cell contents, in seconds from last timestamp + * (use HConstants.FOREVER for unlimited TTL) + * @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. [a-zA-Z_0-9] and does not + * end in a : + * @throws IllegalArgumentException if the number of versions is <= 0 + */ + public HColumnDescriptor(final byte [] columnName, final int maxVersions, + final CompressionType compression, final boolean inMemory, + final boolean blockCacheEnabled, final int maxValueLength, + final int timeToLive, final boolean bloomFilter) { + isLegalFamilyName(columnName); + this.name = stripColon(columnName); + if (maxVersions <= 0) { + // TODO: Allow maxVersion of 0 to be the way you say "Keep all versions". + // Until there is support, consider 0 or < 0 -- a configuration error. + throw new IllegalArgumentException("Maximum versions must be positive"); + } + this.maxVersions = maxVersions; + this.inMemory = inMemory; + this.blockCacheEnabled = blockCacheEnabled; + this.maxValueLength = maxValueLength; + this.timeToLive = timeToLive; + this.bloomFilter = bloomFilter; + this.compressionType = compression; + } + + private static byte [] stripColon(final byte [] n) { + byte [] result = new byte [n.length - 1]; + // Have the stored family name be absent the colon delimiter + System.arraycopy(n, 0, result, 0, n.length - 1); + return result; + } + + /** + * @param b Family name. + * @return b + * @throws IllegalArgumentException If not null and not a legitimate family + * name: i.e. 'printable' and ends in a ':' (Null passes are allowed because + * b can be null when deserializing). + */ + public static byte [] isLegalFamilyName(final byte [] b) { + if (b == null) { + return b; + } + if (b[b.length - 1] != ':') { + throw new IllegalArgumentException("Family names must end in a colon: " + + Bytes.toString(b)); + } + for (int i = 0; i < (b.length - 1); i++) { + if (Character.isLetterOrDigit(b[i]) || b[i] == '_' || b[i] == '.') { + continue; + } + throw new IllegalArgumentException("Illegal character <" + b[i] + + ">. Family names can only contain 'word characters' and must end" + + "with a colon: " + Bytes.toString(b)); + } + return b; + } + + /** + * @return Name of this column family + */ + public byte [] getName() { + return name; + } + + /** + * @return Name of this column family + */ + public String getNameAsString() { + return Bytes.toString(this.name); + } + + /** @return compression type being used for the column family */ + public CompressionType getCompression() { + return this.compressionType; + } + + /** @return maximum number of versions */ + public int getMaxVersions() { + return this.maxVersions; + } + + /** + * @return Compression type setting. + */ + public CompressionType getCompressionType() { + return this.compressionType; + } + + /** + * @return True if we are to keep all in use HRegionServer cache. + */ + public boolean isInMemory() { + return this.inMemory; + } + + /** + * @return Maximum value length. + */ + public int getMaxValueLength() { + return this.maxValueLength; + } + + /** + * @return Time to live. + */ + public int getTimeToLive() { + return this.timeToLive; + } + + /** + * @return True if MapFile blocks should be cached. + */ + public boolean isBlockCacheEnabled() { + return blockCacheEnabled; + } + + /** + * @return true if a bloom filter is enabled + */ + public boolean isBloomFilterEnabled() { + return this.bloomFilter; + } + + /** {@inheritDoc} */ + @Override + public String toString() { + return "{" + HConstants.NAME + " => '" + Bytes.toString(name) + + "', " + HConstants.VERSIONS + " => " + maxVersions + + ", " + COMPRESSION + " => '" + this.compressionType + + "', " + IN_MEMORY + " => " + inMemory + + ", " + BLOCKCACHE + " => " + blockCacheEnabled + + ", " + LENGTH + " => " + maxValueLength + + ", " + TTL + " => " + + (timeToLive == HConstants.FOREVER ? "FOREVER" : + Integer.toString(timeToLive)) + + ", " + BLOOMFILTER + " => " + bloomFilter + "}"; + } + + /** {@inheritDoc} */ + @Override + public boolean equals(Object obj) { + return compareTo(obj) == 0; + } + + /** {@inheritDoc} */ + @Override + public int hashCode() { + int result = Bytes.hashCode(this.name); + result ^= Integer.valueOf(this.maxVersions).hashCode(); + result ^= this.compressionType.hashCode(); + result ^= Boolean.valueOf(this.inMemory).hashCode(); + result ^= Boolean.valueOf(this.blockCacheEnabled).hashCode(); + result ^= Integer.valueOf(this.maxValueLength).hashCode(); + result ^= Integer.valueOf(this.timeToLive).hashCode(); + result ^= Boolean.valueOf(this.bloomFilter).hashCode(); + result ^= Byte.valueOf(COLUMN_DESCRIPTOR_VERSION).hashCode(); + return result; + } + + // Writable + + /** {@inheritDoc} */ + public void readFields(DataInput in) throws IOException { + int versionNumber = in.readByte(); + if (versionNumber <= 2) { + Text t = new Text(); + t.readFields(in); + this.name = t.getBytes(); + if (HStoreKey.getFamilyDelimiterIndex(this.name) > 0) { + this.name = stripColon(this.name); + } + } else { + this.name = Bytes.readByteArray(in); + } + this.maxVersions = in.readInt(); + int ordinal = in.readInt(); + this.compressionType = CompressionType.values()[ordinal]; + this.inMemory = in.readBoolean(); + this.maxValueLength = in.readInt(); + this.bloomFilter = in.readBoolean(); + if (this.bloomFilter && versionNumber < 5) { + // If a bloomFilter is enabled and the column descriptor is less than + // version 5, we need to skip over it to read the rest of the column + // descriptor. There are no BloomFilterDescriptors written to disk for + // column descriptors with a version number >= 5 + BloomFilterDescriptor junk = new BloomFilterDescriptor(); + junk.readFields(in); + } + if (versionNumber > 1) { + this.blockCacheEnabled = in.readBoolean(); + } + + if (versionNumber > 2) { + this.timeToLive = in.readInt(); + } + } + + /** {@inheritDoc} */ + public void write(DataOutput out) throws IOException { + out.writeByte(COLUMN_DESCRIPTOR_VERSION); + Bytes.writeByteArray(out, this.name); + out.writeInt(this.maxVersions); + out.writeInt(this.compressionType.ordinal()); + out.writeBoolean(this.inMemory); + out.writeInt(this.maxValueLength); + out.writeBoolean(this.bloomFilter); + out.writeBoolean(this.blockCacheEnabled); + out.writeInt(this.timeToLive); + } + + // Comparable + + /** {@inheritDoc} */ + public int compareTo(Object o) { + HColumnDescriptor other = (HColumnDescriptor)o; + int result = Bytes.compareTo(this.name, other.getName()); + if(result == 0) { + result = Integer.valueOf(this.maxVersions).compareTo( + Integer.valueOf(other.maxVersions)); + } + + if(result == 0) { + result = this.compressionType.compareTo(other.compressionType); + } + + if(result == 0) { + if(this.inMemory == other.inMemory) { + result = 0; + + } else if(this.inMemory) { + result = -1; + + } else { + result = 1; + } + } + + if(result == 0) { + if(this.blockCacheEnabled == other.blockCacheEnabled) { + result = 0; + + } else if(this.blockCacheEnabled) { + result = -1; + + } else { + result = 1; + } + } + + if(result == 0) { + result = other.maxValueLength - this.maxValueLength; + } + + if(result == 0) { + result = other.timeToLive - this.timeToLive; + } + + if(result == 0) { + if(this.bloomFilter == other.bloomFilter) { + result = 0; + + } else if(this.bloomFilter) { + result = -1; + + } else { + result = 1; + } + } + return result; + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HConstants.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HConstants.java new file mode 100644 index 00000000000..4f56cf23c43 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HConstants.java @@ -0,0 +1,228 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * HConstants holds a bunch of HBase-related constants + */ +public interface HConstants { + + /** long constant for zero */ + static final Long ZERO_L = Long.valueOf(0L); + + static final String NINES = "99999999999999"; + static final String ZEROES = "00000000000000"; + + // For migration + + /** name of version file */ + static final String VERSION_FILE_NAME = "hbase.version"; + + /** + * Current version of file system + * Version 4 supports only one kind of bloom filter + */ + public static final String FILE_SYSTEM_VERSION = "4"; + + // Configuration parameters + + // TODO: URL for hbase master like hdfs URLs with host and port. + // Like jdbc URLs? URLs could be used to refer to table cells? + // jdbc:mysql://[host][,failoverhost...][:port]/[database] + // jdbc:mysql://[host][,failoverhost...][:port]/[database][?propertyName1][=propertyValue1][&propertyName2][=propertyValue2]... + + // Key into HBaseConfiguration for the hbase.master address. + // TODO: Support 'local': i.e. default of all running in single + // process. Same for regionserver. TODO: Is having HBase homed + // on port 60k OK? + + /** Parameter name for master address */ + static final String MASTER_ADDRESS = "hbase.master"; + + /** default host address */ + static final String DEFAULT_HOST = "0.0.0.0"; + + /** default port that the master listens on */ + static final int DEFAULT_MASTER_PORT = 60000; + + /** Default master address */ + static final String DEFAULT_MASTER_ADDRESS = DEFAULT_HOST + ":" + + DEFAULT_MASTER_PORT; + + /** default port for master web api */ + static final int DEFAULT_MASTER_INFOPORT = 60010; + + /** Parameter name for hbase.regionserver address. */ + static final String REGIONSERVER_ADDRESS = "hbase.regionserver"; + + /** Default region server address */ + static final String DEFAULT_REGIONSERVER_ADDRESS = DEFAULT_HOST + ":60020"; + + /** default port for region server web api */ + static final int DEFAULT_REGIONSERVER_INFOPORT = 60030; + + /** Parameter name for what region server interface to use. */ + static final String REGION_SERVER_CLASS = "hbase.regionserver.class"; + + /** Parameter name for what region server implementation to use. */ + static final String REGION_SERVER_IMPL= "hbase.regionserver.impl"; + + /** Default region server interface class name. */ + static final String DEFAULT_REGION_SERVER_CLASS = HRegionInterface.class.getName(); + + /** Parameter name for how often threads should wake up */ + static final String THREAD_WAKE_FREQUENCY = "hbase.server.thread.wakefrequency"; + + /** Parameter name for HBase instance root directory */ + static final String HBASE_DIR = "hbase.rootdir"; + + /** Used to construct the name of the log directory for a region server */ + static final String HREGION_LOGDIR_NAME = "log"; + + /** Name of old log file for reconstruction */ + static final String HREGION_OLDLOGFILE_NAME = "oldlogfile.log"; + + /** Default maximum file size */ + static final long DEFAULT_MAX_FILE_SIZE = 256 * 1024 * 1024; + + /** Default size of a reservation block */ + static final int DEFAULT_SIZE_RESERVATION_BLOCK = 1024 * 1024 * 5; + + // Always store the location of the root table's HRegion. + // This HRegion is never split. + + // region name = table + startkey + regionid. This is the row key. + // each row in the root and meta tables describes exactly 1 region + // Do we ever need to know all the information that we are storing? + + // Note that the name of the root table starts with "-" and the name of the + // meta table starts with "." Why? it's a trick. It turns out that when we + // store region names in memory, we use a SortedMap. Since "-" sorts before + // "." (and since no other table name can start with either of these + // characters, the root region will always be the first entry in such a Map, + // followed by all the meta regions (which will be ordered by their starting + // row key as well), followed by all user tables. So when the Master is + // choosing regions to assign, it will always choose the root region first, + // followed by the meta regions, followed by user regions. Since the root + // and meta regions always need to be on-line, this ensures that they will + // be the first to be reassigned if the server(s) they are being served by + // should go down. + + /** The root table's name.*/ + static final byte [] ROOT_TABLE_NAME = Bytes.toBytes("-ROOT-"); + + /** The META table's name. */ + static final byte [] META_TABLE_NAME = Bytes.toBytes(".META."); + + // Defines for the column names used in both ROOT and META HBase 'meta' tables. + + /** The ROOT and META column family (string) */ + static final String COLUMN_FAMILY_STR = "info:"; + + /** The META historian column family (string) */ + static final String COLUMN_FAMILY_HISTORIAN_STR = "historian:"; + + /** The ROOT and META column family */ + static final byte [] COLUMN_FAMILY = Bytes.toBytes(COLUMN_FAMILY_STR); + + /** The META historian column family */ + static final byte [] COLUMN_FAMILY_HISTORIAN = Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR); + + /** Array of meta column names */ + static final byte[][] COLUMN_FAMILY_ARRAY = new byte[][] {COLUMN_FAMILY}; + + /** ROOT/META column family member - contains HRegionInfo */ + static final byte [] COL_REGIONINFO = + Bytes.toBytes(COLUMN_FAMILY_STR + "regioninfo"); + + /** Array of column - contains HRegionInfo */ + static final byte[][] COL_REGIONINFO_ARRAY = new byte[][] {COL_REGIONINFO}; + + /** ROOT/META column family member - contains HServerAddress.toString() */ + static final byte[] COL_SERVER = Bytes.toBytes(COLUMN_FAMILY_STR + "server"); + + /** ROOT/META column family member - contains server start code (a long) */ + static final byte [] COL_STARTCODE = + Bytes.toBytes(COLUMN_FAMILY_STR + "serverstartcode"); + + /** the lower half of a split region */ + static final byte [] COL_SPLITA = Bytes.toBytes(COLUMN_FAMILY_STR + "splitA"); + + /** the upper half of a split region */ + static final byte [] COL_SPLITB = Bytes.toBytes(COLUMN_FAMILY_STR + "splitB"); + + /** All the columns in the catalog -ROOT- and .META. tables. + */ + static final byte[][] ALL_META_COLUMNS = {COL_REGIONINFO, COL_SERVER, + COL_STARTCODE, COL_SPLITA, COL_SPLITB}; + + // Other constants + + /** + * An empty instance. + */ + static final byte [] EMPTY_BYTE_ARRAY = new byte [0]; + + /** + * Used by scanners, etc when they want to start at the beginning of a region + */ + static final byte [] EMPTY_START_ROW = EMPTY_BYTE_ARRAY; + + /** + * Last row in a table. + */ + static final byte [] EMPTY_END_ROW = EMPTY_START_ROW; + + /** + * Used by scanners and others when they're trying to detect the end of a + * table + */ + static final byte [] LAST_ROW = EMPTY_BYTE_ARRAY; + + /** When we encode strings, we always specify UTF8 encoding */ + static final String UTF8_ENCODING = "UTF-8"; + + /** + * Timestamp to use when we want to refer to the latest cell. + * This is the timestamp sent by clients when no timestamp is specified on + * commit. + */ + static final long LATEST_TIMESTAMP = Long.MAX_VALUE; + + /** + * Define for 'return-all-versions'. + */ + static final int ALL_VERSIONS = Integer.MAX_VALUE; + + /** + * Unlimited time-to-live. + */ + static final int FOREVER = -1; + + public static final String HBASE_CLIENT_RETRIES_NUMBER_KEY = + "hbase.client.retries.number"; + public static final int DEFAULT_CLIENT_RETRIES = 5; + + public static final String NAME = "NAME"; + public static final String VERSIONS = "VERSIONS"; +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HLog.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HLog.java new file mode 100644 index 00000000000..62daf7a2a93 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HLog.java @@ -0,0 +1,698 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.EOFException; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.SequenceFile.CompressionType; +import org.apache.hadoop.io.SequenceFile.Reader; + +/** + * HLog stores all the edits to the HStore. + * + * It performs logfile-rolling, so external callers are not aware that the + * underlying file is being rolled. + * + *

+ * A single HLog is used by several HRegions simultaneously. + * + *

+ * Each HRegion is identified by a unique long int. HRegions do + * not need to declare themselves before using the HLog; they simply include + * their HRegion-id in the append or + * completeCacheFlush calls. + * + *

+ * An HLog consists of multiple on-disk files, which have a chronological order. + * As data is flushed to other (better) on-disk structures, the log becomes + * obsolete. We can destroy all the log messages for a given HRegion-id up to + * the most-recent CACHEFLUSH message from that HRegion. + * + *

+ * It's only practical to delete entire files. Thus, we delete an entire on-disk + * file F when all of the messages in F have a log-sequence-id that's older + * (smaller) than the most-recent CACHEFLUSH message for every HRegion that has + * a message in F. + * + *

+ * Synchronized methods can never execute in parallel. However, between the + * start of a cache flush and the completion point, appends are allowed but log + * rolling is not. To prevent log rolling taking place during this period, a + * separate reentrant lock is used. + * + *

+ * TODO: Vuk Ercegovac also pointed out that keeping HBase HRegion edit logs in + * HDFS is currently flawed. HBase writes edits to logs and to a memcache. The + * 'atomic' write to the log is meant to serve as insurance against abnormal + * RegionServer exit: on startup, the log is rerun to reconstruct an HRegion's + * last wholesome state. But files in HDFS do not 'exist' until they are cleanly + * closed -- something that will not happen if RegionServer exits without + * running its 'close'. + */ +public class HLog implements HConstants { + private static final Log LOG = LogFactory.getLog(HLog.class); + private static final String HLOG_DATFILE = "hlog.dat."; + static final byte [] METACOLUMN = Bytes.toBytes("METACOLUMN:"); + static final byte [] METAROW = Bytes.toBytes("METAROW"); + final FileSystem fs; + final Path dir; + final Configuration conf; + final LogRollListener listener; + final long threadWakeFrequency; + private final int maxlogentries; + + /* + * Current log file. + */ + SequenceFile.Writer writer; + + /* + * Map of all log files but the current one. + */ + final SortedMap outputfiles = + Collections.synchronizedSortedMap(new TreeMap()); + + /* + * Map of region to last sequence/edit id. + */ + private final Map lastSeqWritten = Collections. + synchronizedSortedMap(new TreeMap(Bytes.BYTES_COMPARATOR)); + + private volatile boolean closed = false; + + private final Integer sequenceLock = new Integer(0); + private volatile long logSeqNum = 0; + + private volatile long filenum = 0; + private volatile long old_filenum = -1; + + private volatile int numEntries = 0; + + // This lock prevents starting a log roll during a cache flush. + // synchronized is insufficient because a cache flush spans two method calls. + private final Lock cacheFlushLock = new ReentrantLock(); + + // We synchronize on updateLock to prevent updates and to prevent a log roll + // during an update + private final Integer updateLock = new Integer(0); + + /** + * Create an edit log at the given dir location. + * + * You should never have to load an existing log. If there is a log at + * startup, it should have already been processed and deleted by the time the + * HLog object is started up. + * + * @param fs + * @param dir + * @param conf + * @param listener + * @throws IOException + */ + public HLog(final FileSystem fs, final Path dir, final Configuration conf, + final LogRollListener listener) throws IOException { + this.fs = fs; + this.dir = dir; + this.conf = conf; + this.listener = listener; + this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000); + this.maxlogentries = + conf.getInt("hbase.regionserver.maxlogentries", 30 * 1000); + if (fs.exists(dir)) { + throw new IOException("Target HLog directory already exists: " + dir); + } + fs.mkdirs(dir); + rollWriter(); + } + + /* + * Accessor for tests. + * @return Current state of the monotonically increasing file id. + */ + long getFilenum() { + return this.filenum; + } + + /** + * Get the compression type for the hlog files. + * @param c Configuration to use. + * @return the kind of compression to use + */ + private static CompressionType getCompressionType(final Configuration c) { + String name = c.get("hbase.io.seqfile.compression.type"); + return name == null? CompressionType.NONE: CompressionType.valueOf(name); + } + + /** + * Called by HRegionServer when it opens a new region to ensure that log + * sequence numbers are always greater than the latest sequence number of the + * region being brought on-line. + * + * @param newvalue We'll set log edit/sequence number to this value if it + * is greater than the current value. + */ + void setSequenceNumber(long newvalue) { + synchronized (sequenceLock) { + if (newvalue > logSeqNum) { + if (LOG.isDebugEnabled()) { + LOG.debug("changing sequence number from " + logSeqNum + " to " + + newvalue); + } + logSeqNum = newvalue; + } + } + } + + /** + * Roll the log writer. That is, start writing log messages to a new file. + * + * Because a log cannot be rolled during a cache flush, and a cache flush + * spans two method calls, a special lock needs to be obtained so that a cache + * flush cannot start when the log is being rolled and the log cannot be + * rolled during a cache flush. + * + *

Note that this method cannot be synchronized because it is possible that + * startCacheFlush runs, obtaining the cacheFlushLock, then this method could + * start which would obtain the lock on this but block on obtaining the + * cacheFlushLock and then completeCacheFlush could be called which would wait + * for the lock on this and consequently never release the cacheFlushLock + * + * @throws IOException + */ + public void rollWriter() throws IOException { + this.cacheFlushLock.lock(); + try { + if (closed) { + return; + } + synchronized (updateLock) { + if (this.writer != null) { + // Close the current writer, get a new one. + this.writer.close(); + Path p = computeFilename(old_filenum); + if (LOG.isDebugEnabled()) { + LOG.debug("Closing current log writer " + FSUtils.getPath(p)); + } + if (filenum > 0) { + synchronized (this.sequenceLock) { + this.outputfiles.put(Long.valueOf(this.logSeqNum - 1), p); + } + } + } + old_filenum = filenum; + filenum = System.currentTimeMillis(); + Path newPath = computeFilename(filenum); + this.writer = SequenceFile.createWriter(this.fs, this.conf, newPath, + HLogKey.class, HLogEdit.class, getCompressionType(this.conf)); + LOG.info("New log writer created at " + FSUtils.getPath(newPath)); + + // Can we delete any of the old log files? + if (this.outputfiles.size() > 0) { + if (this.lastSeqWritten.size() <= 0) { + LOG.debug("Last sequence written is empty. Deleting all old hlogs"); + // If so, then no new writes have come in since all regions were + // flushed (and removed from the lastSeqWritten map). Means can + // remove all but currently open log file. + for (Map.Entry e : this.outputfiles.entrySet()) { + deleteLogFile(e.getValue(), e.getKey()); + } + this.outputfiles.clear(); + } else { + // Get oldest edit/sequence id. If logs are older than this id, + // then safe to remove. + Long oldestOutstandingSeqNum = + Collections.min(this.lastSeqWritten.values()); + // Get the set of all log files whose final ID is older than or + // equal to the oldest pending region operation + TreeSet sequenceNumbers = + new TreeSet(this.outputfiles.headMap( + (Long.valueOf(oldestOutstandingSeqNum.longValue() + 1L))).keySet()); + // Now remove old log files (if any) + if (LOG.isDebugEnabled()) { + // Find region associated with oldest key -- helps debugging. + byte [] oldestRegion = null; + for (Map.Entry e: this.lastSeqWritten.entrySet()) { + if (e.getValue().longValue() == oldestOutstandingSeqNum.longValue()) { + oldestRegion = e.getKey(); + break; + } + } + if (LOG.isDebugEnabled() && sequenceNumbers.size() > 0) { + LOG.debug("Found " + sequenceNumbers.size() + + " logs to remove " + + "using oldest outstanding seqnum of " + + oldestOutstandingSeqNum + " from region " + oldestRegion); + } + } + if (sequenceNumbers.size() > 0) { + for (Long seq : sequenceNumbers) { + deleteLogFile(this.outputfiles.remove(seq), seq); + } + } + } + } + this.numEntries = 0; + } + } finally { + this.cacheFlushLock.unlock(); + } + } + + private void deleteLogFile(final Path p, final Long seqno) throws IOException { + LOG.info("removing old log file " + FSUtils.getPath(p) + + " whose highest sequence/edit id is " + seqno); + this.fs.delete(p, true); + } + + /** + * This is a convenience method that computes a new filename with a given + * file-number. + */ + Path computeFilename(final long fn) { + return new Path(dir, HLOG_DATFILE + fn); + } + + /** + * Shut down the log and delete the log directory + * + * @throws IOException + */ + public void closeAndDelete() throws IOException { + close(); + fs.delete(dir, true); + } + + /** + * Shut down the log. + * + * @throws IOException + */ + void close() throws IOException { + cacheFlushLock.lock(); + try { + synchronized (updateLock) { + if (LOG.isDebugEnabled()) { + LOG.debug("closing log writer in " + this.dir.toString()); + } + this.writer.close(); + this.closed = true; + } + } finally { + cacheFlushLock.unlock(); + } + } + + /** + * Append a set of edits to the log. Log edits are keyed by regionName, + * rowname, and log-sequence-id. + * + * Later, if we sort by these keys, we obtain all the relevant edits for a + * given key-range of the HRegion (TODO). Any edits that do not have a + * matching {@link HConstants#COMPLETE_CACHEFLUSH} message can be discarded. + * + *

+ * Logs cannot be restarted once closed, or once the HLog process dies. Each + * time the HLog starts, it must create a new log. This means that other + * systems should process the log appropriately upon each startup (and prior + * to initializing HLog). + * + * synchronized prevents appends during the completion of a cache flush or for + * the duration of a log roll. + * + * @param regionName + * @param tableName + * @param row + * @param columns + * @param timestamp + * @throws IOException + */ + void append(byte [] regionName, byte [] tableName, + TreeMap edits) + throws IOException { + if (closed) { + throw new IOException("Cannot append; log is closed"); + } + synchronized (updateLock) { + long seqNum[] = obtainSeqNum(edits.size()); + // The 'lastSeqWritten' map holds the sequence number of the oldest + // write for each region. When the cache is flushed, the entry for the + // region being flushed is removed if the sequence number of the flush + // is greater than or equal to the value in lastSeqWritten. + if (!this.lastSeqWritten.containsKey(regionName)) { + this.lastSeqWritten.put(regionName, Long.valueOf(seqNum[0])); + } + int counter = 0; + for (Map.Entry es : edits.entrySet()) { + HStoreKey key = es.getKey(); + HLogKey logKey = + new HLogKey(regionName, tableName, key.getRow(), seqNum[counter++]); + HLogEdit logEdit = + new HLogEdit(key.getColumn(), es.getValue(), key.getTimestamp()); + try { + this.writer.append(logKey, logEdit); + } catch (IOException e) { + LOG.fatal("Could not append. Requesting close of log", e); + requestLogRoll(); + throw e; + } + this.numEntries++; + } + } + if (this.numEntries > this.maxlogentries) { + requestLogRoll(); + } + } + + private void requestLogRoll() { + if (this.listener != null) { + this.listener.logRollRequested(); + } + } + + /** @return How many items have been added to the log */ + int getNumEntries() { + return numEntries; + } + + /** + * Obtain a log sequence number. + */ + private long obtainSeqNum() { + long value; + synchronized (sequenceLock) { + value = logSeqNum++; + } + return value; + } + + /** @return the number of log files in use */ + int getNumLogFiles() { + return outputfiles.size(); + } + + /** + * Obtain a specified number of sequence numbers + * + * @param num number of sequence numbers to obtain + * @return array of sequence numbers + */ + private long[] obtainSeqNum(int num) { + long[] results = new long[num]; + synchronized (this.sequenceLock) { + for (int i = 0; i < num; i++) { + results[i] = this.logSeqNum++; + } + } + return results; + } + + /** + * By acquiring a log sequence ID, we can allow log messages to continue while + * we flush the cache. + * + * Acquire a lock so that we do not roll the log between the start and + * completion of a cache-flush. Otherwise the log-seq-id for the flush will + * not appear in the correct logfile. + * + * @return sequence ID to pass {@link #completeCacheFlush(Text, Text, long)} + * @see #completeCacheFlush(Text, Text, long) + * @see #abortCacheFlush() + */ + long startCacheFlush() { + this.cacheFlushLock.lock(); + return obtainSeqNum(); + } + + /** + * Complete the cache flush + * + * Protected by cacheFlushLock + * + * @param regionName + * @param tableName + * @param logSeqId + * @throws IOException + */ + void completeCacheFlush(final byte [] regionName, final byte [] tableName, + final long logSeqId) throws IOException { + + try { + if (this.closed) { + return; + } + synchronized (updateLock) { + this.writer.append(new HLogKey(regionName, tableName, HLog.METAROW, logSeqId), + new HLogEdit(HLog.METACOLUMN, HLogEdit.completeCacheFlush.get(), + System.currentTimeMillis())); + this.numEntries++; + Long seq = this.lastSeqWritten.get(regionName); + if (seq != null && logSeqId >= seq.longValue()) { + this.lastSeqWritten.remove(regionName); + } + } + } finally { + this.cacheFlushLock.unlock(); + } + } + + /** + * Abort a cache flush. + * Call if the flush fails. Note that the only recovery for an aborted flush + * currently is a restart of the regionserver so the snapshot content dropped + * by the failure gets restored to the memcache. + */ + void abortCacheFlush() { + this.cacheFlushLock.unlock(); + } + + /** + * Split up a bunch of log files, that are no longer being written to, into + * new files, one per region. Delete the old log files when finished. + * + * @param rootDir qualified root directory of the HBase instance + * @param srcDir Directory of log files to split: e.g. + * ${ROOTDIR}/log_HOST_PORT + * @param fs FileSystem + * @param conf HBaseConfiguration + * @throws IOException + */ + public static void splitLog(Path rootDir, Path srcDir, FileSystem fs, + Configuration conf) throws IOException { + if (!fs.exists(srcDir)) { + // Nothing to do + return; + } + FileStatus logfiles[] = fs.listStatus(srcDir); + if (logfiles == null || logfiles.length == 0) { + // Nothing to do + return; + } + LOG.info("splitting " + logfiles.length + " log(s) in " + + srcDir.toString()); + Map logWriters = + new TreeMap(Bytes.BYTES_COMPARATOR); + try { + for (int i = 0; i < logfiles.length; i++) { + if (LOG.isDebugEnabled()) { + LOG.debug("Splitting " + i + " of " + logfiles.length + ": " + + logfiles[i].getPath()); + } + // Check for empty file. + if (logfiles[i].getLen() <= 0) { + LOG.info("Skipping " + logfiles[i].toString() + + " because zero length"); + continue; + } + HLogKey key = new HLogKey(); + HLogEdit val = new HLogEdit(); + SequenceFile.Reader in = + new SequenceFile.Reader(fs, logfiles[i].getPath(), conf); + try { + int count = 0; + for (; in.next(key, val); count++) { + byte [] tableName = key.getTablename(); + byte [] regionName = key.getRegionName(); + SequenceFile.Writer w = logWriters.get(regionName); + if (w == null) { + Path logfile = new Path( + HRegion.getRegionDir( + HTableDescriptor.getTableDir(rootDir, tableName), + HRegionInfo.encodeRegionName(regionName)), + HREGION_OLDLOGFILE_NAME); + Path oldlogfile = null; + SequenceFile.Reader old = null; + if (fs.exists(logfile)) { + LOG.warn("Old log file " + logfile + + " already exists. Copying existing file to new file"); + oldlogfile = new Path(logfile.toString() + ".old"); + fs.rename(logfile, oldlogfile); + old = new SequenceFile.Reader(fs, oldlogfile, conf); + } + w = SequenceFile.createWriter(fs, conf, logfile, HLogKey.class, + HLogEdit.class, getCompressionType(conf)); + // Use copy of regionName; regionName object is reused inside in + // HStoreKey.getRegionName so its content changes as we iterate. + logWriters.put(regionName, w); + if (LOG.isDebugEnabled()) { + LOG.debug("Creating new log file writer for path " + logfile + + " and region " + regionName); + } + + if (old != null) { + // Copy from existing log file + HLogKey oldkey = new HLogKey(); + HLogEdit oldval = new HLogEdit(); + for (; old.next(oldkey, oldval); count++) { + if (LOG.isDebugEnabled() && count > 0 && count % 10000 == 0) { + LOG.debug("Copied " + count + " edits"); + } + w.append(oldkey, oldval); + } + old.close(); + fs.delete(oldlogfile, true); + } + } + w.append(key, val); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Applied " + count + " total edits from " + + logfiles[i].getPath().toString()); + } + } catch (IOException e) { + e = RemoteExceptionHandler.checkIOException(e); + if (!(e instanceof EOFException)) { + LOG.warn("Exception processing " + logfiles[i].getPath() + + " -- continuing. Possible DATA LOSS!", e); + } + } finally { + try { + in.close(); + } catch (IOException e) { + LOG.warn("Close in finally threw exception -- continuing", e); + } + // Delete the input file now so we do not replay edits. We could + // have gotten here because of an exception. If so, probably + // nothing we can do about it. Replaying it, it could work but we + // could be stuck replaying for ever. Just continue though we + // could have lost some edits. + fs.delete(logfiles[i].getPath(), true); + } + } + } finally { + for (SequenceFile.Writer w : logWriters.values()) { + w.close(); + } + } + + try { + fs.delete(srcDir, true); + } catch (IOException e) { + e = RemoteExceptionHandler.checkIOException(e); + IOException io = new IOException("Cannot delete: " + srcDir); + io.initCause(e); + throw io; + } + LOG.info("log file splitting completed for " + srcDir.toString()); + } + + private static void usage() { + System.err.println("Usage: java org.apache.hbase.HLog" + + " {--dump ... | --split ...}"); + } + + /** + * Pass one or more log file names and it will either dump out a text version + * on stdout or split the specified log files. + * + * @param args + * @throws IOException + */ + public static void main(String[] args) throws IOException { + if (args.length < 2) { + usage(); + System.exit(-1); + } + boolean dump = true; + if (args[0].compareTo("--dump") != 0) { + if (args[0].compareTo("--split") == 0) { + dump = false; + + } else { + usage(); + System.exit(-1); + } + } + Configuration conf = new HBaseConfiguration(); + FileSystem fs = FileSystem.get(conf); + Path baseDir = new Path(conf.get(HBASE_DIR)); + + for (int i = 1; i < args.length; i++) { + Path logPath = new Path(args[i]); + if (!fs.exists(logPath)) { + throw new FileNotFoundException(args[i] + " does not exist"); + } + if (dump) { + if (!fs.isFile(logPath)) { + throw new IOException(args[i] + " is not a file"); + } + Reader log = new SequenceFile.Reader(fs, logPath, conf); + try { + HLogKey key = new HLogKey(); + HLogEdit val = new HLogEdit(); + while (log.next(key, val)) { + System.out.println(key.toString() + " " + val.toString()); + } + } finally { + log.close(); + } + } else { + if (!fs.getFileStatus(logPath).isDir()) { + throw new IOException(args[i] + " is not a directory"); + } + splitLog(baseDir, logPath, fs, conf); + } + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HLogEdit.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HLogEdit.java new file mode 100644 index 00000000000..ac21de5592c --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HLogEdit.java @@ -0,0 +1,141 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.*; + +import java.io.*; + +import org.apache.hadoop.hbase.HConstants; + +/** + * A log value. + * + * These aren't sortable; you need to sort by the matching HLogKey. + * The table and row are already identified in HLogKey. + * This just indicates the column and value. + */ +public class HLogEdit implements Writable, HConstants { + + /** Value stored for a deleted item */ + public static ImmutableBytesWritable deleteBytes = null; + + /** Value written to HLog on a complete cache flush */ + public static ImmutableBytesWritable completeCacheFlush = null; + + static { + try { + deleteBytes = + new ImmutableBytesWritable("HBASE::DELETEVAL".getBytes(UTF8_ENCODING)); + + completeCacheFlush = + new ImmutableBytesWritable("HBASE::CACHEFLUSH".getBytes(UTF8_ENCODING)); + + } catch (UnsupportedEncodingException e) { + assert(false); + } + } + + /** + * @param value + * @return True if an entry and its content is {@link #deleteBytes}. + */ + public static boolean isDeleted(final byte [] value) { + return (value == null)? false: deleteBytes.compareTo(value) == 0; + } + + private byte [] column; + private byte [] val; + private long timestamp; + private static final int MAX_VALUE_LEN = 128; + + /** + * Default constructor used by Writable + */ + public HLogEdit() { + super(); + } + + /** + * Construct a fully initialized HLogEdit + * @param c column name + * @param bval value + * @param timestamp timestamp for modification + */ + public HLogEdit(byte [] c, byte [] bval, long timestamp) { + this.column = c; + this.val = bval; + this.timestamp = timestamp; + } + + /** @return the column */ + public byte [] getColumn() { + return this.column; + } + + /** @return the value */ + public byte [] getVal() { + return this.val; + } + + /** @return the timestamp */ + public long getTimestamp() { + return this.timestamp; + } + + /** + * @return First column name, timestamp, and first 128 bytes of the value + * bytes as a String. + */ + @Override + public String toString() { + String value = ""; + try { + value = (this.val.length > MAX_VALUE_LEN)? + new String(this.val, 0, MAX_VALUE_LEN, HConstants.UTF8_ENCODING) + + "...": + new String(getVal(), HConstants.UTF8_ENCODING); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("UTF8 encoding not present?", e); + } + return "(" + Bytes.toString(getColumn()) + "/" + getTimestamp() + "/" + + value + ")"; + } + + // Writable + + /** {@inheritDoc} */ + public void write(DataOutput out) throws IOException { + Bytes.writeByteArray(out, this.column); + out.writeInt(this.val.length); + out.write(this.val); + out.writeLong(timestamp); + } + + /** {@inheritDoc} */ + public void readFields(DataInput in) throws IOException { + this.column = Bytes.readByteArray(in); + this.val = new byte[in.readInt()]; + in.readFully(this.val); + this.timestamp = in.readLong(); + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HLogKey.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HLogKey.java new file mode 100644 index 00000000000..dfead090260 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HLogKey.java @@ -0,0 +1,161 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.*; + +import java.io.*; + +/** + * A Key for an entry in the change log. + * + * The log intermingles edits to many tables and rows, so each log entry + * identifies the appropriate table and row. Within a table and row, they're + * also sorted. + */ +public class HLogKey implements WritableComparable { + private byte [] regionName; + private byte [] tablename; + private byte [] row; + private long logSeqNum; + + /** Create an empty key useful when deserializing */ + public HLogKey() { + this(null, null, null, 0L); + } + + /** + * Create the log key! + * We maintain the tablename mainly for debugging purposes. + * A regionName is always a sub-table object. + * + * @param regionName - name of region + * @param tablename - name of table + * @param row - row key + * @param logSeqNum - log sequence number + */ + public HLogKey(final byte [] regionName, final byte [] tablename, + final byte [] row, long logSeqNum) { + this.regionName = regionName; + this.tablename = tablename; + this.row = row; + this.logSeqNum = logSeqNum; + } + + ////////////////////////////////////////////////////////////////////////////// + // A bunch of accessors + ////////////////////////////////////////////////////////////////////////////// + + byte [] getRegionName() { + return regionName; + } + + byte [] getTablename() { + return tablename; + } + + byte [] getRow() { + return row; + } + + long getLogSeqNum() { + return logSeqNum; + } + + /** + * {@inheritDoc} + */ + @Override + public String toString() { + return Bytes.toString(tablename) + "/" + Bytes.toString(regionName) + "/" + + Bytes.toString(row) + "/" + logSeqNum; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean equals(Object obj) { + return compareTo(obj) == 0; + } + + /** + * {@inheritDoc} + */ + @Override + public int hashCode() { + int result = this.regionName.hashCode(); + result ^= this.row.hashCode(); + result ^= this.logSeqNum; + return result; + } + + // + // Comparable + // + + /** + * {@inheritDoc} + */ + public int compareTo(Object o) { + HLogKey other = (HLogKey) o; + int result = Bytes.compareTo(this.regionName, other.regionName); + + if(result == 0) { + result = Bytes.compareTo(this.row, other.row); + + if(result == 0) { + + if (this.logSeqNum < other.logSeqNum) { + result = -1; + + } else if (this.logSeqNum > other.logSeqNum) { + result = 1; + } + } + } + return result; + } + + // + // Writable + // + + /** + * {@inheritDoc} + */ + public void write(DataOutput out) throws IOException { + Bytes.writeByteArray(out, this.regionName); + Bytes.writeByteArray(out, this.tablename); + Bytes.writeByteArray(out, this.row); + out.writeLong(logSeqNum); + } + + /** + * {@inheritDoc} + */ + public void readFields(DataInput in) throws IOException { + this.regionName = Bytes.readByteArray(in); + this.tablename = Bytes.readByteArray(in); + this.row = Bytes.readByteArray(in); + this.logSeqNum = in.readLong(); + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HRegion.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HRegion.java new file mode 100644 index 00000000000..7efbd7097a2 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HRegion.java @@ -0,0 +1,2153 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ColumnNameParseException; +import org.apache.hadoop.hbase.DroppedSnapshotException; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.NotServingRegionException; +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.io.BatchOperation; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.HbaseMapWritable; +import org.apache.hadoop.hbase.io.RowResult; +import org.apache.hadoop.hbase.ipc.HRegionInterface; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; +import org.apache.hadoop.hbase.regionserver.WrongRegionException; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.StringUtils; + +/** + * HRegion stores data for a certain region of a table. It stores all columns + * for each row. A given table consists of one or more HRegions. + * + *

We maintain multiple HStores for a single HRegion. + * + *

An HStore is a set of rows with some column data; together, + * they make up all the data for the rows. + * + *

Each HRegion has a 'startKey' and 'endKey'. + *

The first is inclusive, the second is exclusive (except for + * the final region) The endKey of region 0 is the same as + * startKey for region 1 (if it exists). The startKey for the + * first region is null. The endKey for the final region is null. + * + *

Locking at the HRegion level serves only one purpose: preventing the + * region from being closed (and consequently split) while other operations + * are ongoing. Each row level operation obtains both a row lock and a region + * read lock for the duration of the operation. While a scanner is being + * constructed, getScanner holds a read lock. If the scanner is successfully + * constructed, it holds a read lock until it is closed. A close takes out a + * write lock and consequently will block for ongoing operations and will block + * new operations from starting while the close is in progress. + * + *

An HRegion is defined by its table and its key extent. + * + *

It consists of at least one HStore. The number of HStores should be + * configurable, so that data which is accessed together is stored in the same + * HStore. Right now, we approximate that by building a single HStore for + * each column family. (This config info will be communicated via the + * tabledesc.) + * + *

The HTableDescriptor contains metainfo about the HRegion's table. + * regionName is a unique identifier for this HRegion. (startKey, endKey] + * defines the keyspace for this HRegion. + */ +public class HRegion implements HConstants { + static final String SPLITDIR = "splits"; + static final String MERGEDIR = "merges"; + static final Random rand = new Random(); + static final Log LOG = LogFactory.getLog(HRegion.class); + final AtomicBoolean closed = new AtomicBoolean(false); + private final RegionHistorian historian; + + /** + * Merge two HRegions. The regions must be adjacent andmust not overlap. + * + * @param srcA + * @param srcB + * @return new merged HRegion + * @throws IOException + */ + public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB) + throws IOException { + + HRegion a = srcA; + HRegion b = srcB; + + // Make sure that srcA comes first; important for key-ordering during + // write of the merged file. + if (srcA.getStartKey() == null) { + if (srcB.getStartKey() == null) { + throw new IOException("Cannot merge two regions with null start key"); + } + // A's start key is null but B's isn't. Assume A comes before B + } else if ((srcB.getStartKey() == null) // A is not null but B is + || (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) { // A > B + a = srcB; + b = srcA; + } + + if (!Bytes.equals(a.getEndKey(), b.getStartKey())) { + throw new IOException("Cannot merge non-adjacent regions"); + } + return merge(a, b); + } + + /** + * Merge two regions whether they are adjacent or not. + * + * @param a region a + * @param b region b + * @return new merged region + * @throws IOException + */ + public static HRegion merge(HRegion a, HRegion b) throws IOException { + if (!a.getRegionInfo().getTableDesc().getNameAsString().equals( + b.getRegionInfo().getTableDesc().getNameAsString())) { + throw new IOException("Regions do not belong to the same table"); + } + FileSystem fs = a.getFilesystem(); + + // Make sure each region's cache is empty + + a.flushcache(); + b.flushcache(); + + // Compact each region so we only have one store file per family + + a.compactStores(true); + if (LOG.isDebugEnabled()) { + LOG.debug("Files for region: " + a); + listPaths(fs, a.getRegionDir()); + } + b.compactStores(true); + if (LOG.isDebugEnabled()) { + LOG.debug("Files for region: " + b); + listPaths(fs, b.getRegionDir()); + } + + HBaseConfiguration conf = a.getConf(); + HTableDescriptor tabledesc = a.getTableDesc(); + HLog log = a.getLog(); + Path basedir = a.getBaseDir(); + final byte [] startKey = Bytes.equals(a.getStartKey(), EMPTY_BYTE_ARRAY) || + Bytes.equals(b.getStartKey(), EMPTY_BYTE_ARRAY) ? EMPTY_BYTE_ARRAY : + Bytes.compareTo(a.getStartKey(), b.getStartKey()) <= 0 ? + a.getStartKey() : b.getStartKey(); + final byte [] endKey = Bytes.equals(a.getEndKey(), EMPTY_BYTE_ARRAY) || + Bytes.equals(b.getEndKey(), EMPTY_BYTE_ARRAY) ? EMPTY_BYTE_ARRAY : + Bytes.compareTo(a.getEndKey(), b.getEndKey()) <= 0 ? + b.getEndKey() : a.getEndKey(); + + HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey); + LOG.info("Creating new region " + newRegionInfo.toString()); + int encodedName = newRegionInfo.getEncodedName(); + Path newRegionDir = HRegion.getRegionDir(a.getBaseDir(), encodedName); + if(fs.exists(newRegionDir)) { + throw new IOException("Cannot merge; target file collision at " + + newRegionDir); + } + fs.mkdirs(newRegionDir); + + LOG.info("starting merge of regions: " + a + " and " + b + + " into new region " + newRegionInfo.toString() + + " with start key <" + startKey + "> and end key <" + endKey + ">"); + + // Move HStoreFiles under new region directory + + Map> byFamily = + new TreeMap>(Bytes.BYTES_COMPARATOR); + byFamily = filesByFamily(byFamily, a.close()); + byFamily = filesByFamily(byFamily, b.close()); + for (Map.Entry> es : byFamily.entrySet()) { + byte [] colFamily = es.getKey(); + makeColumnFamilyDirs(fs, basedir, encodedName, colFamily, tabledesc); + + // Because we compacted the source regions we should have no more than two + // HStoreFiles per family and there will be no reference store + List srcFiles = es.getValue(); + if (srcFiles.size() == 2) { + long seqA = srcFiles.get(0).loadInfo(fs); + long seqB = srcFiles.get(1).loadInfo(fs); + if (seqA == seqB) { + // We can't have duplicate sequence numbers + if (LOG.isDebugEnabled()) { + LOG.debug("Adjusting sequence id of storeFile " + srcFiles.get(1) + + " down by one; sequence id A=" + seqA + ", sequence id B=" + + seqB); + } + srcFiles.get(1).writeInfo(fs, seqB - 1); + } + } + for (HStoreFile hsf: srcFiles) { + HStoreFile dst = new HStoreFile(conf, fs, basedir, + newRegionInfo.getEncodedName(), colFamily, -1, null); + if (LOG.isDebugEnabled()) { + LOG.debug("Renaming " + hsf + " to " + dst); + } + hsf.rename(fs, dst); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("Files for new region"); + listPaths(fs, newRegionDir); + } + HRegion dstRegion = new HRegion(basedir, log, fs, conf, newRegionInfo, + null, null); + dstRegion.compactStores(); + if (LOG.isDebugEnabled()) { + LOG.debug("Files for new region"); + listPaths(fs, dstRegion.getRegionDir()); + } + deleteRegion(fs, a.getRegionDir()); + deleteRegion(fs, b.getRegionDir()); + + LOG.info("merge completed. New region is " + dstRegion); + + return dstRegion; + } + + /* + * Fills a map with a vector of store files keyed by column family. + * @param byFamily Map to fill. + * @param storeFiles Store files to process. + * @return Returns byFamily + */ + private static Map> filesByFamily( + Map> byFamily, List storeFiles) { + for (HStoreFile src: storeFiles) { + List v = byFamily.get(src.getColFamily()); + if (v == null) { + v = new ArrayList(); + byFamily.put(src.getColFamily(), v); + } + v.add(src); + } + return byFamily; + } + + /* + * Method to list files in use by region + */ + static void listFiles(FileSystem fs, HRegion r) throws IOException { + listPaths(fs, r.getRegionDir()); + } + + /* + * List the files under the specified directory + * + * @param fs + * @param dir + * @throws IOException + */ + private static void listPaths(FileSystem fs, Path dir) throws IOException { + if (LOG.isDebugEnabled()) { + FileStatus[] stats = fs.listStatus(dir); + if (stats == null || stats.length == 0) { + return; + } + for (int i = 0; i < stats.length; i++) { + String path = stats[i].getPath().toString(); + if (stats[i].isDir()) { + LOG.debug("d " + path); + listPaths(fs, stats[i].getPath()); + } else { + LOG.debug("f " + path + " size=" + stats[i].getLen()); + } + } + } + } + + ////////////////////////////////////////////////////////////////////////////// + // Members + ////////////////////////////////////////////////////////////////////////////// + + private final Map locksToRows = + new ConcurrentHashMap(); + private final Map> targetColumns = + new ConcurrentHashMap>(); + private volatile boolean flushRequested; + // Default access because read by tests. + final Map stores = new ConcurrentHashMap(); + final AtomicLong memcacheSize = new AtomicLong(0); + + final Path basedir; + final HLog log; + final FileSystem fs; + final HBaseConfiguration conf; + final HRegionInfo regionInfo; + final Path regiondir; + private final Path regionCompactionDir; + + /* + * Data structure of write state flags used coordinating flushes, + * compactions and closes. + */ + static class WriteState { + // Set while a memcache flush is happening. + volatile boolean flushing = false; + // Set while a compaction is running. + volatile boolean compacting = false; + // Gets set in close. If set, cannot compact or flush again. + volatile boolean writesEnabled = true; + } + + private volatile WriteState writestate = new WriteState(); + + final int memcacheFlushSize; + private volatile long lastFlushTime; + final FlushRequester flushListener; + private final int blockingMemcacheSize; + final long threadWakeFrequency; + // Used to guard splits and closes + private final ReentrantReadWriteLock splitsAndClosesLock = + new ReentrantReadWriteLock(); + // Stop updates lock + private final ReentrantReadWriteLock updatesLock = + new ReentrantReadWriteLock(); + private final Integer splitLock = new Integer(0); + private final long minSequenceId; + final AtomicInteger activeScannerCount = new AtomicInteger(0); + + ////////////////////////////////////////////////////////////////////////////// + // Constructor + ////////////////////////////////////////////////////////////////////////////// + + /** + * HRegion constructor. + * + * @param basedir qualified path of directory where region should be located, + * usually the table directory. + * @param log The HLog is the outbound log for any updates to the HRegion + * (There's a single HLog for all the HRegions on a single HRegionServer.) + * The log file is a logfile from the previous execution that's + * custom-computed for this HRegion. The HRegionServer computes and sorts the + * appropriate log info for this HRegion. If there is a previous log file + * (implying that the HRegion has been written-to before), then read it from + * the supplied path. + * @param fs is the filesystem. + * @param conf is global configuration settings. + * @param regionInfo - HRegionInfo that describes the region + * @param initialFiles If there are initial files (implying that the HRegion + * is new), then read them from the supplied path. + * @param flushListener an object that implements CacheFlushListener or null + * or null + * @throws IOException + */ + public HRegion(Path basedir, HLog log, FileSystem fs, HBaseConfiguration conf, + HRegionInfo regionInfo, Path initialFiles, + FlushRequester flushListener) throws IOException { + this(basedir, log, fs, conf, regionInfo, initialFiles, flushListener, null); + } + + /** + * HRegion constructor. + * + * @param log The HLog is the outbound log for any updates to the HRegion + * (There's a single HLog for all the HRegions on a single HRegionServer.) + * The log file is a logfile from the previous execution that's + * custom-computed for this HRegion. The HRegionServer computes and sorts the + * appropriate log info for this HRegion. If there is a previous log file + * (implying that the HRegion has been written-to before), then read it from + * the supplied path. + * @param basedir qualified path of directory where region should be located, + * usually the table directory. + * @param fs is the filesystem. + * @param conf is global configuration settings. + * @param regionInfo - HRegionInfo that describes the region + * @param initialFiles If there are initial files (implying that the HRegion + * is new), then read them from the supplied path. + * @param flushListener an object that implements CacheFlushListener or null + * @param reporter Call on a period so hosting server can report we're + * making progress to master -- otherwise master might think region deploy + * failed. Can be null. + * @throws IOException + */ + public HRegion(Path basedir, HLog log, FileSystem fs, HBaseConfiguration conf, + HRegionInfo regionInfo, Path initialFiles, + FlushRequester flushListener, final Progressable reporter) + throws IOException { + + this.basedir = basedir; + this.log = log; + this.fs = fs; + this.conf = conf; + this.regionInfo = regionInfo; + this.flushListener = flushListener; + this.flushRequested = false; + this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000); + String encodedNameStr = Integer.toString(this.regionInfo.getEncodedName()); + this.regiondir = new Path(basedir, encodedNameStr); + Path oldLogFile = new Path(regiondir, HREGION_OLDLOGFILE_NAME); + this.historian = RegionHistorian.getInstance(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Opening region " + this + "/" + + this.regionInfo.getEncodedName()); + } + + this.regionCompactionDir = + new Path(getCompactionDir(basedir), encodedNameStr); + + // Move prefab HStore files into place (if any). This picks up split files + // and any merges from splits and merges dirs. + if (initialFiles != null && fs.exists(initialFiles)) { + fs.rename(initialFiles, this.regiondir); + } + + // Load in all the HStores. + long maxSeqId = -1; + for (HColumnDescriptor c : this.regionInfo.getTableDesc().getFamilies()) { + HStore store = instantiateHStore(this.basedir, c, oldLogFile, reporter); + stores.put(Bytes.mapKey(c.getName()), store); + long storeSeqId = store.getMaxSequenceId(); + if (storeSeqId > maxSeqId) { + maxSeqId = storeSeqId; + } + } + + doReconstructionLog(oldLogFile, maxSeqId, reporter); + + if (fs.exists(oldLogFile)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Deleting old log file: " + oldLogFile); + } + fs.delete(oldLogFile, false); + } + + // Add one to the current maximum sequence id so new edits are beyond. + this.minSequenceId = maxSeqId + 1; + if (LOG.isDebugEnabled()) { + LOG.debug("Next sequence id for region " + + Bytes.toString(regionInfo.getRegionName()) + " is " + + this.minSequenceId); + } + + // Get rid of any splits or merges that were lost in-progress + Path splits = new Path(regiondir, SPLITDIR); + if (fs.exists(splits)) { + fs.delete(splits, true); + } + Path merges = new Path(regiondir, MERGEDIR); + if (fs.exists(merges)) { + fs.delete(merges, true); + } + + // By default, we flush the cache when 64M. + this.memcacheFlushSize = conf.getInt("hbase.hregion.memcache.flush.size", + 1024*1024*64); + + this.blockingMemcacheSize = this.memcacheFlushSize * + conf.getInt("hbase.hregion.memcache.block.multiplier", 1); + + // HRegion is ready to go! + this.writestate.compacting = false; + this.lastFlushTime = System.currentTimeMillis(); + LOG.info("region " + this + "/" + this.regionInfo.getEncodedName() + + " available"); + } + + /** + * @return Updates to this region need to have a sequence id that is >= to + * the this number. + */ + long getMinSequenceId() { + return this.minSequenceId; + } + + /** @return a HRegionInfo object for this region */ + public HRegionInfo getRegionInfo() { + return this.regionInfo; + } + + /** @return true if region is closed */ + public boolean isClosed() { + return this.closed.get(); + } + + /** + * Close down this HRegion. Flush the cache, shut down each HStore, don't + * service any more calls. + * + *

This method could take some time to execute, so don't call it from a + * time-sensitive thread. + * + * @return Vector of all the storage files that the HRegion's component + * HStores make use of. It's a list of all HStoreFile objects. Returns empty + * vector if already closed and null if judged that it should not close. + * + * @throws IOException + */ + public List close() throws IOException { + return close(false); + } + + /** + * Close down this HRegion. Flush the cache unless abort parameter is true, + * Shut down each HStore, don't service any more calls. + * + * This method could take some time to execute, so don't call it from a + * time-sensitive thread. + * + * @param abort true if server is aborting (only during testing) + * @return Vector of all the storage files that the HRegion's component + * HStores make use of. It's a list of HStoreFile objects. Can be null if + * we are not to close at this time or we are already closed. + * + * @throws IOException + */ + List close(boolean abort) throws IOException { + if (isClosed()) { + LOG.warn("region " + this + " already closed"); + return null; + } + synchronized (splitLock) { + synchronized (writestate) { + // Disable compacting and flushing by background threads for this + // region. + writestate.writesEnabled = false; + LOG.debug("Compactions and cache flushes disabled for region " + this); + while (writestate.compacting || writestate.flushing) { + LOG.debug("waiting for" + + (writestate.compacting ? " compaction" : "") + + (writestate.flushing ? + (writestate.compacting ? "," : "") + " cache flush" : + "") + " to complete for region " + this); + try { + writestate.wait(); + } catch (InterruptedException iex) { + // continue + } + } + } + splitsAndClosesLock.writeLock().lock(); + LOG.debug("Updates and scanners disabled for region " + this); + try { + // Wait for active scanners to finish. The write lock we hold will + // prevent new scanners from being created. + synchronized (activeScannerCount) { + while (activeScannerCount.get() != 0) { + LOG.debug("waiting for " + activeScannerCount.get() + + " scanners to finish"); + try { + activeScannerCount.wait(); + } catch (InterruptedException e) { + // continue + } + } + } + LOG.debug("No more active scanners for region " + this); + + // Write lock means no more row locks can be given out. Wait on + // outstanding row locks to come in before we close so we do not drop + // outstanding updates. + waitOnRowLocks(); + LOG.debug("No more row locks outstanding on region " + this); + + // Don't flush the cache if we are aborting + if (!abort) { + internalFlushcache(); + } + + List result = new ArrayList(); + for (HStore store: stores.values()) { + result.addAll(store.close()); + } + this.closed.set(true); + + LOG.info("closed " + this); + return result; + } finally { + splitsAndClosesLock.writeLock().unlock(); + } + } + } + + ////////////////////////////////////////////////////////////////////////////// + // HRegion accessors + ////////////////////////////////////////////////////////////////////////////// + + /** @return start key for region */ + public byte [] getStartKey() { + return this.regionInfo.getStartKey(); + } + + /** @return end key for region */ + public byte [] getEndKey() { + return this.regionInfo.getEndKey(); + } + + /** @return region id */ + public long getRegionId() { + return this.regionInfo.getRegionId(); + } + + /** @return region name */ + public byte [] getRegionName() { + return this.regionInfo.getRegionName(); + } + + /** @return HTableDescriptor for this region */ + public HTableDescriptor getTableDesc() { + return this.regionInfo.getTableDesc(); + } + + /** @return HLog in use for this region */ + public HLog getLog() { + return this.log; + } + + /** @return Configuration object */ + public HBaseConfiguration getConf() { + return this.conf; + } + + /** @return region directory Path */ + public Path getRegionDir() { + return this.regiondir; + } + + /** @return FileSystem being used by this region */ + public FileSystem getFilesystem() { + return this.fs; + } + + /** @return the last time the region was flushed */ + public long getLastFlushTime() { + return this.lastFlushTime; + } + + /** @param t the lastFlushTime */ + void setLastFlushTime(long t) { + this.lastFlushTime = t; + } + + ////////////////////////////////////////////////////////////////////////////// + // HRegion maintenance. + // + // These methods are meant to be called periodically by the HRegionServer for + // upkeep. + ////////////////////////////////////////////////////////////////////////////// + + /** @return returns size of largest HStore. */ + public long getLargestHStoreSize() { + long size = 0; + for (HStore h: stores.values()) { + long storeSize = h.getSize(); + if (storeSize > size) { + size = storeSize; + } + } + return size; + } + + /* + * Split the HRegion to create two brand-new ones. This also closes + * current HRegion. Split should be fast since we don't rewrite store files + * but instead create new 'reference' store files that read off the top and + * bottom ranges of parent store files. + * @param midKey key on which to split region + * @return two brand-new (and open) HRegions or null if a split is not needed + * @throws IOException + */ + HRegion[] splitRegion(final byte [] midKey) throws IOException { + synchronized (splitLock) { + if (closed.get()) { + return null; + } + // Add start/end key checking: hbase-428. + byte [] startKey = this.regionInfo.getStartKey(); + byte [] endKey = this.regionInfo.getEndKey(); + if (Bytes.equals(startKey, midKey)) { + LOG.debug("Startkey (" + startKey + ") and midkey + (" + + midKey + ") are same, not splitting"); + return null; + } + if (Bytes.equals(midKey, endKey)) { + LOG.debug("Endkey and midkey are same, not splitting"); + return null; + } + LOG.info("Starting split of region " + this); + Path splits = new Path(this.regiondir, SPLITDIR); + if(!this.fs.exists(splits)) { + this.fs.mkdirs(splits); + } + // Calculate regionid to use. Can't be less than that of parent else + // it'll insert into wrong location over in .META. table: HBASE-710. + long rid = System.currentTimeMillis(); + if (rid < this.regionInfo.getRegionId()) { + LOG.warn("Clock skew; parent regions id is " + + this.regionInfo.getRegionId() + " but current time here is " + rid); + rid = this.regionInfo.getRegionId() + 1; + } + HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(), + startKey, midKey, false, rid); + Path dirA = + new Path(splits, Integer.toString(regionAInfo.getEncodedName())); + if(fs.exists(dirA)) { + throw new IOException("Cannot split; target file collision at " + dirA); + } + HRegionInfo regionBInfo = new HRegionInfo(this.regionInfo.getTableDesc(), + midKey, endKey, false, rid); + Path dirB = + new Path(splits, Integer.toString(regionBInfo.getEncodedName())); + if(this.fs.exists(dirB)) { + throw new IOException("Cannot split; target file collision at " + dirB); + } + + // Now close the HRegion. Close returns all store files or null if not + // supposed to close (? What to do in this case? Implement abort of close?) + // Close also does wait on outstanding rows and calls a flush just-in-case. + List hstoreFilesToSplit = close(false); + if (hstoreFilesToSplit == null) { + LOG.warn("Close came back null (Implement abort of close?)"); + throw new RuntimeException("close returned empty vector of HStoreFiles"); + } + + // Split each store file. + for(HStoreFile h: hstoreFilesToSplit) { + // A reference to the bottom half of the hsf store file. + HStoreFile.Reference aReference = new HStoreFile.Reference( + this.regionInfo.getEncodedName(), h.getFileId(), + new HStoreKey(midKey), HStoreFile.Range.bottom); + HStoreFile a = new HStoreFile(this.conf, fs, splits, + regionAInfo.getEncodedName(), h.getColFamily(), -1, aReference); + // Reference to top half of the hsf store file. + HStoreFile.Reference bReference = new HStoreFile.Reference( + this.regionInfo.getEncodedName(), h.getFileId(), + new HStoreKey(midKey), HStoreFile.Range.top); + HStoreFile b = new HStoreFile(this.conf, fs, splits, + regionBInfo.getEncodedName(), h.getColFamily(), -1, bReference); + h.splitStoreFile(a, b, this.fs); + } + + // Done! + // Opening the region copies the splits files from the splits directory + // under each region. + HRegion regionA = + new HRegion(basedir, log, fs, conf, regionAInfo, dirA, null); + regionA.close(); + HRegion regionB = + new HRegion(basedir, log, fs, conf, regionBInfo, dirB, null); + regionB.close(); + + // Cleanup + boolean deleted = fs.delete(splits, true); // Get rid of splits directory + if (LOG.isDebugEnabled()) { + LOG.debug("Cleaned up " + FSUtils.getPath(splits) + " " + deleted); + } + HRegion regions[] = new HRegion [] {regionA, regionB}; + + this.historian.addRegionSplit(this.regionInfo, + regionA.getRegionInfo(), regionB.getRegionInfo()); + + return regions; + } + } + + /* + * @param dir + * @return compaction directory for the passed in dir + */ + static Path getCompactionDir(final Path dir) { + return new Path(dir, "compaction.dir"); + } + + /* + * Do preparation for pending compaction. + * Clean out any vestiges of previous failed compactions. + * @throws IOException + */ + private void doRegionCompactionPrep() throws IOException { + doRegionCompactionCleanup(); + } + + /* + * Removes the compaction directory for this Store. + * @throws IOException + */ + private void doRegionCompactionCleanup() throws IOException { + if (this.fs.exists(this.regionCompactionDir)) { + this.fs.delete(this.regionCompactionDir, true); + } + } + + /** + * Called by compaction thread and after region is opened to compact the + * HStores if necessary. + * + *

This operation could block for a long time, so don't call it from a + * time-sensitive thread. + * + * Note that no locking is necessary at this level because compaction only + * conflicts with a region split, and that cannot happen because the region + * server does them sequentially and not in parallel. + * + * @return mid key if split is needed + * @throws IOException + */ + public byte [] compactStores() throws IOException { + return compactStores(false); + } + + /** + * Called by compaction thread and after region is opened to compact the + * HStores if necessary. + * + *

This operation could block for a long time, so don't call it from a + * time-sensitive thread. + * + * Note that no locking is necessary at this level because compaction only + * conflicts with a region split, and that cannot happen because the region + * server does them sequentially and not in parallel. + * + * @param force True to force a compaction regardless of thresholds (Needed + * by merge). + * @return mid key if split is needed + * @throws IOException + */ + private byte [] compactStores(final boolean force) throws IOException { + byte [] midKey = null; + if (this.closed.get()) { + return midKey; + } + try { + synchronized (writestate) { + if (!writestate.compacting && writestate.writesEnabled) { + writestate.compacting = true; + } else { + LOG.info("NOT compacting region " + this + + ": compacting=" + writestate.compacting + ", writesEnabled=" + + writestate.writesEnabled); + return midKey; + } + } + LOG.info("starting compaction on region " + this); + long startTime = System.currentTimeMillis(); + doRegionCompactionPrep(); + long maxSize = -1; + for (HStore store: stores.values()) { + final HStore.StoreSize size = store.compact(force); + if (size != null && size.getSize() > maxSize) { + maxSize = size.getSize(); + midKey = size.getKey(); + } + } + doRegionCompactionCleanup(); + String timeTaken = StringUtils.formatTimeDiff(System.currentTimeMillis(), + startTime); + LOG.info("compaction completed on region " + this + " in " + timeTaken); + + this.historian.addRegionCompaction(regionInfo, timeTaken); + } finally { + synchronized (writestate) { + writestate.compacting = false; + writestate.notifyAll(); + } + } + return midKey; + } + + /** + * Flush the cache. + * + * When this method is called the cache will be flushed unless: + *

    + *
  1. the cache is empty
  2. + *
  3. the region is closed.
  4. + *
  5. a flush is already in progress
  6. + *
  7. writes are disabled
  8. + *
+ * + *

This method may block for some time, so it should not be called from a + * time-sensitive thread. + * + * @return true if cache was flushed + * + * @throws IOException + * @throws DroppedSnapshotException Thrown when replay of hlog is required + * because a Snapshot was not properly persisted. + */ + public boolean flushcache() throws IOException { + if (this.closed.get()) { + return false; + } + synchronized (writestate) { + if (!writestate.flushing && writestate.writesEnabled) { + writestate.flushing = true; + } else { + if(LOG.isDebugEnabled()) { + LOG.debug("NOT flushing memcache for region " + this + + ", flushing=" + + writestate.flushing + ", writesEnabled=" + + writestate.writesEnabled); + } + return false; + } + } + try { + // Prevent splits and closes + splitsAndClosesLock.readLock().lock(); + try { + return internalFlushcache(); + } finally { + splitsAndClosesLock.readLock().unlock(); + } + } finally { + synchronized (writestate) { + writestate.flushing = false; + writestate.notifyAll(); + } + } + } + + /** + * Flushing the cache is a little tricky. We have a lot of updates in the + * HMemcache, all of which have also been written to the log. We need to + * write those updates in the HMemcache out to disk, while being able to + * process reads/writes as much as possible during the flush operation. Also, + * the log has to state clearly the point in time at which the HMemcache was + * flushed. (That way, during recovery, we know when we can rely on the + * on-disk flushed structures and when we have to recover the HMemcache from + * the log.) + * + *

So, we have a three-step process: + * + *

+ *

This method is protected, but can be accessed via several public + * routes. + * + *

This method may block for some time. + * + * @return true if the region needs compacting + * + * @throws IOException + * @throws DroppedSnapshotException Thrown when replay of hlog is required + * because a Snapshot was not properly persisted. + */ + private boolean internalFlushcache() throws IOException { + final long startTime = System.currentTimeMillis(); + + // Clear flush flag. + this.flushRequested = false; + + // Record latest flush time + this.lastFlushTime = startTime; + + if (LOG.isDebugEnabled()) { + LOG.debug("Started memcache flush for region " + this + + ". Current region memcache size " + + StringUtils.humanReadableInt(this.memcacheSize.get())); + } + + // Stop updates while we snapshot the memcache of all stores. We only have + // to do this for a moment. Its quick. The subsequent sequence id that + // goes into the HLog after we've flushed all these snapshots also goes + // into the info file that sits beside the flushed files. + // We also set the memcache size to zero here before we allow updates + // again so its value will represent the size of the updates received + // during the flush + long sequenceId = -1L; + this.updatesLock.writeLock().lock(); + try { + for (HStore s: stores.values()) { + s.snapshot(); + } + sequenceId = log.startCacheFlush(); + this.memcacheSize.set(0); + } finally { + this.updatesLock.writeLock().unlock(); + } + + // Any failure from here on out will be catastrophic requiring server + // restart so hlog content can be replayed and put back into the memcache. + // Otherwise, the snapshot content while backed up in the hlog, it will not + // be part of the current running servers state. + boolean compactionRequested = false; + try { + // A. Flush memcache to all the HStores. + // Keep running vector of all store files that includes both old and the + // just-made new flush store file. + for (HStore hstore: stores.values()) { + boolean needsCompaction = hstore.flushCache(sequenceId); + if (needsCompaction) { + compactionRequested = true; + } + } + } catch (Throwable t) { + // An exception here means that the snapshot was not persisted. + // The hlog needs to be replayed so its content is restored to memcache. + // Currently, only a server restart will do this. + // We used to only catch IOEs but its possible that we'd get other + // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch + // all and sundry. + this.log.abortCacheFlush(); + DroppedSnapshotException dse = new DroppedSnapshotException("region: " + + Bytes.toString(getRegionName())); + dse.initCause(t); + throw dse; + } + + // If we get to here, the HStores have been written. If we get an + // error in completeCacheFlush it will release the lock it is holding + + // B. Write a FLUSHCACHE-COMPLETE message to the log. + // This tells future readers that the HStores were emitted correctly, + // and that all updates to the log for this regionName that have lower + // log-sequence-ids can be safely ignored. + this.log.completeCacheFlush(getRegionName(), + regionInfo.getTableDesc().getName(), sequenceId); + + // C. Finally notify anyone waiting on memcache to clear: + // e.g. checkResources(). + synchronized (this) { + notifyAll(); + } + + if (LOG.isDebugEnabled()) { + String timeTaken = StringUtils.formatTimeDiff(System.currentTimeMillis(), + startTime); + LOG.debug("Finished memcache flush for region " + this + + " in " + + (System.currentTimeMillis() - startTime) + "ms, sequence id=" + + sequenceId + ", compaction requested=" + compactionRequested); + if (!regionInfo.isMetaRegion()) { + this.historian.addRegionFlush(regionInfo, timeTaken); + } + } + return compactionRequested; + } + + ////////////////////////////////////////////////////////////////////////////// + // get() methods for client use. + ////////////////////////////////////////////////////////////////////////////// + + /** + * Fetch a single data item. + * @param row + * @param column + * @return column value + * @throws IOException + */ + public Cell get(byte [] row, byte [] column) throws IOException { + Cell[] results = get(row, column, Long.MAX_VALUE, 1); + return (results == null || results.length == 0)? null: results[0]; + } + /** + * Fetch multiple versions of a single data item + * + * @param row + * @param column + * @param numVersions + * @return array of values one element per version + * @throws IOException + */ + public Cell[] get(byte [] row, byte [] column, int numVersions) + throws IOException { + return get(row, column, Long.MAX_VALUE, numVersions); + } + + /** + * Fetch multiple versions of a single data item, with timestamp. + * + * @param row + * @param column + * @param timestamp + * @param numVersions + * @return array of values one element per version that matches the timestamp + * @throws IOException + */ + public Cell[] get(byte [] row, byte [] column, long timestamp, + int numVersions) + throws IOException { + if (this.closed.get()) { + throw new IOException("Region " + this + " closed"); + } + + // Make sure this is a valid row and valid column + checkRow(row); + checkColumn(column); + // Don't need a row lock for a simple get + HStoreKey key = new HStoreKey(row, column, timestamp); + return getStore(column).get(key, numVersions); + } + + /** + * Fetch all the columns for the indicated row at a specified timestamp. + * Returns a TreeMap that maps column names to values. + * + * We should eventually use Bloom filters here, to reduce running time. If + * the database has many column families and is very sparse, then we could be + * checking many files needlessly. A small Bloom for each row would help us + * determine which column groups are useful for that row. That would let us + * avoid a bunch of disk activity. + * + * @param row + * @param columns Array of columns you'd like to retrieve. When null, get all. + * @param ts + * @return Map values + * @throws IOException + */ + public Map getFull(final byte [] row, + final Set columns, final long ts) + throws IOException { + // Check columns passed + if (columns != null) { + for (byte [] column: columns) { + checkColumn(column); + } + } + HStoreKey key = new HStoreKey(row, ts); + Integer lid = obtainRowLock(row); + try { + TreeMap result = + new TreeMap(Bytes.BYTES_COMPARATOR); + for (HStore targetStore: stores.values()) { + targetStore.getFull(key, columns, result); + } + // Previous step won't fetch whole families: HBASE-631. + // For each column name that is just a column family, open the store + // related to it and fetch everything for that row. + if (columns != null) { + for (byte[] bs : columns) { + if (HStoreKey.getFamilyDelimiterIndex(bs) == (bs.length - 1)) { + HStore store = stores.get(Bytes.mapKey(HStoreKey.getFamily(bs))); + store.getFull(key, null, result); + } + } + } + return result; + } finally { + releaseRowLock(lid); + } + } + + /** + * Return all the data for the row that matches row exactly, + * or the one that immediately preceeds it, at or immediately before + * ts. + * + * @param row row key + * @return map of values + * @throws IOException + */ + public RowResult getClosestRowBefore(final byte [] row) + throws IOException{ + // look across all the HStores for this region and determine what the + // closest key is across all column families, since the data may be sparse + HStoreKey key = null; + checkRow(row); + splitsAndClosesLock.readLock().lock(); + try { + // examine each column family for the preceeding or matching key + for (HStore store : stores.values()) { + // get the closest key + byte [] closestKey = store.getRowKeyAtOrBefore(row); + // if it happens to be an exact match, we can stop looping + if (Bytes.equals(row, closestKey)) { + key = new HStoreKey(closestKey); + break; + } + // otherwise, we need to check if it's the max and move to the next + if (closestKey != null + && (key == null || Bytes.compareTo(closestKey, key.getRow()) > 0) ) { + key = new HStoreKey(closestKey); + } + } + if (key == null) { + return null; + } + + // now that we've found our key, get the values + HbaseMapWritable cells = + new HbaseMapWritable(); + for (HStore s: stores.values()) { + s.getFull(key, null, cells); + } + return new RowResult(key.getRow(), cells); + } finally { + splitsAndClosesLock.readLock().unlock(); + } + } + + /** + * Get versions keys matching the origin key's + * row/column/timestamp and those of an older vintage + * Default access so can be accessed out of {@link HRegionServer}. + * @param origin Where to start searching. + * @param versions How many versions to return. Pass + * {@link HConstants.ALL_VERSIONS} to retrieve all. + * @return Ordered list of versions keys going from newest back. + * @throws IOException + */ + private Set getKeys(final HStoreKey origin, final int versions) + throws IOException { + Set keys = new TreeSet(); + Collection storesToCheck = null; + if (origin.getColumn() == null || origin.getColumn().length == 0) { + // All families + storesToCheck = this.stores.values(); + } else { + storesToCheck = new ArrayList(1); + storesToCheck.add(getStore(origin.getColumn())); + } + for (HStore targetStore: storesToCheck) { + if (targetStore != null) { + // Pass versions without modification since in the store getKeys, it + // includes the size of the passed keys array when counting. + List r = targetStore.getKeys(origin, versions); + if (r != null) { + keys.addAll(r); + } + } + } + return keys; + } + + /** + * Return an iterator that scans over the HRegion, returning the indicated + * columns for only the rows that match the data filter. This Iterator must + * be closed by the caller. + * + * @param cols columns to scan. If column name is a column family, all + * columns of the specified column family are returned. Its also possible + * to pass a regex in the column qualifier. A column qualifier is judged to + * be a regex if it contains at least one of the following characters: + * \+|^&*$[]]}{)(. + * @param firstRow row which is the starting point of the scan + * @param timestamp only return rows whose timestamp is <= this value + * @param filter row filter + * @return InternalScanner + * @throws IOException + */ + public InternalScanner getScanner(byte[][] cols, byte [] firstRow, + long timestamp, RowFilterInterface filter) + throws IOException { + splitsAndClosesLock.readLock().lock(); + try { + if (this.closed.get()) { + throw new IOException("Region " + this + " closed"); + } + HashSet storeSet = new HashSet(); + for (int i = 0; i < cols.length; i++) { + HStore s = stores.get(Bytes.mapKey(HStoreKey.getFamily(cols[i]))); + if (s != null) { + storeSet.add(s); + } + } + return new HScanner(cols, firstRow, timestamp, + storeSet.toArray(new HStore [storeSet.size()]), filter); + } finally { + splitsAndClosesLock.readLock().unlock(); + } + } + + ////////////////////////////////////////////////////////////////////////////// + // set() methods for client use. + ////////////////////////////////////////////////////////////////////////////// + + /** + * @param b + * @throws IOException + */ + public void batchUpdate(BatchUpdate b) + throws IOException { + + // Do a rough check that we have resources to accept a write. The check is + // 'rough' in that between the resource check and the call to obtain a + // read lock, resources may run out. For now, the thought is that this + // will be extremely rare; we'll deal with it when it happens. + checkResources(); + + // We obtain a per-row lock, so other clients will block while one client + // performs an update. The read lock is released by the client calling + // #commit or #abort or if the HRegionServer lease on the lock expires. + // See HRegionServer#RegionListener for how the expire on HRegionServer + // invokes a HRegion#abort. + byte [] row = b.getRow(); + Integer lid = obtainRowLock(row); + long commitTime = (b.getTimestamp() == LATEST_TIMESTAMP) ? + System.currentTimeMillis() : b.getTimestamp(); + try { + List deletes = null; + for (BatchOperation op: b) { + HStoreKey key = new HStoreKey(row, op.getColumn(), commitTime); + byte[] val = null; + if (op.isPut()) { + val = op.getValue(); + if (HLogEdit.isDeleted(val)) { + throw new IOException("Cannot insert value: " + val); + } + } else { + if (b.getTimestamp() == LATEST_TIMESTAMP) { + // Save off these deletes + if (deletes == null) { + deletes = new ArrayList(); + } + deletes.add(op.getColumn()); + } else { + val = HLogEdit.deleteBytes.get(); + } + } + if (val != null) { + localput(lid, key, val); + } + } + TreeMap edits = + this.targetColumns.remove(lid); + + if (edits != null && edits.size() > 0) { + update(edits); + } + + if (deletes != null && deletes.size() > 0) { + // We have some LATEST_TIMESTAMP deletes to run. + for (byte [] column: deletes) { + deleteMultiple(row, column, LATEST_TIMESTAMP, 1); + } + } + } catch (IOException e) { + this.targetColumns.remove(Long.valueOf(lid)); + throw e; + } finally { + releaseRowLock(lid); + } + } + + /* + * Check if resources to support an update. + * + * For now, just checks memcache saturation. + * + * Here we synchronize on HRegion, a broad scoped lock. Its appropriate + * given we're figuring in here whether this region is able to take on + * writes. This is only method with a synchronize (at time of writing), + * this and the synchronize on 'this' inside in internalFlushCache to send + * the notify. + */ + private synchronized void checkResources() { + boolean blocked = false; + + while (this.memcacheSize.get() >= this.blockingMemcacheSize) { + if (!blocked) { + LOG.info("Blocking updates for '" + Thread.currentThread().getName() + + "' on region " + Bytes.toString(getRegionName()) + ": Memcache size " + + StringUtils.humanReadableInt(this.memcacheSize.get()) + + " is >= than blocking " + + StringUtils.humanReadableInt(this.blockingMemcacheSize) + " size"); + } + + blocked = true; + try { + wait(threadWakeFrequency); + } catch (InterruptedException e) { + // continue; + } + } + if (blocked) { + LOG.info("Unblocking updates for region " + this + " '" + + Thread.currentThread().getName() + "'"); + } + } + + /** + * Delete all cells of the same age as the passed timestamp or older. + * @param row + * @param column + * @param ts Delete all entries that have this timestamp or older + * @throws IOException + */ + public void deleteAll(final byte [] row, final byte [] column, final long ts) + throws IOException { + checkColumn(column); + Integer lid = obtainRowLock(row); + try { + deleteMultiple(row, column, ts, ALL_VERSIONS); + } finally { + releaseRowLock(lid); + } + } + + /** + * Delete all cells of the same age as the passed timestamp or older. + * @param row + * @param ts Delete all entries that have this timestamp or older + * @throws IOException + */ + public void deleteAll(final byte [] row, final long ts) + throws IOException { + Integer lid = obtainRowLock(row); + try { + for (HStore store : stores.values()){ + List keys = store.getKeys(new HStoreKey(row, ts), + ALL_VERSIONS); + TreeMap edits = new TreeMap(); + for (HStoreKey key: keys) { + edits.put(key, HLogEdit.deleteBytes.get()); + } + update(edits); + } + } finally { + releaseRowLock(lid); + } + } + + /** + * Delete all cells for a row with matching column family with timestamps + * less than or equal to timestamp. + * + * @param row The row to operate on + * @param family The column family to match + * @param timestamp Timestamp to match + * @throws IOException + */ + public void deleteFamily(byte [] row, byte [] family, long timestamp) + throws IOException{ + Integer lid = obtainRowLock(row); + try { + // find the HStore for the column family + HStore store = getStore(family); + // find all the keys that match our criteria + List keys = store.getKeys(new HStoreKey(row, timestamp), ALL_VERSIONS); + // delete all the cells + TreeMap edits = new TreeMap(); + for (HStoreKey key: keys) { + edits.put(key, HLogEdit.deleteBytes.get()); + } + update(edits); + } finally { + releaseRowLock(lid); + } + } + + /** + * Delete one or many cells. + * Used to support {@link #deleteAll(byte [], byte [], long)} and deletion of + * latest cell. + * + * @param row + * @param column + * @param ts Timestamp to start search on. + * @param versions How many versions to delete. Pass + * {@link HConstants#ALL_VERSIONS} to delete all. + * @throws IOException + */ + private void deleteMultiple(final byte [] row, final byte [] column, + final long ts, final int versions) + throws IOException { + HStoreKey origin = new HStoreKey(row, column, ts); + Set keys = getKeys(origin, versions); + if (keys.size() > 0) { + TreeMap edits = new TreeMap(); + for (HStoreKey key: keys) { + edits.put(key, HLogEdit.deleteBytes.get()); + } + update(edits); + } + } + + /** + * Private implementation. + * + * localput() is used for both puts and deletes. We just place the values + * into a per-row pending area, until a commit() or abort() call is received. + * (Or until the user's write-lock expires.) + * + * @param lockid + * @param key + * @param val Value to enter into cell + * @throws IOException + */ + private void localput(final Integer lockid, final HStoreKey key, + final byte [] val) + throws IOException { + checkColumn(key.getColumn()); + TreeMap targets = this.targetColumns.get(lockid); + if (targets == null) { + targets = new TreeMap(); + this.targetColumns.put(lockid, targets); + } + targets.put(key, val); + } + + /* + * Add updates first to the hlog and then add values to memcache. + * Warning: Assumption is caller has lock on passed in row. + * @param row Row to update. + * @param timestamp Timestamp to record the updates against + * @param updatesByColumn Cell updates by column + * @throws IOException + */ + private void update(final TreeMap updatesByColumn) + throws IOException { + if (updatesByColumn == null || updatesByColumn.size() <= 0) { + return; + } + boolean flush = false; + this.updatesLock.readLock().lock(); + try { + this.log.append(regionInfo.getRegionName(), + regionInfo.getTableDesc().getName(), updatesByColumn); + long size = 0; + for (Map.Entry e: updatesByColumn.entrySet()) { + HStoreKey key = e.getKey(); + size = this.memcacheSize.addAndGet( + getStore(key.getColumn()).add(key, e.getValue())); + } + flush = this.flushListener != null && !this.flushRequested && + size > this.memcacheFlushSize; + } finally { + this.updatesLock.readLock().unlock(); + } + if (flush) { + // Request a cache flush. Do it outside update lock. + this.flushListener.request(this); + this.flushRequested = true; + } + } + + // Do any reconstruction needed from the log + @SuppressWarnings("unused") + protected void doReconstructionLog(Path oldLogFile, long maxSeqId, + Progressable reporter) + throws UnsupportedEncodingException, IOException { + // Nothing to do (Replaying is done in HStores) + } + + protected HStore instantiateHStore(Path baseDir, + HColumnDescriptor c, Path oldLogFile, Progressable reporter) + throws IOException { + return new HStore(baseDir, this.regionInfo, c, this.fs, oldLogFile, + this.conf, reporter); + } + + /* + * @param column + * @return Store that goes with the family on passed column. + * TODO: Make this lookup faster. + */ + protected HStore getStore(final byte [] column) { + return this.stores.get(HStoreKey.getFamilyMapKey(column)); + } + + ////////////////////////////////////////////////////////////////////////////// + // Support code + ////////////////////////////////////////////////////////////////////////////// + + /** Make sure this is a valid row for the HRegion */ + private void checkRow(final byte [] row) throws IOException { + if(!rowIsInRange(regionInfo, row)) { + throw new WrongRegionException("Requested row out of range for " + + "HRegion " + this + ", startKey='" + + Bytes.toString(regionInfo.getStartKey()) + "', getEndKey()='" + + Bytes.toString(regionInfo.getEndKey()) + "', row='" + + Bytes.toString(row) + "'"); + } + } + + /* + * Make sure this is a valid column for the current table + * @param columnName + * @throws NoSuchColumnFamilyException + */ + private void checkColumn(final byte [] columnName) + throws NoSuchColumnFamilyException, ColumnNameParseException { + if (columnName == null) { + return; + } + + int index = HStoreKey.getFamilyDelimiterIndex(columnName); + if (index <= 0) { + throw new ColumnNameParseException(Bytes.toString(columnName) + + " is missing column family delimiter '" + + HStoreKey.COLUMN_FAMILY_DELIMITER + "'"); + } + if (!regionInfo.getTableDesc().hasFamily(columnName, index)) { + throw new NoSuchColumnFamilyException("Column family on " + + Bytes.toString(columnName) + " does not exist in region " + this + + " in table " + regionInfo.getTableDesc()); + } + } + + /** + * Obtain a lock on the given row. Blocks until success. + * + * I know it's strange to have two mappings: + *

+   *   ROWS  ==> LOCKS
+   * 
+ * as well as + *
+   *   LOCKS ==> ROWS
+   * 
+ * + * But it acts as a guard on the client; a miswritten client just can't + * submit the name of a row and start writing to it; it must know the correct + * lockid, which matches the lock list in memory. + * + *

It would be more memory-efficient to assume a correctly-written client, + * which maybe we'll do in the future. + * + * @param row Name of row to lock. + * @throws IOException + * @return The id of the held lock. + */ + Integer obtainRowLock(final byte [] row) throws IOException { + checkRow(row); + splitsAndClosesLock.readLock().lock(); + try { + if (this.closed.get()) { + throw new NotServingRegionException("Region " + this + " closed"); + } + Integer key = Bytes.mapKey(row); + synchronized (locksToRows) { + while (locksToRows.containsKey(key)) { + try { + locksToRows.wait(); + } catch (InterruptedException ie) { + // Empty + } + } + locksToRows.put(key, row); + locksToRows.notifyAll(); + return key; + } + } finally { + splitsAndClosesLock.readLock().unlock(); + } + } + + /** + * Used by unit tests. + * @param lockid + * @return Row that goes with lockid + */ + byte [] getRowFromLock(final Integer lockid) { + return locksToRows.get(lockid); + } + + /** + * Release the row lock! + * @param row Name of row whose lock we are to release + */ + void releaseRowLock(final Integer lockid) { + synchronized (locksToRows) { + locksToRows.remove(lockid); + locksToRows.notifyAll(); + } + } + + private void waitOnRowLocks() { + synchronized (locksToRows) { + while (this.locksToRows.size() > 0) { + LOG.debug("waiting for " + this.locksToRows.size() + " row locks"); + try { + this.locksToRows.wait(); + } catch (InterruptedException e) { + // Catch. Let while test determine loop-end. + } + } + } + } + + /** {@inheritDoc} */ + @Override + public boolean equals(Object o) { + return this.hashCode() == ((HRegion)o).hashCode(); + } + + /** {@inheritDoc} */ + @Override + public int hashCode() { + return this.regionInfo.getRegionName().hashCode(); + } + + /** {@inheritDoc} */ + @Override + public String toString() { + return this.regionInfo.getRegionNameAsString(); + } + + /** @return Path of region base directory */ + public Path getBaseDir() { + return this.basedir; + } + + /** + * HScanner is an iterator through a bunch of rows in an HRegion. + */ + private class HScanner implements InternalScanner { + private InternalScanner[] scanners; + private TreeMap[] resultSets; + private HStoreKey[] keys; + private RowFilterInterface filter; + + /** Create an HScanner with a handle on many HStores. */ + @SuppressWarnings("unchecked") + HScanner(byte [][] cols, byte [] firstRow, long timestamp, HStore[] stores, + RowFilterInterface filter) + throws IOException { + this.filter = filter; + this.scanners = new InternalScanner[stores.length]; + try { + for (int i = 0; i < stores.length; i++) { + + // Only pass relevant columns to each store + + List columns = new ArrayList(); + for (int j = 0; j < cols.length; j++) { + if (Bytes.equals(HStoreKey.getFamily(cols[j]), + stores[i].getFamily().getName())) { + columns.add(cols[j]); + } + } + + RowFilterInterface f = filter; + if (f != null) { + // Need to replicate filters. + // At least WhileMatchRowFilter will mess up the scan if only + // one shared across many rows. See HADOOP-2467. + f = (RowFilterInterface)WritableUtils.clone(filter, conf); + } + scanners[i] = stores[i].getScanner(timestamp, + columns.toArray(new byte[columns.size()][]), firstRow, f); + } + } catch (IOException e) { + for (int i = 0; i < this.scanners.length; i++) { + if(scanners[i] != null) { + closeScanner(i); + } + } + throw e; + } + + // Advance to the first key in each store. + // All results will match the required column-set and scanTime. + this.resultSets = new TreeMap[scanners.length]; + this.keys = new HStoreKey[scanners.length]; + for (int i = 0; i < scanners.length; i++) { + keys[i] = new HStoreKey(); + resultSets[i] = new TreeMap(Bytes.BYTES_COMPARATOR); + if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) { + closeScanner(i); + } + } + + // As we have now successfully completed initialization, increment the + // activeScanner count. + activeScannerCount.incrementAndGet(); + } + + /** {@inheritDoc} */ + @SuppressWarnings("null") + public boolean next(HStoreKey key, SortedMap results) + throws IOException { + boolean moreToFollow = false; + boolean filtered = false; + + do { + // Find the lowest-possible key. + byte [] chosenRow = null; + long chosenTimestamp = -1; + for (int i = 0; i < this.keys.length; i++) { + if (scanners[i] != null && + (chosenRow == null || + (Bytes.compareTo(keys[i].getRow(), chosenRow) < 0) || + ((Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) && + (keys[i].getTimestamp() > chosenTimestamp)))) { + chosenRow = keys[i].getRow(); + chosenTimestamp = keys[i].getTimestamp(); + } + } + + // Store the key and results for each sub-scanner. Merge them as + // appropriate. + if (chosenTimestamp >= 0) { + // Here we are setting the passed in key with current row+timestamp + key.setRow(chosenRow); + key.setVersion(chosenTimestamp); + key.setColumn(HConstants.EMPTY_BYTE_ARRAY); + + for (int i = 0; i < scanners.length; i++) { + if (scanners[i] != null && + Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) { + // NOTE: We used to do results.putAll(resultSets[i]); + // but this had the effect of overwriting newer + // values with older ones. So now we only insert + // a result if the map does not contain the key. + for (Map.Entry e : resultSets[i].entrySet()) { + if (!results.containsKey(e.getKey())) { + results.put(e.getKey(), e.getValue()); + } + } + resultSets[i].clear(); + if (!scanners[i].next(keys[i], resultSets[i])) { + closeScanner(i); + } + } + } + } + + for (int i = 0; i < scanners.length; i++) { + // If the current scanner is non-null AND has a lower-or-equal + // row label, then its timestamp is bad. We need to advance it. + while ((scanners[i] != null) && + (Bytes.compareTo(keys[i].getRow(), chosenRow) <= 0)) { + resultSets[i].clear(); + if (!scanners[i].next(keys[i], resultSets[i])) { + closeScanner(i); + } + } + } + + moreToFollow = chosenTimestamp >= 0; + if (results == null || results.size() <= 0) { + // If we got no results, then there is no more to follow. + moreToFollow = false; + } + + filtered = filter == null ? false : filter.filterRow(results); + + if (filter != null && filter.filterAllRemaining()) { + moreToFollow = false; + } + + if (moreToFollow) { + if (filter != null) { + filter.rowProcessed(filtered, key.getRow()); + } + if (filtered) { + results.clear(); + } + } + } while(filtered && moreToFollow); + + // Make sure scanners closed if no more results + if (!moreToFollow) { + for (int i = 0; i < scanners.length; i++) { + if (null != scanners[i]) { + closeScanner(i); + } + } + } + + return moreToFollow; + } + + /** Shut down a single scanner */ + void closeScanner(int i) { + try { + try { + scanners[i].close(); + } catch (IOException e) { + LOG.warn("Failed closing scanner " + i, e); + } + } finally { + scanners[i] = null; + // These data members can be null if exception in constructor + if (resultSets != null) { + resultSets[i] = null; + } + if (keys != null) { + keys[i] = null; + } + } + } + + /** + * {@inheritDoc} + */ + public void close() { + try { + for(int i = 0; i < scanners.length; i++) { + if(scanners[i] != null) { + closeScanner(i); + } + } + } finally { + synchronized (activeScannerCount) { + int count = activeScannerCount.decrementAndGet(); + if (count < 0) { + LOG.error("active scanner count less than zero: " + count + + " resetting to zero"); + activeScannerCount.set(0); + count = 0; + } + if (count == 0) { + activeScannerCount.notifyAll(); + } + } + } + } + + /** {@inheritDoc} */ + public boolean isWildcardScanner() { + throw new UnsupportedOperationException("Unimplemented on HScanner"); + } + + /** {@inheritDoc} */ + public boolean isMultipleMatchScanner() { + throw new UnsupportedOperationException("Unimplemented on HScanner"); + } + } + + // Utility methods + + /** + * Convenience method creating new HRegions. Used by createTable and by the + * bootstrap code in the HMaster constructor. + * Note, this method creates an {@link HLog} for the created region. It + * needs to be closed explicitly. Use {@link HRegion#getLog()} to get + * access. + * @param info Info for region to create. + * @param rootDir Root directory for HBase instance + * @param conf + * @return new HRegion + * + * @throws IOException + */ + public static HRegion createHRegion(final HRegionInfo info, final Path rootDir, + final HBaseConfiguration conf) + throws IOException { + Path tableDir = + HTableDescriptor.getTableDir(rootDir, info.getTableDesc().getName()); + Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName()); + FileSystem fs = FileSystem.get(conf); + fs.mkdirs(regionDir); + // Note in historian the creation of new region. + if (!info.isMetaRegion()) { + RegionHistorian.getInstance().addRegionCreation(info); + } + return new HRegion(tableDir, + new HLog(fs, new Path(regionDir, HREGION_LOGDIR_NAME), conf, null), + fs, conf, info, null, null); + } + + /** + * Convenience method to open a HRegion outside of an HRegionServer context. + * @param info Info for region to be opened. + * @param rootDir Root directory for HBase instance + * @param log HLog for region to use. This method will call + * HLog#setSequenceNumber(long) passing the result of the call to + * HRegion#getMinSequenceId() to ensure the log id is properly kept + * up. HRegionStore does this every time it opens a new region. + * @param conf + * @return new HRegion + * + * @throws IOException + */ + public static HRegion openHRegion(final HRegionInfo info, final Path rootDir, + final HLog log, final HBaseConfiguration conf) + throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Opening region: " + info); + } + if (info == null) { + throw new NullPointerException("Passed region info is null"); + } + HRegion r = new HRegion( + HTableDescriptor.getTableDir(rootDir, info.getTableDesc().getName()), + log, FileSystem.get(conf), conf, info, null, null); + if (log != null) { + log.setSequenceNumber(r.getMinSequenceId()); + } + return r; + } + + /** + * Inserts a new region's meta information into the passed + * meta region. Used by the HMaster bootstrap code adding + * new table to ROOT table. + * + * @param meta META HRegion to be updated + * @param r HRegion to add to meta + * + * @throws IOException + */ + public static void addRegionToMETA(HRegion meta, HRegion r) + throws IOException { + meta.checkResources(); + // The row key is the region name + byte [] row = r.getRegionName(); + Integer lid = meta.obtainRowLock(row); + try { + HStoreKey key = new HStoreKey(row, COL_REGIONINFO, System.currentTimeMillis()); + TreeMap edits = new TreeMap(); + edits.put(key, Writables.getBytes(r.getRegionInfo())); + meta.update(edits); + } finally { + meta.releaseRowLock(lid); + } + } + + /** + * Delete a region's meta information from the passed + * meta region. + * + * @param srvr META server to be updated + * @param metaRegionName Meta region name + * @param regionName HRegion to remove from meta + * + * @throws IOException + */ + public static void removeRegionFromMETA(final HRegionInterface srvr, + final byte [] metaRegionName, final byte [] regionName) + throws IOException { + srvr.deleteAll(metaRegionName, regionName, HConstants.LATEST_TIMESTAMP); + } + + /** + * Utility method used by HMaster marking regions offlined. + * @param srvr META server to be updated + * @param metaRegionName Meta region name + * @param info HRegion to update in meta + * + * @throws IOException + */ + public static void offlineRegionInMETA(final HRegionInterface srvr, + final byte [] metaRegionName, final HRegionInfo info) + throws IOException { + BatchUpdate b = new BatchUpdate(info.getRegionName()); + info.setOffline(true); + b.put(COL_REGIONINFO, Writables.getBytes(info)); + b.delete(COL_SERVER); + b.delete(COL_STARTCODE); + // If carrying splits, they'll be in place when we show up on new + // server. + srvr.batchUpdate(metaRegionName, b); + } + + /** + * Deletes all the files for a HRegion + * + * @param fs the file system object + * @param rootdir qualified path of HBase root directory + * @param info HRegionInfo for region to be deleted + * @throws IOException + */ + public static void deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info) + throws IOException { + deleteRegion(fs, HRegion.getRegionDir(rootdir, info)); + } + + private static void deleteRegion(FileSystem fs, Path regiondir) + throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("DELETING region " + regiondir.toString()); + } + fs.delete(regiondir, true); + } + + /** + * Computes the Path of the HRegion + * + * @param tabledir qualified path for table + * @param name ENCODED region name + * @return Path of HRegion directory + */ + public static Path getRegionDir(final Path tabledir, final int name) { + return new Path(tabledir, Integer.toString(name)); + } + + /** + * Computes the Path of the HRegion + * + * @param rootdir qualified path of HBase root directory + * @param info HRegionInfo for the region + * @return qualified path of region directory + */ + public static Path getRegionDir(final Path rootdir, final HRegionInfo info) { + return new Path( + HTableDescriptor.getTableDir(rootdir, info.getTableDesc().getName()), + Integer.toString(info.getEncodedName())); + } + + /** + * Determines if the specified row is within the row range specified by the + * specified HRegionInfo + * + * @param info HRegionInfo that specifies the row range + * @param row row to be checked + * @return true if the row is within the range specified by the HRegionInfo + */ + public static boolean rowIsInRange(HRegionInfo info, final byte [] row) { + return ((info.getStartKey().length == 0) || + (Bytes.compareTo(info.getStartKey(), row) <= 0)) && + ((info.getEndKey().length == 0) || + (Bytes.compareTo(info.getEndKey(), row) > 0)); + } + + /** + * Make the directories for a specific column family + * + * @param fs the file system + * @param basedir base directory where region will live (usually the table dir) + * @param encodedRegionName encoded region name + * @param colFamily the column family + * @param tabledesc table descriptor of table + * @throws IOException + */ + public static void makeColumnFamilyDirs(FileSystem fs, Path basedir, + int encodedRegionName, byte [] colFamily, HTableDescriptor tabledesc) + throws IOException { + fs.mkdirs(HStoreFile.getMapDir(basedir, encodedRegionName, colFamily)); + fs.mkdirs(HStoreFile.getInfoDir(basedir, encodedRegionName, colFamily)); + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HRegionInfo.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HRegionInfo.java new file mode 100644 index 00000000000..97ee4e92b0b --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HRegionInfo.java @@ -0,0 +1,451 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JenkinsHash; +import org.apache.hadoop.io.WritableComparable; + +/** + * HRegion information. + * Contains HRegion id, start and end keys, a reference to this + * HRegions' table descriptor, etc. + * + *

This class has been modified so it instantiates using pre-v5 versions of + * the HTableDescriptor, etc: i.e. it will uses classes that in this + * migration v0_2 package. + */ +public class HRegionInfo implements WritableComparable { + /** + * @param regionName + * @return the encodedName + */ + public static int encodeRegionName(final byte [] regionName) { + return Math.abs(JenkinsHash.hash(regionName, regionName.length, 0)); + } + + /** delimiter used between portions of a region name */ + public static final int DELIMITER = ','; + + /** HRegionInfo for root region */ + public static final HRegionInfo ROOT_REGIONINFO = + new HRegionInfo(0L, HTableDescriptor.ROOT_TABLEDESC); + + /** HRegionInfo for first meta region */ + public static final HRegionInfo FIRST_META_REGIONINFO = + new HRegionInfo(1L, HTableDescriptor.META_TABLEDESC); + + /** + * Extracts table name prefix from a metaregion row name. + * @param regionName A metaregion row name. + * @return The table prefix of a region name. + */ + public static byte [] getTableNameFromRegionName(final byte [] regionName) { + return parseMetaRegionRow(regionName).get(0); + } + + /** + * Parses passed metaregion row into its constituent parts. + * Presumes region names are ASCII characters only. + * @param regionName A metaregion row name. + * @return A list where first element is the tablename, second the row + * portion, and the third the id. + */ + public static List parseMetaRegionRow(final byte [] regionName) { + int offset = -1; + for (int i = 0; i < regionName.length; i++) { + if (regionName[i] == DELIMITER) { + offset = i; + break; + } + } + if (offset == -1) { + throw new IllegalArgumentException(Bytes.toString(regionName) + + " does not contain '" + DELIMITER + "' character"); + } + byte [] tableName = new byte[offset]; + System.arraycopy(regionName, 0, tableName, 0, offset); + // Now move in from the tail till we hit DELIMITER to find the id + offset = -1; + for (int i = regionName.length - 1; i > tableName.length; i--) { + if (regionName[i] == DELIMITER) { + offset = i; + break; + } + } + if (offset == -1) { + throw new IllegalArgumentException(Bytes.toString(regionName) + + " does not have parseable tail"); + } + byte [] row = new byte[offset - (tableName.length + 1)]; + System.arraycopy(regionName, tableName.length + 1, row, 0, + offset - (tableName.length + 1)); + byte [] id = new byte[regionName.length - (offset + 1)]; + System.arraycopy(regionName, offset + 1, id, 0, + regionName.length - (offset + 1)); + // Now make up an array to hold the three parse pieces. + List result = new ArrayList(3); + result.add(tableName); + result.add(row); + result.add(id); + return result; + } + + private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY; + private boolean offLine = false; + private long regionId = -1; + private byte [] regionName = HConstants.EMPTY_BYTE_ARRAY; + private String regionNameStr = ""; + private boolean split = false; + private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY; + protected HTableDescriptor tableDesc = null; + private int hashCode = -1; + public static final int NO_HASH = -1; + private volatile int encodedName = NO_HASH; + + private void setHashCode() { + int result = this.regionName.hashCode(); + result ^= this.regionId; + result ^= this.startKey.hashCode(); + result ^= this.endKey.hashCode(); + result ^= Boolean.valueOf(this.offLine).hashCode(); + result ^= this.tableDesc.hashCode(); + this.hashCode = result; + } + + /** + * Private constructor used constructing HRegionInfo for the catalog root and + * first meta regions + */ + private HRegionInfo(long regionId, HTableDescriptor tableDesc) { + this.regionId = regionId; + this.tableDesc = tableDesc; + this.regionName = createRegionName(tableDesc.getName(), null, regionId); + this.regionNameStr = Bytes.toString(this.regionName); + setHashCode(); + } + + /** Default constructor - creates empty object */ + public HRegionInfo() { + this.tableDesc = new HTableDescriptor(); + } + + /** + * Construct HRegionInfo with explicit parameters + * + * @param tableDesc the table descriptor + * @param startKey first key in region + * @param endKey end of key range + * @throws IllegalArgumentException + */ + public HRegionInfo(final HTableDescriptor tableDesc, final byte [] startKey, + final byte [] endKey) + throws IllegalArgumentException { + this(tableDesc, startKey, endKey, false); + } + + /** + * Construct HRegionInfo with explicit parameters + * + * @param tableDesc the table descriptor + * @param startKey first key in region + * @param endKey end of key range + * @param split true if this region has split and we have daughter regions + * regions that may or may not hold references to this region. + * @throws IllegalArgumentException + */ + public HRegionInfo(HTableDescriptor tableDesc, final byte [] startKey, + final byte [] endKey, final boolean split) + throws IllegalArgumentException { + this(tableDesc, startKey, endKey, split, System.currentTimeMillis()); + } + + /** + * Construct HRegionInfo with explicit parameters + * + * @param tableDesc the table descriptor + * @param startKey first key in region + * @param endKey end of key range + * @param split true if this region has split and we have daughter regions + * regions that may or may not hold references to this region. + * @param regionid Region id to use. + * @throws IllegalArgumentException + */ + public HRegionInfo(HTableDescriptor tableDesc, final byte [] startKey, + final byte [] endKey, final boolean split, final long regionid) + throws IllegalArgumentException { + if (tableDesc == null) { + throw new IllegalArgumentException("tableDesc cannot be null"); + } + this.offLine = false; + this.regionId = regionid; + this.regionName = createRegionName(tableDesc.getName(), startKey, regionId); + this.regionNameStr = Bytes.toString(this.regionName); + this.split = split; + this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone(); + this.startKey = startKey == null? + HConstants.EMPTY_START_ROW: startKey.clone(); + this.tableDesc = tableDesc; + setHashCode(); + } + + /** + * Costruct a copy of another HRegionInfo + * + * @param other + */ + public HRegionInfo(HRegionInfo other) { + this.endKey = other.getEndKey(); + this.offLine = other.isOffline(); + this.regionId = other.getRegionId(); + this.regionName = other.getRegionName(); + this.regionNameStr = Bytes.toString(this.regionName); + this.split = other.isSplit(); + this.startKey = other.getStartKey(); + this.tableDesc = other.getTableDesc(); + this.hashCode = other.hashCode(); + this.encodedName = other.getEncodedName(); + } + + private static byte [] createRegionName(final byte [] tableName, + final byte [] startKey, final long regionid) { + return createRegionName(tableName, startKey, Long.toString(regionid)); + } + + /** + * Make a region name of passed parameters. + * @param tableName + * @param startKey Can be null + * @param id Region id. + * @return Region name made of passed tableName, startKey and id + */ + public static byte [] createRegionName(final byte [] tableName, + final byte [] startKey, final String id) { + return createRegionName(tableName, startKey, Bytes.toBytes(id)); + } + /** + * Make a region name of passed parameters. + * @param tableName + * @param startKey Can be null + * @param id Region id + * @return Region name made of passed tableName, startKey and id + */ + public static byte [] createRegionName(final byte [] tableName, + final byte [] startKey, final byte [] id) { + byte [] b = new byte [tableName.length + 2 + id.length + + (startKey == null? 0: startKey.length)]; + int offset = tableName.length; + System.arraycopy(tableName, 0, b, 0, offset); + b[offset++] = DELIMITER; + if (startKey != null && startKey.length > 0) { + System.arraycopy(startKey, 0, b, offset, startKey.length); + offset += startKey.length; + } + b[offset++] = DELIMITER; + System.arraycopy(id, 0, b, offset, id.length); + return b; + } + + /** @return the endKey */ + public byte [] getEndKey(){ + return endKey; + } + + /** @return the regionId */ + public long getRegionId(){ + return regionId; + } + + /** + * @return the regionName as an array of bytes. + * @see #getRegionNameAsString() + */ + public byte [] getRegionName(){ + return regionName; + } + + /** + * @return Region name as a String for use in logging, etc. + */ + public String getRegionNameAsString() { + return this.regionNameStr; + } + + /** @return the encoded region name */ + public synchronized int getEncodedName() { + if (this.encodedName == NO_HASH) { + this.encodedName = encodeRegionName(this.regionName); + } + return this.encodedName; + } + + /** @return the startKey */ + public byte [] getStartKey(){ + return startKey; + } + + /** @return the tableDesc */ + public HTableDescriptor getTableDesc(){ + return tableDesc; + } + + /** @return true if this is the root region */ + public boolean isRootRegion() { + return this.tableDesc.isRootRegion(); + } + + /** @return true if this is the meta table */ + public boolean isMetaTable() { + return this.tableDesc.isMetaTable(); + } + + /** @return true if this region is a meta region */ + public boolean isMetaRegion() { + return this.tableDesc.isMetaRegion(); + } + + /** + * @return True if has been split and has daughters. + */ + public boolean isSplit() { + return this.split; + } + + /** + * @param split set split status + */ + public void setSplit(boolean split) { + this.split = split; + } + + /** + * @return True if this region is offline. + */ + public boolean isOffline() { + return this.offLine; + } + + /** + * @param offLine set online - offline status + */ + public void setOffline(boolean offLine) { + this.offLine = offLine; + } + + /** + * {@inheritDoc} + */ + @Override + public String toString() { + return "REGION => {" + HConstants.NAME + " => '" + + this.regionNameStr + + "', STARTKEY => '" + + Bytes.toString(this.startKey) + "', ENDKEY => '" + + Bytes.toString(this.endKey) + + "', ENCODED => " + getEncodedName() + "," + + (isOffline()? " OFFLINE => true,": "") + (isSplit()? " SPLIT => true,": "") + + " TABLE => {" + this.tableDesc.toString() + "}"; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean equals(Object o) { + return this.compareTo(o) == 0; + } + + /** + * {@inheritDoc} + */ + @Override + public int hashCode() { + return this.hashCode; + } + + // + // Writable + // + + /** + * {@inheritDoc} + */ + public void write(DataOutput out) throws IOException { + Bytes.writeByteArray(out, endKey); + out.writeBoolean(offLine); + out.writeLong(regionId); + Bytes.writeByteArray(out, regionName); + out.writeBoolean(split); + Bytes.writeByteArray(out, startKey); + tableDesc.write(out); + out.writeInt(hashCode); + } + + /** + * {@inheritDoc} + */ + public void readFields(DataInput in) throws IOException { + this.endKey = Bytes.readByteArray(in); + this.offLine = in.readBoolean(); + this.regionId = in.readLong(); + this.regionName = Bytes.readByteArray(in); + this.regionNameStr = Bytes.toString(this.regionName); + this.split = in.readBoolean(); + this.startKey = Bytes.readByteArray(in); + this.tableDesc.readFields(in); + this.hashCode = in.readInt(); + } + + // + // Comparable + // + + /** + * {@inheritDoc} + */ + public int compareTo(Object o) { + HRegionInfo other = (HRegionInfo) o; + if (other == null) { + return 1; + } + + // Are regions of same table? + int result = this.tableDesc.compareTo(other.tableDesc); + if (result != 0) { + return result; + } + + // Compare start keys. + result = Bytes.compareTo(this.startKey, other.startKey); + if (result != 0) { + return result; + } + + // Compare end keys. + return Bytes.compareTo(this.endKey, other.endKey); + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HStore.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HStore.java new file mode 100644 index 00000000000..416683432ad --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HStore.java @@ -0,0 +1,1816 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.EOFException; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.regionserver.ChangedReadersObserver; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.io.MapFile; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.StringUtils; + +/** + * HStore maintains a bunch of data files. It is responsible for maintaining + * the memory/file hierarchy and for periodic flushes to disk and compacting + * edits to the file. + * + * Locking and transactions are handled at a higher level. This API should not + * be called directly by any writer, but rather by an HRegion manager. + */ +public class HStore implements HConstants { + static final Log LOG = LogFactory.getLog(HStore.class); + + /* + * Regex that will work for straight filenames and for reference names. + * If reference, then the regex has more than just one group. Group 1 is + * this files id. Group 2 the referenced region name, etc. + */ + private static final Pattern REF_NAME_PARSER = + Pattern.compile("^(\\d+)(?:\\.(.+))?$"); + + protected final Memcache memcache; + private final Path basedir; + private final HRegionInfo info; + private final HColumnDescriptor family; + private final SequenceFile.CompressionType compression; + final FileSystem fs; + private final HBaseConfiguration conf; + protected long ttl; + + private final long desiredMaxFileSize; + private volatile long storeSize; + + private final Integer flushLock = new Integer(0); + + final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + final byte [] storeName; + private final String storeNameStr; + + /* + * Sorted Map of readers keyed by sequence id (Most recent should be last in + * in list). + */ + private final SortedMap storefiles = + Collections.synchronizedSortedMap(new TreeMap()); + + /* + * Sorted Map of readers keyed by sequence id (Most recent should be last in + * in list). + */ + private final SortedMap readers = + new TreeMap(); + + // The most-recent log-seq-ID that's present. The most-recent such ID means + // we can ignore all log messages up to and including that ID (because they're + // already reflected in the TreeMaps). + private volatile long maxSeqId; + + private final Path compactionDir; + private final Integer compactLock = new Integer(0); + private final int compactionThreshold; + private final Set changedReaderObservers = + Collections.synchronizedSet(new HashSet()); + + /** + * An HStore is a set of zero or more MapFiles, which stretch backwards over + * time. A given HStore is responsible for a certain set of columns for a + * row in the HRegion. + * + *

The HRegion starts writing to its set of HStores when the HRegion's + * memcache is flushed. This results in a round of new MapFiles, one for + * each HStore. + * + *

There's no reason to consider append-logging at this level; all logging + * and locking is handled at the HRegion level. HStore just provides + * services to manage sets of MapFiles. One of the most important of those + * services is MapFile-compaction services. + * + *

The only thing having to do with logs that HStore needs to deal with is + * the reconstructionLog. This is a segment of an HRegion's log that might + * NOT be present upon startup. If the param is NULL, there's nothing to do. + * If the param is non-NULL, we need to process the log to reconstruct + * a TreeMap that might not have been written to disk before the process + * died. + * + *

It's assumed that after this constructor returns, the reconstructionLog + * file will be deleted (by whoever has instantiated the HStore). + * + * @param basedir qualified path under which the region directory lives + * @param info HRegionInfo for this region + * @param family HColumnDescriptor for this column + * @param fs file system object + * @param reconstructionLog existing log file to apply if any + * @param conf configuration object + * @param reporter Call on a period so hosting server can report we're + * making progress to master -- otherwise master might think region deploy + * failed. Can be null. + * @throws IOException + */ + protected HStore(Path basedir, HRegionInfo info, HColumnDescriptor family, + FileSystem fs, Path reconstructionLog, HBaseConfiguration conf, + final Progressable reporter) + throws IOException { + this.basedir = basedir; + this.info = info; + this.family = family; + this.fs = fs; + this.conf = conf; + this.ttl = family.getTimeToLive(); + if (ttl != HConstants.FOREVER) + this.ttl *= 1000; + this.memcache = new Memcache(this.ttl); + this.compactionDir = HRegion.getCompactionDir(basedir); + this.storeName = Bytes.toBytes(this.info.getEncodedName() + "/" + + Bytes.toString(this.family.getName())); + this.storeNameStr = Bytes.toString(this.storeName); + + // By default, we compact if an HStore has more than + // MIN_COMMITS_FOR_COMPACTION map files + this.compactionThreshold = + conf.getInt("hbase.hstore.compactionThreshold", 3); + + // By default we split region if a file > DEFAULT_MAX_FILE_SIZE. + this.desiredMaxFileSize = + conf.getLong("hbase.hregion.max.filesize", DEFAULT_MAX_FILE_SIZE); + this.storeSize = 0L; + + if (family.getCompression() == HColumnDescriptor.CompressionType.BLOCK) { + this.compression = SequenceFile.CompressionType.BLOCK; + } else if (family.getCompression() == + HColumnDescriptor.CompressionType.RECORD) { + this.compression = SequenceFile.CompressionType.RECORD; + } else { + this.compression = SequenceFile.CompressionType.NONE; + } + + Path mapdir = HStoreFile.getMapDir(basedir, info.getEncodedName(), + family.getName()); + if (!fs.exists(mapdir)) { + fs.mkdirs(mapdir); + } + Path infodir = HStoreFile.getInfoDir(basedir, info.getEncodedName(), + family.getName()); + if (!fs.exists(infodir)) { + fs.mkdirs(infodir); + } + + // Go through the 'mapdir' and 'infodir' together, make sure that all + // MapFiles are in a reliable state. Every entry in 'mapdir' must have a + // corresponding one in 'loginfodir'. Without a corresponding log info + // file, the entry in 'mapdir' must be deleted. + // loadHStoreFiles also computes the max sequence id internally. + this.maxSeqId = -1L; + this.storefiles.putAll(loadHStoreFiles(infodir, mapdir)); + if (LOG.isDebugEnabled() && this.storefiles.size() > 0) { + LOG.debug("Loaded " + this.storefiles.size() + " file(s) in hstore " + + Bytes.toString(this.storeName) + ", max sequence id " + this.maxSeqId); + } + + try { + doReconstructionLog(reconstructionLog, maxSeqId, reporter); + } catch (EOFException e) { + // Presume we got here because of lack of HADOOP-1700; for now keep going + // but this is probably not what we want long term. If we got here there + // has been data-loss + LOG.warn("Exception processing reconstruction log " + reconstructionLog + + " opening " + this.storeName + + " -- continuing. Probably lack-of-HADOOP-1700 causing DATA LOSS!", e); + } catch (IOException e) { + // Presume we got here because of some HDFS issue. Don't just keep going. + // Fail to open the HStore. Probably means we'll fail over and over + // again until human intervention but alternative has us skipping logs + // and losing edits: HBASE-642. + LOG.warn("Exception processing reconstruction log " + reconstructionLog + + " opening " + this.storeName, e); + throw e; + } + + // Finally, start up all the map readers! (There could be more than one + // since we haven't compacted yet.) + boolean first = true; + for(Map.Entry e: this.storefiles.entrySet()) { + MapFile.Reader r = null; + if (first) { + // Use a block cache (if configured) for the first reader only + // so as to control memory usage. + r = e.getValue().getReader(this.fs, this.family.isBloomFilterEnabled(), + family.isBlockCacheEnabled()); + first = false; + } else { + r = e.getValue().getReader(this.fs, this.family.isBloomFilterEnabled(), + false); + } + this.readers.put(e.getKey(), r); + } + } + + HColumnDescriptor getFamily() { + return this.family; + } + + long getMaxSequenceId() { + return this.maxSeqId; + } + + /* + * Read the reconstructionLog to see whether we need to build a brand-new + * MapFile out of non-flushed log entries. + * + * We can ignore any log message that has a sequence ID that's equal to or + * lower than maxSeqID. (Because we know such log messages are already + * reflected in the MapFiles.) + */ + private void doReconstructionLog(final Path reconstructionLog, + final long maxSeqID, final Progressable reporter) + throws UnsupportedEncodingException, IOException { + if (reconstructionLog == null || !fs.exists(reconstructionLog)) { + // Nothing to do. + return; + } + // Check its not empty. + FileStatus[] stats = fs.listStatus(reconstructionLog); + if (stats == null || stats.length == 0) { + LOG.warn("Passed reconstruction log " + reconstructionLog + " is zero-length"); + return; + } + long maxSeqIdInLog = -1; + TreeMap reconstructedCache = + new TreeMap(); + + SequenceFile.Reader logReader = new SequenceFile.Reader(this.fs, + reconstructionLog, this.conf); + + try { + HLogKey key = new HLogKey(); + HLogEdit val = new HLogEdit(); + long skippedEdits = 0; + long editsCount = 0; + // How many edits to apply before we send a progress report. + int reportInterval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000); + while (logReader.next(key, val)) { + maxSeqIdInLog = Math.max(maxSeqIdInLog, key.getLogSeqNum()); + if (key.getLogSeqNum() <= maxSeqID) { + skippedEdits++; + continue; + } + // Check this edit is for me. Also, guard against writing + // METACOLUMN info such as HBASE::CACHEFLUSH entries + byte [] column = val.getColumn(); + if (Bytes.equals(column, HLog.METACOLUMN) + || !Bytes.equals(key.getRegionName(), info.getRegionName()) + || !HStoreKey.matchingFamily(family.getName(), column)) { + continue; + } + HStoreKey k = new HStoreKey(key.getRow(), column, val.getTimestamp()); + reconstructedCache.put(k, val.getVal()); + editsCount++; + // Every 2k edits, tell the reporter we're making progress. + // Have seen 60k edits taking 3minutes to complete. + if (reporter != null && (editsCount % reportInterval) == 0) { + reporter.progress(); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("Applied " + editsCount + ", skipped " + skippedEdits + + " because sequence id <= " + maxSeqID); + } + } finally { + logReader.close(); + } + + if (reconstructedCache.size() > 0) { + // We create a "virtual flush" at maxSeqIdInLog+1. + if (LOG.isDebugEnabled()) { + LOG.debug("flushing reconstructionCache"); + } + internalFlushCache(reconstructedCache, maxSeqIdInLog + 1); + } + } + + /* + * Creates a series of HStoreFiles loaded from the given directory. + * There must be a matching 'mapdir' and 'loginfo' pair of files. + * If only one exists, we'll delete it. Does other consistency tests + * checking files are not zero, etc. + * + * @param infodir qualified path for info file directory + * @param mapdir qualified path for map file directory + * @throws IOException + */ + private SortedMap loadHStoreFiles(Path infodir, Path mapdir) + throws IOException { + // Look first at info files. If a reference, these contain info we need + // to create the HStoreFile. + FileStatus infofiles[] = fs.listStatus(infodir); + SortedMap results = new TreeMap(); + ArrayList mapfiles = new ArrayList(infofiles.length); + for (int i = 0; i < infofiles.length; i++) { + Path p = infofiles[i].getPath(); + // Check for empty info file. Should never be the case but can happen + // after data loss in hdfs for whatever reason (upgrade, etc.): HBASE-646 + if (this.fs.getFileStatus(p).getLen() <= 0) { + LOG.warn("Skipping " + p + " because its empty. DATA LOSS? Can " + + "this scenario be repaired? HBASE-646"); + continue; + } + + Matcher m = REF_NAME_PARSER.matcher(p.getName()); + /* + * * * * * N O T E * * * * * + * + * We call isReference(Path, Matcher) here because it calls + * Matcher.matches() which must be called before Matcher.group(int) + * and we don't want to call Matcher.matches() twice. + * + * * * * * N O T E * * * * * + */ + boolean isReference = isReference(p, m); + long fid = Long.parseLong(m.group(1)); + + HStoreFile curfile = null; + HStoreFile.Reference reference = null; + if (isReference) { + reference = HStoreFile.readSplitInfo(p, fs); + } + curfile = new HStoreFile(conf, fs, basedir, info.getEncodedName(), + family.getName(), fid, reference); + storeSize += curfile.length(); + long storeSeqId = -1; + try { + storeSeqId = curfile.loadInfo(fs); + if (storeSeqId > this.maxSeqId) { + this.maxSeqId = storeSeqId; + } + } catch (IOException e) { + // If the HSTORE_LOGINFOFILE doesn't contain a number, just ignore it. + // That means it was built prior to the previous run of HStore, and so + // it cannot contain any updates also contained in the log. + LOG.info("HSTORE_LOGINFOFILE " + curfile + + " does not contain a sequence number - ignoring"); + } + Path mapfile = curfile.getMapFilePath(); + if (!fs.exists(mapfile)) { + fs.delete(curfile.getInfoFilePath(), false); + LOG.warn("Mapfile " + mapfile.toString() + " does not exist. " + + "Cleaned up info file. Continuing...Probable DATA LOSS!!!"); + continue; + } + if (isEmptyDataFile(mapfile)) { + curfile.delete(); + // We can have empty data file if data loss in hdfs. + LOG.warn("Mapfile " + mapfile.toString() + " has empty data. " + + "Deleting. Continuing...Probable DATA LOSS!!! See HBASE-646."); + continue; + } + if (isEmptyIndexFile(mapfile)) { + try { + // Try fixing this file.. if we can. Use the hbase version of fix. + // Need to remove the old index file first else fix won't go ahead. + this.fs.delete(new Path(mapfile, MapFile.INDEX_FILE_NAME), false); + long count = MapFile.fix(this.fs, mapfile, HStoreFile.HbaseMapFile.KEY_CLASS, + HStoreFile.HbaseMapFile.VALUE_CLASS, false, this.conf); + if (LOG.isDebugEnabled()) { + LOG.debug("Fixed index on " + mapfile.toString() + "; had " + + count + " entries"); + } + } catch (Exception e) { + LOG.warn("Failed fix of " + mapfile.toString() + + "...continuing; Probable DATA LOSS!!!", e); + continue; + } + } + + // TODO: Confirm referent exists. + + // Found map and sympathetic info file. Add this hstorefile to result. + if (LOG.isDebugEnabled()) { + LOG.debug("loaded " + FSUtils.getPath(p) + ", isReference=" + + isReference + ", sequence id=" + storeSeqId); + } + results.put(Long.valueOf(storeSeqId), curfile); + // Keep list of sympathetic data mapfiles for cleaning info dir in next + // section. Make sure path is fully qualified for compare. + mapfiles.add(mapfile); + } + + // List paths by experience returns fully qualified names -- at least when + // running on a mini hdfs cluster. + FileStatus datfiles[] = fs.listStatus(mapdir); + for (int i = 0; i < datfiles.length; i++) { + Path p = datfiles[i].getPath(); + // If does not have sympathetic info file, delete. + if (!mapfiles.contains(fs.makeQualified(p))) { + fs.delete(p, true); + } + } + return results; + } + + /* + * @param mapfile + * @return True if the passed mapfile has a zero-length data component (its + * broken). + * @throws IOException + */ + private boolean isEmptyDataFile(final Path mapfile) + throws IOException { + // Mapfiles are made of 'data' and 'index' files. Confirm 'data' is + // non-null if it exists (may not have been written to yet). + return isEmptyFile(new Path(mapfile, MapFile.DATA_FILE_NAME)); + } + + /* + * @param mapfile + * @return True if the passed mapfile has a zero-length index component (its + * broken). + * @throws IOException + */ + private boolean isEmptyIndexFile(final Path mapfile) + throws IOException { + // Mapfiles are made of 'data' and 'index' files. Confirm 'data' is + // non-null if it exists (may not have been written to yet). + return isEmptyFile(new Path(mapfile, MapFile.INDEX_FILE_NAME)); + } + + /* + * @param mapfile + * @return True if the passed mapfile has a zero-length index component (its + * broken). + * @throws IOException + */ + private boolean isEmptyFile(final Path f) + throws IOException { + return this.fs.exists(f) && + this.fs.getFileStatus(f).getLen() == 0; + } + + /** + * Adds a value to the memcache + * + * @param key + * @param value + * @return memcache size delta + */ + protected long add(HStoreKey key, byte[] value) { + lock.readLock().lock(); + try { + return this.memcache.add(key, value); + } finally { + lock.readLock().unlock(); + } + } + + /** + * Close all the MapFile readers + * + * We don't need to worry about subsequent requests because the HRegion holds + * a write lock that will prevent any more reads or writes. + * + * @throws IOException + */ + List close() throws IOException { + ArrayList result = null; + this.lock.writeLock().lock(); + try { + for (MapFile.Reader reader: this.readers.values()) { + reader.close(); + } + synchronized (this.storefiles) { + result = new ArrayList(storefiles.values()); + } + LOG.debug("closed " + this.storeNameStr); + return result; + } finally { + this.lock.writeLock().unlock(); + } + } + + ////////////////////////////////////////////////////////////////////////////// + // Flush changes to disk + ////////////////////////////////////////////////////////////////////////////// + + /** + * Snapshot this stores memcache. Call before running + * {@link #flushCache(long)} so it has some work to do. + */ + void snapshot() { + this.memcache.snapshot(); + } + + /** + * Write out current snapshot. Presumes {@link #snapshot()} has been called + * previously. + * @param logCacheFlushId flush sequence number + * @return true if a compaction is needed + * @throws IOException + */ + boolean flushCache(final long logCacheFlushId) throws IOException { + // Get the snapshot to flush. Presumes that a call to + // this.memcache.snapshot() has happened earlier up in the chain. + SortedMap cache = this.memcache.getSnapshot(); + boolean compactionNeeded = internalFlushCache(cache, logCacheFlushId); + // If an exception happens flushing, we let it out without clearing + // the memcache snapshot. The old snapshot will be returned when we say + // 'snapshot', the next time flush comes around. + this.memcache.clearSnapshot(cache); + return compactionNeeded; + } + + private boolean internalFlushCache(SortedMap cache, + long logCacheFlushId) throws IOException { + long flushed = 0; + // Don't flush if there are no entries. + if (cache.size() == 0) { + return false; + } + + // TODO: We can fail in the below block before we complete adding this + // flush to list of store files. Add cleanup of anything put on filesystem + // if we fail. + synchronized(flushLock) { + long now = System.currentTimeMillis(); + // A. Write the Maps out to the disk + HStoreFile flushedFile = new HStoreFile(conf, fs, basedir, + info.getEncodedName(), family.getName(), -1L, null); + MapFile.Writer out = flushedFile.getWriter(this.fs, this.compression, + this.family.isBloomFilterEnabled(), cache.size()); + + // Here we tried picking up an existing HStoreFile from disk and + // interlacing the memcache flush compacting as we go. The notion was + // that interlacing would take as long as a pure flush with the added + // benefit of having one less file in the store. Experiments showed that + // it takes two to three times the amount of time flushing -- more column + // families makes it so the two timings come closer together -- but it + // also complicates the flush. The code was removed. Needed work picking + // which file to interlace (favor references first, etc.) + // + // Related, looks like 'merging compactions' in BigTable paper interlaces + // a memcache flush. We don't. + int entries = 0; + try { + for (Map.Entry es: cache.entrySet()) { + HStoreKey curkey = es.getKey(); + byte[] bytes = es.getValue(); + if (HStoreKey.matchingFamily(this.family.getName(), curkey.getColumn())) { + if (ttl == HConstants.FOREVER || + now < curkey.getTimestamp() + ttl) { + entries++; + out.append(curkey, new ImmutableBytesWritable(bytes)); + flushed += curkey.getSize() + (bytes == null ? 0 : bytes.length); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("internalFlushCache: " + curkey + + ": expired, skipped"); + } + } + } + } + } finally { + out.close(); + } + long newStoreSize = flushedFile.length(); + storeSize += newStoreSize; + + // B. Write out the log sequence number that corresponds to this output + // MapFile. The MapFile is current up to and including the log seq num. + flushedFile.writeInfo(fs, logCacheFlushId); + + // C. Finally, make the new MapFile available. + updateReaders(logCacheFlushId, flushedFile); + if(LOG.isDebugEnabled()) { + LOG.debug("Added " + FSUtils.getPath(flushedFile.getMapFilePath()) + + " with " + entries + + " entries, sequence id " + logCacheFlushId + ", data size " + + StringUtils.humanReadableInt(flushed) + ", file size " + + StringUtils.humanReadableInt(newStoreSize)); + } + } + return storefiles.size() >= compactionThreshold; + } + + /* + * Change readers adding into place the Reader produced by this new flush. + * @param logCacheFlushId + * @param flushedFile + * @throws IOException + */ + private void updateReaders(final long logCacheFlushId, + final HStoreFile flushedFile) + throws IOException { + this.lock.writeLock().lock(); + try { + Long flushid = Long.valueOf(logCacheFlushId); + // Open the map file reader. + this.readers.put(flushid, + flushedFile.getReader(this.fs, this.family.isBloomFilterEnabled(), + this.family.isBlockCacheEnabled())); + this.storefiles.put(flushid, flushedFile); + // Tell listeners of the change in readers. + notifyChangedReadersObservers(); + } finally { + this.lock.writeLock().unlock(); + } + } + + /* + * Notify all observers that set of Readers has changed. + * @throws IOException + */ + private void notifyChangedReadersObservers() throws IOException { + synchronized (this.changedReaderObservers) { + for (ChangedReadersObserver o: this.changedReaderObservers) { + o.updateReaders(); + } + } + } + + /* + * @param o Observer who wants to know about changes in set of Readers + */ + void addChangedReaderObserver(ChangedReadersObserver o) { + this.changedReaderObservers.add(o); + } + + /* + * @param o Observer no longer interested in changes in set of Readers. + */ + void deleteChangedReaderObserver(ChangedReadersObserver o) { + if (!this.changedReaderObservers.remove(o)) { + LOG.warn("Not in set" + o); + } + } + + ////////////////////////////////////////////////////////////////////////////// + // Compaction + ////////////////////////////////////////////////////////////////////////////// + + /** + * Compact the back-HStores. This method may take some time, so the calling + * thread must be able to block for long periods. + * + *

During this time, the HStore can work as usual, getting values from + * MapFiles and writing new MapFiles from the Memcache. + * + * Existing MapFiles are not destroyed until the new compacted TreeMap is + * completely written-out to disk. + * + * The compactLock prevents multiple simultaneous compactions. + * The structureLock prevents us from interfering with other write operations. + * + * We don't want to hold the structureLock for the whole time, as a compact() + * can be lengthy and we want to allow cache-flushes during this period. + * + * @param force True to force a compaction regardless of thresholds (Needed + * by merge). + * @return mid key if a split is needed, null otherwise + * @throws IOException + */ + StoreSize compact(final boolean force) throws IOException { + synchronized (compactLock) { + long maxId = -1; + int nrows = -1; + List filesToCompact = null; + synchronized (storefiles) { + if (this.storefiles.size() <= 0) { + return null; + } + filesToCompact = new ArrayList(this.storefiles.values()); + + // The max-sequenceID in any of the to-be-compacted TreeMaps is the + // last key of storefiles. + maxId = this.storefiles.lastKey().longValue(); + } + if (!force && filesToCompact.size() < compactionThreshold) { + return checkSplit(); + } + if (!fs.exists(compactionDir) && !fs.mkdirs(compactionDir)) { + LOG.warn("Mkdir on " + compactionDir.toString() + " failed"); + return checkSplit(); + } + /* + * We create a new list of MapFile.Reader objects so we don't screw up the + * caching associated with the currently-loaded ones. Our iteration-based + * access pattern is practically designed to ruin the cache. + */ + List readers = new ArrayList(); + for (HStoreFile file: filesToCompact) { + try { + HStoreFile.BloomFilterMapFile.Reader reader = + file.getReader(fs, false, false); + readers.add(reader); + + // Compute the size of the new bloomfilter if needed + if (this.family.isBloomFilterEnabled()) { + nrows += reader.getBloomFilterSize(); + } + } catch (IOException e) { + // Add info about which file threw exception. It may not be in the + // exception message so output a message here where we know the + // culprit. + LOG.warn("Failed with " + e.toString() + ": " + file.toString()); + closeCompactionReaders(readers); + throw e; + } + } + + // Storefiles are keyed by sequence id. The oldest file comes first. + // We need to return out of here a List that has the newest file first. + Collections.reverse(readers); + + // Step through them, writing to the brand-new MapFile + HStoreFile compactedOutputFile = new HStoreFile(conf, fs, + this.compactionDir, info.getEncodedName(), family.getName(), + -1L, null); + if (LOG.isDebugEnabled()) { + LOG.debug("started compaction of " + readers.size() + " files into " + + FSUtils.getPath(compactedOutputFile.getMapFilePath())); + } + MapFile.Writer writer = compactedOutputFile.getWriter(this.fs, + this.compression, this.family.isBloomFilterEnabled(), nrows); + try { + compactHStoreFiles(writer, readers); + } finally { + writer.close(); + } + + // Now, write out an HSTORE_LOGINFOFILE for the brand-new TreeMap. + compactedOutputFile.writeInfo(fs, maxId); + + // Move the compaction into place. + completeCompaction(filesToCompact, compactedOutputFile); + + if (LOG.isDebugEnabled()) { + LOG.debug("Completed compaction of " + this.storeNameStr + + " store size is " + StringUtils.humanReadableInt(storeSize)); + } + } + return checkSplit(); + } + + /* + * Compact a list of MapFile.Readers into MapFile.Writer. + * + * We work by iterating through the readers in parallel. We always increment + * the lowest-ranked one. + * Updates to a single row/column will appear ranked by timestamp. This allows + * us to throw out deleted values or obsolete versions. + */ + private void compactHStoreFiles(final MapFile.Writer compactedOut, + final List readers) throws IOException { + + MapFile.Reader[] rdrs = readers.toArray(new MapFile.Reader[readers.size()]); + try { + HStoreKey[] keys = new HStoreKey[rdrs.length]; + ImmutableBytesWritable[] vals = new ImmutableBytesWritable[rdrs.length]; + boolean[] done = new boolean[rdrs.length]; + for(int i = 0; i < rdrs.length; i++) { + keys[i] = new HStoreKey(); + vals[i] = new ImmutableBytesWritable(); + done[i] = false; + } + + // Now, advance through the readers in order. This will have the + // effect of a run-time sort of the entire dataset. + int numDone = 0; + for(int i = 0; i < rdrs.length; i++) { + rdrs[i].reset(); + done[i] = ! rdrs[i].next(keys[i], vals[i]); + if(done[i]) { + numDone++; + } + } + + long now = System.currentTimeMillis(); + int timesSeen = 0; + byte [] lastRow = null; + byte [] lastColumn = null; + // Map of a row deletes keyed by column with a list of timestamps for value + Map> deletes = null; + while (numDone < done.length) { + // Find the reader with the smallest key. If two files have same key + // but different values -- i.e. one is delete and other is non-delete + // value -- we will find the first, the one that was written later and + // therefore the one whose value should make it out to the compacted + // store file. + int smallestKey = -1; + for(int i = 0; i < rdrs.length; i++) { + if(done[i]) { + continue; + } + if(smallestKey < 0) { + smallestKey = i; + } else { + if(keys[i].compareTo(keys[smallestKey]) < 0) { + smallestKey = i; + } + } + } + + // Reflect the current key/val in the output + HStoreKey sk = keys[smallestKey]; + if (Bytes.equals(lastRow, sk.getRow()) + && Bytes.equals(lastColumn, sk.getColumn())) { + timesSeen++; + } else { + timesSeen = 0; + // We are on to a new row. Create a new deletes list. + deletes = new TreeMap>(Bytes.BYTES_COMPARATOR); + } + + byte [] value = (vals[smallestKey] == null)? + null: vals[smallestKey].get(); + if (!isDeleted(sk, value, false, deletes) && + timesSeen <= family.getMaxVersions()) { + // Keep old versions until we have maxVersions worth. + // Then just skip them. + if (sk.getRow().length != 0 && sk.getColumn().length != 0) { + // Only write out objects which have a non-zero length key and + // value + if (ttl == HConstants.FOREVER || now < sk.getTimestamp() + ttl) { + compactedOut.append(sk, vals[smallestKey]); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("compactHStoreFiles: " + sk + ": expired, deleted"); + } + } + } + } + + // Update last-seen items + lastRow = sk.getRow(); + lastColumn = sk.getColumn(); + + // Advance the smallest key. If that reader's all finished, then + // mark it as done. + if(!rdrs[smallestKey].next(keys[smallestKey], + vals[smallestKey])) { + done[smallestKey] = true; + rdrs[smallestKey].close(); + rdrs[smallestKey] = null; + numDone++; + } + } + } finally { + closeCompactionReaders(readers); + } + } + + private void closeCompactionReaders(final List rdrs) { + for (MapFile.Reader r: rdrs) { + try { + r.close(); + } catch (IOException e) { + LOG.warn("Exception closing reader for " + this.storeNameStr, e); + } + } + } + + /* + * Check if this is cell is deleted. + * If a memcache and a deletes, check key does not have an entry filled. + * Otherwise, check value is not the HGlobals.deleteBytes value. + * If passed value IS deleteBytes, then it is added to the passed + * deletes map. + * @param hsk + * @param value + * @param checkMemcache true if the memcache should be consulted + * @param deletes Map keyed by column with a value of timestamp. Can be null. + * If non-null and passed value is HGlobals.deleteBytes, then we add to this + * map. + * @return True if this is a deleted cell. Adds the passed deletes map if + * passed value is HGlobals.deleteBytes. + */ + private boolean isDeleted(final HStoreKey hsk, final byte [] value, + final boolean checkMemcache, final Map> deletes) { + if (checkMemcache && memcache.isDeleted(hsk)) { + return true; + } + List timestamps = + (deletes == null) ? null: deletes.get(hsk.getColumn()); + if (timestamps != null && + timestamps.contains(Long.valueOf(hsk.getTimestamp()))) { + return true; + } + if (value == null) { + // If a null value, shouldn't be in here. Mark it as deleted cell. + return true; + } + if (!HLogEdit.isDeleted(value)) { + return false; + } + // Cell has delete value. Save it into deletes. + if (deletes != null) { + if (timestamps == null) { + timestamps = new ArrayList(); + deletes.put(hsk.getColumn(), timestamps); + } + // We know its not already in the deletes array else we'd have returned + // earlier so no need to test if timestamps already has this value. + timestamps.add(Long.valueOf(hsk.getTimestamp())); + } + return true; + } + + /* + * It's assumed that the compactLock will be acquired prior to calling this + * method! Otherwise, it is not thread-safe! + * + * It works by processing a compaction that's been written to disk. + * + *

It is usually invoked at the end of a compaction, but might also be + * invoked at HStore startup, if the prior execution died midway through. + * + *

Moving the compacted TreeMap into place means: + *

+   * 1) Moving the new compacted MapFile into place
+   * 2) Unload all replaced MapFiles, close and collect list to delete.
+   * 3) Loading the new TreeMap.
+   * 4) Compute new store size
+   * 
+ * + * @param compactedFiles list of files that were compacted + * @param compactedFile HStoreFile that is the result of the compaction + * @throws IOException + */ + private void completeCompaction(final List compactedFiles, + final HStoreFile compactedFile) + throws IOException { + this.lock.writeLock().lock(); + try { + // 1. Moving the new MapFile into place. + HStoreFile finalCompactedFile = new HStoreFile(conf, fs, basedir, + info.getEncodedName(), family.getName(), -1, null); + if (LOG.isDebugEnabled()) { + LOG.debug("moving " + FSUtils.getPath(compactedFile.getMapFilePath()) + + " to " + FSUtils.getPath(finalCompactedFile.getMapFilePath())); + } + if (!compactedFile.rename(this.fs, finalCompactedFile)) { + LOG.error("Failed move of compacted file " + + finalCompactedFile.getMapFilePath().toString()); + return; + } + + // 2. Unload all replaced MapFiles, close and collect list to delete. + synchronized (storefiles) { + Map toDelete = new HashMap(); + for (Map.Entry e : this.storefiles.entrySet()) { + if (!compactedFiles.contains(e.getValue())) { + continue; + } + Long key = e.getKey(); + MapFile.Reader reader = this.readers.remove(key); + if (reader != null) { + reader.close(); + } + toDelete.put(key, e.getValue()); + } + + try { + // 3. Loading the new TreeMap. + // Change this.storefiles so it reflects new state but do not + // delete old store files until we have sent out notification of + // change in case old files are still being accessed by outstanding + // scanners. + for (Long key : toDelete.keySet()) { + this.storefiles.remove(key); + } + // Add new compacted Reader and store file. + Long orderVal = Long.valueOf(finalCompactedFile.loadInfo(fs)); + this.readers.put(orderVal, + // Use a block cache (if configured) for this reader since + // it is the only one. + finalCompactedFile.getReader(this.fs, + this.family.isBloomFilterEnabled(), + this.family.isBlockCacheEnabled())); + this.storefiles.put(orderVal, finalCompactedFile); + // Tell observers that list of Readers has changed. + notifyChangedReadersObservers(); + // Finally, delete old store files. + for (HStoreFile hsf : toDelete.values()) { + hsf.delete(); + } + } catch (IOException e) { + e = RemoteExceptionHandler.checkIOException(e); + LOG.error("Failed replacing compacted files for " + + this.storeNameStr + + ". Compacted file is " + finalCompactedFile.toString() + + ". Files replaced are " + compactedFiles.toString() + + " some of which may have been already removed", e); + } + // 4. Compute new store size + storeSize = 0L; + for (HStoreFile hsf : storefiles.values()) { + storeSize += hsf.length(); + } + } + } finally { + this.lock.writeLock().unlock(); + } + } + + // //////////////////////////////////////////////////////////////////////////// + // Accessors. + // (This is the only section that is directly useful!) + ////////////////////////////////////////////////////////////////////////////// + + /** + * Return all the available columns for the given key. The key indicates a + * row and timestamp, but not a column name. + * + * The returned object should map column names to Cells. + */ + void getFull(HStoreKey key, final Set columns, + Map results) + throws IOException { + Map deletes = + new TreeMap(Bytes.BYTES_COMPARATOR); + + // if the key is null, we're not even looking for anything. return. + if (key == null) { + return; + } + + this.lock.readLock().lock(); + + // get from the memcache first. + memcache.getFull(key, columns, deletes, results); + + try { + MapFile.Reader[] maparray = getReaders(); + + // examine each mapfile + for (int i = maparray.length - 1; i >= 0; i--) { + MapFile.Reader map = maparray[i]; + + // synchronize on the map so that no one else iterates it at the same + // time + getFullFromMapFile(map, key, columns, deletes, results); + } + + } finally { + this.lock.readLock().unlock(); + } + } + + private void getFullFromMapFile(MapFile.Reader map, HStoreKey key, + Set columns, Map deletes, Map results) + throws IOException { + synchronized(map) { + long now = System.currentTimeMillis(); + + // seek back to the beginning + map.reset(); + + // seek to the closest key that should match the row we're looking for + ImmutableBytesWritable readval = new ImmutableBytesWritable(); + HStoreKey readkey = (HStoreKey)map.getClosest(key, readval); + if (readkey == null) { + return; + } + do { + byte [] readcol = readkey.getColumn(); + + // if we're looking for this column (or all of them), and there isn't + // already a value for this column in the results map, and the key we + // just read matches, then we'll consider it + if ((columns == null || columns.contains(readcol)) + && !results.containsKey(readcol) + && key.matchesWithoutColumn(readkey)) { + // if the value of the cell we're looking at right now is a delete, + // we need to treat it differently + if(HLogEdit.isDeleted(readval.get())) { + // if it's not already recorded as a delete or recorded with a more + // recent delete timestamp, record it for later + if (!deletes.containsKey(readcol) + || deletes.get(readcol).longValue() < readkey.getTimestamp()) { + deletes.put(readcol, readkey.getTimestamp()); + } + } else if (!(deletes.containsKey(readcol) + && deletes.get(readcol).longValue() >= readkey.getTimestamp()) ) { + // So the cell itself isn't a delete, but there may be a delete + // pending from earlier in our search. Only record this result if + // there aren't any pending deletes. + if (!(deletes.containsKey(readcol) && + deletes.get(readcol).longValue() >= readkey.getTimestamp())) { + if (ttl == HConstants.FOREVER || + now < readkey.getTimestamp() + ttl) { + results.put(readcol, + new Cell(readval.get(), readkey.getTimestamp())); + // need to reinstantiate the readval so we can reuse it, + // otherwise next iteration will destroy our result + readval = new ImmutableBytesWritable(); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("getFullFromMapFile: " + readkey + ": expired, skipped"); + } + } + } + } + } else if (Bytes.compareTo(key.getRow(), readkey.getRow()) < 0) { + // if we've crossed into the next row, then we can just stop + // iterating + break; + } + + } while(map.next(readkey, readval)); + } + } + + MapFile.Reader [] getReaders() { + return this.readers.values(). + toArray(new MapFile.Reader[this.readers.size()]); + } + + /** + * Get the value for the indicated HStoreKey. Grab the target value and the + * previous numVersions - 1 values, as well. + * + * Use {@link HConstants.ALL_VERSIONS} to retrieve all versions. + * @param key + * @param numVersions Number of versions to fetch. Must be > 0. + * @return values for the specified versions + * @throws IOException + */ + Cell[] get(HStoreKey key, int numVersions) throws IOException { + if (numVersions <= 0) { + throw new IllegalArgumentException("Number of versions must be > 0"); + } + + this.lock.readLock().lock(); + long now = System.currentTimeMillis(); + try { + // Check the memcache + List results = this.memcache.get(key, numVersions); + // If we got sufficient versions from memcache, return. + if (results.size() == numVersions) { + return results.toArray(new Cell[results.size()]); + } + + // Keep a list of deleted cell keys. We need this because as we go through + // the store files, the cell with the delete marker may be in one file and + // the old non-delete cell value in a later store file. If we don't keep + // around the fact that the cell was deleted in a newer record, we end up + // returning the old value if user is asking for more than one version. + // This List of deletes should not large since we are only keeping rows + // and columns that match those set on the scanner and which have delete + // values. If memory usage becomes an issue, could redo as bloom filter. + Map> deletes = + new TreeMap>(Bytes.BYTES_COMPARATOR); + // This code below is very close to the body of the getKeys method. + MapFile.Reader[] maparray = getReaders(); + for(int i = maparray.length - 1; i >= 0; i--) { + MapFile.Reader map = maparray[i]; + synchronized(map) { + map.reset(); + ImmutableBytesWritable readval = new ImmutableBytesWritable(); + HStoreKey readkey = (HStoreKey)map.getClosest(key, readval); + if (readkey == null) { + // 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. + continue; + } + if (!readkey.matchesRowCol(key)) { + continue; + } + if (!isDeleted(readkey, readval.get(), true, deletes)) { + if (ttl == HConstants.FOREVER || + now < readkey.getTimestamp() + ttl) { + results.add(new Cell(readval.get(), readkey.getTimestamp())); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("get: " + readkey + ": expired, skipped"); + } + } + // Perhaps only one version is wanted. I could let this + // test happen later in the for loop test but it would cost + // the allocation of an ImmutableBytesWritable. + if (hasEnoughVersions(numVersions, results)) { + break; + } + } + for (readval = new ImmutableBytesWritable(); + map.next(readkey, readval) && + readkey.matchesRowCol(key) && + !hasEnoughVersions(numVersions, results); + readval = new ImmutableBytesWritable()) { + if (!isDeleted(readkey, readval.get(), true, deletes)) { + if (ttl == HConstants.FOREVER || + now < readkey.getTimestamp() + ttl) { + results.add(new Cell(readval.get(), readkey.getTimestamp())); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("get: " + readkey + ": expired, skipped"); + } + } + } + } + } + if (hasEnoughVersions(numVersions, results)) { + break; + } + } + return results.size() == 0 ? + null : results.toArray(new Cell[results.size()]); + } finally { + this.lock.readLock().unlock(); + } + } + + /** + * Small method to check if we are over the max number of versions + * or we acheived this family max versions. + * The later happens when we have the situation described in HBASE-621. + * @param numVersions + * @param results + * @return + */ + private boolean hasEnoughVersions(final int numVersions, + final List results) { + return (results.size() >= numVersions || results.size() >= family + .getMaxVersions()); + } + + /** + * Get versions keys matching the origin key's + * row/column/timestamp and those of an older vintage + * Default access so can be accessed out of {@link HRegionServer}. + * @param origin Where to start searching. + * @param versions How many versions to return. Pass + * {@link HConstants.ALL_VERSIONS} to retrieve all. Versions will include + * size of passed allKeys in its count. + * @param allKeys List of keys prepopulated by keys we found in memcache. + * This method returns this passed list with all matching keys found in + * stores appended. + * @return The passed allKeys with versions of + * matching keys found in store files appended. + * @throws IOException + */ + List getKeys(final HStoreKey origin, final int versions) + throws IOException { + + List keys = this.memcache.getKeys(origin, versions); + if (keys.size() >= versions) { + return keys; + } + + // This code below is very close to the body of the get method. + this.lock.readLock().lock(); + long now = System.currentTimeMillis(); + try { + MapFile.Reader[] maparray = getReaders(); + for(int i = maparray.length - 1; i >= 0; i--) { + MapFile.Reader map = maparray[i]; + synchronized(map) { + map.reset(); + + // do the priming read + ImmutableBytesWritable readval = new ImmutableBytesWritable(); + HStoreKey readkey = (HStoreKey)map.getClosest(origin, readval); + if (readkey == null) { + // 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. + continue; + } + + do{ + // if the row matches, we might want this one. + if (rowMatches(origin, readkey)) { + // if the cell matches, then we definitely want this key. + if (cellMatches(origin, readkey)) { + // store the key if it isn't deleted or superceeded by what's + // in the memcache + if (!isDeleted(readkey, readval.get(), false, null) && + !keys.contains(readkey)) { + if (ttl == HConstants.FOREVER || + now < readkey.getTimestamp() + ttl) { + keys.add(new HStoreKey(readkey)); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("getKeys: " + readkey + + ": expired, skipped"); + } + } + + // if we've collected enough versions, then exit the loop. + if (keys.size() >= versions) { + break; + } + } + } else { + // the cell doesn't match, but there might be more with different + // timestamps, so move to the next key + continue; + } + } else { + // the row doesn't match, so we've gone too far. + break; + } + } while (map.next(readkey, readval)); // advance to the next key + } + } + + return keys; + } finally { + this.lock.readLock().unlock(); + } + } + + /** + * Find the key that matches row exactly, or the one that immediately + * preceeds it. WARNING: Only use this method on a table where writes occur + * with stricly increasing timestamps. This method assumes this pattern of + * writes in order to make it reasonably performant. + */ + byte [] getRowKeyAtOrBefore(final byte [] row) + throws IOException{ + // Map of HStoreKeys that are candidates for holding the row key that + // most closely matches what we're looking for. We'll have to update it + // deletes found all over the place as we go along before finally reading + // the best key out of it at the end. + SortedMap candidateKeys = new TreeMap(); + + // Obtain read lock + this.lock.readLock().lock(); + try { + // Process each store file + MapFile.Reader[] maparray = getReaders(); + for (int i = maparray.length - 1; i >= 0; i--) { + // update the candidate keys from the current map file + rowAtOrBeforeFromMapFile(maparray[i], row, candidateKeys); + } + + // Finally, check the memcache + this.memcache.getRowKeyAtOrBefore(row, candidateKeys); + + // Return the best key from candidateKeys + return candidateKeys.isEmpty()? null: candidateKeys.lastKey().getRow(); + } finally { + this.lock.readLock().unlock(); + } + } + + /** + * Check an individual MapFile for the row at or before a given key + * and timestamp + */ + private void rowAtOrBeforeFromMapFile(MapFile.Reader map, final byte [] row, + SortedMap candidateKeys) + throws IOException { + ImmutableBytesWritable readval = new ImmutableBytesWritable(); + HStoreKey readkey = new HStoreKey(); + + synchronized(map) { + // don't bother with the rest of this if the file is empty + map.reset(); + if (!map.next(readkey, readval)) { + return; + } + + long now = System.currentTimeMillis(); + + // if there aren't any candidate keys yet, we'll do some things slightly + // different + if (candidateKeys.isEmpty()) { + rowKeyFromMapFileEmptyKeys(map, row, candidateKeys, now); + } else { + rowKeyAtOrBeforeExistingCandKeys(map, row, candidateKeys, now); + } + } + } + + private void rowKeyFromMapFileEmptyKeys(MapFile.Reader map, byte[] row, + SortedMap candidateKeys, long now) + throws IOException { + + HStoreKey searchKey = new HStoreKey(row); + ImmutableBytesWritable readval = new ImmutableBytesWritable(); + HStoreKey readkey = new HStoreKey(); + + // if the row we're looking for is past the end of this mapfile, just + // save time and add the last key to the candidates. + HStoreKey finalKey = new HStoreKey(); + map.finalKey(finalKey); + if (Bytes.compareTo(finalKey.getRow(), row) < 0) { + candidateKeys.put(stripTimestamp(finalKey), + new Long(finalKey.getTimestamp())); + return; + } + + HStoreKey deletedOrExpiredRow = null; + boolean foundCandidate = false; + while (!foundCandidate) { + // seek to the exact row, or the one that would be immediately before it + readkey = (HStoreKey)map.getClosest(searchKey, readval, true); + + if (readkey == null) { + // didn't find anything that would match, so return + return; + } + + do { + // if we have an exact match on row, and it's not a delete, save this + // as a candidate key + if (Bytes.equals(readkey.getRow(), row)) { + if (!HLogEdit.isDeleted(readval.get())) { + if (ttl == HConstants.FOREVER || + now < readkey.getTimestamp() + ttl) { + candidateKeys.put(stripTimestamp(readkey), + new Long(readkey.getTimestamp())); + foundCandidate = true; + continue; + } + if (LOG.isDebugEnabled()) { + LOG.debug("rowAtOrBeforeFromMapFile:" + readkey + + ": expired, skipped"); + } + } + deletedOrExpiredRow = stripTimestamp(readkey); + } else if (Bytes.compareTo(readkey.getRow(), row) > 0 ) { + // if the row key we just read is beyond the key we're searching for, + // then we're done. return. + break; + } else { + // so, the row key doesn't match, but we haven't gone past the row + // we're seeking yet, so this row is a candidate for closest + // (assuming that it isn't a delete). + if (!HLogEdit.isDeleted(readval.get())) { + if (ttl == HConstants.FOREVER || + now < readkey.getTimestamp() + ttl) { + candidateKeys.put(stripTimestamp(readkey), + new Long(readkey.getTimestamp())); + foundCandidate = true; + continue; + } + if (LOG.isDebugEnabled()) { + LOG.debug("rowAtOrBeforeFromMapFile:" + readkey + + ": expired, skipped"); + } + } + deletedOrExpiredRow = stripTimestamp(readkey); + } + } while(map.next(readkey, readval)); + + // If we get here and have no candidates but we did find a deleted or + // expired candidate, we need to look at the key before that + + if (!foundCandidate && deletedOrExpiredRow != null) { + searchKey = deletedOrExpiredRow; + deletedOrExpiredRow = null; + + } else { + // No candidates and no deleted or expired candidates. Give up. + break; + } + } + + // arriving here just means that we consumed the whole rest of the map + // without going "past" the key we're searching for. we can just fall + // through here. + } + + + private void rowKeyAtOrBeforeExistingCandKeys(MapFile.Reader map, byte[] row, + SortedMap candidateKeys, long now) + throws IOException { + + HStoreKey strippedKey = null; + ImmutableBytesWritable readval = new ImmutableBytesWritable(); + HStoreKey readkey = new HStoreKey(); + + + // if there are already candidate keys, we need to start our search + // at the earliest possible key so that we can discover any possible + // deletes for keys between the start and the search key. + HStoreKey searchKey = new HStoreKey(candidateKeys.firstKey().getRow()); + + // if the row we're looking for is past the end of this mapfile, just + // save time and add the last key to the candidates. + HStoreKey finalKey = new HStoreKey(); + map.finalKey(finalKey); + if (Bytes.compareTo(finalKey.getRow(), searchKey.getRow()) < 0) { + strippedKey = stripTimestamp(finalKey); + + // if the candidate keys has a cell like this one already, + // then we might want to update the timestamp we're using on it + if (candidateKeys.containsKey(strippedKey)) { + long bestCandidateTs = + candidateKeys.get(strippedKey).longValue(); + if (bestCandidateTs < finalKey.getTimestamp()) { + candidateKeys.put(strippedKey, new Long(finalKey.getTimestamp())); + } + } else { + // otherwise, this is a new key, so put it up as a candidate + candidateKeys.put(strippedKey, new Long(finalKey.getTimestamp())); + } + return; + } + + // seek to the exact row, or the one that would be immediately before it + readkey = (HStoreKey)map.getClosest(searchKey, readval, true); + + if (readkey == null) { + // didn't find anything that would match, so return + return; + } + + do { + // if we have an exact match on row, and it's not a delete, save this + // as a candidate key + if (Bytes.equals(readkey.getRow(), row)) { + strippedKey = stripTimestamp(readkey); + if (!HLogEdit.isDeleted(readval.get())) { + if (ttl == HConstants.FOREVER || + now < readkey.getTimestamp() + ttl) { + candidateKeys.put(strippedKey, + new Long(readkey.getTimestamp())); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("rowAtOrBeforeFromMapFile: " + readkey + + ": expired, skipped"); + } + } + } else { + // if the candidate keys contain any that might match by timestamp, + // then check for a match and remove it if it's too young to + // survive the delete + if (candidateKeys.containsKey(strippedKey)) { + long bestCandidateTs = + candidateKeys.get(strippedKey).longValue(); + if (bestCandidateTs <= readkey.getTimestamp()) { + candidateKeys.remove(strippedKey); + } + } + } + } else if (Bytes.compareTo(readkey.getRow(), row) > 0 ) { + // if the row key we just read is beyond the key we're searching for, + // then we're done. return. + return; + } else { + strippedKey = stripTimestamp(readkey); + + // so, the row key doesn't match, but we haven't gone past the row + // we're seeking yet, so this row is a candidate for closest + // (assuming that it isn't a delete). + if (!HLogEdit.isDeleted(readval.get())) { + if (ttl == HConstants.FOREVER || + now < readkey.getTimestamp() + ttl) { + candidateKeys.put(strippedKey, readkey.getTimestamp()); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("rowAtOrBeforeFromMapFile: " + readkey + + ": expired, skipped"); + } + } + } else { + // if the candidate keys contain any that might match by timestamp, + // then check for a match and remove it if it's too young to + // survive the delete + if (candidateKeys.containsKey(strippedKey)) { + long bestCandidateTs = + candidateKeys.get(strippedKey).longValue(); + if (bestCandidateTs <= readkey.getTimestamp()) { + candidateKeys.remove(strippedKey); + } + } + } + } + } while(map.next(readkey, readval)); + } + + static HStoreKey stripTimestamp(HStoreKey key) { + return new HStoreKey(key.getRow(), key.getColumn()); + } + + /** + * Test that the target matches the origin. If the + * origin has an empty column, then it's assumed to mean any column + * matches and only match on row and timestamp. Otherwise, it compares the + * keys with HStoreKey.matchesRowCol(). + * @param origin The key we're testing against + * @param target The key we're testing + */ + private boolean cellMatches(HStoreKey origin, HStoreKey target){ + // if the origin's column is empty, then we're matching any column + if (Bytes.equals(origin.getColumn(), HConstants.EMPTY_BYTE_ARRAY)){ + // if the row matches, then... + if (Bytes.equals(target.getRow(), origin.getRow())) { + // check the timestamp + return target.getTimestamp() <= origin.getTimestamp(); + } + return false; + } + // otherwise, we want to match on row and column + return target.matchesRowCol(origin); + } + + /** + * Test that the target matches the origin. If the origin + * has an empty column, then it just tests row equivalence. Otherwise, it uses + * HStoreKey.matchesRowCol(). + * @param origin Key we're testing against + * @param target Key we're testing + */ + private boolean rowMatches(HStoreKey origin, HStoreKey target){ + // if the origin's column is empty, then we're matching any column + if (Bytes.equals(origin.getColumn(), HConstants.EMPTY_BYTE_ARRAY)) { + // if the row matches, then... + return Bytes.equals(target.getRow(), origin.getRow()); + } + // otherwise, we want to match on row and column + return target.matchesRowCol(origin); + } + + /** + * Determines if HStore can be split + * + * @return a StoreSize if store can be split, null otherwise + */ + StoreSize checkSplit() { + if (this.storefiles.size() <= 0) { + return null; + } + if (storeSize < this.desiredMaxFileSize) { + return null; + } + this.lock.readLock().lock(); + try { + // Not splitable if we find a reference store file present in the store. + boolean splitable = true; + long maxSize = 0L; + Long mapIndex = Long.valueOf(0L); + // Iterate through all the MapFiles + synchronized (storefiles) { + for (Map.Entry e: storefiles.entrySet()) { + HStoreFile curHSF = e.getValue(); + long size = curHSF.length(); + if (size > maxSize) { + // This is the largest one so far + maxSize = size; + mapIndex = e.getKey(); + } + if (splitable) { + splitable = !curHSF.isReference(); + } + } + } + if (!splitable) { + return null; + } + MapFile.Reader r = this.readers.get(mapIndex); + + // seek back to the beginning of mapfile + r.reset(); + + // get the first and last keys + HStoreKey firstKey = new HStoreKey(); + HStoreKey lastKey = new HStoreKey(); + Writable value = new ImmutableBytesWritable(); + r.next(firstKey, value); + r.finalKey(lastKey); + + // get the midkey + 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 (Bytes.equals(mk.getRow(), firstKey.getRow()) && + Bytes.equals(mk.getRow(), lastKey.getRow())) { + return null; + } + return new StoreSize(maxSize, mk.getRow()); + } + } catch(IOException e) { + LOG.warn("Failed getting store size for " + this.storeNameStr, e); + } finally { + this.lock.readLock().unlock(); + } + return null; + } + + /** @return aggregate size of HStore */ + public long getSize() { + return storeSize; + } + + ////////////////////////////////////////////////////////////////////////////// + // File administration + ////////////////////////////////////////////////////////////////////////////// + + /** + * Return a scanner for both the memcache and the HStore files + */ + protected InternalScanner getScanner(long timestamp, byte [][] targetCols, + byte [] firstRow, RowFilterInterface filter) + throws IOException { + lock.readLock().lock(); + try { + return new HStoreScanner(this, targetCols, firstRow, timestamp, filter); + } finally { + lock.readLock().unlock(); + } + } + + /** {@inheritDoc} */ + @Override + public String toString() { + return this.storeNameStr; + } + + /** + * @param p Path to check. + * @return True if the path has format of a HStoreFile reference. + */ + public static boolean isReference(final Path p) { + return isReference(p, REF_NAME_PARSER.matcher(p.getName())); + } + + private static boolean isReference(final Path p, final Matcher m) { + if (m == null || !m.matches()) { + LOG.warn("Failed match of store file name " + p.toString()); + throw new RuntimeException("Failed match of store file name " + + p.toString()); + } + return m.groupCount() > 1 && m.group(2) != null; + } + + /** + * @return Current list of store files. + */ + SortedMap getStorefiles() { + synchronized (this.storefiles) { + SortedMap copy = + new TreeMap(this.storefiles); + return copy; + } + } + + class StoreSize { + private final long size; + private final byte[] key; + StoreSize(long size, byte[] key) { + this.size = size; + this.key = new byte[key.length]; + System.arraycopy(key, 0, this.key, 0, key.length); + } + /* @return the size */ + long getSize() { + return size; + } + /* @return the key */ + byte[] getKey() { + return key; + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HStoreFile.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HStoreFile.java new file mode 100644 index 00000000000..2df45a097c2 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HStoreFile.java @@ -0,0 +1,1032 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Random; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.BlockFSInputStream; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Writables; +import org.apache.hadoop.io.MapFile; +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; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HStoreKey; + +/** + * A HStore data file. HStores usually have one or more of these files. They + * are produced by flushing the memcache to disk. + * + *

Each HStore maintains a bunch of different data files. The filename is a + * mix of the parent dir, the region name, the column name, and a file + * identifier. The name may also be a reference to a store file located + * elsewhere. This class handles all that path-building stuff for you. + * + *

An HStoreFile usually tracks 4 things: its parent dir, the region + * identifier, the column family, and the file identifier. If you know those + * four things, you know how to obtain the right HStoreFile. HStoreFiles may + * also refernce store files in another region serving either from + * the top-half of the remote file or from the bottom-half. Such references + * are made fast splitting regions. + * + *

Plain HStoreFiles are named for a randomly generated id as in: + * 1278437856009925445 A file by this name is made in both the + * mapfiles and info subdirectories of a + * HStore columnfamily directoy: E.g. If the column family is 'anchor:', then + * under the region directory there is a subdirectory named 'anchor' within + * which is a 'mapfiles' and 'info' subdirectory. In each will be found a + * file named something like 1278437856009925445, one to hold the + * data in 'mapfiles' and one under 'info' that holds the sequence id for this + * store file. + * + *

References to store files located over in some other region look like + * this: + * 1278437856009925445.hbaserepository,qAReLZD-OyQORZWq_vqR1k==,959247014679548184: + * i.e. an id followed by the name of the referenced region. The data + * ('mapfiles') of HStoreFile references are empty. The accompanying + * info file contains the + * midkey, the id of the remote store we're referencing and whether we're + * to serve the top or bottom region of the remote store file. Note, a region + * is not splitable if it has instances of store file references (References + * are cleaned up by compactions). + * + *

When merging or splitting HRegions, we might want to modify one of the + * params for an HStoreFile (effectively moving it elsewhere). + */ +public class HStoreFile implements HConstants { + static final Log LOG = LogFactory.getLog(HStoreFile.class.getName()); + static final byte INFO_SEQ_NUM = 0; + static final String HSTORE_DATFILE_DIR = "mapfiles"; + static final String HSTORE_INFO_DIR = "info"; + static final String HSTORE_FILTER_DIR = "filter"; + + /** + * For split HStoreFiles, specifies if the file covers the lower half or + * the upper half of the key range + */ + public static enum Range { + /** HStoreFile contains upper half of key range */ + top, + /** HStoreFile contains lower half of key range */ + bottom + } + + private final static Random rand = new Random(); + + private final Path basedir; + private final int encodedRegionName; + private final byte [] colFamily; + private final long fileId; + private final HBaseConfiguration conf; + private final FileSystem fs; + private final Reference reference; + + /** + * Constructor that fully initializes the object + * @param conf Configuration object + * @param basedir qualified path that is parent of region directory + * @param encodedRegionName file name friendly name of the region + * @param colFamily name of the column family + * @param fileId file identifier + * @param ref Reference to another HStoreFile. + * @throws IOException + */ + HStoreFile(HBaseConfiguration conf, FileSystem fs, Path basedir, + int encodedRegionName, byte [] colFamily, long fileId, + final Reference ref) throws IOException { + this.conf = conf; + this.fs = fs; + this.basedir = basedir; + this.encodedRegionName = encodedRegionName; + this.colFamily = colFamily; + + long id = fileId; + if (id == -1) { + Path mapdir = HStoreFile.getMapDir(basedir, encodedRegionName, colFamily); + Path testpath = null; + do { + id = Math.abs(rand.nextLong()); + testpath = new Path(mapdir, createHStoreFilename(id, -1)); + } while(fs.exists(testpath)); + } + this.fileId = id; + + // If a reference, construction does not write the pointer files. Thats + // done by invocations of writeReferenceFiles(hsf, fs). Happens at fast + // split time. + this.reference = ref; + } + + /** @return the region name */ + boolean isReference() { + return reference != null; + } + + Reference getReference() { + return reference; + } + + int getEncodedRegionName() { + return this.encodedRegionName; + } + + /** @return the column family */ + byte [] getColFamily() { + return colFamily; + } + + /** @return the file identifier */ + long getFileId() { + return fileId; + } + + // Build full filenames from those components + + /** @return path for MapFile */ + Path getMapFilePath() { + if (isReference()) { + return getMapFilePath(encodedRegionName, fileId, + reference.getEncodedRegionName()); + } + return getMapFilePath(this.encodedRegionName, fileId); + } + + private Path getMapFilePath(final Reference r) { + if (r == null) { + return getMapFilePath(); + } + return getMapFilePath(r.getEncodedRegionName(), r.getFileId()); + } + + private Path getMapFilePath(final int encodedName, final long fid) { + return getMapFilePath(encodedName, fid, HRegionInfo.NO_HASH); + } + + private Path getMapFilePath(final int encodedName, final long fid, + final int ern) { + return new Path(HStoreFile.getMapDir(basedir, encodedName, colFamily), + createHStoreFilename(fid, ern)); + } + + /** @return path for info file */ + Path getInfoFilePath() { + if (isReference()) { + return getInfoFilePath(encodedRegionName, fileId, + reference.getEncodedRegionName()); + + } + return getInfoFilePath(encodedRegionName, fileId); + } + + private Path getInfoFilePath(final int encodedName, final long fid) { + return getInfoFilePath(encodedName, fid, HRegionInfo.NO_HASH); + } + + private Path getInfoFilePath(final int encodedName, final long fid, + final int ern) { + return new Path(HStoreFile.getInfoDir(basedir, encodedName, colFamily), + createHStoreFilename(fid, ern)); + } + + // File handling + + /* + * Split by making two new store files that reference top and bottom regions + * of original store file. + * @param midKey + * @param dstA + * @param dstB + * @param fs + * @param c + * @throws IOException + * + * @param midKey the key which will be the starting key of the second region + * @param dstA the file which will contain keys from the start of the source + * @param dstB the file which will contain keys from midKey to end of source + * @param fs file system + * @param c configuration + * @throws IOException + */ + void splitStoreFile(final HStoreFile dstA, final HStoreFile dstB, + final FileSystem fs) + throws IOException { + dstA.writeReferenceFiles(fs); + dstB.writeReferenceFiles(fs); + } + + void writeReferenceFiles(final FileSystem fs) + throws IOException { + createOrFail(fs, getMapFilePath()); + writeSplitInfo(fs); + } + + /* + * If reference, create and write the remote store file id, the midkey and + * whether we're going against the top file region of the referent out to + * the info file. + * @param p Path to info file. + * @param hsf + * @param fs + * @throws IOException + */ + private void writeSplitInfo(final FileSystem fs) throws IOException { + Path p = getInfoFilePath(); + if (fs.exists(p)) { + throw new IOException("File already exists " + p.toString()); + } + FSDataOutputStream out = fs.create(p); + try { + reference.write(out); + } finally { + out.close(); + } + } + + /** + * @see #writeSplitInfo(FileSystem fs) + */ + static HStoreFile.Reference readSplitInfo(final Path p, final FileSystem fs) + throws IOException { + FSDataInputStream in = fs.open(p); + try { + HStoreFile.Reference r = new HStoreFile.Reference(); + r.readFields(in); + return r; + } finally { + in.close(); + } + } + + private void createOrFail(final FileSystem fs, final Path p) + throws IOException { + if (fs.exists(p)) { + throw new IOException("File already exists " + p.toString()); + } + if (!fs.createNewFile(p)) { + throw new IOException("Failed create of " + p); + } + } + + /** + * Reads in an info file + * + * @param fs file system + * @return The sequence id contained in the info file + * @throws IOException + */ + long loadInfo(FileSystem fs) throws IOException { + Path p = null; + if (isReference()) { + p = getInfoFilePath(reference.getEncodedRegionName(), reference.getFileId()); + } else { + p = getInfoFilePath(); + } + DataInputStream in = new DataInputStream(fs.open(p)); + try { + byte flag = in.readByte(); + if(flag == INFO_SEQ_NUM) { + return in.readLong(); + } + throw new IOException("Cannot process log file: " + p); + } finally { + in.close(); + } + } + + /** + * Writes the file-identifier to disk + * + * @param fs file system + * @param infonum file id + * @throws IOException + */ + void writeInfo(FileSystem fs, long infonum) throws IOException { + Path p = getInfoFilePath(); + FSDataOutputStream out = fs.create(p); + try { + out.writeByte(INFO_SEQ_NUM); + out.writeLong(infonum); + } finally { + out.close(); + } + } + + /** + * Delete store map files. + * @throws IOException + */ + public void delete() throws IOException { + fs.delete(getMapFilePath(), true); + fs.delete(getInfoFilePath(), true); + } + + /** + * Renames the mapfiles and info directories under the passed + * hsf directory. + * @param fs + * @param hsf + * @return True if succeeded. + * @throws IOException + */ + public boolean rename(final FileSystem fs, final HStoreFile hsf) + throws IOException { + Path src = getMapFilePath(); + if (!fs.exists(src)) { + throw new FileNotFoundException(src.toString()); + } + boolean success = fs.rename(src, hsf.getMapFilePath()); + if (!success) { + LOG.warn("Failed rename of " + src + " to " + hsf.getMapFilePath()); + } else { + src = getInfoFilePath(); + if (!fs.exists(src)) { + throw new FileNotFoundException(src.toString()); + } + success = fs.rename(src, hsf.getInfoFilePath()); + if (!success) { + LOG.warn("Failed rename of " + src + " to " + hsf.getInfoFilePath()); + } + } + return success; + } + + /** + * Get reader for the store file map file. + * Client is responsible for closing file when done. + * @param fs + * @param bloomFilter If true, a bloom filter exists + * @param blockCacheEnabled If true, MapFile blocks should be cached. + * @return BloomFilterMapFile.Reader + * @throws IOException + */ + public synchronized BloomFilterMapFile.Reader getReader(final FileSystem fs, + final boolean bloomFilter, final boolean blockCacheEnabled) + throws IOException { + if (isReference()) { + return new HStoreFile.HalfMapFileReader(fs, + getMapFilePath(reference).toString(), conf, + reference.getFileRegion(), reference.getMidkey(), bloomFilter, + blockCacheEnabled); + } + return new BloomFilterMapFile.Reader(fs, getMapFilePath().toString(), + conf, bloomFilter, blockCacheEnabled); + } + + /** + * Get a store file writer. + * Client is responsible for closing file when done. + * @param fs + * @param compression Pass SequenceFile.CompressionType.NONE + * for none. + * @param bloomFilter If true, create a bloom filter + * @param nrows number of rows expected. Required if bloomFilter is true. + * @return MapFile.Writer + * @throws IOException + */ + public MapFile.Writer getWriter(final FileSystem fs, + final SequenceFile.CompressionType compression, + final boolean bloomFilter, int nrows) + throws IOException { + if (isReference()) { + throw new IOException("Illegal Access: Cannot get a writer on a" + + "HStoreFile reference"); + } + return new BloomFilterMapFile.Writer(conf, fs, + getMapFilePath().toString(), compression, bloomFilter, nrows); + } + + /** + * @return Length of the store map file. If a reference, size is + * approximation. + * @throws IOException + */ + public long length() throws IOException { + Path p = new Path(getMapFilePath(reference), MapFile.DATA_FILE_NAME); + long l = p.getFileSystem(conf).getFileStatus(p).getLen(); + return (isReference())? l / 2: l; + } + + /** {@inheritDoc} */ + @Override + public String toString() { + return encodedRegionName + "/" + Bytes.toString(colFamily) + "/" + fileId + + (isReference()? "-" + reference.toString(): ""); + } + + static boolean isTopFileRegion(final Range r) { + return r.equals(Range.top); + } + + private static String createHStoreFilename(final long fid, + final int encodedRegionName) { + return Long.toString(fid) + + ((encodedRegionName != HRegionInfo.NO_HASH)? + "." + encodedRegionName : ""); + } + + /** + * @param dir Base directory + * @param encodedRegionName Encoding of region name. + * @param f Column family. + * @return path for map file directory + */ + public static Path getMapDir(Path dir, int encodedRegionName, + final byte [] f) { + return getFamilySubDir(dir, encodedRegionName, f, HSTORE_DATFILE_DIR); + } + + /** + * @param dir Base directory + * @param encodedRegionName Encoding of region name. + * @param f Column family. + * @return the info directory path + */ + public static Path getInfoDir(Path dir, int encodedRegionName, byte [] f) { + return getFamilySubDir(dir, encodedRegionName, f, HSTORE_INFO_DIR); + } + + /** + * @param dir Base directory + * @param encodedRegionName Encoding of region name. + * @param f Column family. + * @return the bloom filter directory path + */ + @Deprecated + public static Path getFilterDir(Path dir, int encodedRegionName, + final byte [] f) { + return getFamilySubDir(dir, encodedRegionName, f, HSTORE_FILTER_DIR); + } + + /* + * @param base Base directory + * @param encodedRegionName Encoding of region name. + * @param f Column family. + * @param subdir Subdirectory to create under column family/store directory. + * @return + */ + private static Path getFamilySubDir(final Path base, + final int encodedRegionName, final byte [] f, final String subdir) { + return new Path(base, new Path(Integer.toString(encodedRegionName), + new Path(Bytes.toString(f), subdir))); + } + + /* + * Data structure to hold reference to a store file over in another region. + */ + static class Reference implements Writable { + private int encodedRegionName; + private long fileid; + private Range region; + private HStoreKey midkey; + + Reference(final int ern, final long fid, final HStoreKey m, + final Range fr) { + this.encodedRegionName = ern; + this.fileid = fid; + this.region = fr; + this.midkey = m; + } + + Reference() { + this(-1, -1, null, Range.bottom); + } + + long getFileId() { + return fileid; + } + + Range getFileRegion() { + return region; + } + + HStoreKey getMidkey() { + return midkey; + } + + int getEncodedRegionName() { + return this.encodedRegionName; + } + + /** {@inheritDoc} */ + @Override + public String toString() { + return encodedRegionName + "/" + fileid + "/" + region; + } + + // Make it serializable. + + /** {@inheritDoc} */ + public void write(DataOutput out) throws IOException { + // Write out the encoded region name as a String. Doing it as a String + // keeps a Reference's serialziation backword compatible with + // pre-HBASE-82 serializations. ALternative is rewriting all + // info files in hbase (Serialized References are written into the + // 'info' file that accompanies HBase Store files). + out.writeUTF(Integer.toString(encodedRegionName)); + out.writeLong(fileid); + // Write true if we're doing top of the file. + out.writeBoolean(isTopFileRegion(region)); + midkey.write(out); + } + + /** {@inheritDoc} */ + public void readFields(DataInput in) throws IOException { + this.encodedRegionName = Integer.parseInt(in.readUTF()); + fileid = in.readLong(); + boolean tmp = in.readBoolean(); + // If true, set region to top. + region = tmp? Range.top: Range.bottom; + midkey = new HStoreKey(); + midkey.readFields(in); + } + } + + /** + * Hbase customizations of MapFile. + */ + static class HbaseMapFile extends MapFile { + static final Class KEY_CLASS = HStoreKey.class; + static final Class 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. + */ + static class HbaseReader extends MapFile.Reader { + + private final boolean blockCacheEnabled; + + /** + * @param fs + * @param dirName + * @param conf + * @throws IOException + */ + public HbaseReader(FileSystem fs, String dirName, Configuration conf) + throws IOException { + this(fs, dirName, conf, false); + } + + /** + * @param fs + * @param dirName + * @param conf + * @param blockCacheEnabled + * @throws IOException + */ + public HbaseReader(FileSystem fs, String dirName, Configuration conf, + boolean blockCacheEnabled) + throws IOException { + super(fs, dirName, null, conf, false); // defer opening streams + this.blockCacheEnabled = blockCacheEnabled; + open(fs, dirName, null, 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. + midKey(); + } + + @Override + protected org.apache.hadoop.io.SequenceFile.Reader createDataFileReader( + FileSystem fs, Path dataFile, Configuration conf) + throws IOException { + if (!blockCacheEnabled) { + return super.createDataFileReader(fs, dataFile, conf); + } + LOG.info("Block Cache enabled"); + final int blockSize = conf.getInt("hbase.hstore.blockCache.blockSize", + 64 * 1024); + return new SequenceFile.Reader(fs, dataFile, conf) { + @Override + protected FSDataInputStream openFile(FileSystem fs, Path file, + int bufferSize, long length) throws IOException { + + return new FSDataInputStream(new BlockFSInputStream( + super.openFile(fs, file, bufferSize, length), length, + blockSize)); + } + }; + } + } + + static class HbaseWriter extends MapFile.Writer { + /** + * @param conf + * @param fs + * @param dirName + * @param compression + * @throws IOException + */ + public HbaseWriter(Configuration conf, FileSystem fs, String dirName, + SequenceFile.CompressionType compression) + throws IOException { + super(conf, fs, dirName, KEY_CLASS, VALUE_CLASS, compression); + // 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)); + } + } + } + + /** + * On write, all keys are added to a bloom filter. On read, all keys are + * tested first against bloom filter. Keys are HStoreKey. If passed bloom + * filter is null, just passes invocation to parent. + */ + static class BloomFilterMapFile extends HbaseMapFile { + protected static final String BLOOMFILTER_FILE_NAME = "filter"; + + static class Reader extends HbaseReader { + private final BloomFilter bloomFilter; + + /** + * @param fs + * @param dirName + * @param conf + * @param filter + * @param blockCacheEnabled + * @throws IOException + */ + public Reader(FileSystem fs, String dirName, Configuration conf, + final boolean filter, final boolean blockCacheEnabled) + throws IOException { + super(fs, dirName, conf, blockCacheEnabled); + if (filter) { + this.bloomFilter = loadBloomFilter(fs, dirName); + } else { + this.bloomFilter = null; + } + } + + private BloomFilter loadBloomFilter(FileSystem fs, String dirName) + throws IOException { + Path filterFile = new Path(dirName, BLOOMFILTER_FILE_NAME); + if(!fs.exists(filterFile)) { + throw new FileNotFoundException("Could not find bloom filter: " + + filterFile); + } + BloomFilter filter = new BloomFilter(); + FSDataInputStream in = fs.open(filterFile); + try { + bloomFilter.readFields(in); + } finally { + fs.close(); + } + return filter; + } + + /** {@inheritDoc} */ + @Override + public Writable get(WritableComparable key, Writable val) + throws IOException { + if (bloomFilter == null) { + return super.get(key, val); + } + if(bloomFilter.membershipTest(getBloomFilterKey(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; + } + + /** {@inheritDoc} */ + @Override + public WritableComparable getClosest(WritableComparable key, + Writable val) throws IOException { + if (bloomFilter == null) { + return super.getClosest(key, val); + } + // Note - the key being passed to us is always a HStoreKey + if(bloomFilter.membershipTest(getBloomFilterKey(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; + } + + /* @return size of the bloom filter */ + int getBloomFilterSize() { + return bloomFilter == null ? 0 : bloomFilter.getVectorSize(); + } + } + + static class Writer extends HbaseWriter { + private static final double DEFAULT_NUMBER_OF_HASH_FUNCTIONS = 4.0; + private final BloomFilter bloomFilter; + private final String dirName; + private final FileSystem fs; + + /** + * @param conf + * @param fs + * @param dirName + * @param compression + * @param filter + * @param nrows + * @throws IOException + */ + @SuppressWarnings("unchecked") + public Writer(Configuration conf, FileSystem fs, String dirName, + SequenceFile.CompressionType compression, final boolean filter, + int nrows) + throws IOException { + super(conf, fs, dirName, compression); + this.dirName = dirName; + this.fs = fs; + if (filter) { + /* + * There is no way to automatically determine the vector size and the + * number of hash functions to use. In particular, bloom filters are + * very sensitive to the number of elements inserted into them. For + * HBase, the number of entries depends on the size of the data stored + * in the column. Currently the default region size is 256MB, so the + * number of entries is approximately + * 256MB / (average value size for column). + * + * If m denotes the number of bits in the Bloom filter (vectorSize), + * n denotes the number of elements inserted into the Bloom filter and + * k represents the number of hash functions used (nbHash), then + * according to Broder and Mitzenmacher, + * + * ( http://www.eecs.harvard.edu/~michaelm/NEWWORK/postscripts/BloomFilterSurvey.pdf ) + * + * the probability of false positives is minimized when k is + * approximately m/n ln(2). + */ + this.bloomFilter = new BloomFilter( + (int) DEFAULT_NUMBER_OF_HASH_FUNCTIONS, + (int) Math.ceil( + (DEFAULT_NUMBER_OF_HASH_FUNCTIONS * (1.0 * nrows)) / + Math.log(2.0)) + ); + } else { + this.bloomFilter = null; + } + } + + /** {@inheritDoc} */ + @Override + public void append(WritableComparable key, Writable val) + throws IOException { + if (bloomFilter != null) { + bloomFilter.add(getBloomFilterKey(key)); + } + super.append(key, val); + } + + /** {@inheritDoc} */ + @Override + public synchronized void close() throws IOException { + super.close(); + if (this.bloomFilter != null) { + flushBloomFilter(); + } + } + + /** + * Flushes bloom filter to disk + * + * @throws IOException + */ + private void flushBloomFilter() throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("flushing bloom filter for " + this.dirName); + } + FSDataOutputStream out = + fs.create(new Path(dirName, BLOOMFILTER_FILE_NAME)); + try { + bloomFilter.write(out); + } finally { + out.close(); + } + if (LOG.isDebugEnabled()) { + LOG.debug("flushed bloom filter for " + this.dirName); + } + } + } + } + + /** + * A facade for a {@link MapFile.Reader} that serves up either the top or + * bottom half of a MapFile (where 'bottom' is the first half of the file + * containing the keys that sort lowest and 'top' is the second half of the + * file with keys that sort greater than those of the bottom half). + * Subclasses BloomFilterMapFile.Reader in case + * + *

This file is not splitable. Calls to {@link #midKey()} return null. + */ + static class HalfMapFileReader extends BloomFilterMapFile.Reader { + private final boolean top; + private final WritableComparable midkey; + private boolean firstNextCall = true; + + HalfMapFileReader(final FileSystem fs, final String dirName, + final Configuration conf, final Range r, + final WritableComparable midKey) + throws IOException { + this(fs, dirName, conf, r, midKey, false, false); + } + + HalfMapFileReader(final FileSystem fs, final String dirName, + final Configuration conf, final Range r, + final WritableComparable midKey, final boolean filter, + final boolean blockCacheEnabled) + throws IOException { + super(fs, dirName, conf, filter, blockCacheEnabled); + top = isTopFileRegion(r); + midkey = midKey; + } + + @SuppressWarnings("unchecked") + private void checkKey(final WritableComparable key) + throws IOException { + if (top) { + if (key.compareTo(midkey) < 0) { + throw new IOException("Illegal Access: Key is less than midKey of " + + "backing mapfile"); + } + } else if (key.compareTo(midkey) >= 0) { + throw new IOException("Illegal Access: Key is greater than or equal " + + "to midKey of backing mapfile"); + } + } + + /** {@inheritDoc} */ + @Override + public synchronized void finalKey(WritableComparable key) + throws IOException { + if (top) { + super.finalKey(key); + } else { + reset(); + Writable value = new ImmutableBytesWritable(); + WritableComparable k = super.getClosest(midkey, value, true); + ByteArrayOutputStream byteout = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(byteout); + k.write(out); + ByteArrayInputStream bytein = + new ByteArrayInputStream(byteout.toByteArray()); + DataInputStream in = new DataInputStream(bytein); + key.readFields(in); + } + } + + /** {@inheritDoc} */ + @Override + public synchronized Writable get(WritableComparable key, Writable val) + throws IOException { + checkKey(key); + return super.get(key, val); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override + public synchronized WritableComparable getClosest(WritableComparable key, + Writable val) + throws IOException { + WritableComparable closest = null; + if (top) { + // If top, the lowest possible key is midkey. Do not have to check + // what comes back from super getClosest. Will return exact match or + // greater. + 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) { + // Check key is within range for bottom. + 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). + if (closest != null && closest.compareTo(this.midkey) >= 0) { + // Don't let this value out. + closest = null; + } + } + // Else, key is > midkey so let out closest = null. + } + return closest; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unused") + @Override + public synchronized WritableComparable midKey() throws IOException { + // Returns null to indicate file is not splitable. + return null; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override + public synchronized boolean next(WritableComparable key, Writable val) + throws IOException { + if (firstNextCall) { + firstNextCall = false; + 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(midkey, val); + // Now copy the mid key into the passed key. + if (nearest != null) { + Writables.copyWritable(nearest, key); + return true; + } + return false; + } + } + boolean result = super.next(key, val); + if (!top && key.compareTo(midkey) >= 0) { + result = false; + } + return result; + } + + /** {@inheritDoc} */ + @Override + public synchronized void reset() throws IOException { + if (top) { + firstNextCall = true; + seek(midkey); + return; + } + super.reset(); + } + + /** {@inheritDoc} */ + @Override + public synchronized boolean seek(WritableComparable key) + throws IOException { + checkKey(key); + return super.seek(key); + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HStoreScanner.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HStoreScanner.java new file mode 100644 index 00000000000..81ced6e8a6c --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HStoreScanner.java @@ -0,0 +1,269 @@ +/** + * Copyright 2008 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Scanner scans both the memcache and the HStore + */ +class HStoreScanner implements InternalScanner { + static final Log LOG = LogFactory.getLog(HStoreScanner.class); + + private InternalScanner[] scanners; + private TreeMap[] resultSets; + private HStoreKey[] keys; + private boolean wildcardMatch = false; + private boolean multipleMatchers = false; + private RowFilterInterface dataFilter; + private HStore store; + + /** Create an Scanner with a handle on the memcache and HStore files. */ + @SuppressWarnings("unchecked") + HStoreScanner(HStore store, byte [][] targetCols, byte [] firstRow, + long timestamp, RowFilterInterface filter) + throws IOException { + this.store = store; + this.dataFilter = filter; + if (null != dataFilter) { + dataFilter.reset(); + } + this.scanners = new InternalScanner[2]; + this.resultSets = new TreeMap[scanners.length]; + this.keys = new HStoreKey[scanners.length]; + + try { + scanners[0] = store.memcache.getScanner(timestamp, targetCols, firstRow); + scanners[1] = new StoreFileScanner(store, timestamp, targetCols, firstRow); + for (int i = 0; i < scanners.length; i++) { + if (scanners[i].isWildcardScanner()) { + this.wildcardMatch = true; + } + if (scanners[i].isMultipleMatchScanner()) { + this.multipleMatchers = true; + } + } + } catch(IOException e) { + for (int i = 0; i < this.scanners.length; i++) { + if(scanners[i] != null) { + closeScanner(i); + } + } + throw e; + } + + // Advance to the first key in each scanner. + // All results will match the required column-set and scanTime. + for (int i = 0; i < scanners.length; i++) { + keys[i] = new HStoreKey(); + resultSets[i] = new TreeMap(Bytes.BYTES_COMPARATOR); + if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) { + closeScanner(i); + } + } + } + + /** @return true if the scanner is a wild card scanner */ + public boolean isWildcardScanner() { + return wildcardMatch; + } + + /** @return true if the scanner is a multiple match scanner */ + public boolean isMultipleMatchScanner() { + return multipleMatchers; + } + + /** {@inheritDoc} */ + public boolean next(HStoreKey key, SortedMap results) + throws IOException { + + // Filtered flag is set by filters. If a cell has been 'filtered out' + // -- i.e. it is not to be returned to the caller -- the flag is 'true'. + boolean filtered = true; + boolean moreToFollow = true; + while (filtered && moreToFollow) { + // Find the lowest-possible key. + byte [] chosenRow = null; + long chosenTimestamp = -1; + for (int i = 0; i < this.keys.length; i++) { + if (scanners[i] != null && + (chosenRow == null || + (Bytes.compareTo(keys[i].getRow(), chosenRow) < 0) || + ((Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) && + (keys[i].getTimestamp() > chosenTimestamp)))) { + chosenRow = keys[i].getRow(); + chosenTimestamp = keys[i].getTimestamp(); + } + } + + // Filter whole row by row key? + filtered = dataFilter != null? dataFilter.filterRowKey(chosenRow) : false; + + // Store the key and results for each sub-scanner. Merge them as + // appropriate. + if (chosenTimestamp >= 0 && !filtered) { + // Here we are setting the passed in key with current row+timestamp + key.setRow(chosenRow); + key.setVersion(chosenTimestamp); + key.setColumn(HConstants.EMPTY_BYTE_ARRAY); + // Keep list of deleted cell keys within this row. We need this + // because as we go through scanners, the delete record may be in an + // early scanner and then the same record with a non-delete, non-null + // value in a later. Without history of what we've seen, we'll return + // deleted values. This List should not ever grow too large since we + // are only keeping rows and columns that match those set on the + // scanner and which have delete values. If memory usage becomes a + // problem, could redo as bloom filter. + List deletes = new ArrayList(); + for (int i = 0; i < scanners.length && !filtered; i++) { + while ((scanners[i] != null + && !filtered + && moreToFollow) + && (Bytes.compareTo(keys[i].getRow(), chosenRow) == 0)) { + // If we are doing a wild card match or there are multiple + // matchers per column, we need to scan all the older versions of + // this row to pick up the rest of the family members + if (!wildcardMatch + && !multipleMatchers + && (keys[i].getTimestamp() != chosenTimestamp)) { + break; + } + + // NOTE: We used to do results.putAll(resultSets[i]); + // but this had the effect of overwriting newer + // values with older ones. So now we only insert + // a result if the map does not contain the key. + HStoreKey hsk = new HStoreKey(key.getRow(), HConstants.EMPTY_BYTE_ARRAY, + key.getTimestamp()); + for (Map.Entry e : resultSets[i].entrySet()) { + hsk.setColumn(e.getKey()); + if (HLogEdit.isDeleted(e.getValue().getValue())) { + if (!deletes.contains(hsk)) { + // Key changes as we cycle the for loop so add a copy to + // the set of deletes. + deletes.add(new HStoreKey(hsk)); + } + } else if (!deletes.contains(hsk) && + !filtered && + moreToFollow && + !results.containsKey(e.getKey())) { + if (dataFilter != null) { + // Filter whole row by column data? + filtered = dataFilter.filterColumn(chosenRow, e.getKey(), + e.getValue().getValue()); + if (filtered) { + results.clear(); + break; + } + } + results.put(e.getKey(), e.getValue()); + } + } + resultSets[i].clear(); + if (!scanners[i].next(keys[i], resultSets[i])) { + closeScanner(i); + } + } + } + } + + for (int i = 0; i < scanners.length; i++) { + // If the current scanner is non-null AND has a lower-or-equal + // row label, then its timestamp is bad. We need to advance it. + while ((scanners[i] != null) && + (Bytes.compareTo(keys[i].getRow(), chosenRow) <= 0)) { + resultSets[i].clear(); + if (!scanners[i].next(keys[i], resultSets[i])) { + closeScanner(i); + } + } + } + + moreToFollow = chosenTimestamp >= 0; + + if (dataFilter != null) { + if (dataFilter.filterAllRemaining()) { + moreToFollow = false; + } + } + + if (results.size() <= 0 && !filtered) { + // There were no results found for this row. Marked it as + // 'filtered'-out otherwise we will not move on to the next row. + filtered = true; + } + } + + // If we got no results, then there is no more to follow. + if (results == null || results.size() <= 0) { + moreToFollow = false; + } + + // Make sure scanners closed if no more results + if (!moreToFollow) { + for (int i = 0; i < scanners.length; i++) { + if (null != scanners[i]) { + closeScanner(i); + } + } + } + + return moreToFollow; + } + + /** Shut down a single scanner */ + void closeScanner(int i) { + try { + try { + scanners[i].close(); + } catch (IOException e) { + LOG.warn(store.storeName + " failed closing scanner " + i, e); + } + } finally { + scanners[i] = null; + keys[i] = null; + resultSets[i] = null; + } + } + + /** {@inheritDoc} */ + public void close() { + for(int i = 0; i < scanners.length; i++) { + if(scanners[i] != null) { + closeScanner(i); + } + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/HTableDescriptor.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/HTableDescriptor.java new file mode 100644 index 00000000000..254dddb1b7d --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/HTableDescriptor.java @@ -0,0 +1,328 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.WritableComparable; + +/** + * HTableDescriptor contains the name of an HTable, and its + * column families. + */ +public class HTableDescriptor implements WritableComparable { + /** Table descriptor for -ROOT- catalog table */ + public static final HTableDescriptor ROOT_TABLEDESC = new HTableDescriptor( + HConstants.ROOT_TABLE_NAME, + new HColumnDescriptor[] { new HColumnDescriptor(HConstants.COLUMN_FAMILY, + 1, HColumnDescriptor.CompressionType.NONE, false, false, + Integer.MAX_VALUE, HConstants.FOREVER, false) }); + + /** Table descriptor for .META. catalog table */ + public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor( + HConstants.META_TABLE_NAME, new HColumnDescriptor[] { + new HColumnDescriptor(HConstants.COLUMN_FAMILY, 1, + HColumnDescriptor.CompressionType.NONE, false, false, + Integer.MAX_VALUE, HConstants.FOREVER, false), + new HColumnDescriptor(HConstants.COLUMN_FAMILY_HISTORIAN, + HConstants.ALL_VERSIONS, HColumnDescriptor.CompressionType.NONE, + false, false, Integer.MAX_VALUE, HConstants.FOREVER, false) }); + + private boolean rootregion = false; + private boolean metaregion = false; + private byte [] name = HConstants.EMPTY_BYTE_ARRAY; + private String nameAsString = ""; + + public static final String FAMILIES = "FAMILIES"; + + // Key is hash of the family name. + private final Map families = + new HashMap(); + + /** + * Private constructor used internally creating table descriptors for + * catalog tables: e.g. .META. and -ROOT-. + */ + private HTableDescriptor(final byte [] name, HColumnDescriptor[] families) { + this.name = name.clone(); + setMetaFlags(name); + for(HColumnDescriptor descriptor : families) { + this.families.put(Bytes.mapKey(descriptor.getName()), descriptor); + } + } + + /** + * Constructs an empty object. + * For deserializing an HTableDescriptor instance only. + * @see #HTableDescriptor(byte[]) + */ + public HTableDescriptor() { + super(); + } + + /** + * Constructor. + * @param name Table name. + * @throws IllegalArgumentException if passed a table name + * that is made of other than 'word' characters, underscore or period: i.e. + * [a-zA-Z_0-9.]. + * @see HADOOP-1581 HBASE: Un-openable tablename bug + */ + public HTableDescriptor(final String name) { + this(Bytes.toBytes(name)); + } + + /** + * Constructor. + * @param name Table name. + * @throws IllegalArgumentException if passed a table name + * that is made of other than 'word' characters, underscore or period: i.e. + * [a-zA-Z_0-9.]. + * @see HADOOP-1581 HBASE: Un-openable tablename bug + */ + public HTableDescriptor(final byte [] name) { + setMetaFlags(name); + this.name = this.metaregion? name: isLegalTableName(name); + this.nameAsString = Bytes.toString(this.name); + } + + /* + * Set meta flags on this table. + * Called by constructors. + * @param name + */ + private void setMetaFlags(final byte [] name) { + this.rootregion = Bytes.equals(name, HConstants.ROOT_TABLE_NAME); + this.metaregion = + this.rootregion? true: Bytes.equals(name, HConstants.META_TABLE_NAME); + } + + /** + * Check passed buffer is legal user-space table name. + * @param b Table name. + * @return Returns passed b param + * @throws NullPointerException If passed b is null + * @throws IllegalArgumentException if passed a table name + * that is made of other than 'word' characters or underscores: i.e. + * [a-zA-Z_0-9]. + */ + public static byte [] isLegalTableName(final byte [] b) { + if (b == null || b.length <= 0) { + throw new IllegalArgumentException("Name is null or empty"); + } + for (int i = 0; i < b.length; i++) { + if (Character.isLetterOrDigit(b[i]) || b[i] == '_') { + continue; + } + throw new IllegalArgumentException("Illegal character <" + b[i] + ">. " + + "User-space table names can only contain 'word characters':" + + "i.e. [a-zA-Z_0-9]: " + Bytes.toString(b)); + } + return b; + } + + /** @return true if this is the root region */ + public boolean isRootRegion() { + return rootregion; + } + + /** @return true if table is the meta table */ + public boolean isMetaTable() { + return metaregion && !rootregion; + } + + /** @return true if this is a meta region (part of the root or meta tables) */ + public boolean isMetaRegion() { + return metaregion; + } + + /** @return name of table */ + public byte [] getName() { + return name; + } + + /** @return name of table */ + public String getNameAsString() { + return this.nameAsString; + } + + /** + * Adds a column family. + * @param family HColumnDescriptor of familyto add. + */ + public void addFamily(final HColumnDescriptor family) { + if (family.getName() == null || family.getName().length <= 0) { + throw new NullPointerException("Family name cannot be null or empty"); + } + this.families.put(Bytes.mapKey(family.getName()), family); + } + + /** + * Checks to see if this table contains the given column family + * @param c Family name or column name. + * @return true if the table contains the specified family name + */ + public boolean hasFamily(final byte [] c) { + return hasFamily(c, HStoreKey.getFamilyDelimiterIndex(c)); + } + + /** + * Checks to see if this table contains the given column family + * @param c Family name or column name. + * @param index Index to column family delimiter + * @return true if the table contains the specified family name + */ + public boolean hasFamily(final byte [] c, final int index) { + // If index is -1, then presume we were passed a column family name minus + // the colon delimiter. + return families.containsKey(Bytes.mapKey(c, index == -1? c.length: index)); + } + + /** + * @return Name of this table and then a map of all of the column family + * descriptors. + * @see #getNameAsString() + */ + @Override + public String toString() { + return HConstants.NAME + " => '" + Bytes.toString(this.name) + + "', " + FAMILIES + " => " + this.families.values(); + } + + /** {@inheritDoc} */ + @Override + public boolean equals(Object obj) { + return compareTo(obj) == 0; + } + + /** {@inheritDoc} */ + @Override + public int hashCode() { + // TODO: Cache. + int result = Bytes.hashCode(this.name); + if (this.families != null && this.families.size() > 0) { + for (HColumnDescriptor e: this.families.values()) { + result ^= e.hashCode(); + } + } + return result; + } + + // Writable + + /** {@inheritDoc} */ + public void write(DataOutput out) throws IOException { + out.writeBoolean(rootregion); + out.writeBoolean(metaregion); + Bytes.writeByteArray(out, name); + out.writeInt(families.size()); + for(Iterator it = families.values().iterator(); + it.hasNext(); ) { + it.next().write(out); + } + } + + /** {@inheritDoc} */ + public void readFields(DataInput in) throws IOException { + this.rootregion = in.readBoolean(); + this.metaregion = in.readBoolean(); + this.name = Bytes.readByteArray(in); + this.nameAsString = Bytes.toString(this.name); + int numCols = in.readInt(); + this.families.clear(); + for (int i = 0; i < numCols; i++) { + HColumnDescriptor c = new HColumnDescriptor(); + c.readFields(in); + this.families.put(Bytes.mapKey(c.getName()), c); + } + } + + // Comparable + + /** {@inheritDoc} */ + public int compareTo(Object o) { + HTableDescriptor other = (HTableDescriptor) o; + int result = Bytes.compareTo(this.name, other.name); + if (result == 0) { + result = families.size() - other.families.size(); + } + + if (result == 0 && families.size() != other.families.size()) { + result = Integer.valueOf(families.size()).compareTo( + Integer.valueOf(other.families.size())); + } + + if (result == 0) { + for (Iterator it = families.values().iterator(), + it2 = other.families.values().iterator(); it.hasNext(); ) { + result = it.next().compareTo(it2.next()); + if (result != 0) { + break; + } + } + } + return result; + } + + /** + * @return Immutable sorted map of families. + */ + public Collection getFamilies() { + return Collections.unmodifiableCollection(this.families.values()); + } + + /** + * @param column + * @return Column descriptor for the passed family name or the family on + * passed in column. + */ + public HColumnDescriptor getFamily(final byte [] column) { + return this.families.get(HStoreKey.getFamilyMapKey(column)); + } + + /** + * @param column + * @return Column descriptor for the passed family name or the family on + * passed in column. + */ + public HColumnDescriptor removeFamily(final byte [] column) { + return this.families.remove(HStoreKey.getFamilyMapKey(column)); + } + + /** + * @param rootdir qualified path of HBase root directory + * @param tableName name of table + * @return path for table + */ + public static Path getTableDir(Path rootdir, final byte [] tableName) { + return new Path(rootdir, Bytes.toString(tableName)); + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/LogRollListener.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/LogRollListener.java new file mode 100644 index 00000000000..9dadeb14538 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/LogRollListener.java @@ -0,0 +1,29 @@ +/** + * Copyright 2007 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +/** + * Mechanism by which the HLog requests a log roll + */ +public interface LogRollListener { + /** Request that the log be rolled */ + public void logRollRequested(); +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/Memcache.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/Memcache.java new file mode 100644 index 00000000000..9eb2e926f54 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/Memcache.java @@ -0,0 +1,760 @@ +/** + * Copyright 2008 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.IOException; +import java.rmi.UnexpectedException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.regionserver.HAbstractScanner; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; + + +/** + * The Memcache holds in-memory modifications to the HRegion. + * Keeps a current map. When asked to flush the map, current map is moved + * to snapshot and is cleared. We continue to serve edits out of new map + * and backing snapshot until flusher reports in that the flush succeeded. At + * this point we let the snapshot go. + */ +class Memcache { + private final Log LOG = LogFactory.getLog(this.getClass().getName()); + + private long ttl; + + // Note that since these structures are always accessed with a lock held, + // so no additional synchronization is required. + + // The currently active sorted map of edits. + private volatile SortedMap memcache = + createSynchronizedSortedMap(); + + // Snapshot of memcache. Made for flusher. + private volatile SortedMap snapshot = + createSynchronizedSortedMap(); + + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + /** + * Default constructor. Used for tests. + */ + public Memcache() + { + ttl = HConstants.FOREVER; + } + + /** + * Constructor. + * @param ttl The TTL for cache entries, in milliseconds. + */ + public Memcache(long ttl) { + this.ttl = ttl; + } + + /* + * Utility method. + * @return sycnhronized sorted map of HStoreKey to byte arrays. + */ + private static SortedMap createSynchronizedSortedMap() { + return Collections.synchronizedSortedMap(new TreeMap()); + } + + /** + * Creates a snapshot of the current Memcache. + * Snapshot must be cleared by call to {@link #clearSnapshot(SortedMap)} + * To get the snapshot made by this method, use + * {@link #getSnapshot}. + */ + void snapshot() { + this.lock.writeLock().lock(); + try { + // If snapshot currently has entries, then flusher failed or didn't call + // cleanup. Log a warning. + if (this.snapshot.size() > 0) { + LOG.debug("Snapshot called again without clearing previous. " + + "Doing nothing. Another ongoing flush or did we fail last attempt?"); + } else { + // We used to synchronize on the memcache here but we're inside a + // write lock so removed it. Comment is left in case removal was a + // mistake. St.Ack + if (this.memcache.size() != 0) { + this.snapshot = this.memcache; + this.memcache = createSynchronizedSortedMap(); + } + } + } finally { + this.lock.writeLock().unlock(); + } + } + + /** + * Return the current snapshot. + * Called by flusher to get current snapshot made by a previous + * call to {@link snapshot}. + * @return Return snapshot. + * @see {@link #snapshot()} + * @see {@link #clearSnapshot(SortedMap)} + */ + SortedMap getSnapshot() { + return this.snapshot; + } + + /** + * The passed snapshot was successfully persisted; it can be let go. + * @param ss The snapshot to clean out. + * @throws UnexpectedException + * @see {@link #snapshot()} + */ + void clearSnapshot(final SortedMap ss) + throws UnexpectedException { + this.lock.writeLock().lock(); + try { + if (this.snapshot != ss) { + throw new UnexpectedException("Current snapshot is " + + this.snapshot + ", was passed " + ss); + } + // OK. Passed in snapshot is same as current snapshot. If not-empty, + // create a new snapshot and let the old one go. + if (ss.size() != 0) { + this.snapshot = createSynchronizedSortedMap(); + } + } finally { + this.lock.writeLock().unlock(); + } + } + + /** + * Write an update + * @param key + * @param value + * @return memcache size delta + */ + long add(final HStoreKey key, final byte[] value) { + this.lock.readLock().lock(); + try { + byte[] oldValue = this.memcache.remove(key); + this.memcache.put(key, value); + return key.getSize() + (value == null ? 0 : value.length) - + (oldValue == null ? 0 : oldValue.length); + } finally { + this.lock.readLock().unlock(); + } + } + + /** + * Look back through all the backlog TreeMaps to find the target. + * @param key + * @param numVersions + * @return An array of byte arrays ordered by timestamp. + */ + List get(final HStoreKey key, final int numVersions) { + this.lock.readLock().lock(); + try { + List results; + // The synchronizations here are because internalGet iterates + synchronized (this.memcache) { + results = internalGet(this.memcache, key, numVersions); + } + synchronized (this.snapshot) { + results.addAll(results.size(), + internalGet(this.snapshot, key, numVersions - results.size())); + } + return results; + } finally { + this.lock.readLock().unlock(); + } + } + + /** + * @param a + * @param b + * @return Return lowest of a or b or null if both a and b are null + */ + @SuppressWarnings("unchecked") + private byte [] getLowest(final byte [] a, + final byte [] b) { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + return Bytes.compareTo(a, b) <= 0? a: b; + } + + /** + * @param row Find the row that comes after this one. + * @return Next row or null if none found + */ + byte [] getNextRow(final byte [] row) { + this.lock.readLock().lock(); + try { + return getLowest(getNextRow(row, this.memcache), + getNextRow(row, this.snapshot)); + } finally { + this.lock.readLock().unlock(); + } + } + + /* + * @param row Find row that follows this one. + * @param map Map to look in for a row beyond row. + * This method synchronizes on passed map while iterating it. + * @return Next row or null if none found. + */ + private byte [] getNextRow(final byte [] row, + final SortedMap map) { + byte [] result = null; + // Synchronize on the map to make the tailMap making 'safe'. + synchronized (map) { + // Make an HSK with maximum timestamp so we get past most of the current + // rows cell entries. + HStoreKey hsk = new HStoreKey(row, HConstants.LATEST_TIMESTAMP); + SortedMap tailMap = map.tailMap(hsk); + // Iterate until we fall into the next row; i.e. move off current row + for (Map.Entry es: tailMap.entrySet()) { + HStoreKey itKey = es.getKey(); + if (Bytes.compareTo(itKey.getRow(), row) <= 0) { + continue; + } + // Note: Not suppressing deletes or expired cells. + result = itKey.getRow(); + break; + } + } + return result; + } + + /** + * Return all the available columns for the given key. The key indicates a + * row and timestamp, but not a column name. + * @param key + * @param columns Pass null for all columns else the wanted subset. + * @param deletes Map to accumulate deletes found. + * @param results Where to stick row results found. + */ + void getFull(HStoreKey key, Set columns, Map deletes, + Map results) { + this.lock.readLock().lock(); + try { + // The synchronizations here are because internalGet iterates + synchronized (this.memcache) { + internalGetFull(this.memcache, key, columns, deletes, results); + } + synchronized (this.snapshot) { + internalGetFull(this.snapshot, key, columns, deletes, results); + } + } finally { + this.lock.readLock().unlock(); + } + } + + private void internalGetFull(SortedMap map, HStoreKey key, + Set columns, Map deletes, + Map results) { + if (map.isEmpty() || key == null) { + return; + } + List victims = new ArrayList(); + SortedMap tailMap = map.tailMap(key); + long now = System.currentTimeMillis(); + for (Map.Entry es: tailMap.entrySet()) { + HStoreKey itKey = es.getKey(); + byte [] itCol = itKey.getColumn(); + if (results.get(itCol) == null && key.matchesWithoutColumn(itKey)) { + if (columns == null || columns.contains(itKey.getColumn())) { + byte [] val = tailMap.get(itKey); + if (HLogEdit.isDeleted(val)) { + if (!deletes.containsKey(itCol) + || deletes.get(itCol).longValue() < itKey.getTimestamp()) { + deletes.put(itCol, Long.valueOf(itKey.getTimestamp())); + } + } else if (!(deletes.containsKey(itCol) + && deletes.get(itCol).longValue() >= itKey.getTimestamp())) { + // Skip expired cells + if (ttl == HConstants.FOREVER || + now < itKey.getTimestamp() + ttl) { + results.put(itCol, new Cell(val, itKey.getTimestamp())); + } else { + victims.add(itKey); + if (LOG.isDebugEnabled()) { + LOG.debug("internalGetFull: " + itKey + ": expired, skipped"); + } + } + } + } + } else if (Bytes.compareTo(key.getRow(), itKey.getRow()) < 0) { + break; + } + } + // Remove expired victims from the map. + for (HStoreKey v: victims) + map.remove(v); + } + + /** + * @param row Row to look for. + * @param candidateKeys Map of candidate keys (Accumulation over lots of + * lookup over stores and memcaches) + */ + void getRowKeyAtOrBefore(final byte [] row, + SortedMap candidateKeys) { + this.lock.readLock().lock(); + try { + synchronized (memcache) { + internalGetRowKeyAtOrBefore(memcache, row, candidateKeys); + } + synchronized (snapshot) { + internalGetRowKeyAtOrBefore(snapshot, row, candidateKeys); + } + } finally { + this.lock.readLock().unlock(); + } + } + + private void internalGetRowKeyAtOrBefore(SortedMap map, + byte [] key, SortedMap candidateKeys) { + HStoreKey strippedKey = null; + + // we want the earliest possible to start searching from + HStoreKey search_key = candidateKeys.isEmpty() ? + new HStoreKey(key) : new HStoreKey(candidateKeys.firstKey().getRow()); + Iterator key_iterator = null; + HStoreKey found_key = null; + ArrayList victims = new ArrayList(); + long now = System.currentTimeMillis(); + // get all the entries that come equal or after our search key + SortedMap tailMap = map.tailMap(search_key); + + // if there are items in the tail map, there's either a direct match to + // the search key, or a range of values between the first candidate key + // and the ultimate search key (or the end of the cache) + if (!tailMap.isEmpty() && + Bytes.compareTo(tailMap.firstKey().getRow(), key) <= 0) { + key_iterator = tailMap.keySet().iterator(); + + // keep looking at cells as long as they are no greater than the + // ultimate search key and there's still records left in the map. + do { + found_key = key_iterator.next(); + if (Bytes.compareTo(found_key.getRow(), key) <= 0) { + strippedKey = stripTimestamp(found_key); + if (HLogEdit.isDeleted(tailMap.get(found_key))) { + if (candidateKeys.containsKey(strippedKey)) { + long bestCandidateTs = + candidateKeys.get(strippedKey).longValue(); + if (bestCandidateTs <= found_key.getTimestamp()) { + candidateKeys.remove(strippedKey); + } + } + } else { + if (ttl == HConstants.FOREVER || + now < found_key.getTimestamp() + ttl) { + candidateKeys.put(strippedKey, + new Long(found_key.getTimestamp())); + } else { + victims.add(found_key); + if (LOG.isDebugEnabled()) { + LOG.debug(":" + found_key + ": expired, skipped"); + } + } + } + } + } while (Bytes.compareTo(found_key.getRow(), key) <= 0 + && key_iterator.hasNext()); + } else { + // the tail didn't contain any keys that matched our criteria, or was + // empty. examine all the keys that preceed our splitting point. + SortedMap headMap = map.headMap(search_key); + + // if we tried to create a headMap and got an empty map, then there are + // no keys at or before the search key, so we're done. + if (headMap.isEmpty()) { + return; + } + + // if there aren't any candidate keys at this point, we need to search + // backwards until we find at least one candidate or run out of headMap. + if (candidateKeys.isEmpty()) { + HStoreKey[] cells = + headMap.keySet().toArray(new HStoreKey[headMap.keySet().size()]); + + byte [] lastRowFound = null; + for(int i = cells.length - 1; i >= 0; i--) { + HStoreKey thisKey = cells[i]; + + // if the last row we found a candidate key for is different than + // the row of the current candidate, we can stop looking. + if (lastRowFound != null && + !Bytes.equals(lastRowFound, thisKey.getRow())) { + break; + } + + // if this isn't a delete, record it as a candidate key. also + // take note of the row of this candidate so that we'll know when + // we cross the row boundary into the previous row. + if (!HLogEdit.isDeleted(headMap.get(thisKey))) { + if (ttl == HConstants.FOREVER) { + lastRowFound = thisKey.getRow(); + candidateKeys.put(stripTimestamp(thisKey), + new Long(thisKey.getTimestamp())); + } else { + victims.add(found_key); + if (LOG.isDebugEnabled()) { + LOG.debug("internalGetRowKeyAtOrBefore: " + found_key + + ": expired, skipped"); + } + } + } + } + } else { + // if there are already some candidate keys, we only need to consider + // the very last row's worth of keys in the headMap, because any + // smaller acceptable candidate keys would have caused us to start + // our search earlier in the list, and we wouldn't be searching here. + SortedMap thisRowTailMap = + headMap.tailMap(new HStoreKey(headMap.lastKey().getRow())); + + key_iterator = thisRowTailMap.keySet().iterator(); + + do { + found_key = key_iterator.next(); + + if (HLogEdit.isDeleted(thisRowTailMap.get(found_key))) { + strippedKey = stripTimestamp(found_key); + if (candidateKeys.containsKey(strippedKey)) { + long bestCandidateTs = + candidateKeys.get(strippedKey).longValue(); + if (bestCandidateTs <= found_key.getTimestamp()) { + candidateKeys.remove(strippedKey); + } + } + } else { + if (ttl == HConstants.FOREVER || + now < found_key.getTimestamp() + ttl) { + candidateKeys.put(stripTimestamp(found_key), + Long.valueOf(found_key.getTimestamp())); + } else { + victims.add(found_key); + if (LOG.isDebugEnabled()) { + LOG.debug("internalGetRowKeyAtOrBefore: " + found_key + + ": expired, skipped"); + } + } + } + } while (key_iterator.hasNext()); + } + } + // Remove expired victims from the map. + for (HStoreKey victim: victims) + map.remove(victim); + } + + static HStoreKey stripTimestamp(HStoreKey key) { + return new HStoreKey(key.getRow(), key.getColumn()); + } + + /** + * Examine a single map for the desired key. + * + * TODO - This is kinda slow. We need a data structure that allows for + * proximity-searches, not just precise-matches. + * + * @param map + * @param key + * @param numVersions + * @return Ordered list of items found in passed map. If no + * matching values, returns an empty list (does not return null). + */ + private ArrayList internalGet( + final SortedMap map, final HStoreKey key, + final int numVersions) { + + ArrayList result = new ArrayList(); + + // TODO: If get is of a particular version -- numVersions == 1 -- we + // should be able to avoid all of the tailmap creations and iterations + // below. + long now = System.currentTimeMillis(); + List victims = new ArrayList(); + SortedMap tailMap = map.tailMap(key); + for (Map.Entry es: tailMap.entrySet()) { + HStoreKey itKey = es.getKey(); + if (itKey.matchesRowCol(key)) { + if (!HLogEdit.isDeleted(es.getValue())) { + // Filter out expired results + if (ttl == HConstants.FOREVER || + now < itKey.getTimestamp() + ttl) { + result.add(new Cell(tailMap.get(itKey), itKey.getTimestamp())); + if (numVersions > 0 && result.size() >= numVersions) { + break; + } + } else { + victims.add(itKey); + if (LOG.isDebugEnabled()) { + LOG.debug("internalGet: " + itKey + ": expired, skipped"); + } + } + } + } else { + // By L.N. HBASE-684, map is sorted, so we can't find match any more. + break; + } + } + // Remove expired victims from the map. + for (HStoreKey v: victims) { + map.remove(v); + } + return result; + } + + /** + * Get versions keys matching the origin key's + * row/column/timestamp and those of an older vintage + * Default access so can be accessed out of {@link HRegionServer}. + * @param origin Where to start searching. + * @param versions How many versions to return. Pass + * {@link HConstants.ALL_VERSIONS} to retrieve all. + * @return Ordered list of versions keys going from newest back. + * @throws IOException + */ + List getKeys(final HStoreKey origin, final int versions) { + this.lock.readLock().lock(); + try { + List results; + synchronized (memcache) { + results = internalGetKeys(this.memcache, origin, versions); + } + synchronized (snapshot) { + results.addAll(results.size(), internalGetKeys(snapshot, origin, + versions == HConstants.ALL_VERSIONS ? versions : + (versions - results.size()))); + } + return results; + + } finally { + this.lock.readLock().unlock(); + } + } + + /* + * @param origin Where to start searching. + * @param versions How many versions to return. Pass + * {@link HConstants.ALL_VERSIONS} to retrieve all. + * @return List of all keys that are of the same row and column and of + * equal or older timestamp. If no keys, returns an empty List. Does not + * return null. + */ + private List internalGetKeys( + final SortedMap map, final HStoreKey origin, + final int versions) { + + long now = System.currentTimeMillis(); + List result = new ArrayList(); + List victims = new ArrayList(); + SortedMap tailMap = map.tailMap(origin); + for (Map.Entry es: tailMap.entrySet()) { + HStoreKey key = es.getKey(); + + // if there's no column name, then compare rows and timestamps + if (origin.getColumn() != null && origin.getColumn().length == 0) { + // if the current and origin row don't match, then we can jump + // out of the loop entirely. + if (!Bytes.equals(key.getRow(), origin.getRow())) { + break; + } + // if the rows match but the timestamp is newer, skip it so we can + // get to the ones we actually want. + if (key.getTimestamp() > origin.getTimestamp()) { + continue; + } + } + else{ // compare rows and columns + // if the key doesn't match the row and column, then we're done, since + // all the cells are ordered. + if (!key.matchesRowCol(origin)) { + break; + } + } + if (!HLogEdit.isDeleted(es.getValue())) { + if (ttl == HConstants.FOREVER || now < key.getTimestamp() + ttl) { + result.add(key); + } else { + victims.add(key); + if (LOG.isDebugEnabled()) { + LOG.debug("internalGetKeys: " + key + ": expired, skipped"); + } + } + if (result.size() >= versions) { + // We have enough results. Return. + break; + } + } + } + + // Clean expired victims from the map. + for (HStoreKey v: victims) + map.remove(v); + + return result; + } + + + /** + * @param key + * @return True if an entry and its content is {@link HGlobals.deleteBytes}. + * Use checking values in store. On occasion the memcache has the fact that + * the cell has been deleted. + */ + boolean isDeleted(final HStoreKey key) { + return HLogEdit.isDeleted(this.memcache.get(key)); + } + + /** + * @return a scanner over the keys in the Memcache + */ + InternalScanner getScanner(long timestamp, + final byte [][] targetCols, final byte [] firstRow) + throws IOException { + this.lock.readLock().lock(); + try { + return new MemcacheScanner(timestamp, targetCols, firstRow); + } finally { + this.lock.readLock().unlock(); + } + } + + ////////////////////////////////////////////////////////////////////////////// + // MemcacheScanner implements the InternalScanner. + // It lets the caller scan the contents of the Memcache. + ////////////////////////////////////////////////////////////////////////////// + + private class MemcacheScanner extends HAbstractScanner { + private byte [] currentRow; + private Set columns = null; + + MemcacheScanner(final long timestamp, final byte [] targetCols[], + final byte [] firstRow) + throws IOException { + // Call to super will create ColumnMatchers and whether this is a regex + // scanner or not. Will also save away timestamp. Also sorts rows. + super(timestamp, targetCols); + this.currentRow = firstRow; + // If we're being asked to scan explicit columns rather than all in + // a family or columns that match regexes, cache the sorted array of + // columns. + this.columns = null; + if (!isWildcardScanner()) { + this.columns = new TreeSet(Bytes.BYTES_COMPARATOR); + for (int i = 0; i < targetCols.length; i++) { + this.columns.add(targetCols[i]); + } + } + } + + /** {@inheritDoc} */ + @Override + public boolean next(HStoreKey key, SortedMap results) + throws IOException { + if (this.scannerClosed) { + return false; + } + // This is a treemap rather than a Hashmap because then I can have a + // byte array as key -- because I can independently specify a comparator. + Map deletes = + new TreeMap(Bytes.BYTES_COMPARATOR); + // Catch all row results in here. These results are ten filtered to + // ensure they match column name regexes, or if none, added to results. + Map rowResults = + new TreeMap(Bytes.BYTES_COMPARATOR); + if (results.size() > 0) { + results.clear(); + } + long latestTimestamp = -1; + while (results.size() <= 0 && this.currentRow != null) { + if (deletes.size() > 0) { + deletes.clear(); + } + if (rowResults.size() > 0) { + rowResults.clear(); + } + key.setRow(this.currentRow); + key.setVersion(this.timestamp); + getFull(key, isWildcardScanner() ? null : this.columns, deletes, + rowResults); + for (Map.Entry e: deletes.entrySet()) { + rowResults.put(e.getKey(), + new Cell(HLogEdit.deleteBytes.get(), e.getValue().longValue())); + } + for (Map.Entry e: rowResults.entrySet()) { + byte [] column = e.getKey(); + Cell c = e.getValue(); + if (isWildcardScanner()) { + // Check the results match. We only check columns, not timestamps. + // We presume that timestamps have been handled properly when we + // called getFull. + if (!columnMatch(column)) { + continue; + } + } + // We should never return HConstants.LATEST_TIMESTAMP as the time for + // the row. As a compromise, we return the largest timestamp for the + // entries that we find that match. + if (c.getTimestamp() != HConstants.LATEST_TIMESTAMP && + c.getTimestamp() > latestTimestamp) { + latestTimestamp = c.getTimestamp(); + } + results.put(column, c); + } + this.currentRow = getNextRow(this.currentRow); + + } + // Set the timestamp to the largest one for the row if we would otherwise + // return HConstants.LATEST_TIMESTAMP + if (key.getTimestamp() == HConstants.LATEST_TIMESTAMP) { + key.setVersion(latestTimestamp); + } + return results.size() > 0; + } + + /** {@inheritDoc} */ + public void close() { + if (!scannerClosed) { + scannerClosed = true; + } + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/MetaUtils.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/MetaUtils.java new file mode 100644 index 00000000000..f3440202f3d --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/MetaUtils.java @@ -0,0 +1,452 @@ +/** + * Copyright 2008 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.Writables; + +/** + * Contains utility methods for manipulating HBase meta tables. + * Be sure to call {@link #shutdown()} when done with this class so it closes + * resources opened during meta processing (ROOT, META, etc.). Be careful + * how you use this class. If used during migrations, be careful when using + * this class to check whether migration is needed. + */ +public class MetaUtils { + private static final Log LOG = LogFactory.getLog(MetaUtils.class); + private final HBaseConfiguration conf; + private FileSystem fs; + private Path rootdir; + private HLog log; + private HRegion rootRegion; + private Map metaRegions = Collections.synchronizedSortedMap( + new TreeMap(Bytes.BYTES_COMPARATOR)); + + /** Default constructor + * @throws IOException */ + public MetaUtils() throws IOException { + this(new HBaseConfiguration()); + } + + /** @param conf HBaseConfiguration + * @throws IOException */ + public MetaUtils(HBaseConfiguration conf) throws IOException { + this.conf = conf; + conf.setInt("hbase.client.retries.number", 1); + this.rootRegion = null; + initialize(); + } + + /** + * Verifies that DFS is available and that HBase is off-line. + * @throws IOException + */ + private void initialize() throws IOException { + this.fs = FileSystem.get(this.conf); // get DFS handle + // Get root directory of HBase installation + this.rootdir = fs.makeQualified(new Path(this.conf.get(HConstants.HBASE_DIR))); + if (!fs.exists(rootdir)) { + String message = "HBase root directory " + rootdir.toString() + + " does not exist."; + LOG.error(message); + throw new FileNotFoundException(message); + } + } + + /** @return the HLog + * @throws IOException */ + public synchronized HLog getLog() throws IOException { + if (this.log == null) { + Path logdir = new Path(this.fs.getHomeDirectory(), + HConstants.HREGION_LOGDIR_NAME + "_" + System.currentTimeMillis()); + this.log = new HLog(this.fs, logdir, this.conf, null); + } + return this.log; + } + + /** + * @return HRegion for root region + * @throws IOException + */ + public HRegion getRootRegion() throws IOException { + if (this.rootRegion == null) { + openRootRegion(); + } + return this.rootRegion; + } + + /** + * Open or return cached opened meta region + * + * @param metaInfo HRegionInfo for meta region + * @return meta HRegion + * @throws IOException + */ + public HRegion getMetaRegion(HRegionInfo metaInfo) throws IOException { + HRegion meta = metaRegions.get(metaInfo.getRegionName()); + if (meta == null) { + meta = openMetaRegion(metaInfo); + this.metaRegions.put(metaInfo.getRegionName(), meta); + } + return meta; + } + + /** + * Closes catalog regions if open. Also closes and deletes the HLog. You + * must call this method if you want to persist changes made during a + * MetaUtils edit session. + */ + public void shutdown() { + if (this.rootRegion != null) { + try { + this.rootRegion.close(); + } catch (IOException e) { + LOG.error("closing root region", e); + } finally { + this.rootRegion = null; + } + } + try { + for (HRegion r: metaRegions.values()) { + r.close(); + } + } catch (IOException e) { + LOG.error("closing meta region", e); + } finally { + metaRegions.clear(); + } + try { + if (this.log != null) { + this.log.rollWriter(); + this.log.closeAndDelete(); + } + } catch (IOException e) { + LOG.error("closing HLog", e); + } finally { + this.log = null; + } + } + + /** + * Used by scanRootRegion and scanMetaRegion to call back the caller so it + * can process the data for a row. + */ + public interface ScannerListener { + /** + * Callback so client of scanner can process row contents + * + * @param info HRegionInfo for row + * @return false to terminate the scan + * @throws IOException + */ + public boolean processRow(HRegionInfo info) throws IOException; + } + + /** + * Scans the root region. For every meta region found, calls the listener with + * the HRegionInfo of the meta region. + * + * @param listener method to be called for each meta region found + * @throws IOException + */ + public void scanRootRegion(ScannerListener listener) throws IOException { + // Open root region so we can scan it + if (this.rootRegion == null) { + openRootRegion(); + } + InternalScanner rootScanner = rootRegion.getScanner( + HConstants.COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW, + HConstants.LATEST_TIMESTAMP, null); + try { + HStoreKey key = new HStoreKey(); + SortedMap results = + new TreeMap(Bytes.BYTES_COMPARATOR); + while (rootScanner.next(key, results)) { + HRegionInfo info = (HRegionInfo)Writables.getWritable( + results.get(HConstants.COL_REGIONINFO).getValue(), + new HRegionInfo()); + if (info == null) { + LOG.warn("region info is null for row " + key.getRow() + + " in table " + HConstants.ROOT_TABLE_NAME); + continue; + } + if (!listener.processRow(info)) { + break; + } + results.clear(); + } + } finally { + rootScanner.close(); + } + } + + /** + * Scans a meta region. For every region found, calls the listener with + * the HRegionInfo of the region. + * TODO: Use Visitor rather than Listener pattern. Allow multiple Visitors. + * Use this everywhere we scan meta regions: e.g. in metascanners, in close + * handling, etc. Have it pass in the whole row, not just HRegionInfo. + * + * @param metaRegionInfo HRegionInfo for meta region + * @param listener method to be called for each meta region found + * @throws IOException + */ + public void scanMetaRegion(HRegionInfo metaRegionInfo, + ScannerListener listener) + throws IOException { + // Open meta region so we can scan it + HRegion metaRegion = openMetaRegion(metaRegionInfo); + scanMetaRegion(metaRegion, listener); + } + + /** + * Scan the passed in metaregion m invoking the passed + * listener per row found. + * @param m + * @param listener + * @throws IOException + */ + public void scanMetaRegion(final HRegion m, final ScannerListener listener) + throws IOException { + InternalScanner metaScanner = m.getScanner(HConstants.COL_REGIONINFO_ARRAY, + HConstants.EMPTY_START_ROW, HConstants.LATEST_TIMESTAMP, null); + try { + HStoreKey key = new HStoreKey(); + SortedMap results = + new TreeMap(Bytes.BYTES_COMPARATOR); + while (metaScanner.next(key, results)) { + HRegionInfo info = (HRegionInfo)Writables.getWritable( + results.get(HConstants.COL_REGIONINFO).getValue(), + new HRegionInfo()); + if (info == null) { + LOG.warn("regioninfo null for row " + key.getRow() + " in table " + + Bytes.toString(m.getTableDesc().getName())); + continue; + } + if (!listener.processRow(info)) { + break; + } + results.clear(); + } + } finally { + metaScanner.close(); + } + } + + private synchronized HRegion openRootRegion() throws IOException { + if (this.rootRegion != null) { + return this.rootRegion; + } + this.rootRegion = HRegion.openHRegion(HRegionInfo.ROOT_REGIONINFO, + this.rootdir, getLog(), this.conf); + this.rootRegion.compactStores(); + return this.rootRegion; + } + + private HRegion openMetaRegion(HRegionInfo metaInfo) throws IOException { + HRegion meta = + HRegion.openHRegion(metaInfo, this.rootdir, getLog(), this.conf); + meta.compactStores(); + return meta; + } + + /** + * Set a single region on/offline. + * This is a tool to repair tables that have offlined tables in their midst. + * Can happen on occasion. Use at your own risk. Call from a bit of java + * or jython script. This method is 'expensive' in that it creates a + * {@link HTable} instance per invocation to go against .META. + * @param c A configuration that has its hbase.master + * properly set. + * @param row Row in the catalog .META. table whose HRegionInfo's offline + * status we want to change. + * @param onlineOffline Pass true to OFFLINE the region. + * @throws IOException + */ + public static void changeOnlineStatus (final HBaseConfiguration c, + final byte [] row, final boolean onlineOffline) + throws IOException { + HTable t = new HTable(c, HConstants.META_TABLE_NAME); + Cell cell = t.get(row, HConstants.COL_REGIONINFO); + if (cell == null) { + throw new IOException("no information for row " + row); + } + // Throws exception if null. + HRegionInfo info = (HRegionInfo)Writables. + getWritable(cell.getValue(), new HRegionInfo()); + BatchUpdate b = new BatchUpdate(row); + info.setOffline(onlineOffline); + b.put(HConstants.COL_REGIONINFO, Writables.getBytes(info)); + b.delete(HConstants.COL_SERVER); + b.delete(HConstants.COL_STARTCODE); + t.commit(b); + } + + /** + * Offline version of the online TableOperation, + * org.apache.hadoop.hbase.master.AddColumn. + * @param tableName + * @param hcd Add this column to tableName + * @throws IOException + */ + public void addColumn(final byte [] tableName, + final HColumnDescriptor hcd) + throws IOException { + List metas = getMETARows(tableName); + for (HRegionInfo hri: metas) { + final HRegion m = getMetaRegion(hri); + scanMetaRegion(m, new ScannerListener() { + private boolean inTable = true; + + @SuppressWarnings("synthetic-access") + public boolean processRow(HRegionInfo info) throws IOException { + LOG.debug("Testing " + Bytes.toString(tableName) + " against " + + Bytes.toString(info.getTableDesc().getName())); + if (Bytes.equals(info.getTableDesc().getName(), tableName)) { + this.inTable = false; + info.getTableDesc().addFamily(hcd); + updateMETARegionInfo(m, info); + return true; + } + // If we got here and we have not yet encountered the table yet, + // inTable will be false. Otherwise, we've passed out the table. + // Stop the scanner. + return this.inTable; + }}); + } + } + + /** + * Offline version of the online TableOperation, + * org.apache.hadoop.hbase.master.DeleteColumn. + * @param tableName + * @param columnFamily Name of column name to remove. + * @throws IOException + */ + public void deleteColumn(final byte [] tableName, + final byte [] columnFamily) throws IOException { + List metas = getMETARows(tableName); + final Path tabledir = new Path(rootdir, Bytes.toString(tableName)); + for (HRegionInfo hri: metas) { + final HRegion m = getMetaRegion(hri); + scanMetaRegion(m, new ScannerListener() { + private boolean inTable = true; + + @SuppressWarnings("synthetic-access") + public boolean processRow(HRegionInfo info) throws IOException { + if (Bytes.equals(info.getTableDesc().getName(), tableName)) { + this.inTable = false; + info.getTableDesc().removeFamily(columnFamily); + updateMETARegionInfo(m, info); + FSUtils.deleteColumnFamily(fs, tabledir, info.getEncodedName(), + HStoreKey.getFamily(columnFamily)); + return false; + } + // If we got here and we have not yet encountered the table yet, + // inTable will be false. Otherwise, we've passed out the table. + // Stop the scanner. + return this.inTable; + }}); + } + } + + /** + * Update COL_REGIONINFO in meta region r with HRegionInfo hri + * + * @param r + * @param hri + * @throws IOException + */ + public void updateMETARegionInfo(HRegion r, final HRegionInfo hri) + throws IOException { + if (LOG.isDebugEnabled()) { + HRegionInfo h = (HRegionInfo)Writables.getWritable( + r.get(hri.getRegionName(), HConstants.COL_REGIONINFO).getValue(), + new HRegionInfo()); + LOG.debug("Old " + Bytes.toString(HConstants.COL_REGIONINFO) + + " for " + hri.toString() + " in " + r.toString() + " is: " + + h.toString()); + } + BatchUpdate b = new BatchUpdate(hri.getRegionName()); + b.put(HConstants.COL_REGIONINFO, Writables.getBytes(hri)); + r.batchUpdate(b); + if (LOG.isDebugEnabled()) { + HRegionInfo h = (HRegionInfo)Writables.getWritable( + r.get(hri.getRegionName(), HConstants.COL_REGIONINFO).getValue(), + new HRegionInfo()); + LOG.debug("New " + Bytes.toString(HConstants.COL_REGIONINFO) + + " for " + hri.toString() + " in " + r.toString() + " is: " + + h.toString()); + } + } + + /** + * @return List of {@link HRegionInfo} rows found in the ROOT or META + * catalog table. + * @param tableName Name of table to go looking for. + * @throws IOException + * @see #getMetaRegion(HRegionInfo) + */ + public List getMETARows(final byte [] tableName) + throws IOException { + final List result = new ArrayList(); + // If passed table name is META, then return the root region. + if (Bytes.equals(HConstants.META_TABLE_NAME, tableName)) { + result.add(openRootRegion().getRegionInfo()); + return result; + } + // Return all meta regions that contain the passed tablename. + scanRootRegion(new ScannerListener() { + private final Log SL_LOG = LogFactory.getLog(this.getClass()); + + @SuppressWarnings("unused") + public boolean processRow(HRegionInfo info) throws IOException { + SL_LOG.debug("Testing " + info); + if (Bytes.equals(info.getTableDesc().getName(), + HConstants.META_TABLE_NAME)) { + result.add(info); + return false; + } + return true; + }}); + return result; + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/RegionHistorian.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/RegionHistorian.java new file mode 100644 index 00000000000..528aa109567 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/RegionHistorian.java @@ -0,0 +1,322 @@ +/** + * Copyright 2008 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Collections; +import java.util.GregorianCalendar; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HServerAddress; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.io.BatchUpdate; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * The Region Historian task is to keep track of every modification a region + * has to go through. Public methods are used to update the information in the + * .META. table and to retrieve it. This is a Singleton. By + * default, the Historian is offline; it will not log. Its enabled in the + * regionserver and master down in their guts after there's some certainty the + * .META. has been deployed. + */ +public class RegionHistorian implements HConstants { + private static final Log LOG = LogFactory.getLog(RegionHistorian.class); + + private HTable metaTable; + + private GregorianCalendar cal = new GregorianCalendar(); + + /** Singleton reference */ + private static RegionHistorian historian; + + /** Date formater for the timestamp in RegionHistoryInformation */ + private static SimpleDateFormat dateFormat = new SimpleDateFormat( + "EEE, d MMM yyyy HH:mm:ss"); + + public static enum HistorianColumnKey { + REGION_CREATION ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"creation")), + REGION_OPEN ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"open")), + REGION_SPLIT ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"split")), + REGION_COMPACTION ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"compaction")), + REGION_FLUSH ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"flush")), + REGION_ASSIGNMENT ( Bytes.toBytes(COLUMN_FAMILY_HISTORIAN_STR+"assignment")); + + public byte[] key; + + HistorianColumnKey(byte[] key) { + this.key = key; + } + } + + /** + * Default constructor. Initializes reference to .META. table. Inaccessible. + * Use {@link #getInstance(HBaseConfiguration)} to obtain the Singleton + * instance of this class. + */ + private RegionHistorian() { + super(); + } + + /** + * Get the RegionHistorian Singleton instance. + * @return The region historian + */ + public static RegionHistorian getInstance() { + if (historian == null) { + historian = new RegionHistorian(); + } + return historian; + } + + /** + * Returns, for a given region name, an ordered list by timestamp of all + * values in the historian column of the .META. table. + * @param regionName + * Region name as a string + * @return List of RegionHistoryInformation or null if we're offline. + */ + public List getRegionHistory(String regionName) { + if (!isOnline()) { + return null; + } + List informations = + new ArrayList(); + try { + /* + * TODO REGION_HISTORIAN_KEYS is used because there is no other for the + * moment to retrieve all version and to have the column key information. + * To be changed when HTable.getRow handles versions. + */ + for (HistorianColumnKey keyEnu : HistorianColumnKey.values()) { + byte[] columnKey = keyEnu.key; + Cell[] cells = this.metaTable.get(Bytes.toBytes(regionName), + columnKey, ALL_VERSIONS); + if (cells != null) { + for (Cell cell : cells) { + informations.add(historian.new RegionHistoryInformation(cell + .getTimestamp(), Bytes.toString(columnKey).split(":")[1], Bytes + .toString(cell.getValue()))); + } + } + } + } catch (IOException ioe) { + LOG.warn("Unable to retrieve region history", ioe); + } + Collections.sort(informations); + return informations; + } + + /** + * Method to add a creation event to the row in the .META table + * @param info + */ + public void addRegionAssignment(HRegionInfo info, String serverName) { + add(HistorianColumnKey.REGION_ASSIGNMENT.key, "Region assigned to server " + + serverName, info); + } + + /** + * Method to add a creation event to the row in the .META table + * @param info + */ + public void addRegionCreation(HRegionInfo info) { + add(HistorianColumnKey.REGION_CREATION.key, "Region creation", info); + } + + /** + * Method to add a opening event to the row in the .META table + * @param info + * @param address + */ + public void addRegionOpen(HRegionInfo info, HServerAddress address) { + add(HistorianColumnKey.REGION_OPEN.key, "Region opened on server : " + + address.getHostname(), info); + } + + /** + * Method to add a split event to the rows in the .META table with + * information from oldInfo. + * @param oldInfo + * @param newInfo1 + * @param newInfo2 + */ + public void addRegionSplit(HRegionInfo oldInfo, HRegionInfo newInfo1, + HRegionInfo newInfo2) { + HRegionInfo[] infos = new HRegionInfo[] { newInfo1, newInfo2 }; + for (HRegionInfo info : infos) { + add(HistorianColumnKey.REGION_SPLIT.key, "Region split from : " + + oldInfo.getRegionNameAsString(), info); + } + } + + /** + * Method to add a compaction event to the row in the .META table + * @param info + */ + public void addRegionCompaction(final HRegionInfo info, + final String timeTaken) { + // While historian can not log flushes because it could deadlock the + // regionserver -- see the note in addRegionFlush -- there should be no + // such danger compacting; compactions are not allowed when + // Flusher#flushSomeRegions is run. + if (LOG.isDebugEnabled()) { + add(HistorianColumnKey.REGION_COMPACTION.key, + "Region compaction completed in " + timeTaken, info); + } + } + + /** + * Method to add a flush event to the row in the .META table + * @param info + */ + public void addRegionFlush(HRegionInfo info, + @SuppressWarnings("unused") String timeTaken) { + // Disabled. Noop. If this regionserver is hosting the .META. AND is + // holding the reclaimMemcacheMemory global lock -- + // see Flusher#flushSomeRegions -- we deadlock. For now, just disable + // logging of flushes. + } + + /** + * Method to add an event with LATEST_TIMESTAMP. + * @param column + * @param text + * @param info + */ + private void add(byte[] column, + String text, HRegionInfo info) { + add(column, text, info, LATEST_TIMESTAMP); + } + + /** + * Method to add an event with provided information. + * @param column + * @param text + * @param info + * @param timestamp + */ + private void add(byte[] column, + String text, HRegionInfo info, long timestamp) { + if (!isOnline()) { + // Its a noop + return; + } + if (!info.isMetaRegion()) { + BatchUpdate batch = new BatchUpdate(info.getRegionName()); + batch.setTimestamp(timestamp); + batch.put(column, Bytes.toBytes(text)); + try { + this.metaTable.commit(batch); + } catch (IOException ioe) { + LOG.warn("Unable to '" + text + "'", ioe); + } + } + } + + /** + * Inner class that only contains information about an event. + * + */ + public class RegionHistoryInformation implements + Comparable { + + private long timestamp; + + private String event; + + private String description; + + public RegionHistoryInformation(long timestamp, String event, + String description) { + this.timestamp = timestamp; + this.event = event; + this.description = description; + } + + /** + * Returns the inverse value of Long.compareTo + */ + public int compareTo(RegionHistoryInformation otherInfo) { + return -1 * Long.valueOf(timestamp).compareTo(otherInfo.getTimestamp()); + } + + public String getEvent() { + return event; + } + + public String getDescription() { + return description; + } + + public long getTimestamp() { + return timestamp; + } + + /** + * @return The value of the timestamp processed with the date formater. + */ + public String getTimestampAsString() { + cal.setTimeInMillis(timestamp); + return dateFormat.format(cal.getTime()); + } + } + + /** + * @return True if the historian is online. When offline, will not add + * updates to the .META. table. + */ + public boolean isOnline() { + return this.metaTable != null; + } + + /** + * @param c Online the historian. Invoke after cluster has spun up. + */ + public void online(final HBaseConfiguration c) { + try { + this.metaTable = new HTable(c, META_TABLE_NAME); + if (LOG.isDebugEnabled()) { + LOG.debug("Onlined"); + } + } catch (IOException ioe) { + LOG.error("Unable to create RegionHistorian", ioe); + } + } + + /** + * Offlines the historian. + * @see #online(HBaseConfiguration) + */ + public void offline() { + this.metaTable = null; + if (LOG.isDebugEnabled()) { + LOG.debug("Offlined"); + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/StoreFileScanner.java b/src/java/org/apache/hadoop/hbase/util/migration/v5/StoreFileScanner.java new file mode 100644 index 00000000000..c226ccf3031 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/StoreFileScanner.java @@ -0,0 +1,391 @@ +/** + * Copyright 2008 The Apache Software Foundation + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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.util.migration.v5; + +import java.io.IOException; +import java.util.SortedMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HStoreKey; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.io.Cell; +import org.apache.hadoop.hbase.regionserver.ChangedReadersObserver; +import org.apache.hadoop.hbase.regionserver.HAbstractScanner; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.io.MapFile; + +/** + * A scanner that iterates through HStore files + */ +class StoreFileScanner extends HAbstractScanner +implements ChangedReadersObserver { + private final Log LOG = LogFactory.getLog(this.getClass()); + + // Keys retrieved from the sources + private volatile HStoreKey keys[]; + // Values that correspond to those keys + private volatile byte [][] vals; + + // Readers we go against. + private volatile MapFile.Reader[] readers; + + // Store this scanner came out of. + private final HStore store; + + // Used around replacement of Readers if they change while we're scanning. + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + /** + * @param store + * @param timestamp + * @param targetCols + * @param firstRow + * @throws IOException + */ + public StoreFileScanner(final HStore store, final long timestamp, + final byte [][] targetCols, final byte [] firstRow) + throws IOException { + super(timestamp, targetCols); + this.store = store; + this.store.addChangedReaderObserver(this); + this.store.lock.readLock().lock(); + try { + openReaders(firstRow); + } catch (Exception ex) { + close(); + IOException e = new IOException("HStoreScanner failed construction"); + e.initCause(ex); + throw e; + } finally { + this.store.lock.readLock().unlock(); + } + } + + /* + * Go open new Reader iterators and cue them at firstRow. + * Closes existing Readers if any. + * @param firstRow + * @throws IOException + */ + private void openReaders(final byte [] firstRow) throws IOException { + if (this.readers != null) { + for (int i = 0; i < this.readers.length; i++) { + if (this.readers[i] != null) { + this.readers[i].close(); + } + } + } + // Open our own copies of the Readers here inside in the scanner. + this.readers = new MapFile.Reader[this.store.getStorefiles().size()]; + + // Most recent map file should be first + int i = readers.length - 1; + for(HStoreFile curHSF: store.getStorefiles().values()) { + readers[i--] = curHSF.getReader(store.fs, false, false); + } + + this.keys = new HStoreKey[readers.length]; + this.vals = new byte[readers.length][]; + + // Advance the readers to the first pos. + for (i = 0; i < readers.length; i++) { + keys[i] = new HStoreKey(); + if (firstRow != null && firstRow.length != 0) { + if (findFirstRow(i, firstRow)) { + continue; + } + } + while (getNext(i)) { + if (columnMatch(i)) { + break; + } + } + } + } + + /** + * For a particular column i, find all the matchers defined for the column. + * Compare the column family and column key using the matchers. The first one + * that matches returns true. If no matchers are successful, return false. + * + * @param i index into the keys array + * @return true if any of the matchers for the column match the column family + * and the column key. + * @throws IOException + */ + boolean columnMatch(int i) throws IOException { + return columnMatch(keys[i].getColumn()); + } + + /** + * Get the next set of values for this scanner. + * + * @param key The key that matched + * @param results All the results for key + * @return true if a match was found + * @throws IOException + * + * @see org.apache.hadoop.hbase.regionserver.InternalScanner#next(org.apache.hadoop.hbase.HStoreKey, java.util.SortedMap) + */ + @Override + public boolean next(HStoreKey key, SortedMap results) + throws IOException { + if (this.scannerClosed) { + return false; + } + this.lock.readLock().lock(); + try { + // Find the next viable row label (and timestamp). + ViableRow viableRow = getNextViableRow(); + + // Grab all the values that match this row/timestamp + boolean insertedItem = false; + if (viableRow.getRow() != null) { + key.setRow(viableRow.getRow()); + key.setVersion(viableRow.getTimestamp()); + + for (int i = 0; i < keys.length; i++) { + // Fetch the data + while ((keys[i] != null) + && (Bytes.compareTo(keys[i].getRow(), viableRow.getRow()) == 0)) { + + // If we are doing a wild card match or there are multiple matchers + // per column, we need to scan all the older versions of this row + // to pick up the rest of the family members + if(!isWildcardScanner() + && !isMultipleMatchScanner() + && (keys[i].getTimestamp() != viableRow.getTimestamp())) { + break; + } + + if(columnMatch(i)) { + // We only want the first result for any specific family member + if(!results.containsKey(keys[i].getColumn())) { + results.put(keys[i].getColumn(), + new Cell(vals[i], keys[i].getTimestamp())); + insertedItem = true; + } + } + + if (!getNext(i)) { + closeSubScanner(i); + } + } + + // Advance the current scanner beyond the chosen row, to + // a valid timestamp, so we're ready next time. + while ((keys[i] != null) + && ((Bytes.compareTo(keys[i].getRow(), viableRow.getRow()) <= 0) + || (keys[i].getTimestamp() > this.timestamp) + || (! columnMatch(i)))) { + getNext(i); + } + } + } + return insertedItem; + } finally { + this.lock.readLock().unlock(); + } + } + + // Data stucture to hold next, viable row (and timestamp). + class ViableRow { + private final byte [] row; + private final long ts; + + ViableRow(final byte [] r, final long t) { + this.row = r; + this.ts = t; + } + + byte [] getRow() { + return this.row; + } + + long getTimestamp() { + return this.ts; + } + } + + /* + * @return An instance of ViableRow + * @throws IOException + */ + private ViableRow getNextViableRow() throws IOException { + // Find the next viable row label (and timestamp). + byte [] viableRow = null; + long viableTimestamp = -1; + long now = System.currentTimeMillis(); + long ttl = store.ttl; + for(int i = 0; i < keys.length; i++) { + // The first key that we find that matches may have a timestamp greater + // than the one we're looking for. We have to advance to see if there + // is an older version present, since timestamps are sorted descending + while (keys[i] != null && + keys[i].getTimestamp() > this.timestamp && + columnMatch(i) && + getNext(i)) { + if (columnMatch(i)) { + break; + } + } + if((keys[i] != null) + // If we get here and keys[i] is not null, we already know that the + // column matches and the timestamp of the row is less than or equal + // to this.timestamp, so we do not need to test that here + && ((viableRow == null) + || (Bytes.compareTo(keys[i].getRow(), viableRow) < 0) + || ((Bytes.compareTo(keys[i].getRow(), viableRow) == 0) + && (keys[i].getTimestamp() > viableTimestamp)))) { + if (ttl == HConstants.FOREVER || now < keys[i].getTimestamp() + ttl) { + viableRow = keys[i].getRow(); + viableTimestamp = keys[i].getTimestamp(); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("getNextViableRow :" + keys[i] + ": expired, skipped"); + } + } + } + } + return new ViableRow(viableRow, viableTimestamp); + } + + /** + * The user didn't want to start scanning at the first row. This method + * seeks to the requested row. + * + * @param i which iterator to advance + * @param firstRow seek to this row + * @return true if this is the first row or if the row was not found + */ + private boolean findFirstRow(int i, final byte [] firstRow) throws IOException { + ImmutableBytesWritable ibw = new ImmutableBytesWritable(); + HStoreKey firstKey + = (HStoreKey)readers[i].getClosest(new HStoreKey(firstRow), ibw); + if (firstKey == null) { + // Didn't find it. Close the scanner and return TRUE + closeSubScanner(i); + return true; + } + long now = System.currentTimeMillis(); + long ttl = store.ttl; + if (ttl != HConstants.FOREVER && now >= firstKey.getTimestamp() + ttl) { + // Didn't find it. Close the scanner and return TRUE + closeSubScanner(i); + return true; + } + this.vals[i] = ibw.get(); + keys[i].setRow(firstKey.getRow()); + keys[i].setColumn(firstKey.getColumn()); + keys[i].setVersion(firstKey.getTimestamp()); + return columnMatch(i); + } + + /** + * Get the next value from the specified reader. + * + * @param i which reader to fetch next value from + * @return true if there is more data available + */ + private boolean getNext(int i) throws IOException { + boolean result = false; + ImmutableBytesWritable ibw = new ImmutableBytesWritable(); + long now = System.currentTimeMillis(); + long ttl = store.ttl; + while (true) { + if (!readers[i].next(keys[i], ibw)) { + closeSubScanner(i); + break; + } + if (keys[i].getTimestamp() <= this.timestamp) { + if (ttl == HConstants.FOREVER || now < keys[i].getTimestamp() + ttl) { + vals[i] = ibw.get(); + result = true; + break; + } + if (LOG.isDebugEnabled()) { + LOG.debug("getNext: " + keys[i] + ": expired, skipped"); + } + } + } + return result; + } + + /** Close down the indicated reader. */ + private void closeSubScanner(int i) { + try { + if(readers[i] != null) { + try { + readers[i].close(); + } catch(IOException e) { + LOG.error(store.storeName + " closing sub-scanner", e); + } + } + + } finally { + readers[i] = null; + keys[i] = null; + vals[i] = null; + } + } + + /** Shut it down! */ + public void close() { + if (!this.scannerClosed) { + this.store.deleteChangedReaderObserver(this); + try { + for(int i = 0; i < readers.length; i++) { + if(readers[i] != null) { + try { + readers[i].close(); + } catch(IOException e) { + LOG.error(store.storeName + " closing scanner", e); + } + } + } + + } finally { + this.scannerClosed = true; + } + } + } + + // Implementation of ChangedReadersObserver + + /** {@inheritDoc} */ + public void updateReaders() throws IOException { + this.lock.writeLock().lock(); + try { + // The keys are currently lined up at the next row to fetch. Pass in + // the current row as 'first' row and readers will be opened and cue'd + // up so future call to next will start here. + ViableRow viableRow = getNextViableRow(); + openReaders(viableRow.getRow()); + LOG.debug("Replaced Scanner Readers at row " + + Bytes.toString(viableRow.getRow())); + } finally { + this.lock.writeLock().unlock(); + } + } +} diff --git a/src/java/org/apache/hadoop/hbase/util/migration/v5/package.html b/src/java/org/apache/hadoop/hbase/util/migration/v5/package.html new file mode 100644 index 00000000000..58b9bfc3ae9 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/util/migration/v5/package.html @@ -0,0 +1,36 @@ + + + + + + + +Package of classes used instantiating objects written with pre-version 5 +versions of HBase. + +Under the hbase.rootdir, a file named hbase.version +holds the version number for the data persisted by HBase. The version number +is upped every time a change is made in HBase on-filesystem formats. Version +0.2.0 of HBase shipped with an on-filesystem version of 5. This +package holds classes from previous to version 5 used during the migration of +an HBase instance up to version 5. See +How To Migrate +for more on the migration of HBase across versions and for notes on design +of the HBase migration system. + + \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/TestTable.java b/src/test/org/apache/hadoop/hbase/TestTable.java index f47d631ec72..312d763544e 100644 --- a/src/test/org/apache/hadoop/hbase/TestTable.java +++ b/src/test/org/apache/hadoop/hbase/TestTable.java @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.io.BatchUpdate; /** Tests table creation restrictions*/ public class TestTable extends HBaseClusterTestCase { @@ -123,4 +124,26 @@ public class TestTable extends HBaseClusterTestCase { @SuppressWarnings("unused") HTable table = new HTable(conf, getName()); } + + /** + * Test read only tables + */ + public void testReadOnlyTable() throws Exception { + HBaseAdmin admin = new HBaseAdmin(conf); + HTableDescriptor desc = new HTableDescriptor(getName()); + byte[] colName = "test:".getBytes(); + desc.addFamily(new HColumnDescriptor(colName)); + desc.setReadOnly(true); + admin.createTable(desc); + HTable table = new HTable(conf, getName()); + try { + byte[] value = "somedata".getBytes(); + BatchUpdate update = new BatchUpdate(); + update.put(colName, value); + table.commit(update); + fail("BatchUpdate on read only table succeeded"); + } catch (Exception e) { + // expected + } + } } \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/client/TestHTable.java b/src/test/org/apache/hadoop/hbase/client/TestHTable.java index ba24d2b2c4d..ff4ed8a1e0f 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestHTable.java +++ b/src/test/org/apache/hadoop/hbase/client/TestHTable.java @@ -21,6 +21,9 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseClusterTestCase; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; @@ -31,11 +34,14 @@ import org.apache.hadoop.hbase.io.Cell; import org.apache.hadoop.hbase.io.RowResult; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; /** * Tests HTable */ public class TestHTable extends HBaseClusterTestCase implements HConstants { + private static final Log LOG = LogFactory.getLog(TestHTable.class); private static final HColumnDescriptor column = new HColumnDescriptor(COLUMN_FAMILY); @@ -45,6 +51,9 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants { private static final byte [] row = Bytes.toBytes("row"); + private static final byte [] attrName = Bytes.toBytes("TESTATTR"); + private static final byte [] attrValue = Bytes.toBytes("somevalue"); + /** * the test * @throws IOException @@ -123,7 +132,57 @@ public class TestHTable extends HBaseClusterTestCase implements HConstants { // We can still access A through newA because it has the table information // cached. And if it needs to recalibrate, that will cause the information // to be reloaded. - + + // Test user metadata + + try { + // make a modifiable descriptor + HTableDescriptor desc = new HTableDescriptor(a.getMetadata()); + // offline the table + admin.disableTable(tableAname); + // add a user attribute to HTD + desc.setValue(attrName, attrValue); + // add a user attribute to HCD + for (HColumnDescriptor c: desc.getFamilies()) + c.setValue(attrName, attrValue); + // update metadata for all regions of this table + admin.modifyTableMeta(tableAname, desc); + // enable the table + admin.enableTable(tableAname); + + // Use a metascanner to avoid client API caching (HConnection has a + // metadata cache) + MetaScanner.MetaScannerVisitor visitor = new MetaScanner.MetaScannerVisitor() { + public boolean processRow( + @SuppressWarnings("unused") RowResult rowResult, + HRegionLocation regionLocation, + HRegionInfo info) { + LOG.info("visiting " + regionLocation.toString()); + HTableDescriptor desc = info.getTableDesc(); + if (Bytes.compareTo(desc.getName(), tableAname) == 0) { + // check HTD attribute + byte[] value = desc.getValue(attrName); + if (value == null) + fail("missing HTD attribute value"); + if (Bytes.compareTo(value, attrValue) != 0) + fail("HTD attribute value is incorrect"); + // check HCD attribute + for (HColumnDescriptor c: desc.getFamilies()) { + value = c.getValue(attrName); + if (value == null) + fail("missing HCD attribute value"); + if (Bytes.compareTo(value, attrValue) != 0) + fail("HCD attribute value is incorrect"); + } + } + return true; + } + }; + MetaScanner.metaScan(conf, visitor); + } catch (Exception e) { + e.printStackTrace(); + fail(); + } } /**