You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 21:13:10 UTC

svn commit: r1182034 [1/2] - in /hbase/branches/0.89/src: main/java/org/apache/hadoop/hbase/io/hfile/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/regionserver/metrics/ test/java/org/apache/hadoop/hbase/io/hfile/ te...

Author: nspiegelberg
Date: Tue Oct 11 19:13:09 2011
New Revision: 1182034

URL: http://svn.apache.org/viewvc?rev=1182034&view=rev
Log:
Per-(table, columnFamily) metrics with configurable table name inclusion

Summary:
This is based on Liyin's D326097 diff with some refactoring done to push the
table name and column family configuration to a base class, SchemaConfigured.
This was convenient as many of existing classes having these properties (HFile
readers/writers, HFile blocks, etc.) did not have a base class.

Whether to collect per-(table, columnFamily) or per-columnFamily only metrics
can be configured with the hbase.metrics.showTableName configuration key. We
don't expect this configuration to change at runtime, so we cache the setting
statically and log an error when an attempt is made to flip it once already set.
This way we don't have to pass configuration to a lot more places, e.g.
everywhere an HFile reader is instantiated.

Test Plan: Unit tests, dev cluster, dark launch. Look at what metrics are being
collected. A new unit test will be written to test both per-table and per-CF
settings.

Reviewers: liyintang, kannan, kranganathan

Reviewed By: liyintang

CC: hbase-eng@lists, puma-eng@lists, liyintang

Differential Revision: 331777

Revert Plan: OK

Added:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java
      - copied, changed from r1182033, hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockInfo.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaConfigured.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/metrics/
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/metrics/TestSchemaMetrics.java
Removed:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/ColumnFamilyMetrics.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockInfo.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestColumnFamilyMetrics.java
Modified:
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/CachedBlock.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderV1.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLruBlockCache.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java Tue Oct 11 19:13:09 2011
@@ -25,16 +25,15 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.hfile.ColumnFamilyMetrics.
-    ColumnFamilyConfigured;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
 import org.apache.hadoop.io.RawComparator;
 
 /**
  * Common functionality needed by all versions of {@link HFile} readers.
  */
-public abstract class AbstractHFileReader extends ColumnFamilyConfigured
+public abstract class AbstractHFileReader extends SchemaConfigured
     implements HFile.Reader {
 
   /** Filesystem-level block reader for this HFile format version. */
@@ -96,14 +95,12 @@ public abstract class AbstractHFileReade
 
   protected FileInfo fileInfo;
 
-  protected final ColumnFamilyMetrics cfMetrics;
-
   protected AbstractHFileReader(Path path, FixedFileTrailer trailer,
       final FSDataInputStream fsdis, final long fileSize,
       final boolean closeIStream,
       final BlockCache blockCache, final boolean inMemory,
       final boolean evictOnClose) {
-    super(path);
+    super(null, path);  // We don't have configuration to pass to super.
     this.trailer = trailer;
     this.compressAlgo = trailer.getCompressionCodec();
     this.blockCache = blockCache;
@@ -114,7 +111,6 @@ public abstract class AbstractHFileReade
     this.evictOnClose = evictOnClose;
     this.path = path;
     this.name = path.getName();
-    cfMetrics = ColumnFamilyMetrics.getInstance(getColumnFamilyName());
   }
 
   @SuppressWarnings("serial")
@@ -322,8 +318,4 @@ public abstract class AbstractHFileReade
     return path;
   }
 
-  public ColumnFamilyMetrics getColumnFamilyMetrics() {
-    return cfMetrics;
-  }
-
 }

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java Tue Oct 11 19:13:09 2011
@@ -26,18 +26,15 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
-import org.apache.hadoop.hbase.io.hfile.ColumnFamilyMetrics.
-    ColumnFamilyConfigured;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.Writable;
@@ -45,11 +42,9 @@ import org.apache.hadoop.io.Writable;
 /**
  * Common functionality needed by all versions of {@link HFile} writers.
  */
-public abstract class AbstractHFileWriter extends ColumnFamilyConfigured
+public abstract class AbstractHFileWriter extends SchemaConfigured
     implements HFile.Writer {
 
-  private static final Log LOG = LogFactory.getLog(AbstractHFileWriter.class);
-
   /** Key previously appended. Becomes the last key in the file. */
   protected byte[] lastKeyBuffer = null;
 
@@ -119,7 +114,7 @@ public abstract class AbstractHFileWrite
   public AbstractHFileWriter(Configuration conf,
       FSDataOutputStream outputStream, Path path, int blockSize,
       Compression.Algorithm compressAlgo, KeyComparator comparator) {
-    super(path);
+    super(conf, path);
     this.outputStream = outputStream;
     this.path = path;
     this.name = path != null ? path.getName() : outputStream.toString();

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java Tue Oct 11 19:13:09 2011
@@ -19,7 +19,6 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.hfile.LruBlockCache.CacheStats;
 
 /**
@@ -33,14 +32,14 @@ public interface BlockCache {
    * @param buf The block contents wrapped in a ByteBuffer.
    * @param inMemory Whether block should be treated as in-memory
    */
-  public void cacheBlock(String blockName, HeapSize buf, boolean inMemory);
+  public void cacheBlock(String blockName, Cacheable buf, boolean inMemory);
 
   /**
    * Add block to cache (defaults to not in-memory).
    * @param blockName Zero-based file block number.
    * @param buf The block contents wrapped in a ByteBuffer.
    */
-  public void cacheBlock(String blockName, HeapSize buf);
+  public void cacheBlock(String blockName, Cacheable buf);
 
   /**
    * Fetch block from cache.
@@ -48,7 +47,7 @@ public interface BlockCache {
    * @param caching true if the caller caches blocks on a miss
    * @return Block or null if block is not in the cache.
    */
-  public HeapSize getBlock(String blockName, boolean caching);
+  public Cacheable getBlock(String blockName, boolean caching);
 
   /**
    * Evict block from cache.

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockType.java Tue Oct 11 19:13:09 2011
@@ -75,7 +75,18 @@ public enum BlockType {
   INDEX_V1("IDXBLK)+", BlockCategory.INDEX);
 
   public enum BlockCategory {
-    DATA, META, INDEX, BLOOM, ALL_CATEGORIES, UNKNOWN
+    DATA, META, INDEX, BLOOM, ALL_CATEGORIES, UNKNOWN;
+
+    /**
+     * Throws an exception if the block category passed is the special category
+     * meaning "all categories".
+     */
+    public void expectSpecific() {
+      if (this == ALL_CATEGORIES) {
+        throw new IllegalArgumentException("Expected a specific block " +
+            "category but got " + this);
+      }
+    }
   }
 
   public static final int MAGIC_LENGTH = 8;

Copied: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java (from r1182033, hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockInfo.java)
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java?p2=hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java&p1=hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockInfo.java&r1=1182033&r2=1182034&rev=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockInfo.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java Tue Oct 11 19:13:09 2011
@@ -1,4 +1,4 @@
-/*
+/**
  * Copyright 2011 The Apache Software Foundation
  *
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -17,23 +17,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hbase.io.hfile;
 
-import org.apache.hadoop.hbase.io.hfile.BlockType;
-import org.apache.hadoop.hbase.io.hfile.ColumnFamilyMetrics.ColumnFamilyAware;
+package org.apache.hadoop.hbase.io.hfile;
 
-/**
- * An interface that exposes methods to retrieve the column type and BlockType
- * of a particular cached block. This is more information than that which is
- * required by most cache implementations, but is used for more specific
- * metrics, for example. Used by implementations of HeapSize, such as
- * {@link HFileBlock}
- */
-public interface HFileBlockInfo extends ColumnFamilyAware {
+import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 
+public interface Cacheable extends HeapSize {
   /**
-   * @return BlockType descriptor of this cached item. Indicates the type of
-   *         data, such as a data block or an index one.
+   * @return the block type of this cached HFile block
    */
   public BlockType getBlockType();
-}
\ No newline at end of file
+
+  /**
+   * @return the metrics object identified by table and column family
+   */
+  public SchemaMetrics getSchemaMetrics();
+}

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/CachedBlock.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/CachedBlock.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/CachedBlock.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/CachedBlock.java Tue Oct 11 19:13:09 2011
@@ -53,16 +53,16 @@ public class CachedBlock implements Heap
   };
 
   private final String blockName;
-  private final HeapSize buf;
+  private final Cacheable buf;
   private volatile long accessTime;
   private long size;
   private BlockPriority priority;
 
-  public CachedBlock(String blockName, HeapSize buf, long accessTime) {
+  public CachedBlock(String blockName, Cacheable buf, long accessTime) {
     this(blockName, buf, accessTime, false);
   }
 
-  public CachedBlock(String blockName, HeapSize buf, long accessTime,
+  public CachedBlock(String blockName, Cacheable buf, long accessTime,
       boolean inMemory) {
     this.blockName = blockName;
     this.buf = buf;
@@ -101,7 +101,7 @@ public class CachedBlock implements Heap
     return this.accessTime < that.accessTime ? 1 : -1;
   }
 
-  public HeapSize getBuffer() {
+  public Cacheable getBuffer() {
     return this.buf;
   }
 

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java Tue Oct 11 19:13:09 2011
@@ -39,6 +39,8 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
 import org.apache.hadoop.hbase.io.HbaseMapWritable;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.SchemaAware;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -145,6 +147,11 @@ public class HFile {
   /** Separator between HFile name and offset in block cache key */
   static final char CACHE_KEY_SEPARATOR = '_';
 
+  /**
+   *  ROOT_DIR/TABLE_NAME/REGION_NAME/CF_NAME/HFILE
+   */
+  public final static int HFILE_PATH_LENGTH = 5;
+
   // For measuring latency of "typical" reads and writes
   static volatile int readOps;
   static volatile long readTime;
@@ -303,7 +310,7 @@ public class HFile {
   }
 
   /** An interface used by clients to open and iterate an {@link HFile}. */
-  public interface Reader extends Closeable, CachingBlockReader {
+  public interface Reader extends Closeable, CachingBlockReader, SchemaAware {
     /**
      * Returns this reader's "name". Usually the last component of the path.
      * Needs to be constant as the file is being moved to support caching on

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java Tue Oct 11 19:13:09 2011
@@ -35,9 +35,8 @@ import org.apache.hadoop.fs.FSDataInputS
 import org.apache.hadoop.fs.FSDataOutputStream;
 
 import org.apache.hadoop.hbase.io.DoubleOutputStream;
-import org.apache.hadoop.hbase.io.HeapSize;
-import org.apache.hadoop.hbase.io.hfile.HFileBlockInfo;
 import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.IOUtils;
@@ -73,8 +72,9 @@ import static org.apache.hadoop.hbase.io
  * The version 2 block representation in the block cache is the same as above,
  * except that the data section is always uncompressed in the cache.
  */
-public class HFileBlock implements HeapSize, HFileBlockInfo {
+public class HFileBlock extends SchemaConfigured implements Cacheable {
 
+  // Constants
   /** The size of a version 2 {@link HFile} block header */
   public static final int HEADER_SIZE = MAGIC_LENGTH + 2 * Bytes.SIZEOF_INT
       + Bytes.SIZEOF_LONG;
@@ -85,6 +85,11 @@ public class HFileBlock implements HeapS
   public static final int BYTE_BUFFER_HEAP_SIZE = (int) ClassSize.estimateBase(
       ByteBuffer.wrap(new byte[0], 0, 0).getClass(), false);
 
+  // Static counters
+  private static final AtomicLong numSeekRead = new AtomicLong();
+  private static final AtomicLong numPositionalRead = new AtomicLong();
+
+  // Instance variables
   private BlockType blockType;
   private final int onDiskSizeWithoutHeader;
   private final int uncompressedSizeWithoutHeader;
@@ -104,9 +109,6 @@ public class HFileBlock implements HeapS
    */
   private int nextBlockOnDiskSizeWithHeader = -1;
 
-  private static final AtomicLong numSeekRead = new AtomicLong();
-  private static final AtomicLong numPositionalRead = new AtomicLong();
-
   /**
    * Creates a new {@link HFile} block from the given fields. This constructor
    * is mostly used when the block data has already been read and uncompressed,
@@ -139,16 +141,6 @@ public class HFileBlock implements HeapS
     this.offset = offset;
   }
 
-  private String cfName = "cf.unknown";
-
-  public String getColumnFamilyName() {
-    return this.cfName;
-  }
-
-  public void setColumnFamilyName(String cfName) {
-    this.cfName = cfName;
-  }
-
   /**
    * Creates a block from an existing buffer starting with a header. Rewinds
    * and takes ownership of the buffer. By definition of rewind, ignores the
@@ -1380,16 +1372,23 @@ public class HFileBlock implements HeapS
 
   @Override
   public long heapSize() {
-    // This object, block type and byte buffer reference, on-disk and
-    // uncompressed size, next block's on-disk size, offset and previous
-    // offset, byte buffer object, and its byte array. Might also need to add
-    // some fields inside the byte buffer.
-    // Also includes size of string cfName, which has its length multiplied by
-    // two because Java characters are unicode and thus 2 bytes each.
-    return ClassSize.align(ClassSize.OBJECT + 3 * ClassSize.REFERENCE + 3
-        * Bytes.SIZEOF_INT + 2 * Bytes.SIZEOF_LONG)
-        + ClassSize.align(buf.capacity() + BYTE_BUFFER_HEAP_SIZE)
-        + ClassSize.align(ClassSize.STRING + 2 * this.cfName.length());
+    long size = ClassSize.align(
+        // This object
+        ClassSize.OBJECT +
+        // Block type and byte buffer references
+        2 * ClassSize.REFERENCE +
+        // On-disk size, uncompressed size, and next block's on-disk size
+        3 * Bytes.SIZEOF_INT +
+        // This and previous block offset
+        2 * Bytes.SIZEOF_LONG
+    );
+
+    // Byte buffer
+    size += ClassSize.align(buf.capacity() + BYTE_BUFFER_HEAP_SIZE);
+
+    // SchemaConfigured (but don't count object overhead twice).
+    size += super.heapSize() - ClassSize.OBJECT;
+    return size;
   }
 
   /**

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java Tue Oct 11 19:13:09 2011
@@ -227,7 +227,7 @@ public class HFileReaderV1 extends Abstr
             cacheBlock);
         if (cachedBlock != null) {
           cacheHits++;
-          cfMetrics.updateOnCacheHit(BlockCategory.META, false);
+          getSchemaMetrics().updateOnCacheHit(BlockCategory.META, false);
           return cachedBlock.getBufferWithoutHeader();
         }
         // Cache Miss, please load.
@@ -236,13 +236,13 @@ public class HFileReaderV1 extends Abstr
       HFileBlock hfileBlock = fsBlockReader.readBlockData(offset,
           nextOffset - offset, metaBlockIndexReader.getRootBlockDataSize(block),
           true);
-      hfileBlock.setColumnFamilyName(this.getColumnFamilyName());
+      configureWithSchema(hfileBlock);
       hfileBlock.expectType(BlockType.META);
 
       long delta = System.currentTimeMillis() - now;
       HFile.readTime += delta;
       HFile.readOps++;
-      cfMetrics.updateOnCacheMiss(BlockCategory.META, false, delta);
+      getSchemaMetrics().updateOnCacheMiss(BlockCategory.META, false, delta);
 
       // Cache the block
       if (cacheBlock && blockCache != null) {
@@ -289,7 +289,8 @@ public class HFileReaderV1 extends Abstr
             cacheBlock);
         if (cachedBlock != null) {
           cacheHits++;
-          cfMetrics.updateOnCacheHit(BlockCategory.DATA, isCompaction);
+          getSchemaMetrics().updateOnCacheHit(BlockCategory.DATA,
+              isCompaction);
           return cachedBlock.getBufferWithoutHeader();
         }
         // Carry on, please load.
@@ -311,14 +312,15 @@ public class HFileReaderV1 extends Abstr
 
       HFileBlock hfileBlock = fsBlockReader.readBlockData(offset, nextOffset
           - offset, dataBlockIndexReader.getRootBlockDataSize(block), pread);
-      hfileBlock.setColumnFamilyName(this.getColumnFamilyName());
+      configureWithSchema(hfileBlock);
       hfileBlock.expectType(BlockType.DATA);
       ByteBuffer buf = hfileBlock.getBufferWithoutHeader();
 
       long delta = System.currentTimeMillis() - now;
       HFile.readTime += delta;
       HFile.readOps++;
-      cfMetrics.updateOnCacheMiss(BlockCategory.DATA, isCompaction, delta);
+      getSchemaMetrics().updateOnCacheMiss(BlockCategory.DATA, isCompaction,
+          delta);
 
       // Cache the block
       if (cacheBlock && blockCache != null) {

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java Tue Oct 11 19:13:09 2011
@@ -180,7 +180,7 @@ public class HFileReaderV2 extends Abstr
           // Return a distinct 'shallow copy' of the block,
           // so pos does not get messed by the scanner
           cacheHits++;
-          cfMetrics.updateOnCacheHit(BlockCategory.META, false);
+          getSchemaMetrics().updateOnCacheHit(BlockCategory.META, false);
           return cachedBlock.getBufferWithoutHeader();
         }
         // Cache Miss, please load.
@@ -188,12 +188,12 @@ public class HFileReaderV2 extends Abstr
 
       HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset,
           blockSize, -1, true);
-      metaBlock.setColumnFamilyName(this.getColumnFamilyName());
+      configureWithSchema(metaBlock);
 
       long delta = System.currentTimeMillis() - now;
       HFile.readTime += delta;
       HFile.readOps++;
-      cfMetrics.updateOnCacheMiss(BlockCategory.META, false, delta);
+      getSchemaMetrics().updateOnCacheMiss(BlockCategory.META, false, delta);
 
       // Cache the block
       if (cacheBlock && blockCache != null) {
@@ -246,7 +246,7 @@ public class HFileReaderV2 extends Abstr
           BlockCategory blockCategory =
               cachedBlock.getBlockType().getCategory();
           cacheHits++;
-          cfMetrics.updateOnCacheHit(blockCategory, isCompaction);
+          getSchemaMetrics().updateOnCacheHit(blockCategory, isCompaction);
           return cachedBlock;
         }
         // Carry on, please load.
@@ -256,13 +256,13 @@ public class HFileReaderV2 extends Abstr
       long now = System.currentTimeMillis();
       HFileBlock dataBlock = fsBlockReader.readBlockData(dataBlockOffset,
           onDiskBlockSize, -1, pread);
-      dataBlock.setColumnFamilyName(this.getColumnFamilyName());
+      configureWithSchema(dataBlock);
       BlockCategory blockCategory = dataBlock.getBlockType().getCategory();
 
       long delta = System.currentTimeMillis() - now;
       HFile.readTime += delta;
       HFile.readOps++;
-      cfMetrics.updateOnCacheMiss(blockCategory, isCompaction, delta);
+      getSchemaMetrics().updateOnCacheMiss(blockCategory, isCompaction, delta);
 
       // Cache the block
       if (cacheBlock && blockCache != null) {

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java Tue Oct 11 19:13:09 2011
@@ -201,7 +201,7 @@ public class HFileWriterV1 extends Abstr
       HFileBlock cBlock = new HFileBlock(BlockType.DATA,
           (int) (outputStream.getPos() - blockBegin), bytes.length, -1,
           ByteBuffer.wrap(bytes, 0, bytes.length), true, blockBegin);
-      cBlock.setColumnFamilyName(this.getColumnFamilyName());
+      configureWithSchema(cBlock);
       blockCache.cacheBlock(HFile.getBlockCacheKey(name, blockBegin),cBlock);
       baosDos.close();
     }

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java Tue Oct 11 19:13:09 2011
@@ -210,10 +210,7 @@ public class HFileWriterV2 extends Abstr
     HFile.writeOps++;
 
     if (cacheDataBlocksOnWrite) {
-      HFileBlock cBlock = fsBlockWriter.getBlockForCaching();
-      cBlock.setColumnFamilyName(this.getColumnFamilyName());
-      blockCache.cacheBlock(HFile.getBlockCacheKey(name, lastDataBlockOffset),
-          cBlock);
+      cacheBlockOnWrite(lastDataBlockOffset);
     }
   }
 
@@ -230,16 +227,19 @@ public class HFileWriterV2 extends Abstr
             fsBlockWriter.getUncompressedSizeWithoutHeader());
 
         if (cacheThisBlock) {
-          // Cache this block on write.
-          HFileBlock cBlock = fsBlockWriter.getBlockForCaching();
-          cBlock.setColumnFamilyName(this.getColumnFamilyName());
-          blockCache.cacheBlock(HFile.getBlockCacheKey(name, offset),
-              cBlock);
+          cacheBlockOnWrite(offset);
         }
       }
     }
   }
 
+  private void cacheBlockOnWrite(long offset) {
+    final HFileBlock cBlock = fsBlockWriter.getBlockForCaching();
+    configureWithSchema(cBlock);
+    blockCache.cacheBlock(HFile.getBlockCacheKey(name, offset),
+        cBlock);
+  }
+
   /**
    * Ready a new block for writing.
    *

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java Tue Oct 11 19:13:09 2011
@@ -31,11 +31,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
-import org.apache.hadoop.hbase.io.hfile.ColumnFamilyMetrics;
-import org.apache.hadoop.hbase.io.hfile.ColumnFamilyMetrics.BlockMetricType;
-import org.apache.hadoop.hbase.io.hfile.HFileBlockInfo;
 import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 
@@ -273,7 +270,7 @@ public class LruBlockCache implements Bl
    * @param buf block buffer
    * @param inMemory if block is in-memory
    */
-  public void cacheBlock(String blockName, HeapSize buf, boolean inMemory) {
+  public void cacheBlock(String blockName, Cacheable buf, boolean inMemory) {
     CachedBlock cb = map.get(blockName);
     if(cb != null) {
       throw new RuntimeException("Cached an already cached block");
@@ -297,7 +294,7 @@ public class LruBlockCache implements Bl
    * @param blockName block name
    * @param buf block buffer
    */
-  public void cacheBlock(String blockName, HeapSize buf) {
+  public void cacheBlock(String blockName, Cacheable buf) {
     cacheBlock(blockName, buf, false);
   }
 
@@ -314,12 +311,11 @@ public class LruBlockCache implements Bl
     if (evict) {
       heapsize *= -1;
     }
-    if (cb.getBuffer() instanceof HFileBlockInfo) {
-      final HFileBlockInfo blockInfo = (HFileBlockInfo) cb.getBuffer();
-      ColumnFamilyMetrics cfMetrics = ColumnFamilyMetrics.getInstance(
-          blockInfo.getColumnFamilyName());
-      cfMetrics.updateBlockCacheMetrics(blockInfo.getBlockType().getCategory(),
-          heapsize, evict);
+    Cacheable cachedBlock = cb.getBuffer();
+    SchemaMetrics schemaMetrics = cachedBlock.getSchemaMetrics();
+    if (schemaMetrics != null) {
+      schemaMetrics.updateOnCachePutOrEvict(
+          cachedBlock.getBlockType().getCategory(), heapsize, evict);
     }
     return size.addAndGet(heapsize);
   }
@@ -331,7 +327,7 @@ public class LruBlockCache implements Bl
    * @return buffer of specified block name, or null if not in cache
    */
   @Override
-  public HeapSize getBlock(String blockName, boolean caching) {
+  public Cacheable getBlock(String blockName, boolean caching) {
     CachedBlock cb = map.get(blockName);
     if(cb == null) {
       stats.miss(caching);

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/io/hfile/SimpleBlockCache.java Tue Oct 11 19:13:09 2011
@@ -33,9 +33,9 @@ import org.apache.hadoop.hbase.io.hfile.
  * Simple one RFile soft reference cache.
  */
 public class SimpleBlockCache implements BlockCache {
-  private static class Ref extends SoftReference<HeapSize> {
+  private static class Ref extends SoftReference<Cacheable> {
     public String blockId;
-    public Ref(String blockId, HeapSize block, ReferenceQueue q) {
+    public Ref(String blockId, Cacheable block, ReferenceQueue q) {
       super(block, q);
       this.blockId = blockId;
     }
@@ -69,7 +69,7 @@ public class SimpleBlockCache implements
     return cache.size();
   }
 
-  public synchronized HeapSize getBlock(String blockName, boolean caching) {
+  public synchronized Cacheable getBlock(String blockName, boolean caching) {
     processQueue(); // clear out some crap.
     Ref ref = cache.get(blockName);
     if (ref == null)
@@ -77,11 +77,11 @@ public class SimpleBlockCache implements
     return ref.get();
   }
 
-  public synchronized void cacheBlock(String blockName, HeapSize block) {
+  public synchronized void cacheBlock(String blockName, Cacheable block) {
     cache.put(blockName, new Ref(blockName, block, q));
   }
 
-  public synchronized void cacheBlock(String blockName, HeapSize block,
+  public synchronized void cacheBlock(String blockName, Cacheable block,
       boolean inMemory) {
     cache.put(blockName, new Ref(blockName, block, q));
   }

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Tue Oct 11 19:13:09 2011
@@ -29,6 +29,7 @@ import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -79,6 +80,7 @@ import org.apache.hadoop.hbase.ipc.HRegi
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
@@ -276,13 +278,17 @@ public class HRegion implements HeapSize
   public static String createMutationSignature(Set<byte[]> families) {
     int limit = families.size();
     if (1 == limit) {
-      return "cf." + Bytes.toString(families.iterator().next());
+      return SchemaMetrics.CF_PREFIX +
+          Bytes.toString(families.iterator().next());
     }
 
-    StringBuilder sb = new StringBuilder("cf.");
+    List<byte[]> sortedFamilies = new ArrayList<byte[]>(families);
+    Collections.sort(sortedFamilies, Bytes.BYTES_COMPARATOR);
+
+    StringBuilder sb = new StringBuilder(SchemaMetrics.CF_PREFIX);
 
     int MAX_SIZE = 256;
-    for (byte[] family : families) {
+    for (byte[] family : sortedFamilies) {
       if (sb.length() > MAX_SIZE) {
         sb.append("__more");
         break;
@@ -309,7 +315,7 @@ public class HRegion implements HeapSize
    * @return the string to print out in metrics
    */
   public static String createMutationSignature(byte[] family) {
-    return "cf." + Bytes.toString(family);
+    return SchemaMetrics.CF_PREFIX + Bytes.toString(family);
   }
 
   public static void incrNumericMetric(String key, long amount) {
@@ -356,6 +362,16 @@ public class HRegion implements HeapSize
     return m.get();
   }
 
+  public static Pair<Long, Integer> getTimeVaryingMetric(String key) {
+    Pair<AtomicLong, AtomicInteger> pair = timeVaryingMetrics.get(key);
+    if (pair == null) {
+      return new Pair<Long, Integer>(0L, 0);
+    }
+
+    return new Pair<Long, Integer>(pair.getFirst().get(),
+        pair.getSecond().get());
+  }
+
   static long getNumericPersistentMetric(String key) {
     AtomicLong m = numericPersistentMetrics.get(key);
     if (m == null)
@@ -1812,7 +1828,7 @@ public class HRegion implements HeapSize
             if (!signature.equals(HRegion.createMutationSignature(put
                 .getFamilyMap().keySet()))) {
               isSignatureClear = false;
-              signature = "cf.__unknown";
+              signature = SchemaMetrics.CF_UNKNOWN_PREFIX;
             }
           }
         }
@@ -1874,7 +1890,7 @@ public class HRegion implements HeapSize
       // do after lock
       long after = EnvironmentEdgeManager.currentTimeMillis();
       if (null == signature) {
-        signature = "cf.__badfamily";
+        signature = SchemaMetrics.CF_BAD_FAMILY_PREFIX;
       }
       HRegion.incrTimeVaryingMetric(signature + ".multiput_", after - now);
 

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Tue Oct 11 19:13:09 2011
@@ -104,6 +104,9 @@ import org.apache.hadoop.hbase.ipc.HMast
 import org.apache.hadoop.hbase.ipc.HRegionInterface;
 import org.apache.hadoop.hbase.regionserver.metrics.RegionServerDynamicMetrics;
 import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.StoreMetricType;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -316,6 +319,7 @@ public class HRegionServer implements HR
         HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
 
     reinitialize();
+    SchemaMetrics.configureGlobally(conf);
   }
 
   /**
@@ -1074,7 +1078,7 @@ public class HRegionServer implements HR
     long tmpfilesize;
     long tmpbloomsize;
     long tmpstaticsize;
-    String cfname;
+    SchemaMetrics schemaMetrics;
 
     // Note that this is a map of Doubles instead of Longs. This is because we
     // do effective integer division, which would perhaps truncate more than it
@@ -1097,19 +1101,29 @@ public class HRegionServer implements HR
             tmpbloomsize = store.getTotalStaticBloomSize();
             tmpstaticsize = store.getTotalStaticIndexSize();
 
-            // Note that there is only one store per CF so setting is safe
-            cfname = "cf." + store.toString();
-            this.incrMap(tempVals, cfname + ".storeFileCount", tmpfiles);
-            this.incrMap(tempVals, cfname + ".storeFileIndexSizeMB",
-                (tmpindex / (1024.0 * 1024)));
-            this.incrMap(tempVals, cfname + ".storeFileSizeMB",
-                (tmpfilesize / (1024.0 * 1024)));
-            this.incrMap(tempVals, cfname + ".staticBloomSizeKB",
-                (tmpbloomsize / 1024.0));
-            this.incrMap(tempVals, cfname + ".memstoreSizeMB",
-                (store.getMemStoreSize() / (1024.0 * 1024)));
-            this.incrMap(tempVals, cfname + ".staticIndexSizeKB",
-                tmpstaticsize / 1024.0);
+            schemaMetrics = store.getSchemaMetrics();
+            schemaMetrics.updateStoreMetric(
+                StoreMetricType.STORE_FILE_COUNT,
+                tmpfiles);
+
+            schemaMetrics.updateStoreMetric(
+                StoreMetricType.STORE_FILE_INDEX_SIZE,
+                (long) (tmpindex / (1024.0 * 1024)));
+
+            schemaMetrics.updateStoreMetric(
+                StoreMetricType.STORE_FILE_SIZE_MB,
+                (long) (tmpfilesize / (1024.0 * 1024)));
+
+            schemaMetrics.updateStoreMetric(
+                StoreMetricType.STATIC_BLOOM_SIZE_KB,
+                (long)(tmpbloomsize / 1024.0));
+
+            schemaMetrics.updateStoreMetric(StoreMetricType.MEMSTORE_SIZE_MB,
+                (long)(store.getMemStoreSize() / (1024.0 * 1024)));
+
+            schemaMetrics.updateStoreMetric(
+                StoreMetricType.STATIC_INDEX_SIZE_KB,
+                (long)(tmpstaticsize / 1024.0));
 
             storefiles += tmpfiles;
             storefileIndexSize += tmpindex;

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Tue Oct 11 19:13:09 2011
@@ -53,6 +53,8 @@ import org.apache.hadoop.hbase.monitorin
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.util.StringUtils;
@@ -87,7 +89,7 @@ import com.google.common.collect.Lists;
  * <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 HeapSize {
+public class Store extends SchemaConfigured implements HeapSize {
   static final Log LOG = LogFactory.getLog(Store.class);
   protected final MemStore memstore;
   // This stores directory in the filesystem.
@@ -151,6 +153,8 @@ public class Store implements HeapSize {
   protected Store(Path basedir, HRegion region, HColumnDescriptor family,
     FileSystem fs, Configuration conf)
   throws IOException {
+    super(conf, region.getTableDesc().getNameAsString(),
+        Bytes.toString(family.getName()));
     HRegionInfo info = region.regionInfo;
     this.fs = fs;
     this.homedir = getStoreHomedir(basedir, info.getEncodedName(), family.getName());
@@ -178,7 +182,7 @@ public class Store implements HeapSize {
       this.ttl *= 1000;
     }
     this.memstore = new MemStore(this.comparator);
-    this.storeNameStr = Bytes.toString(this.family.getName());
+    this.storeNameStr = getColumnFamilyName();
 
     // By default, compact if storefile.count >= minFilesToCompact
     this.minFilesToCompact = Math.max(2,
@@ -528,8 +532,8 @@ public class Store implements HeapSize {
     // retrieved from HRegion.recentFlushes, which is set within
     // HRegion.internalFlushcache, which indirectly calls this to actually do
     // the flushing through the StoreFlusherImpl class
-    HRegion.incrNumericPersistentMetric("cf." + this.toString() + ".flushSize",
-        flushed);
+    getSchemaMetrics().updatePersistentStoreMetric(
+        SchemaMetrics.StoreMetricType.FLUSH_SIZE, flushed);
     if(LOG.isInfoEnabled()) {
       LOG.info("Added " + sf + ", entries=" + r.getEntries() +
         ", sequenceid=" + logCacheFlushId +
@@ -1748,7 +1752,7 @@ public class Store implements HeapSize {
   }
 
   public static final long FIXED_OVERHEAD = ClassSize.align(
-      ClassSize.OBJECT + (14 * ClassSize.REFERENCE) +
+      new SchemaConfigured().heapSize() + (14 * ClassSize.REFERENCE) +
       (7 * Bytes.SIZEOF_LONG) + (6 * Bytes.SIZEOF_INT) + (3 * Bytes.SIZEOF_BOOLEAN));
 
   public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java Tue Oct 11 19:13:09 2011
@@ -48,12 +48,14 @@ import org.apache.hadoop.hbase.client.Sc
 import org.apache.hadoop.hbase.io.HalfStoreFileReader;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.ColumnFamilyMetrics;
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.io.hfile.HFileWriterV1;
 import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.SchemaAware;
 import org.apache.hadoop.hbase.util.BloomFilter;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
@@ -411,6 +413,7 @@ public class StoreFile {
       this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
           getBlockCache(), this.reference);
     } else {
+      SchemaMetrics.configureGlobally(conf);
       this.reader = new Reader(this.fs, this.path, getBlockCache(),
           this.inMemory,
           this.conf.getBoolean(HFile.EVICT_BLOCKS_ON_CLOSE_KEY, true));
@@ -754,8 +757,8 @@ public class StoreFile {
         throws IOException {
 
       writer = HFile.getWriterFactory(conf).createWriter(
-		fs, path, blocksize, HFile.getBytesPerChecksum(conf, fs.getConf()),
-		compress, comparator.getRawComparator());
+          fs, path, blocksize, HFile.getBytesPerChecksum(conf, fs.getConf()),
+          compress, comparator.getRawComparator());
 
       this.kvComparator = comparator;
 
@@ -954,7 +957,7 @@ public class StoreFile {
   /**
    * Reader for a StoreFile.
    */
-  public static class Reader {
+  public static class Reader extends SchemaConfigured {
     static final Log LOG = LogFactory.getLog(Reader.class.getName());
 
     protected BloomFilter bloomFilter = null;
@@ -963,16 +966,17 @@ public class StoreFile {
     protected TimeRangeTracker timeRangeTracker = null;
     protected long sequenceID = -1;
     private byte[] lastBloomKey;
-    private final ColumnFamilyMetrics cfMetrics;
+
+    private Reader(HFile.Reader reader) {
+      super(reader);
+      this.reader = reader;
+    }
 
     public Reader(FileSystem fs, Path path, BlockCache blockCache,
         boolean inMemory, boolean evictOnClose)
         throws IOException {
-      reader = HFile.createReader(fs, path, blockCache, inMemory, evictOnClose);
-
-      // prepare the text (key) for the metrics
-      cfMetrics = ColumnFamilyMetrics.getInstance(
-          reader.getColumnFamilyName());
+      this(HFile.createReader(fs, path, blockCache, inMemory,
+          evictOnClose));
       bloomFilterType = BloomType.NONE;
     }
 
@@ -981,7 +985,6 @@ public class StoreFile {
      */
     Reader() {
       this.reader = null;
-      this.cfMetrics = ColumnFamilyMetrics.getInstance(null);
     }
 
     public RawComparator<byte []> getComparator() {
@@ -1206,7 +1209,7 @@ public class StoreFile {
                 && bloomFilter.contains(key, 0, key.length, bloom);
           }
 
-          cfMetrics.updateBloomMetrics(exists);
+          getSchemaMetrics().updateBloomMetrics(exists);
           return exists;
         }
       } catch (IOException e) {

Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaConfigured.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaConfigured.java?rev=1182034&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaConfigured.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaConfigured.java Tue Oct 11 19:13:09 2011
@@ -0,0 +1,142 @@
+/*
+ * Copyright 2011 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.regionserver.metrics;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileBlock;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.SchemaAware;
+import org.apache.hadoop.hbase.util.ClassSize;
+
+/**
+ * A base class for objects that are associated with a particular table and
+ * column family. Provides a way to obtain the schema metrics object.
+ */
+public class SchemaConfigured implements HeapSize, SchemaAware {
+  private static final Log LOG = LogFactory.getLog(SchemaMetrics.class);
+
+  // These are not final because we set them at runtime e.g. for HFile blocks.
+  private String cfName = SchemaMetrics.UNKNOWN;
+  private String tableName = SchemaMetrics.UNKNOWN;
+  private SchemaMetrics schemaMetrics;
+
+  /** A helper constructor that configures the "use table name" flag. */
+  private SchemaConfigured(Configuration conf) {
+    if (conf != null) {
+      SchemaMetrics.configureGlobally(conf);
+    }
+  }
+
+  /**
+   * Default constructor. Only use when column/family name are not known at
+   * construction (i.e. for HFile blocks).
+   */
+  public SchemaConfigured() {
+  }
+
+  /**
+   * Initialize table and column family name from an HFile path. If
+   * configuration is null,
+   * {@link SchemaMetrics#configureGlobally(Configuration)} should have been
+   * called already.
+   */
+  public SchemaConfigured(Configuration conf, Path path) {
+    this(conf);
+
+    if (path != null) {
+      String splits[] = path.toString().split("/");
+      if (splits.length < HFile.HFILE_PATH_LENGTH) {
+        LOG.warn("Could not determine table and column family of the HFile " +
+            "path " + path);
+      } else {
+        cfName = splits[splits.length - 2].intern();
+        tableName = splits[splits.length - 4].intern();
+      }
+    }
+
+    initializeMetrics();
+  }
+
+  /**
+   * Used when we know table and column family name. If configuration is null,
+   * {@link SchemaMetrics#configureGlobally(Configuration)} should have been
+   * called already.
+   */
+  public SchemaConfigured(Configuration conf, String tableName, String cfName)
+  {
+    this(conf);
+    this.tableName = tableName.intern();
+    this.cfName = cfName.intern();
+    initializeMetrics();
+  }
+
+  public SchemaConfigured(SchemaAware that) {
+    tableName = that.getTableName();
+    cfName = that.getColumnFamilyName();
+    schemaMetrics = that.getSchemaMetrics();
+    if (schemaMetrics == null) {
+      initializeMetrics();
+    }
+  }
+
+  private void initializeMetrics() {
+    schemaMetrics = SchemaMetrics.getInstance(tableName, cfName);
+  }
+
+  @Override
+  public String getTableName() {
+    return tableName;
+  }
+
+  @Override
+  public String getColumnFamilyName() {
+    return cfName;
+  }
+
+  @Override
+  public SchemaMetrics getSchemaMetrics() {
+    return schemaMetrics;
+  }
+
+  /**
+   * Configures the given object (most commonly an HFile block) with the
+   * current table and column family name, and the associated collection of
+   * metrics.
+   */
+  public void configureWithSchema(SchemaConfigured block) {
+    SchemaConfigured upcast = block;  // need this to assign private fields
+    upcast.tableName = tableName;
+    upcast.cfName = cfName;
+    upcast.schemaMetrics = schemaMetrics;
+  }
+
+  @Override
+  public long heapSize() {
+    // We don't count table name and column family name characters because
+    // these strings are shared among many objects.
+    return ClassSize.align(ClassSize.OBJECT + 3 * ClassSize.REFERENCE);
+  }
+
+}

Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java?rev=1182034&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/regionserver/metrics/SchemaMetrics.java Tue Oct 11 19:13:09 2011
@@ -0,0 +1,720 @@
+/*
+ * Copyright 2011 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.regionserver.metrics;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+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.conf.Configuration;
+import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * A collection of metric names in a given column family or a (table, column
+ * family) combination. The following "dimensions" are supported:
+ * <ul>
+ * <li>Table name (optional; enabled based on configuration)</li>
+ * <li>Per-column family vs. aggregated. The aggregated mode is only supported
+ * when table name is not included.</li>
+ * <li>Block category (data, index, bloom filter, etc.)</li>
+ * <li>Whether the request is part of a compaction</li>
+ * <li>Metric type (read time, block read count, cache hits/misses, etc.)</li>
+ * </ul>
+ * <p>
+ * An instance of this class does not store any metric values. It just allows
+ * to determine the correct metric name for each combination of the above
+ * dimensions.
+ * <p>
+ * <table>
+ * <tr>
+ * <th rowspan="2">Metric key</th>
+ * <th colspan="2">Per-table metrics conf setting</th>
+ * <th rowspan="2">Description</th>
+ * </tr>
+ * <tr>
+ * <th>On</th>
+ * <th>Off</th>
+ * </th>
+ * <tr>
+ *   <td> tab.T.cf.CF.M </td> <td> Include </td> <td> Skip    </td>
+ *   <td> A specific column family of a specific table        </td>
+ * </tr>
+ * <tr>
+ *   <td> tab.T.M       </td> <td> Skip    </td> <td> Skip    </td>
+ *   <td> All column families in the given table              </td>
+ * </tr>
+ * <tr>
+ *   <td> cf.CF.M       </td> <td> Skip    </td> <td> Include </td>
+ *   <td> A specific column family in all tables              </td>
+ * </tr>
+ * <tr>
+ *   <td> M             </td> <td> Include </td> <td> Include </td>
+ *   <td> All column families in all tables                   </td>
+ * </tr>
+ * </table>
+ */
+public class SchemaMetrics {
+
+  public interface SchemaAware {
+    public String getTableName();
+    public String getColumnFamilyName();
+    public SchemaMetrics getSchemaMetrics();
+  }
+
+  private static final Log LOG = LogFactory.getLog(SchemaMetrics.class);
+
+  public static enum BlockMetricType {
+    // Metric configuration: compactionAware | timeVarying
+    READ_TIME("Read",                   true,  true),
+    READ_COUNT("BlockReadCnt",          true,  false),
+    CACHE_HIT("BlockReadCacheHitCnt",   true,  false),
+    CACHE_MISS("BlockReadCacheMissCnt", true,  false),
+
+    CACHE_SIZE("blockCacheSize",        false, false),
+    CACHED("blockCacheNumCached",       false, false),
+    EVICTED("blockCacheNumEvicted",     false, false);
+
+    private final String metricStr;
+    private final boolean compactionAware;
+    private final boolean timeVarying;
+
+    BlockMetricType(String metricStr, boolean compactionAware,
+          boolean timeVarying) {
+      this.metricStr = metricStr;
+      this.compactionAware = compactionAware;
+      this.timeVarying = timeVarying;
+    }
+
+    @Override
+    public String toString() {
+      return metricStr;
+    }
+
+    private static final String BLOCK_METRIC_TYPE_RE;
+    static {
+      StringBuilder sb = new StringBuilder();
+      for (BlockMetricType bmt : values()) {
+        if (sb.length() > 0)
+          sb.append("|");
+        sb.append(bmt);
+      }
+      BLOCK_METRIC_TYPE_RE = sb.toString();
+    }
+  };
+
+  public static enum StoreMetricType {
+    STORE_FILE_COUNT("storeFileCount"),
+    STORE_FILE_INDEX_SIZE("storeFileIndexSizeMB"),
+    STORE_FILE_SIZE_MB("storeFileSizeMB"),
+    STATIC_BLOOM_SIZE_KB("staticBloomSizeKB"),
+    MEMSTORE_SIZE_MB("memstoreSizeMB"),
+    STATIC_INDEX_SIZE_KB("staticIndexSizeKB"),
+    FLUSH_SIZE("flushSize");
+
+    private final String metricStr;
+
+    StoreMetricType(String metricStr) {
+      this.metricStr = metricStr;
+    }
+
+    @Override
+    public String toString() {
+      return metricStr;
+    }
+  };
+
+  // Constants
+  /**
+   * A string used when column family or table name is unknown, and in some
+   * unit tests. This should not normally show up in metric names but if it
+   * does it is better than creating a silent discrepancy in total vs.
+   * per-CF/table metrics.
+   */
+  public static final String UNKNOWN = "__unknown";
+
+  private static final String TABLE_PREFIX = "tab.";
+  public static final String CF_PREFIX = "cf.";
+  public static final String BLOCK_TYPE_PREFIX = "bt.";
+  public static final String CF_UNKNOWN_PREFIX = CF_PREFIX + UNKNOWN;
+  public static final String CF_BAD_FAMILY_PREFIX = CF_PREFIX + "__badfamily";
+
+  /**
+   * A special schema metric value that means "all tables aggregated" or
+   * "all column families aggregated" when used as a table name or a column
+   * family name.
+   */
+  public static final String TOTAL_KEY = "";
+
+  /**
+   * Special handling for meta-block-specific metrics for
+   * backwards-compatibility.
+   */
+  private static final String META_BLOCK_CATEGORY_STR = "Meta";
+
+  private static final int NUM_BLOCK_CATEGORIES =
+      BlockCategory.values().length;
+
+  private static final int NUM_METRIC_TYPES =
+      BlockMetricType.values().length;
+
+  static final boolean[] BOOL_VALUES = new boolean[] { false, true };
+
+  private static final int NUM_BLOCK_METRICS =
+      NUM_BLOCK_CATEGORIES *  // blockCategory
+      BOOL_VALUES.length *    // isCompaction
+      NUM_METRIC_TYPES;       // metricType
+
+  private static final int NUM_STORE_METRIC_TYPES =
+      StoreMetricType.values().length;
+
+  /** Conf key controlling whether we include table name in metric names */
+  private static final String SHOW_TABLE_NAME_CONF_KEY =
+      "hbase.metrics.showTableName";
+
+  // Global variables
+  /** All instances of this class */
+  private static final ConcurrentHashMap<String, SchemaMetrics>
+      cfToMetrics = new ConcurrentHashMap<String, SchemaMetrics>();
+
+  /** Metrics for all tables and column families. */
+  // This has to be initialized after cfToMetrics.
+  public static final SchemaMetrics ALL_SCHEMA_METRICS =
+    getInstance(TOTAL_KEY, TOTAL_KEY);
+
+  /**
+   * Whether to include table name in metric names. If this is null, it has not
+   * been initialized. This is a global instance, but we also have a copy of it
+   * per a {@link SchemaMetrics} object to avoid synchronization overhead.
+   */
+  private static volatile Boolean useTableNameGlobally;
+
+  /** Whether we logged a message about configuration inconsistency */
+  private static volatile boolean loggedConfInconsistency;
+
+  // Instance variables
+  private final String[] blockMetricNames = new String[NUM_BLOCK_METRICS];
+  private final boolean[] blockMetricTimeVarying =
+      new boolean[NUM_BLOCK_METRICS];
+
+  private final String[] bloomMetricNames = new String[2];
+  private final String[] storeMetricNames = new String[NUM_STORE_METRIC_TYPES];
+
+  private SchemaMetrics(final String tableName, final String cfName) {
+    String metricPrefix =
+        tableName.equals(TOTAL_KEY) ? "" : TABLE_PREFIX + tableName + ".";
+    metricPrefix += cfName.equals(TOTAL_KEY) ? "" : CF_PREFIX + cfName + ".";
+
+    for (BlockCategory blockCategory : BlockCategory.values()) {
+      for (boolean isCompaction : BOOL_VALUES) {
+        for (BlockMetricType metricType : BlockMetricType.values()) {
+          if (!metricType.compactionAware && isCompaction) {
+            continue;
+          }
+
+          StringBuilder sb = new StringBuilder(metricPrefix);
+          if (blockCategory != BlockCategory.ALL_CATEGORIES
+              && blockCategory != BlockCategory.META) {
+            String categoryStr = blockCategory.toString();
+            categoryStr = categoryStr.charAt(0)
+                + categoryStr.substring(1).toLowerCase();
+            sb.append(BLOCK_TYPE_PREFIX + categoryStr + ".");
+          }
+
+          if (metricType.compactionAware) {
+            sb.append(isCompaction ? "compaction" : "fs");
+          }
+
+          // A special-case for meta blocks for backwards-compatibility.
+          if (blockCategory == BlockCategory.META) {
+            sb.append(META_BLOCK_CATEGORY_STR);
+          }
+
+          sb.append(metricType);
+
+          int i = getBlockMetricIndex(blockCategory, isCompaction, metricType);
+          blockMetricNames[i] = sb.toString().intern();
+          blockMetricTimeVarying[i] = metricType.timeVarying;
+        }
+      }
+    }
+
+    for (boolean isInBloom : BOOL_VALUES) {
+      bloomMetricNames[isInBloom ? 1 : 0] = metricPrefix
+          + (isInBloom ? "keyMaybeInBloomCnt" : "keyNotInBloomCnt");
+    }
+
+    for (StoreMetricType storeMetric : StoreMetricType.values()) {
+      storeMetricNames[storeMetric.ordinal()] = metricPrefix +
+        storeMetric.toString();
+    }
+  }
+
+  /**
+   * Returns a {@link SchemaMetrics} object for the given table and column
+   * family, instantiating it if necessary.
+   *
+   * @param tableName table name (null is interpreted as "unknown"). This is
+   *          ignored
+   * @param cfName column family name (null is interpreted as "unknown")
+   */
+  public static SchemaMetrics getInstance(String tableName, String cfName) {
+    if (tableName == null) {
+      tableName = UNKNOWN;
+    }
+
+    if (!tableName.equals(TOTAL_KEY)) {
+      // We are provided with a non-trivial table name (including "unknown").
+      // We need to know whether table name should be included into metrics.
+      if (useTableNameGlobally == null) {
+        throw new IllegalStateException("The value of the "
+            + SHOW_TABLE_NAME_CONF_KEY + " conf option has not been specified "
+            + "in SchemaMetrics");
+      }
+      final boolean useTableName = useTableNameGlobally;
+      if (!useTableName) {
+        // Don't include table name in metric keys.
+        tableName = TOTAL_KEY;
+      }
+    }
+
+    if (cfName == null) {
+      cfName = UNKNOWN;
+    }
+
+    final String instanceKey = tableName + "\t" + cfName;
+    SchemaMetrics schemaMetrics = cfToMetrics.get(instanceKey);
+    if (schemaMetrics != null) {
+      return schemaMetrics;
+    }
+
+    schemaMetrics = new SchemaMetrics(tableName, cfName);
+    SchemaMetrics existingMetrics = cfToMetrics.putIfAbsent(instanceKey,
+        schemaMetrics);
+    return existingMetrics != null ? existingMetrics : schemaMetrics;
+  }
+
+  private static final int getBlockMetricIndex(BlockCategory blockCategory,
+      boolean isCompaction, BlockMetricType metricType) {
+    int i = 0;
+    i = i * NUM_BLOCK_CATEGORIES + blockCategory.ordinal();
+    i = i * BOOL_VALUES.length + (isCompaction ? 1 : 0);
+    i = i * NUM_METRIC_TYPES + metricType.ordinal();
+    return i;
+  }
+
+  public String getBlockMetricName(BlockCategory blockCategory,
+      boolean isCompaction, BlockMetricType metricType) {
+    if (isCompaction && !metricType.compactionAware) {
+      throw new IllegalArgumentException("isCompaction cannot be true for "
+          + metricType);
+    }
+    return blockMetricNames[getBlockMetricIndex(blockCategory, isCompaction,
+        metricType)];
+  }
+
+  public String getBloomMetricName(boolean isInBloom) {
+    return bloomMetricNames[isInBloom ? 1 : 0];
+  }
+
+  /**
+   * Increments the given metric, both per-CF and aggregate, for both the given
+   * category and all categories in aggregate (four counters total).
+   */
+  private void incrNumericMetric(BlockCategory blockCategory,
+      boolean isCompaction, BlockMetricType metricType) {
+    if (blockCategory == null) {
+      blockCategory = BlockCategory.UNKNOWN;  // So that we see this in stats.
+    }
+    HRegion.incrNumericMetric(getBlockMetricName(blockCategory,
+        isCompaction, metricType), 1);
+
+    if (blockCategory != BlockCategory.ALL_CATEGORIES) {
+      incrNumericMetric(BlockCategory.ALL_CATEGORIES, isCompaction,
+          metricType);
+    }
+  }
+
+  private void addToReadTime(BlockCategory blockCategory,
+      boolean isCompaction, long timeMs) {
+    HRegion.incrTimeVaryingMetric(getBlockMetricName(blockCategory,
+        isCompaction, BlockMetricType.READ_TIME), timeMs);
+
+    // Also update the read time aggregated across all block categories
+    if (blockCategory != BlockCategory.ALL_CATEGORIES) {
+      addToReadTime(BlockCategory.ALL_CATEGORIES, isCompaction, timeMs);
+    }
+  }
+
+  /**
+   * Update the store metric to a certain value.
+   * @param storeMetricType the store metric to update
+   * @param value the value to update the metric to
+   */
+  public void updateStoreMetric(StoreMetricType storeMetricType, long value) {
+    HRegion.setNumericMetric(storeMetricNames[storeMetricType.ordinal()],
+        value);
+  }
+
+  /**
+   * Update a metric that does not get reset on every poll.
+   * @param storeMetricType the store metric to update
+   * @param value the value to update the metric to
+   */
+  public void updatePersistentStoreMetric(StoreMetricType storeMetricType,
+      long value) {
+    HRegion.incrNumericPersistentMetric(
+        storeMetricNames[storeMetricType.ordinal()], value);
+  }
+
+  /**
+   * Updates the number of hits and the total number of block reads on a block
+   * cache hit.
+   */
+  public void updateOnCacheHit(BlockCategory blockCategory,
+      boolean isCompaction) {
+    blockCategory.expectSpecific();
+    incrNumericMetric(blockCategory, isCompaction, BlockMetricType.CACHE_HIT);
+    incrNumericMetric(blockCategory, isCompaction, BlockMetricType.READ_COUNT);
+    if (this != ALL_SCHEMA_METRICS) {
+      ALL_SCHEMA_METRICS.updateOnCacheHit(blockCategory, isCompaction);
+    }
+  }
+
+  /**
+   * Updates read time, the number of misses, and the total number of block
+   * reads on a block cache miss.
+   */
+  public void updateOnCacheMiss(BlockCategory blockCategory,
+      boolean isCompaction, long timeMs) {
+    blockCategory.expectSpecific();
+    addToReadTime(blockCategory, isCompaction, timeMs);
+    incrNumericMetric(blockCategory, isCompaction, BlockMetricType.CACHE_MISS);
+    incrNumericMetric(blockCategory, isCompaction, BlockMetricType.READ_COUNT);
+    if (this != ALL_SCHEMA_METRICS) {
+      ALL_SCHEMA_METRICS.updateOnCacheMiss(blockCategory, isCompaction,
+          timeMs);
+    }
+  }
+
+  /**
+   * Adds the given delta to the cache size for the given block category and
+   * the aggregate metric for all block categories. Updates both the per-CF
+   * counter and the counter for all CFs (four metrics total). The cache size
+   * metric is "persistent", i.e. it does not get reset when metrics are
+   * collected.
+   */
+  public void addToCacheSize(BlockCategory category, long cacheSizeDelta) {
+    if (category == null) {
+      category = BlockCategory.ALL_CATEGORIES;
+    }
+    HRegion.incrNumericPersistentMetric(getBlockMetricName(category, false,
+        BlockMetricType.CACHE_SIZE), cacheSizeDelta);
+
+    if (category != BlockCategory.ALL_CATEGORIES) {
+      addToCacheSize(BlockCategory.ALL_CATEGORIES, cacheSizeDelta);
+    }
+  }
+
+  public void updateOnCachePutOrEvict(BlockCategory blockCategory,
+      long cacheSizeDelta, boolean isEviction) {
+    addToCacheSize(blockCategory, cacheSizeDelta);
+    incrNumericMetric(blockCategory, false,
+        isEviction ? BlockMetricType.EVICTED : BlockMetricType.CACHED);
+    if (this != ALL_SCHEMA_METRICS) {
+      ALL_SCHEMA_METRICS.updateOnCachePutOrEvict(blockCategory, cacheSizeDelta,
+          isEviction);
+    }
+  }
+
+  /**
+   * Increments both the per-CF and the aggregate counter of bloom
+   * positives/negatives as specified by the argument.
+   */
+  public void updateBloomMetrics(boolean isInBloom) {
+    HRegion.incrNumericMetric(getBloomMetricName(isInBloom), 1);
+    if (this != ALL_SCHEMA_METRICS) {
+      ALL_SCHEMA_METRICS.updateBloomMetrics(isInBloom);
+    }
+  }
+
+  /**
+   * Sets the flag whether to use table name in metric names according to the
+   * given configuration. This must be called at least once before
+   * instantiating HFile readers/writers.
+   */
+  public static void configureGlobally(Configuration conf) {
+    final boolean useTableNameNew =
+        conf.getBoolean(SHOW_TABLE_NAME_CONF_KEY, false);
+    setUseTableName(useTableNameNew);
+  }
+
+  /**
+   * Sets the flag of whether to use table name in metric names. This flag
+   * is specified in configuration and is not expected to change at runtime,
+   * so we log an error message when it does change.
+   */
+  private static void setUseTableName(final boolean useTableNameNew) {
+    if (useTableNameGlobally == null) {
+      // This configuration option has not yet been set.
+      useTableNameGlobally = useTableNameNew;
+    } else if (useTableNameGlobally != useTableNameNew
+        && !loggedConfInconsistency) {
+      // The configuration is inconsistent and we have not reported it
+      // previously. Once we report it, just keep ignoring the new setting.
+      LOG.error("Inconsistent configuration. Previous configuration "
+          + "for using table name in metrics: " + useTableNameGlobally + ", "
+          + "new configuration: " + useTableNameNew);
+      loggedConfInconsistency = true;
+    }
+  }
+
+  // Methods used in testing
+
+  private static final String WORD_AND_DOT_RE_STR = "[^.]+\\.";
+  private static final String TABLE_NAME_RE_STR =
+      "\\b" + TABLE_PREFIX.replace(".", "\\.") + WORD_AND_DOT_RE_STR;
+  private static final String CF_NAME_RE_STR =
+      "\\b" + CF_PREFIX.replace(".", "\\.") + WORD_AND_DOT_RE_STR;
+  private static final Pattern CF_NAME_RE = Pattern.compile(
+      CF_NAME_RE_STR);
+  private static final Pattern TABLE_AND_CF_NAME_RE = Pattern.compile(
+      TABLE_NAME_RE_STR + CF_NAME_RE_STR);
+  private static final Pattern BLOCK_CATEGORY_RE = Pattern.compile(
+      "\\b" + BLOCK_TYPE_PREFIX.replace(".", "\\.") + "[^.]+\\." +
+      // Also remove the special-case block type marker for meta blocks
+      "|" + META_BLOCK_CATEGORY_STR + "(?=" +
+      BlockMetricType.BLOCK_METRIC_TYPE_RE + ")");
+
+  /**
+   * A suffix for the "number of operations" part of "time-varying metrics". We
+   * only use this for metric verification in unit testing. Time-varying
+   * metrics are handled by a different code path in production.
+   */
+  private static String NUM_OPS_SUFFIX = "numops";
+
+  /**
+   * A custom suffix that we use for verifying the second component of
+   * a "time-varying metric".
+   */
+  private static String TOTAL_SUFFIX = "_total";
+  private static final Pattern TIME_VARYING_SUFFIX_RE = Pattern.compile(
+      "(" + NUM_OPS_SUFFIX + "|" + TOTAL_SUFFIX + ")$");
+
+  void printMetricNames() {
+    for (BlockCategory blockCategory : BlockCategory.values()) {
+      for (boolean isCompaction : BOOL_VALUES) {
+        for (BlockMetricType metricType : BlockMetricType.values()) {
+          int i = getBlockMetricIndex(blockCategory, isCompaction, metricType);
+          System.err.println("blockCategory=" + blockCategory + ", "
+              + "metricType=" + metricType + ", isCompaction=" + isCompaction
+              + ", metricName=" + blockMetricNames[i]);
+        }
+      }
+    }
+  }
+
+  private Collection<String> getAllMetricNames() {
+    List<String> allMetricNames = new ArrayList<String>();
+    for (int i = 0; i < blockMetricNames.length; ++i) {
+      final String blockMetricName = blockMetricNames[i];
+      final boolean timeVarying = blockMetricTimeVarying[i];
+      if (blockMetricName != null) {
+        if (timeVarying) {
+          allMetricNames.add(blockMetricName + NUM_OPS_SUFFIX);
+          allMetricNames.add(blockMetricName + TOTAL_SUFFIX);
+        } else {
+          allMetricNames.add(blockMetricName);
+        }
+      }
+    }
+    allMetricNames.addAll(Arrays.asList(bloomMetricNames));
+    return allMetricNames;
+  }
+
+  private static final boolean isTimeVaryingKey(String metricKey) {
+    return metricKey.endsWith(NUM_OPS_SUFFIX)
+        || metricKey.endsWith(TOTAL_SUFFIX);
+  }
+
+  private static final String stripTimeVaryingSuffix(String metricKey) {
+    return TIME_VARYING_SUFFIX_RE.matcher(metricKey).replaceAll("");
+  }
+
+  public static Map<String, Long> getMetricsSnapshot() {
+    Map<String, Long> metricsSnapshot = new TreeMap<String, Long>();
+    for (SchemaMetrics cfm : cfToMetrics.values()) {
+      for (String metricName : cfm.getAllMetricNames()) {
+        long metricValue;
+        if (isTimeVaryingKey(metricName)) {
+          Pair<Long, Integer> totalAndCount =
+              HRegion.getTimeVaryingMetric(stripTimeVaryingSuffix(metricName));
+          metricValue = metricName.endsWith(TOTAL_SUFFIX) ?
+              totalAndCount.getFirst() : totalAndCount.getSecond();
+        } else {
+          metricValue = HRegion.getNumericMetric(metricName);
+        }
+
+        metricsSnapshot.put(metricName, metricValue);
+      }
+    }
+    return metricsSnapshot;
+  }
+
+  private static long getLong(Map<String, Long> m, String k) {
+    Long l = m.get(k);
+    return l != null ? l : 0;
+  }
+
+  private static void putLong(Map<String, Long> m, String k, long v) {
+    if (v != 0) {
+      m.put(k, v);
+    } else {
+      m.remove(k);
+    }
+  }
+  private static Map<String, Long> diffMetrics(Map<String, Long> a,
+      Map<String, Long> b) {
+    Set<String> allKeys = new TreeSet<String>(a.keySet());
+    allKeys.addAll(b.keySet());
+    Map<String, Long> diff = new TreeMap<String, Long>();
+    for (String k : allKeys) {
+      long aVal = getLong(a, k);
+      long bVal = getLong(b, k);
+      if (aVal != bVal) {
+        diff.put(k, bVal - aVal);
+      }
+    }
+    return diff;
+  }
+
+  public static void validateMetricChanges(Map<String, Long> oldMetrics) {
+    final Map<String, Long> newMetrics = getMetricsSnapshot();
+    final Map<String, Long> allCfDeltas = new TreeMap<String, Long>();
+    final Map<String, Long> allBlockCategoryDeltas =
+        new TreeMap<String, Long>();
+    final Map<String, Long> deltas = diffMetrics(oldMetrics, newMetrics);
+    final Pattern cfTableMetricRE =
+        useTableNameGlobally ? TABLE_AND_CF_NAME_RE : CF_NAME_RE;
+    final Set<String> allKeys = new TreeSet<String>(oldMetrics.keySet());
+    allKeys.addAll(newMetrics.keySet());
+
+    for (SchemaMetrics cfm : cfToMetrics.values()) {
+      for (String metricName : cfm.getAllMetricNames()) {
+        if (metricName.startsWith(CF_PREFIX + CF_PREFIX)) {
+          throw new AssertionError("Column family prefix used twice: " +
+              metricName);
+        }
+
+        final long oldValue = getLong(oldMetrics, metricName);
+        final long newValue = getLong(newMetrics, metricName);
+        final long delta = newValue - oldValue;
+        if (oldValue != newValue) {
+          // Debug output for the unit test
+          System.err.println("Metric=" + metricName + ", delta=" + delta);
+        }
+
+        if (cfm != ALL_SCHEMA_METRICS) {
+          // Re-calculate values of metrics with no column family (or CF/table)
+          // specified based on all metrics with CF (or CF/table) specified.
+          final String aggregateMetricName =
+              cfTableMetricRE.matcher(metricName).replaceAll("");
+          putLong(allCfDeltas, aggregateMetricName,
+              getLong(allCfDeltas, aggregateMetricName) + delta);
+        }
+
+        Matcher matcher = BLOCK_CATEGORY_RE.matcher(metricName);
+        if (matcher.find()) {
+           // Only process per-block-category metrics
+          String metricNoBlockCategory = matcher.replaceAll("");
+          putLong(allBlockCategoryDeltas, metricNoBlockCategory,
+              getLong(allBlockCategoryDeltas, metricNoBlockCategory) + delta);
+        }
+      }
+    }
+
+    StringBuilder errors = new StringBuilder();
+    for (String key : ALL_SCHEMA_METRICS.getAllMetricNames()) {
+      long actual = getLong(deltas, key);
+      long expected = getLong(allCfDeltas, key);
+      if (actual != expected) {
+        if (errors.length() > 0)
+          errors.append("\n");
+        errors.append("The all-CF metric " + key + " changed by "
+            + actual + " but the aggregation of per-column-family metrics "
+            + "yields " + expected);
+      }
+    }
+
+    // Verify metrics computed for all block types based on the aggregation
+    // of per-block-type metrics.
+    for (String key : allKeys) {
+      if (BLOCK_CATEGORY_RE.matcher(key).find() ||
+          key.contains(ALL_SCHEMA_METRICS.getBloomMetricName(false)) ||
+          key.contains(ALL_SCHEMA_METRICS.getBloomMetricName(true))){
+        // Skip per-block-category metrics. Also skip bloom filters, because
+        // they are not aggregated per block type.
+        continue;
+      }
+      long actual = getLong(deltas, key);
+      long expected = getLong(allBlockCategoryDeltas, key);
+      if (actual != expected) {
+        if (errors.length() > 0)
+          errors.append("\n");
+        errors.append("The all-block-category metric " + key
+            + " changed by " + actual + " but the aggregation of "
+            + "per-block-category metrics yields " + expected);
+      }
+    }
+
+    if (errors.length() > 0) {
+      throw new AssertionError(errors.toString());
+    }
+  }
+
+  /**
+   * Creates an instance pretending both the table and column family are
+   * unknown. Used in unit tests.
+   */
+  public static SchemaMetrics getUnknownInstanceForTest() {
+    return getInstance(UNKNOWN, UNKNOWN);
+  }
+
+  /**
+   * Set the flag to use or not use table name in metric names. Used in unit
+   * tests, so the flag can be set arbitrarily.
+   */
+  public static void setUseTableNameInTest(final boolean useTableNameNew) {
+    useTableNameGlobally = useTableNameNew;
+  }
+
+}

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCachedBlockQueue.java Tue Oct 11 19:13:09 2011
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.io.hfile
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 
 import junit.framework.TestCase;
 
@@ -129,11 +130,22 @@ public class TestCachedBlockQueue extend
   {
     public CachedBlock(final long heapSize, String name, long accessTime) {
       super(name,
-          new HeapSize(){
+          new Cacheable() {
             @Override
             public long heapSize() {
               return ((int)(heapSize - CachedBlock.PER_BLOCK_OVERHEAD));
-            }},
+            }
+
+            @Override
+            public BlockType getBlockType() {
+              return BlockType.DATA;
+            }
+
+            @Override
+            public SchemaMetrics getSchemaMetrics() {
+              return SchemaMetrics.ALL_SCHEMA_METRICS;
+            }
+          },
           accessTime,false);
     }
   }

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java Tue Oct 11 19:13:09 2011
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Writable;
 
@@ -73,14 +74,14 @@ public class TestHFile extends HBaseTest
 
   @Override
   public void setUp() throws Exception {
-    startingMetrics = ColumnFamilyMetrics.getMetricsSnapshot();
+    startingMetrics = SchemaMetrics.getMetricsSnapshot();
     super.setUp();
   }
 
   @Override
   public void tearDown() throws Exception {
     super.tearDown();
-    ColumnFamilyMetrics.validateMetricChanges(startingMetrics);
+    SchemaMetrics.validateMetricChanges(startingMetrics);
   }
 
   // write some records into the tfile

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java?rev=1182034&r1=1182033&r2=1182034&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java Tue Oct 11 19:13:09 2011
@@ -44,6 +44,7 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.compress.Compressor;
@@ -471,13 +472,16 @@ public class TestHFileBlock {
       HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
           true, -1);
       assertEquals(80, HFileBlock.BYTE_BUFFER_HEAP_SIZE);
-      long expected = ClassSize.align(ClassSize.estimateBase(HFileBlock.class,
-          true))
-          + ClassSize.align(ClassSize.estimateBase(buf.getClass(), true)
-              + HFileBlock.HEADER_SIZE + size)
-          + ClassSize.align(ClassSize.estimateBase(String.class, true) + 2
-              * block.getColumnFamilyName().length());
-      assertEquals(expected, block.heapSize());
+      long byteBufferExpectedSize =
+          ClassSize.align(ClassSize.estimateBase(buf.getClass(), true)
+              + HFileBlock.HEADER_SIZE + size);
+      long hfileBlockExpectedSize =
+          ClassSize.align(ClassSize.estimateBase(HFileBlock.class, true));
+      long expected = hfileBlockExpectedSize + byteBufferExpectedSize;
+      assertEquals("Block data size: " + size + ", byte buffer expected " +
+          "size: " + byteBufferExpectedSize + ", HFileBlock class expected " +
+          "size: " + hfileBlockExpectedSize + ";", expected,
+          block.heapSize());
     }
   }