You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/07/03 21:44:07 UTC

svn commit: r790999 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/regionserver/ src/java/org/apache/hadoop/hbase/util/ src/test/org/apache/hadoop/hbase/io/

Author: stack
Date: Fri Jul  3 19:44:06 2009
New Revision: 790999

URL: http://svn.apache.org/viewvc?rev=790999&view=rev
Log:
HBASE-1607 Redo MemStore heap sizing to be accurate, testable, and more like new LruBlockCache

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/MemStore.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/ClassSize.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/TestHeapSize.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=790999&r1=790998&r2=790999&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Fri Jul  3 19:44:06 2009
@@ -435,6 +435,8 @@
                Watcher
    HBASE-1597  Prevent unnecessary caching of blocks during compactions
                (Jon Gray via Stack)
+   HBASE-1607  Redo MemStore heap sizing to be accurate, testable, and more
+               like new LruBlockCache (Jon Gray via Stack)
 
   OPTIMIZATIONS
    HBASE-1412  Change values for delete column and column family in KeyValue

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java?rev=790999&r1=790998&r2=790999&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/RegionHistorian.java Fri Jul  3 19:44:06 2009
@@ -33,8 +33,8 @@
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
 
 /**
  * The Region Historian task is to keep track of every modification a region
@@ -49,8 +49,6 @@
   
   private HTable metaTable;
 
-
-
   /** Singleton reference */
   private static RegionHistorian historian;
 
@@ -333,4 +331,7 @@
       LOG.debug("Offlined");
     }
   }
+
+  public static final long FIXED_OVERHEAD = ClassSize.align(
+      ClassSize.OBJECT + ClassSize.REFERENCE);
 }
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java?rev=790999&r1=790998&r2=790999&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HLog.java Fri Jul  3 19:44:06 2009
@@ -55,6 +55,7 @@
 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.ClassSize;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.io.SequenceFile;
@@ -111,7 +112,7 @@
   private final int flushlogentries;
   private final AtomicInteger unflushedEntries = new AtomicInteger(0);
   private volatile long lastLogFlushTime;
-
+  
   /*
    * Current log file.
    */
@@ -1117,4 +1118,9 @@
       }
     }
   }
+
+  public static final long FIXED_OVERHEAD = ClassSize.align(
+      ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
+      ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
+  
 }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=790999&r1=790998&r2=790999&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java Fri Jul  3 19:44:06 2009
@@ -54,9 +54,11 @@
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.Reference.Range;
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.util.Progressable;
@@ -98,7 +100,7 @@
  * regionName is a unique identifier for this HRegion. (startKey, endKey]
  * defines the keyspace for this HRegion.
  */
-public class HRegion implements HConstants { // , Writable{
+public class HRegion implements HConstants, HeapSize { // , Writable{
   static final Log LOG = LogFactory.getLog(HRegion.class);
   static final String SPLITDIR = "splits";
   static final String MERGEDIR = "merges";
@@ -2322,45 +2324,28 @@
             + " in table " + regionInfo.getTableDesc());
     }
   }
+
+  public static final long FIXED_OVERHEAD = ClassSize.align(
+      (3 * Bytes.SIZEOF_LONG) + (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN +
+      (21 * ClassSize.REFERENCE) + ClassSize.OBJECT);
+  
+  public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
+      ClassSize.OBJECT + (2 * ClassSize.ATOMIC_BOOLEAN) + 
+      ClassSize.ATOMIC_LONG + ClassSize.ATOMIC_INTEGER +
+      ClassSize.CONCURRENT_HASHMAP + 
+      (16 * ClassSize.CONCURRENT_HASHMAP_ENTRY) + 
+      (16 * ClassSize.CONCURRENT_HASHMAP_SEGMENT) +
+      ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY +
+      RegionHistorian.FIXED_OVERHEAD + HLog.FIXED_OVERHEAD +
+      ClassSize.align(ClassSize.OBJECT + (5 * Bytes.SIZEOF_BOOLEAN)) +
+      (3 * ClassSize.REENTRANT_LOCK));
   
-  
-//  //HBaseAdmin Debugging 
-//  /**
-//   * @return number of stores in the region
-//   */
-//  public int getNumStores() {
-//    return this.numStores;
-//  }
-//  /**
-//   * @return the name of the region
-//   */
-//  public byte [] getRegionsName() {
-//    return this.name;
-//  }
-//  /**
-//   * @return the number of files in every store
-//   */
-//  public int [] getStoresSize() {
-//    return this.storeSize;
-//  }
-//  
-//  //Writable, used for debugging purposes only
-//  public void readFields(final DataInput in)
-//  throws IOException {
-//    this.name = Bytes.readByteArray(in);
-//    this.numStores = in.readInt();
-//    this.storeSize = new int [numStores];
-//    for(int i=0; i<this.numStores; i++) {
-//      this.storeSize[i] = in.readInt();
-//    }
-//  }
-//
-//  public void write(final DataOutput out)
-//  throws IOException {
-//    Bytes.writeByteArray(out, this.regionInfo.getRegionName());
-//    out.writeInt(this.stores.size());
-//    for(Store store : this.stores.values()) {
-//      out.writeInt(store.getNumberOfstorefiles());
-//    }
-//  }
+  @Override
+  public long heapSize() {
+    long heapSize = DEEP_OVERHEAD;
+    for(Store store : this.stores.values()) {
+      heapSize += store.heapSize();
+    }
+    return heapSize;
+  }
 }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/MemStore.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/MemStore.java?rev=790999&r1=790998&r2=790999&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/MemStore.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/MemStore.java Fri Jul  3 19:44:06 2009
@@ -30,14 +30,17 @@
 import java.util.NavigableSet;
 import java.util.SortedSet;
 import java.util.TreeSet;
+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.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.regionserver.DeleteCompare.DeleteCode;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
 
 /**
  * The MemStore holds in-memory modifications to the Store.  Modifications
@@ -50,7 +53,7 @@
  * TODO: With new KVSLS, need to make sure we update HeapSize with difference
  * in KV size.
  */
-class MemStore {
+public class MemStore implements HeapSize {
   private static final Log LOG = LogFactory.getLog(MemStore.class);
 
   private final long ttl;
@@ -75,8 +78,8 @@
   // Used comparing versions -- same r/c and type but different timestamp.
   final KeyValue.KVComparator comparatorIgnoreTimestamp;
 
-  // TODO: Fix this guess by studying jprofiler
-  private final static int ESTIMATED_KV_HEAP_TAX = 60;
+  // Used to track own heapSize
+  final AtomicLong size;
 
   /**
    * Default constructor. Used for tests.
@@ -98,6 +101,7 @@
     this.comparatorIgnoreType = this.comparator.getComparatorIgnoringType();
     this.kvset = new KeyValueSkipListSet(c);
     this.snapshot = new KeyValueSkipListSet(c);
+    this.size = new AtomicLong(DEEP_OVERHEAD);
   }
 
   void dump() {
@@ -129,6 +133,8 @@
         if (!this.kvset.isEmpty()) {
           this.snapshot = this.kvset;
           this.kvset = new KeyValueSkipListSet(this.comparator);
+          // Reset heap to not include any keys
+          this.size.set(DEEP_OVERHEAD);
         }
       }
     } finally {
@@ -181,7 +187,8 @@
     long size = -1;
     this.lock.readLock().lock();
     try {
-      size = heapSize(kv, this.kvset.add(kv));
+      size = heapSizeChange(kv, this.kvset.add(kv));
+      this.size.addAndGet(size);
     } finally {
       this.lock.readLock().unlock();
     }
@@ -254,33 +261,19 @@
       //Delete all the entries effected by the last added delete
       for (KeyValue kv : deletes) {
         notpresent = this.kvset.remove(kv);
-        size -= heapSize(kv, notpresent);
+        size -= heapSizeChange(kv, notpresent);
       }
       
       // Adding the delete to memstore. Add any value, as long as
       // same instance each time.
-      size += heapSize(delete, this.kvset.add(delete));
+      size += heapSizeChange(delete, this.kvset.add(delete));
     } finally {
       this.lock.readLock().unlock();
     }
+    this.size.addAndGet(size);
     return size;
   }
   
-  /*
-   * Calculate how the memstore size has changed, approximately.  Be careful.
-   * If class changes, be sure to change the size calculation.
-   * Add in tax of Map.Entry.
-   * @param kv
-   * @param notpresent True if the kv was NOT present in the set.
-   * @return Size
-   */
-  long heapSize(final KeyValue kv, final boolean notpresent) {
-    return notpresent?
-      // Add overhead for value byte array and for Map.Entry -- 57 bytes
-      // on x64 according to jprofiler.
-      ESTIMATED_KV_HEAP_TAX + 57 + kv.getLength(): 0; // Guess no change in size.
-  }
-
   /**
    * @param kv Find the row that comes after this one.  If null, we return the
    * first.
@@ -694,6 +687,42 @@
       }
     }
   }
+  
+  public final static long FIXED_OVERHEAD = ClassSize.align(
+      ClassSize.OBJECT + Bytes.SIZEOF_LONG + (7 * ClassSize.REFERENCE));
+  
+  public final static long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
+      ClassSize.REENTRANT_LOCK + ClassSize.ATOMIC_LONG +
+      (2 * ClassSize.CONCURRENT_SKIPLISTMAP));
+
+  /*
+   * Calculate how the MemStore size has changed.  Includes overhead of the
+   * backing Map.
+   * @param kv
+   * @param notpresent True if the kv was NOT present in the set.
+   * @return Size
+   */
+  long heapSizeChange(final KeyValue kv, final boolean notpresent) {
+    return notpresent ? 
+        ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + kv.heapSize()):
+        0;
+  }
+  
+  /**
+   * Get the entire heap usage for this MemStore not including keys in the
+   * snapshot.
+   */
+  @Override
+  public long heapSize() {
+    return size.get();
+  }
+  
+  /**
+   * Get the heap usage of KVs in this MemStore.
+   */
+  public long keySize() {
+    return heapSize() - DEEP_OVERHEAD;
+  }
 
   /**
    * Code to help figure if our approximation of object heap sizes is close

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=790999&r1=790998&r2=790999&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/Store.java Fri Jul  3 19:44:06 2009
@@ -49,6 +49,7 @@
 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.SequenceFile;
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -56,6 +57,7 @@
 import org.apache.hadoop.hbase.io.hfile.HFile.CompactionReader;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
@@ -83,7 +85,7 @@
  * <p>Locking and transactions are handled at a higher level.  This API should
  * not be called directly but by an HRegion manager.
  */
-public class Store implements HConstants {
+public class Store implements HConstants, HeapSize {
   static final Log LOG = LogFactory.getLog(Store.class);
   /**
    * Comparator that looks at columns and compares their family portions.
@@ -510,7 +512,7 @@
           if (!isExpired(kv, oldestTimestamp)) {
             writer.append(kv);
             entries++;
-            flushed += this.memstore.heapSize(kv, true);
+            flushed += this.memstore.heapSizeChange(kv, true);
           }
         }
         // B. Write out the log sequence number that corresponds to this output
@@ -1627,4 +1629,19 @@
         Bytes.toBytes(newValue));
     return new ICVResult(newValue, newKv.heapSize(), newKv);
   }
+
+  public static final long FIXED_OVERHEAD = ClassSize.align(
+      ClassSize.OBJECT + (17 * ClassSize.REFERENCE) +
+      (5 * Bytes.SIZEOF_LONG) + (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN +
+      ClassSize.align(ClassSize.ARRAY));
+  
+  public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
+      ClassSize.OBJECT + ClassSize.REENTRANT_LOCK + 
+      ClassSize.CONCURRENT_SKIPLISTMAP + 
+      ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT);
+      
+  @Override
+  public long heapSize() {
+    return DEEP_OVERHEAD + this.memstore.heapSize();
+  }
 }
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/ClassSize.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/ClassSize.java?rev=790999&r1=790998&r2=790999&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/ClassSize.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/ClassSize.java Fri Jul  3 19:44:06 2009
@@ -73,7 +73,25 @@
   
   /** Overhead for ConcurrentHashMap.Segment */
   public static int CONCURRENT_HASHMAP_SEGMENT = 0;
-
+  
+  /** Overhead for ConcurrentSkipListMap */
+  public static int CONCURRENT_SKIPLISTMAP = 0;
+  
+  /** Overhead for ConcurrentSkipListMap Entry */
+  public static int CONCURRENT_SKIPLISTMAP_ENTRY = 0;
+  
+  /** Overhead for ReentrantReadWriteLock */
+  public static int REENTRANT_LOCK = 0;
+  
+  /** Overhead for AtomicLong */
+  public static int ATOMIC_LONG = 0;
+  
+  /** Overhead for AtomicInteger */
+  public static int ATOMIC_INTEGER = 0;
+  
+  /** Overhead for AtomicBoolean */
+  public static int ATOMIC_BOOLEAN = 0;
+  
   private static final String THIRTY_TWO = "32";
 
   /**
@@ -118,6 +136,21 @@
       
     CONCURRENT_HASHMAP_SEGMENT = align(REFERENCE + OBJECT + 
         (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_FLOAT + ARRAY);
+    
+    CONCURRENT_SKIPLISTMAP = align(Bytes.SIZEOF_INT + OBJECT + (8 * REFERENCE));
+    
+    CONCURRENT_SKIPLISTMAP_ENTRY = align(
+        align(OBJECT + (3 * REFERENCE)) + /* one node per entry */ 
+        align((OBJECT + (3 * REFERENCE))/2)); /* one index per two entries */
+    
+    REENTRANT_LOCK = align(OBJECT + (3 * REFERENCE));
+    
+    ATOMIC_LONG = align(OBJECT + Bytes.SIZEOF_LONG);
+    
+    ATOMIC_INTEGER = align(OBJECT + Bytes.SIZEOF_INT);
+    
+    ATOMIC_BOOLEAN = align(OBJECT + Bytes.SIZEOF_BOOLEAN);
+    
   }
   
   /**

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/TestHeapSize.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/TestHeapSize.java?rev=790999&r1=790998&r2=790999&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/TestHeapSize.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/TestHeapSize.java Fri Jul  3 19:44:06 2009
@@ -5,13 +5,23 @@
 import java.util.ArrayList;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+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.hbase.KeyValue;
+import org.apache.hadoop.hbase.RegionHistorian;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.hfile.CachedBlock;
 import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.MemStore;
+import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 
@@ -99,6 +109,60 @@
       ClassSize.estimateBase(cl, true);
       assertEquals(expected, actual);
     }
+
+    // ConcurrentHashMap
+    cl = ConcurrentHashMap.class;
+    expected = ClassSize.estimateBase(cl, false);
+    actual = ClassSize.CONCURRENT_HASHMAP;
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      assertEquals(expected, actual);
+    }
+
+    // ConcurrentSkipListMap
+    cl = ConcurrentSkipListMap.class;
+    expected = ClassSize.estimateBase(cl, false);
+    actual = ClassSize.CONCURRENT_SKIPLISTMAP;
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      assertEquals(expected, actual);
+    }
+
+    // ReentrantReadWriteLock
+    cl = ReentrantReadWriteLock.class;
+    expected = ClassSize.estimateBase(cl, false);
+    actual = ClassSize.REENTRANT_LOCK;
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      assertEquals(expected, actual);
+    }
+
+    // AtomicLong
+    cl = AtomicLong.class;
+    expected = ClassSize.estimateBase(cl, false);
+    actual = ClassSize.ATOMIC_LONG;
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      assertEquals(expected, actual);
+    }
+
+    // AtomicInteger
+    cl = AtomicInteger.class;
+    expected = ClassSize.estimateBase(cl, false);
+    actual = ClassSize.ATOMIC_INTEGER;
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      assertEquals(expected, actual);
+    }
+
+    // AtomicBoolean
+    cl = AtomicBoolean.class;
+    expected = ClassSize.estimateBase(cl, false);
+    actual = ClassSize.ATOMIC_BOOLEAN;
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      assertEquals(expected, actual);
+    }
     
   }
   
@@ -124,18 +188,21 @@
       assertEquals(expected, actual);
     }
     
-    //LruBlockCache Overhead
-    cl = LruBlockCache.class;
-    actual = LruBlockCache.CACHE_FIXED_OVERHEAD;
+    //Put
+    cl = Put.class;
     expected = ClassSize.estimateBase(cl, false);
+    //The actual TreeMap is not included in the above calculation
+    expected += ClassSize.TREEMAP;
+    Put put = new Put(Bytes.toBytes(""));
+    actual = put.heapSize();
     if(expected != actual) {
       ClassSize.estimateBase(cl, true);
       assertEquals(expected, actual);
     }
     
-    // LruBlockCache Map Fixed Overhead
-    cl = ConcurrentHashMap.class;
-    actual = ClassSize.CONCURRENT_HASHMAP;
+    //LruBlockCache Overhead
+    cl = LruBlockCache.class;
+    actual = LruBlockCache.CACHE_FIXED_OVERHEAD;
     expected = ClassSize.estimateBase(cl, false);
     if(expected != actual) {
       ClassSize.estimateBase(cl, true);
@@ -157,17 +224,64 @@
       assertEquals(expected, actual);
     }
     
-    //Put
-    cl = Put.class;
+    // MemStore Overhead
+    cl = MemStore.class;
+    actual = MemStore.FIXED_OVERHEAD;
     expected = ClassSize.estimateBase(cl, false);
-    //The actual TreeMap is not included in the above calculation
-    expected += ClassSize.TREEMAP;
-    Put put = new Put(Bytes.toBytes(""));
-    actual = put.heapSize();
     if(expected != actual) {
       ClassSize.estimateBase(cl, true);
       assertEquals(expected, actual);
     }
+    
+    // MemStore Deep Overhead
+    actual = MemStore.DEEP_OVERHEAD;
+    expected = ClassSize.estimateBase(cl, false);
+    expected += ClassSize.estimateBase(ReentrantReadWriteLock.class, false);
+    expected += ClassSize.estimateBase(AtomicLong.class, false);
+    expected += ClassSize.estimateBase(ConcurrentSkipListMap.class, false);
+    expected += ClassSize.estimateBase(ConcurrentSkipListMap.class, false);
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      ClassSize.estimateBase(ReentrantReadWriteLock.class, true);
+      ClassSize.estimateBase(AtomicLong.class, true);
+      ClassSize.estimateBase(ConcurrentSkipListMap.class, true);
+      assertEquals(expected, actual);
+    }
+    
+    // Store Overhead
+    cl = Store.class;
+    actual = Store.FIXED_OVERHEAD;
+    expected = ClassSize.estimateBase(cl, false);
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      assertEquals(expected, actual);
+    }
+    
+    // Region Overhead
+    cl = HRegion.class;
+    actual = HRegion.FIXED_OVERHEAD;
+    expected = ClassSize.estimateBase(cl, false);
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      assertEquals(expected, actual);
+    }
+    
+    // RegionHistorian Overhead
+    cl = RegionHistorian.class;
+    actual = RegionHistorian.FIXED_OVERHEAD;
+    expected = ClassSize.estimateBase(cl, false);
+    if(expected != actual) {
+      ClassSize.estimateBase(cl, true);
+      assertEquals(expected, actual);
+    }
+    
+    // Currently NOT testing Deep Overheads of many of these classes.
+    // Deep overheads cover a vast majority of stuff, but will not be 100%
+    // accurate because it's unclear when we're referencing stuff that's already
+    // accounted for.  But we have satisfied our two core requirements.
+    // Sizing is quite accurate now, and our tests will throw errors if 
+    // any of these classes are modified without updating overhead sizes.
+
   }
 
 }