You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jg...@apache.org on 2011/10/12 06:15:31 UTC

svn commit: r1182194 [2/2] - in /hbase/trunk: ./ src/main/java/org/apache/hadoop/hbase/io/ src/main/java/org/apache/hadoop/hbase/io/hfile/ src/main/java/org/apache/hadoop/hbase/io/hfile/slab/ src/main/java/org/apache/hadoop/hbase/mapreduce/ src/main/ja...

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java Wed Oct 12 04:15:30 2011
@@ -27,6 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
@@ -75,10 +76,6 @@ public final class BloomFilterFactory {
   public static final String IO_STOREFILE_BLOOM_BLOCK_SIZE =
       "io.storefile.bloom.block.size";
 
-  /** Whether to cache compound Bloom filter blocks on write */
-  public static final String IO_STOREFILE_BLOOM_CACHE_ON_WRITE =
-      "io.storefile.bloom.cacheonwrite";
-
   /** Maximum number of times a Bloom filter can be "folded" if oversized */
   private static final int MAX_ALLOWED_FOLD_FACTOR = 7;
 
@@ -140,7 +137,8 @@ public final class BloomFilterFactory {
    *         or when failed to create one.
    */
   public static BloomFilterWriter createBloomAtWrite(Configuration conf,
-      BloomType bloomType, int maxKeys, HFile.Writer writer) {
+      CacheConfig cacheConf, BloomType bloomType, int maxKeys,
+      HFile.Writer writer) {
     if (!isBloomEnabled(conf)) {
       LOG.info("Bloom filters are disabled by configuration for "
           + writer.getPath()
@@ -169,7 +167,7 @@ public final class BloomFilterFactory {
       // In case of compound Bloom filters we ignore the maxKeys hint.
       CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(
           getBloomBlockSize(conf), err, Hash.getHashType(conf), maxFold,
-          cacheChunksOnWrite(conf), bloomType == BloomType.ROWCOL
+          cacheConf.shouldCacheBloomsOnWrite(), bloomType == BloomType.ROWCOL
               ? KeyValue.KEY_COMPARATOR : Bytes.BYTES_RAWCOMPARATOR);
       writer.addInlineBlockWriter(bloomWriter);
       return bloomWriter;
@@ -200,10 +198,4 @@ public final class BloomFilterFactory {
   public static int getBloomBlockSize(Configuration conf) {
     return conf.getInt(IO_STOREFILE_BLOOM_BLOCK_SIZE, 128 * 1024);
   }
-
-  /** @return whether to cache compound Bloom filter chunks on write */
-  public static boolean cacheChunksOnWrite(Configuration conf) {
-    return conf.getBoolean(IO_STOREFILE_BLOOM_CACHE_ON_WRITE, false);
-  }
-
 };

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java Wed Oct 12 04:15:30 2011
@@ -19,20 +19,19 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import java.io.IOException;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.io.compress.Compressor;
 
-import java.io.IOException;
-import java.net.URI;
-
 /**
  * Compression validation test.  Checks compression is working.  Be sure to run
  * on every node in your cluster.
@@ -106,13 +105,14 @@ public class CompressionTest {
   public static void doSmokeTest(FileSystem fs, Path path, String codec)
   throws Exception {
     Configuration conf = HBaseConfiguration.create();
-    HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(
+    HFile.Writer writer =
+      HFile.getWriterFactory(conf).createWriter(
       fs, path, HFile.DEFAULT_BLOCKSIZE, codec, null);
     writer.append(Bytes.toBytes("testkey"), Bytes.toBytes("testval"));
     writer.appendFileInfo(Bytes.toBytes("infokey"), Bytes.toBytes("infoval"));
     writer.close();
 
-    HFile.Reader reader = HFile.createReader(fs, path, null, false, false);
+    HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf));
     reader.loadFileInfo();
     byte[] key = reader.getFirstKey();
     boolean rc = Bytes.toString(key).equals("testkey");

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java Wed Oct 12 04:15:30 2011
@@ -31,9 +31,10 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+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.Compression;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -188,9 +189,9 @@ public class HFilePerformanceEvaluation 
 
     @Override
     void setUp() throws Exception {
-      writer = HFile.getWriterFactory(conf).createWriter(this.fs, this.mf,
-          RFILE_BLOCKSIZE,
-          (Compression.Algorithm) null, null);
+      writer =
+        HFile.getWriterFactory(conf).createWriter(this.fs,
+            this.mf, RFILE_BLOCKSIZE, (Compression.Algorithm) null, null);
     }
 
     @Override
@@ -226,7 +227,7 @@ public class HFilePerformanceEvaluation 
 
     @Override
     void setUp() throws Exception {
-      reader = HFile.createReader(this.fs, this.mf, null, false, false);
+      reader = HFile.createReader(this.fs, this.mf, new CacheConfig(this.conf));
       this.reader.loadFileInfo();
     }
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java Wed Oct 12 04:15:30 2011
@@ -20,22 +20,23 @@
 
 package org.apache.hadoop.hbase.io;
 
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertTrue;
-
 
 public class TestHalfStoreFileReader {
 
@@ -63,8 +64,10 @@ public class TestHalfStoreFileReader {
 
     Configuration conf = test_util.getConfiguration();
     FileSystem fs = FileSystem.get(conf);
+    CacheConfig cacheConf = new CacheConfig(conf);
 
-    HFile.Writer w = HFile.getWriterFactory(conf).createWriter(fs, p, 1024,
+    HFile.Writer w =
+      HFile.getWriterFactory(conf, cacheConf).createWriter(fs, p, 1024,
         "none", KeyValue.KEY_COMPARATOR);
 
     // write some things.
@@ -74,7 +77,7 @@ public class TestHalfStoreFileReader {
     }
     w.close();
 
-    HFile.Reader r = HFile.createReader(fs, p, null, false, false);
+    HFile.Reader r = HFile.createReader(fs, p, cacheConf);
     r.loadFileInfo();
     byte [] midkey = r.midkey();
     KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);
@@ -83,16 +86,17 @@ public class TestHalfStoreFileReader {
     //System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey));
 
     Reference bottom = new Reference(midkey, Reference.Range.bottom);
-    doTestOfScanAndReseek(p, fs, bottom);
+    doTestOfScanAndReseek(p, fs, bottom, cacheConf);
 
     Reference top = new Reference(midkey, Reference.Range.top);
-    doTestOfScanAndReseek(p, fs, top);
+    doTestOfScanAndReseek(p, fs, top, cacheConf);
   }
 
-  private void doTestOfScanAndReseek(Path p, FileSystem fs, Reference bottom)
+  private void doTestOfScanAndReseek(Path p, FileSystem fs, Reference bottom,
+      CacheConfig cacheConf)
       throws IOException {
     final HalfStoreFileReader halfreader =
-        new HalfStoreFileReader(fs, p, null, bottom);
+        new HalfStoreFileReader(fs, p, cacheConf, bottom);
     halfreader.loadFileInfo();
     final HFileScanner scanner = halfreader.getScanner(false, false);
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java Wed Oct 12 04:15:30 2011
@@ -67,7 +67,10 @@ public class RandomSeek {
     Path path = new Path("/Users/ryan/rfile.big.txt");
     long start = System.currentTimeMillis();
     SimpleBlockCache cache = new SimpleBlockCache();
-    Reader reader = HFile.createReader(lfs, path, cache, false, false);
+    CacheConfig cacheConf = new CacheConfig(cache, true, false, false, false,
+        false, false, false);
+
+    Reader reader = HFile.createReader(lfs, path, cacheConf);
     reader.loadFileInfo();
     System.out.println(reader.getTrailer());
     long end = System.currentTimeMillis();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java Wed Oct 12 04:15:30 2011
@@ -64,6 +64,7 @@ public class TestCacheOnWrite {
   private static final HBaseTestingUtility TEST_UTIL =
     new HBaseTestingUtility();
   private Configuration conf;
+  private CacheConfig cacheConf;
   private FileSystem fs;
   private Random rand = new Random(12983177L);
   private Path storeFilePath;
@@ -82,11 +83,11 @@ public class TestCacheOnWrite {
       KeyValue.Type.values().length - 2;
 
   private static enum CacheOnWriteType {
-    DATA_BLOCKS(BlockType.DATA, HFile.CACHE_BLOCKS_ON_WRITE_KEY),
+    DATA_BLOCKS(BlockType.DATA, CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY),
     BLOOM_BLOCKS(BlockType.BLOOM_CHUNK,
-        BloomFilterFactory.IO_STOREFILE_BLOOM_CACHE_ON_WRITE),
+        CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY),
     INDEX_BLOCKS(BlockType.LEAF_INDEX,
-        HFileBlockIndex.CACHE_INDEX_BLOCKS_ON_WRITE_KEY);
+        CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY);
 
     private final String confKey;
     private final BlockType inlineBlockType;
@@ -114,6 +115,7 @@ public class TestCacheOnWrite {
     this.cowType = cowType;
     this.compress = compress;
     testName = "[cacheOnWrite=" + cowType + ", compress=" + compress + "]";
+    System.out.println(testName);
   }
 
   @Parameters
@@ -134,9 +136,17 @@ public class TestCacheOnWrite {
     conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, INDEX_BLOCK_SIZE);
     conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
         BLOOM_BLOCK_SIZE);
+    conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY,
+        cowType.shouldBeCached(BlockType.DATA));
+    conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY,
+        cowType.shouldBeCached(BlockType.LEAF_INDEX));
+    conf.setBoolean(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY,
+        cowType.shouldBeCached(BlockType.BLOOM_CHUNK));
     cowType.modifyConf(conf);
     fs = FileSystem.get(conf);
-    blockCache = StoreFile.getBlockCache(conf);
+    cacheConf = new CacheConfig(conf);
+    blockCache = cacheConf.getBlockCache();
+    System.out.println("setUp()");
   }
 
   @After
@@ -152,7 +162,7 @@ public class TestCacheOnWrite {
 
   private void readStoreFile() throws IOException {
     HFileReaderV2 reader = (HFileReaderV2) HFile.createReader(fs,
-        storeFilePath, null, false, false);
+        storeFilePath, cacheConf);
     LOG.info("HFile information: " + reader);
     HFileScanner scanner = reader.getScanner(false, false);
     assertTrue(testName, scanner.seekTo());
@@ -167,8 +177,8 @@ public class TestCacheOnWrite {
       if (prevBlock != null) {
          onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
       }
-      // Flags: cache the block, use pread, this is not a compaction.
-      HFileBlock block = reader.readBlock(offset, onDiskSize, true, true,
+      // Flags: don't cache the block, use pread, this is not a compaction.
+      HFileBlock block = reader.readBlock(offset, onDiskSize, false, true,
           false);
       String blockCacheKey = HFile.getBlockCacheKey(reader.getName(), offset);
       boolean isCached = blockCache.getBlock(blockCacheKey, true) != null;
@@ -210,7 +220,7 @@ public class TestCacheOnWrite {
         "test_cache_on_write");
     StoreFile.Writer sfw = StoreFile.createWriter(fs, storeFileParentDir,
         DATA_BLOCK_SIZE, compress, KeyValue.COMPARATOR, conf,
-        StoreFile.BloomType.ROWCOL, NUM_KV);
+        cacheConf, StoreFile.BloomType.ROWCOL, NUM_KV);
 
     final int rowLen = 32;
     for (int i = 0; i < NUM_KV; ++i) {

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java Wed Oct 12 04:15:30 2011
@@ -53,6 +53,7 @@ public class TestHFile extends HBaseTest
     HBaseTestingUtility.getTestDir("TestHFile").toString();
   private final int minBlockSize = 512;
   private static String localFormatter = "%010d";
+  private static CacheConfig cacheConf = null;
 
   /**
    * Test empty HFile.
@@ -60,10 +61,11 @@ public class TestHFile extends HBaseTest
    * @throws IOException
    */
   public void testEmptyHFile() throws IOException {
+    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path f = new Path(ROOT_DIR, getName());
-    Writer w = HFile.getWriterFactory(conf).createWriter(this.fs, f);
+    Writer w = HFile.getWriterFactory(conf, cacheConf).createWriter(this.fs, f);
     w.close();
-    Reader r = HFile.createReader(fs, f, null, false, false);
+    Reader r = HFile.createReader(fs, f, cacheConf);
     r.loadFileInfo();
     assertNull(r.getFirstKey());
     assertNull(r.getLastKey());
@@ -130,16 +132,18 @@ public class TestHFile extends HBaseTest
    * test none codecs
    */
   void basicWithSomeCodec(String codec) throws IOException {
-    Path ncTFile = new Path(ROOT_DIR, "basic.hfile");
+    if (cacheConf == null) cacheConf = new CacheConfig(conf);
+    Path ncTFile = new Path(ROOT_DIR, "basic.hfile." + codec.toString());
     FSDataOutputStream fout = createFSOutput(ncTFile);
-    Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+    Writer writer = HFile.getWriterFactory(conf, cacheConf).createWriter(fout,
         minBlockSize, Compression.getCompressionAlgorithmByName(codec), null);
     LOG.info(writer);
     writeRecords(writer);
     fout.close();
     FSDataInputStream fin = fs.open(ncTFile);
     Reader reader = HFile.createReader(ncTFile, fs.open(ncTFile),
-      fs.getFileStatus(ncTFile).getLen(), null, false, false);
+      fs.getFileStatus(ncTFile).getLen(), cacheConf);
+    System.out.println(cacheConf.toString());
     // Load up the index.
     reader.loadFileInfo();
     // Get a scanner that caches and that does not use pread.
@@ -205,9 +209,10 @@ public class TestHFile extends HBaseTest
   }
 
   private void metablocks(final String compress) throws Exception {
+    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path mFile = new Path(ROOT_DIR, "meta.hfile");
     FSDataOutputStream fout = createFSOutput(mFile);
-    Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+    Writer writer = HFile.getWriterFactory(conf, cacheConf).createWriter(fout,
         minBlockSize, Compression.getCompressionAlgorithmByName(compress),
         null);
     someTestingWithMetaBlock(writer);
@@ -215,7 +220,7 @@ public class TestHFile extends HBaseTest
     fout.close();
     FSDataInputStream fin = fs.open(mFile);
     Reader reader = HFile.createReader(mFile, fs.open(mFile),
-        this.fs.getFileStatus(mFile).getLen(), null, false, false);
+        this.fs.getFileStatus(mFile).getLen(), cacheConf);
     reader.loadFileInfo();
     // No data -- this should return false.
     assertFalse(reader.getScanner(false, false).seekTo());
@@ -232,16 +237,17 @@ public class TestHFile extends HBaseTest
   }
 
   public void testNullMetaBlocks() throws Exception {
+    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     for (Compression.Algorithm compressAlgo : 
         HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
       Path mFile = new Path(ROOT_DIR, "nometa_" + compressAlgo + ".hfile");
       FSDataOutputStream fout = createFSOutput(mFile);
-      Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+      Writer writer = HFile.getWriterFactory(conf, cacheConf).createWriter(fout,
           minBlockSize, compressAlgo, null);
       writer.append("foo".getBytes(), "value".getBytes());
       writer.close();
       fout.close();
-      Reader reader = HFile.createReader(fs, mFile, null, false, false);
+      Reader reader = HFile.createReader(fs, mFile, cacheConf);
       reader.loadFileInfo();
       assertNull(reader.getMetaBlock("non-existant", false));
     }
@@ -257,9 +263,10 @@ public class TestHFile extends HBaseTest
   }
 
   public void testComparator() throws IOException {
+    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path mFile = new Path(ROOT_DIR, "meta.tfile");
     FSDataOutputStream fout = createFSOutput(mFile);
-    Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+    Writer writer = HFile.getWriterFactory(conf, cacheConf).createWriter(fout,
       minBlockSize, (Compression.Algorithm) null, new KeyComparator() {
         @Override
         public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2,

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java Wed Oct 12 04:15:30 2011
@@ -460,7 +460,8 @@ public class TestHFileBlockIndex {
   public void testHFileWriterAndReader() throws IOException {
     Path hfilePath = new Path(HBaseTestingUtility.getTestDir(),
         "hfile_for_block_index");
-    BlockCache blockCache = StoreFile.getBlockCache(conf);
+    CacheConfig cacheConf = new CacheConfig(conf);
+    BlockCache blockCache = cacheConf.getBlockCache();
 
     for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; ++testI) {
       int indexBlockSize = INDEX_CHUNK_SIZES[testI];
@@ -478,7 +479,8 @@ public class TestHFileBlockIndex {
 
       // Write the HFile
       {
-        HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(fs,
+        HFile.Writer writer =
+          HFile.getWriterFactory(conf, cacheConf).createWriter(fs,
             hfilePath, SMALL_BLOCK_SIZE, compr, KeyValue.KEY_COMPARATOR);
         Random rand = new Random(19231737);
 
@@ -505,8 +507,7 @@ public class TestHFileBlockIndex {
       }
 
       // Read the HFile
-      HFile.Reader reader = HFile.createReader(fs, hfilePath, blockCache,
-          false, true);
+      HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf);
       assertEquals(expectedNumLevels,
           reader.getTrailer().getNumDataIndexLevels());
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java Wed Oct 12 04:15:30 2011
@@ -157,7 +157,8 @@ public class TestHFilePerformance extend
 
     if ("HFile".equals(fileType)){
         System.out.println("HFile write method: ");
-        HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+        HFile.Writer writer =
+          HFile.getWriterFactory(conf).createWriter(fout,
              minBlockSize, codecName, null);
 
         // Writing value in one shot.
@@ -237,7 +238,7 @@ public class TestHFilePerformance extend
 
     if ("HFile".equals(fileType)){
         HFile.Reader reader = HFile.createReader(path, fs.open(path),
-          fs.getFileStatus(path).getLen(), null, false, false);
+          fs.getFileStatus(path).getLen(), new CacheConfig(conf));
         reader.loadFileInfo();
         switch (method) {
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderV1.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderV1.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderV1.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileReaderV1.java Wed Oct 12 04:15:30 2011
@@ -56,7 +56,7 @@ public class TestHFileReaderV1 {
         "8e8ab58dcf39412da19833fcd8f687ac");
     Path existingHFilePath = new Path(url.getPath());
     HFile.Reader reader =
-      HFile.createReader(fs, existingHFilePath, null, false, false);
+      HFile.createReader(fs, existingHFilePath, new CacheConfig(conf));
     reader.loadFileInfo();
     FixedFileTrailer trailer = reader.getTrailer();
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java Wed Oct 12 04:15:30 2011
@@ -118,7 +118,8 @@ public class TestHFileSeek extends TestC
     long totalBytes = 0;
     FSDataOutputStream fout = createFSOutput(path, fs);
     try {
-      Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+      Writer writer =
+        HFile.getWriterFactory(conf).createWriter(fout,
           options.minBlockSize, options.compress, null);
       try {
         BytesWritable key = new BytesWritable();
@@ -164,7 +165,7 @@ public class TestHFileSeek extends TestC
     long totalBytes = 0;
     FSDataInputStream fsdis = fs.open(path);
     Reader reader = HFile.createReader(path, fsdis,
-        fs.getFileStatus(path).getLen(), null, false, false);
+        fs.getFileStatus(path).getLen(), new CacheConfig(conf));
     reader.loadFileInfo();
     KeySampler kSampler =
         new KeySampler(rng, reader.getFirstKey(), reader.getLastKey(),

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java Wed Oct 12 04:15:30 2011
@@ -67,8 +67,8 @@ public class TestHFileWriterV2 {
         "testHFileFormatV2");
 
     final Compression.Algorithm COMPRESS_ALGO = Compression.Algorithm.GZ;
-    HFileWriterV2 writer = new HFileWriterV2(conf, fs, hfilePath, 4096,
-        COMPRESS_ALGO, KeyValue.KEY_COMPARATOR);
+    HFileWriterV2 writer = new HFileWriterV2(conf, new CacheConfig(conf), fs,
+        hfilePath, 4096, COMPRESS_ALGO, KeyValue.KEY_COMPARATOR);
 
     long totalKeyLength = 0;
     long totalValueLength = 0;

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java Wed Oct 12 04:15:30 2011
@@ -42,8 +42,10 @@ public class TestReseekTo {
 
     Path ncTFile = new Path(HBaseTestingUtility.getTestDir(), "basic.hfile");
     FSDataOutputStream fout = TEST_UTIL.getTestFileSystem().create(ncTFile);
+    CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
     HFile.Writer writer = HFile.getWriterFactory(
-        TEST_UTIL.getConfiguration()).createWriter(fout, 4000, "none", null);
+        TEST_UTIL.getConfiguration(), cacheConf).createWriter(
+            fout, 4000, "none", null);
     int numberOfKeys = 1000;
 
     String valueString = "Value";
@@ -61,7 +63,7 @@ public class TestReseekTo {
     fout.close();
 
     HFile.Reader reader = HFile.createReader(TEST_UTIL.getTestFileSystem(),
-        ncTFile, null, false, false);
+        ncTFile, cacheConf);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, true);
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java Wed Oct 12 04:15:30 2011
@@ -45,7 +45,8 @@ public class TestSeekTo extends HBaseTes
     Path ncTFile = new Path(this.testDir, "basic.hfile");
     FSDataOutputStream fout = this.fs.create(ncTFile);
     int blocksize = toKV("a").getLength() * 3;
-    HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+    HFile.Writer writer =
+      HFile.getWriterFactory(conf).createWriter(fout,
         blocksize, "none", null);
     // 4 bytes * 3 * 2 for each key/value +
     // 3 for keys, 15 for values = 42 (woot)
@@ -62,7 +63,7 @@ public class TestSeekTo extends HBaseTes
 
   public void testSeekBefore() throws Exception {
     Path p = makeNewFile();
-    HFile.Reader reader = HFile.createReader(fs, p, null, false, false);
+    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf));
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, true);
     assertEquals(false, scanner.seekBefore(toKV("a").getKey()));
@@ -95,7 +96,7 @@ public class TestSeekTo extends HBaseTes
 
   public void testSeekTo() throws Exception {
     Path p = makeNewFile();
-    HFile.Reader reader = HFile.createReader(fs, p, null, false, false);
+    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf));
     reader.loadFileInfo();
     assertEquals(2, reader.getDataBlockIndexReader().getRootBlockCount());
     HFileScanner scanner = reader.getScanner(false, true);
@@ -115,7 +116,7 @@ public class TestSeekTo extends HBaseTes
 
   public void testBlockContainingKey() throws Exception {
     Path p = makeNewFile();
-    HFile.Reader reader = HFile.createReader(fs, p, null, false, false);
+    HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf));
     reader.loadFileInfo();
     HFileBlockIndex.BlockIndexReader blockIndexReader = 
       reader.getDataBlockIndexReader();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java Wed Oct 12 04:15:30 2011
@@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.Re
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.hfile.HFile;
@@ -281,8 +282,8 @@ public class TestHFileOutputFormat  {
       FileStatus[] file = fs.listStatus(sub3[0].getPath());
 
       // open as HFile Reader and pull out TIMERANGE FileInfo.
-      HFile.Reader rd = HFile.createReader(fs, file[0].getPath(), null, true,
-          false);
+      HFile.Reader rd = HFile.createReader(fs, file[0].getPath(),
+          new CacheConfig(conf));
       Map<byte[],byte[]> finfo = rd.loadFileInfo();
       byte[] range = finfo.get("TIMERANGE".getBytes());
       assertNotNull(range);
@@ -608,8 +609,8 @@ public class TestHFileOutputFormat  {
             // verify that the compression on this file matches the configured
             // compression
             Path dataFilePath = fileSystem.listStatus(f.getPath())[0].getPath();
-            Reader reader = HFile.createReader(fileSystem, dataFilePath, null,
-                false, true);
+            Reader reader = HFile.createReader(fileSystem, dataFilePath,
+                new CacheConfig(conf));
             reader.loadFileInfo();
             assertEquals("Incorrect compression used for column family " + familyStr
                          + "(reader: " + reader + ")",

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java Wed Oct 12 04:15:30 2011
@@ -19,10 +19,11 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
@@ -41,8 +43,6 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
-
 /**
  * Test cases for the "load" half of the HFileOutputFormat bulk load
  * functionality. These tests run faster than the full MR cluster
@@ -179,7 +179,7 @@ public class TestLoadIncrementalHFiles {
   private int verifyHFile(Path p) throws IOException {
     Configuration conf = util.getConfiguration();
     HFile.Reader reader = HFile.createReader(
-        p.getFileSystem(conf), p, null, false, false);
+        p.getFileSystem(conf), p, new CacheConfig(conf));
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, false);
     scanner.seekTo();
@@ -203,7 +203,8 @@ public class TestLoadIncrementalHFiles {
       byte[] family, byte[] qualifier,
       byte[] startKey, byte[] endKey, int numRows) throws IOException
   {
-    HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(fs, path,
+    HFile.Writer writer =
+      HFile.getWriterFactory(conf, new CacheConfig(conf)).createWriter(fs, path,
         BLOCKSIZE, COMPRESSION,
         KeyValue.KEY_COMPARATOR);
     long now = System.currentTimeMillis();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java Wed Oct 12 04:15:30 2011
@@ -21,6 +21,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
@@ -31,18 +32,18 @@ import org.apache.hadoop.hbase.HBaseConf
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.CacheStats;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-
 import org.junit.Test;
 
 public class TestBlocksRead extends HBaseTestCase {
@@ -98,7 +99,7 @@ public class TestBlocksRead extends HBas
     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
     Path path = new Path(DIR + callingMethod);
     region = HRegion.createHRegion(info, path, conf, htd);
-    blockCache = StoreFile.getBlockCache(conf);
+    blockCache = new CacheConfig(conf).getBlockCache();
   }
 
   private void putData(byte[] cf, String row, String col, long version)
@@ -178,6 +179,10 @@ public class TestBlocksRead extends HBas
       return blockCache.getStats().getRequestCount();
   }
 
+  private static long getBlkCount() {
+    return blockCache.getBlockCount();
+  }
+
   /**
    * Test # of blocks read for some simple seek cases.
    * @throws Exception
@@ -316,4 +321,49 @@ public class TestBlocksRead extends HBas
     verifyData(kvs[1], "row", "col2", 12);
     verifyData(kvs[2], "row", "col3", 13);
   }
+
+  /**
+   * Test # of blocks read to ensure disabling cache-fill on Scan works.
+   * @throws Exception
+   */
+  @Test
+  public void testBlocksStoredWhenCachingDisabled() throws Exception {
+    byte [] TABLE = Bytes.toBytes("testBlocksReadWhenCachingDisabled");
+    byte [] FAMILY = Bytes.toBytes("cf1");
+    byte [][] FAMILIES = new byte[][] { FAMILY };
+
+    HBaseConfiguration conf = getConf();
+    initHRegion(TABLE, getName(), conf, FAMILIES);
+
+    putData(FAMILY, "row", "col1", 1);
+    putData(FAMILY, "row", "col2", 2);
+    region.flushcache();
+
+    // Execute a scan with caching turned off
+    // Expected blocks stored: 0
+    long blocksStart = getBlkCount();
+    Scan scan = new Scan();
+    scan.setCacheBlocks(false);
+    RegionScanner rs = region.getScanner(scan);
+    List<KeyValue> result = new ArrayList<KeyValue>(2);
+    rs.next(result);
+    assertEquals(2, result.size());
+    rs.close();
+    long blocksEnd = getBlkCount();
+
+    assertEquals(blocksStart, blocksEnd);
+
+    // Execute with caching turned on
+    // Expected blocks stored: 2
+    blocksStart = blocksEnd;
+    scan.setCacheBlocks(true);
+    rs = region.getScanner(scan);
+    result = new ArrayList<KeyValue>(2);
+    rs.next(result);
+    assertEquals(2, result.size());
+    rs.close();
+    blocksEnd = getBlkCount();
+    
+    assertEquals(2, blocksEnd - blocksStart);
+  }
 }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java Wed Oct 12 04:15:30 2011
@@ -36,9 +36,8 @@ import org.apache.hadoop.hbase.HColumnDe
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.regionserver.wal.TestWALReplay;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.collect.Lists;
@@ -100,8 +99,9 @@ public class TestCompactSelection extend
     boolean isRef = false;
 
     MockStoreFile(long length, boolean isRef) throws IOException {
-      super(TEST_UTIL.getTestFileSystem(), TEST_FILE, false,
-            TEST_UTIL.getConfiguration(), BloomType.NONE, false);
+      super(TEST_UTIL.getTestFileSystem(), TEST_FILE,
+            TEST_UTIL.getConfiguration(),
+            new CacheConfig(TEST_UTIL.getConfiguration()), BloomType.NONE);
       this.length = length;
       this.isRef  = isRef;
     }

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java Wed Oct 12 04:15:30 2011
@@ -20,7 +20,10 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -38,6 +41,7 @@ import org.apache.hadoop.hbase.HBaseTest
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2;
 import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
@@ -105,6 +109,7 @@ public class TestCompoundBloomFilter {
   }
 
   private static Configuration conf;
+  private static CacheConfig cacheConf;
   private FileSystem fs;
   private BlockCache blockCache;
 
@@ -123,7 +128,8 @@ public class TestCompoundBloomFilter {
 
     fs = FileSystem.get(conf);
 
-    blockCache = StoreFile.getBlockCache(conf);
+    cacheConf = new CacheConfig(conf);
+    blockCache = cacheConf.getBlockCache();
     assertNotNull(blockCache);
   }
 
@@ -187,7 +193,7 @@ public class TestCompoundBloomFilter {
 
   private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
       Path sfPath) throws IOException {
-    StoreFile sf = new StoreFile(fs, sfPath, true, conf, bt, false);
+    StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt);
     StoreFile.Reader r = sf.createReader();
     final boolean pread = true; // does not really matter
     StoreFileScanner scanner = r.getStoreFileScanner(true, pread);
@@ -283,11 +289,12 @@ public class TestCompoundBloomFilter {
       throws IOException {
     conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
         BLOOM_BLOCK_SIZES[t]);
-    conf.setBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, true);
+    conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
+    cacheConf = new CacheConfig(conf);
 
     StoreFile.Writer w = StoreFile.createWriter(fs,
         HBaseTestingUtility.getTestDir(), BLOCK_SIZES[t], null, null, conf,
-        bt, 0);
+        cacheConf, bt, 0);
 
     assertTrue(w.hasBloom());
     assertTrue(w.getBloomWriter() instanceof CompoundBloomFilterWriter);

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java Wed Oct 12 04:15:30 2011
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.HTableDes
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -70,13 +71,14 @@ public class TestFSErrorsExposed {
         HBaseTestingUtility.getTestDir("internalScannerExposesErrors"),
         "regionname"), "familyname");
     FaultyFileSystem fs = new FaultyFileSystem(util.getTestFileSystem());
+    CacheConfig cacheConf = new CacheConfig(util.getConfiguration());
     StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2*1024,
-        util.getConfiguration());
+        util.getConfiguration(), cacheConf);
     TestStoreFile.writeStoreFile(
         writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
 
-    StoreFile sf = new StoreFile(fs, writer.getPath(), false,
-        util.getConfiguration(), StoreFile.BloomType.NONE, false);
+    StoreFile sf = new StoreFile(fs, writer.getPath(),
+        util.getConfiguration(), cacheConf, StoreFile.BloomType.NONE);
     StoreFile.Reader reader = sf.createReader();
     HFileScanner scanner = reader.getScanner(false, true);
 
@@ -112,13 +114,14 @@ public class TestFSErrorsExposed {
         HBaseTestingUtility.getTestDir("internalScannerExposesErrors"),
         "regionname"), "familyname");
     FaultyFileSystem fs = new FaultyFileSystem(util.getTestFileSystem());
+    CacheConfig cacheConf = new CacheConfig(util.getConfiguration());
     StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2 * 1024,
-        util.getConfiguration());
+        util.getConfiguration(), cacheConf);
     TestStoreFile.writeStoreFile(
         writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
 
-    StoreFile sf = new StoreFile(fs, writer.getPath(), false,
-        util.getConfiguration(), BloomType.NONE, false);
+    StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(),
+        cacheConf, BloomType.NONE);
     List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(
         Collections.singletonList(sf), false, true);
     KeyValueScanner scanner = scanners.get(0);

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java Wed Oct 12 04:15:30 2011
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.security.User;
@@ -209,7 +210,7 @@ public class TestStore extends TestCase 
     Configuration c = HBaseConfiguration.create();
     FileSystem fs = FileSystem.get(c);
     StoreFile.Writer w = StoreFile.createWriter(fs, storedir,
-        StoreFile.DEFAULT_BLOCKSIZE_SMALL, c);
+        StoreFile.DEFAULT_BLOCKSIZE_SMALL, c, new CacheConfig(c));
     w.appendMetadata(seqid + 1, false);
     w.close();
     this.store.close();

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java Wed Oct 12 04:15:30 2011
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.Reference.Range;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CacheStats;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
@@ -58,6 +59,7 @@ import com.google.common.collect.Lists;
 public class TestStoreFile extends HBaseTestCase {
   static final Log LOG = LogFactory.getLog(TestStoreFile.class);
   private MiniDFSCluster cluster;
+  private CacheConfig cacheConf;
 
   @Override
   public void setUp() throws Exception {
@@ -66,6 +68,7 @@ public class TestStoreFile extends HBase
       // Set the hbase.rootdir to be the home directory in mini dfs.
       this.conf.set(HConstants.HBASE_DIR,
         this.cluster.getFileSystem().getHomeDirectory().toString());
+      this.cacheConf = new CacheConfig(conf);
     } catch (IOException e) {
       shutdownDfs(cluster);
     }
@@ -89,10 +92,10 @@ public class TestStoreFile extends HBase
     // Make up a directory hierarchy that has a regiondir and familyname.
     StoreFile.Writer writer = StoreFile.createWriter(this.fs,
       new Path(new Path(this.testDir, "regionname"), "familyname"), 2 * 1024,
-      conf);
+      conf, cacheConf);
     writeStoreFile(writer);
-    checkHalfHFile(new StoreFile(this.fs, writer.getPath(), true, conf,
-        StoreFile.BloomType.NONE, false));
+    checkHalfHFile(new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
+        StoreFile.BloomType.NONE));
   }
 
   private void writeStoreFile(final StoreFile.Writer writer) throws IOException {
@@ -130,10 +133,10 @@ public class TestStoreFile extends HBase
     Path dir = new Path(storedir, "1234567890");
     // Make a store file and write data to it.
     StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024,
-        conf);
+        conf, cacheConf);
     writeStoreFile(writer);
-    StoreFile hsf = new StoreFile(this.fs, writer.getPath(), true, conf,
-        StoreFile.BloomType.NONE, false);
+    StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
+        StoreFile.BloomType.NONE);
     StoreFile.Reader reader = hsf.createReader();
     // Split on a row, not in middle of row.  Midkey returned by reader
     // may be in middle of row.  Create new one with empty column and
@@ -144,8 +147,8 @@ public class TestStoreFile extends HBase
     byte [] finalRow = kv.getRow();
     // Make a reference
     Path refPath = StoreFile.split(fs, dir, hsf, midRow, Range.top);
-    StoreFile refHsf = new StoreFile(this.fs, refPath, true, conf,
-        StoreFile.BloomType.NONE, false);
+    StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf,
+        StoreFile.BloomType.NONE);
     // Now confirm that I can read from the reference and that it only gets
     // keys from top half of the file.
     HFileScanner s = refHsf.createReader().getScanner(false, false);
@@ -181,10 +184,10 @@ public class TestStoreFile extends HBase
     Path bottomPath = StoreFile.split(this.fs, bottomDir,
       f, midRow, Range.bottom);
     // Make readers on top and bottom.
-    StoreFile.Reader top = new StoreFile(this.fs, topPath, true, conf,
-        StoreFile.BloomType.NONE, false).createReader();
-    StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath, true, conf,
-        StoreFile.BloomType.NONE, false).createReader();
+    StoreFile.Reader top = new StoreFile(this.fs, topPath, conf, cacheConf,
+        StoreFile.BloomType.NONE).createReader();
+    StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
+        StoreFile.BloomType.NONE).createReader();
     ByteBuffer previous = null;
     LOG.info("Midkey: " + midKV.toString());
     ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midkey);
@@ -239,10 +242,10 @@ public class TestStoreFile extends HBase
       topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
       bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
         Range.bottom);
-      top = new StoreFile(this.fs, topPath, true, conf,
-          StoreFile.BloomType.NONE, false).createReader();
-      bottom = new StoreFile(this.fs, bottomPath, true, conf,
-          StoreFile.BloomType.NONE, false).createReader();
+      top = new StoreFile(this.fs, topPath, conf, cacheConf,
+          StoreFile.BloomType.NONE).createReader();
+      bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
+          StoreFile.BloomType.NONE).createReader();
       bottomScanner = bottom.getScanner(false, false);
       int count = 0;
       while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
@@ -284,10 +287,10 @@ public class TestStoreFile extends HBase
       topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
       bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
         Range.bottom);
-      top = new StoreFile(this.fs, topPath, true, conf,
-          StoreFile.BloomType.NONE, false).createReader();
-      bottom = new StoreFile(this.fs, bottomPath, true, conf,
-          StoreFile.BloomType.NONE, false).createReader();
+      top = new StoreFile(this.fs, topPath, conf, cacheConf,
+          StoreFile.BloomType.NONE).createReader();
+      bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
+          StoreFile.BloomType.NONE).createReader();
       first = true;
       bottomScanner = bottom.getScanner(false, false);
       while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
@@ -345,7 +348,7 @@ public class TestStoreFile extends HBase
     }
     writer.close();
 
-    StoreFile.Reader reader = new StoreFile.Reader(fs, f, null, false, false);
+    StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
     reader.loadFileInfo();
     reader.loadBloomfilter();
     StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
@@ -386,7 +389,7 @@ public class TestStoreFile extends HBase
     Path f = new Path(ROOT_DIR, getName());
     StoreFile.Writer writer = new StoreFile.Writer(fs, f,
         StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
-        conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
+        conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
 
     bloomWriteRead(writer, fs);
   }
@@ -413,11 +416,11 @@ public class TestStoreFile extends HBase
 
     for (int x : new int[]{0,1}) {
       // write the file
-      Path f = new Path(ROOT_DIR, getName());
+      Path f = new Path(ROOT_DIR, getName() + x);
       StoreFile.Writer writer = new StoreFile.Writer(fs, f,
           StoreFile.DEFAULT_BLOCKSIZE_SMALL,
           HFile.DEFAULT_COMPRESSION_ALGORITHM,
-          conf, KeyValue.COMPARATOR, bt[x], expKeys[x]);
+          conf, cacheConf, KeyValue.COMPARATOR, bt[x], expKeys[x]);
 
       long now = System.currentTimeMillis();
       for (int i = 0; i < rowCount*2; i += 2) { // rows
@@ -434,10 +437,10 @@ public class TestStoreFile extends HBase
       }
       writer.close();
 
-      StoreFile.Reader reader = new StoreFile.Reader(fs, f, null, false, false);
+      StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
       reader.loadFileInfo();
       reader.loadBloomfilter();
-      StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
+      StoreFileScanner scanner = reader.getStoreFileScanner(true, true);
       assertEquals(expKeys[x], reader.bloomFilter.getKeyCount());
 
       // check false positives rate
@@ -487,7 +490,7 @@ public class TestStoreFile extends HBase
     // this should not create a bloom because the max keys is too small
     StoreFile.Writer writer = new StoreFile.Writer(fs, f,
         StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
-        conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
+        conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
     assertFalse(writer.hasBloom());
     writer.close();
     fs.delete(f, true);
@@ -510,7 +513,8 @@ public class TestStoreFile extends HBase
     // because Java can't create a contiguous array > MAX_INT
     writer = new StoreFile.Writer(fs, f,
         StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
-        conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, Integer.MAX_VALUE);
+        conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW,
+        Integer.MAX_VALUE);
     assertFalse(writer.hasBloom());
     writer.close();
     fs.delete(f, true);
@@ -602,7 +606,7 @@ public class TestStoreFile extends HBase
     "familyname");
     Path dir = new Path(storedir, "1234567890");
     StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024,
-        conf);
+        conf, cacheConf);
 
     List<KeyValue> kvList = getKeyValueSet(timestamps,numRows,
         family, qualifier);
@@ -613,8 +617,8 @@ public class TestStoreFile extends HBase
     writer.appendMetadata(0, false);
     writer.close();
 
-    StoreFile hsf = new StoreFile(this.fs, writer.getPath(), true, conf,
-        StoreFile.BloomType.NONE, false);
+    StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
+        StoreFile.BloomType.NONE);
     StoreFile.Reader reader = hsf.createReader();
     StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
     TreeSet<byte[]> columns = new TreeSet<byte[]>();
@@ -640,14 +644,13 @@ public class TestStoreFile extends HBase
 
   public void testCacheOnWriteEvictOnClose() throws Exception {
     Configuration conf = this.conf;
-    conf.setBoolean("hbase.rs.evictblocksonclose", false);
 
     // Find a home for our files
     Path baseDir = new Path(new Path(this.testDir, "regionname"),
     "twoCOWEOC");
 
     // Grab the block cache and get the initial hit/miss counts
-    BlockCache bc = StoreFile.getBlockCache(conf);
+    BlockCache bc = new CacheConfig(conf).getBlockCache();
     assertNotNull(bc);
     CacheStats cs = bc.getStats();
     long startHit = cs.getHitCount();
@@ -655,11 +658,12 @@ public class TestStoreFile extends HBase
     long startEvicted = cs.getEvictedCount();
 
     // Let's write a StoreFile with three blocks, with cache on write off
-    conf.setBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, false);
+    conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false);
+    CacheConfig cacheConf = new CacheConfig(conf);
     Path pathCowOff = new Path(baseDir, "123456789");
-    StoreFile.Writer writer = writeStoreFile(conf, pathCowOff, 3);
-    StoreFile hsf = new StoreFile(this.fs, writer.getPath(), true, conf,
-        StoreFile.BloomType.NONE, false);
+    StoreFile.Writer writer = writeStoreFile(conf, cacheConf, pathCowOff, 3);
+    StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
+        StoreFile.BloomType.NONE);
     LOG.debug(hsf.getPath().toString());
 
     // Read this file, we should see 3 misses
@@ -676,11 +680,12 @@ public class TestStoreFile extends HBase
     reader.close();
 
     // Now write a StoreFile with three blocks, with cache on write on
-    conf.setBoolean(HFile.CACHE_BLOCKS_ON_WRITE_KEY, true);
+    conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
+    cacheConf = new CacheConfig(conf);
     Path pathCowOn = new Path(baseDir, "123456788");
-    writer = writeStoreFile(conf, pathCowOn, 3);
-    hsf = new StoreFile(this.fs, writer.getPath(), true, conf,
-        StoreFile.BloomType.NONE, false);
+    writer = writeStoreFile(conf, cacheConf, pathCowOn, 3);
+    hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
+        StoreFile.BloomType.NONE);
 
     // Read this file, we should see 3 hits
     reader = hsf.createReader();
@@ -695,14 +700,14 @@ public class TestStoreFile extends HBase
     reader.close();
 
     // Let's read back the two files to ensure the blocks exactly match
-    hsf = new StoreFile(this.fs, pathCowOff, true, conf,
-        StoreFile.BloomType.NONE, false);
+    hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf,
+        StoreFile.BloomType.NONE);
     StoreFile.Reader readerOne = hsf.createReader();
     readerOne.loadFileInfo();
     StoreFileScanner scannerOne = readerOne.getStoreFileScanner(true, true);
     scannerOne.seek(KeyValue.LOWESTKEY);
-    hsf = new StoreFile(this.fs, pathCowOn, true, conf,
-        StoreFile.BloomType.NONE, false);
+    hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf,
+        StoreFile.BloomType.NONE);
     StoreFile.Reader readerTwo = hsf.createReader();
     readerTwo.loadFileInfo();
     StoreFileScanner scannerTwo = readerTwo.getStoreFileScanner(true, true);
@@ -731,8 +736,9 @@ public class TestStoreFile extends HBase
 
     // Let's close the first file with evict on close turned on
     conf.setBoolean("hbase.rs.evictblocksonclose", true);
-    hsf = new StoreFile(this.fs, pathCowOff, true, conf,
-        StoreFile.BloomType.NONE, false);
+    cacheConf = new CacheConfig(conf);
+    hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf,
+        StoreFile.BloomType.NONE);
     reader = hsf.createReader();
     reader.close();
 
@@ -744,8 +750,9 @@ public class TestStoreFile extends HBase
 
     // Let's close the second file with evict on close turned off
     conf.setBoolean("hbase.rs.evictblocksonclose", false);
-    hsf = new StoreFile(this.fs, pathCowOn, true, conf,
-        StoreFile.BloomType.NONE, false);
+    cacheConf = new CacheConfig(conf);
+    hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf,
+        StoreFile.BloomType.NONE);
     reader = hsf.createReader();
     reader.close();
 
@@ -755,8 +762,8 @@ public class TestStoreFile extends HBase
     assertEquals(startEvicted, cs.getEvictedCount());
   }
 
-  private StoreFile.Writer writeStoreFile(Configuration conf, Path path,
-      int numBlocks)
+  private StoreFile.Writer writeStoreFile(Configuration conf,
+      CacheConfig cacheConf, Path path, int numBlocks)
   throws IOException {
     // Let's put ~5 small KVs in each block, so let's make 5*numBlocks KVs
     int numKVs = 5 * numBlocks;
@@ -771,7 +778,7 @@ public class TestStoreFile extends HBase
     int blockSize = totalSize / numBlocks;
     StoreFile.Writer writer = new StoreFile.Writer(fs, path, blockSize,
         HFile.DEFAULT_COMPRESSION_ALGORITHM,
-        conf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000);
+        conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000);
     // We'll write N-1 KVs to ensure we don't write an extra block
     kvs.remove(kvs.size()-1);
     for (KeyValue kv : kvs) {

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileBlockCacheSummary.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileBlockCacheSummary.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileBlockCacheSummary.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileBlockCacheSummary.java Wed Oct 12 04:15:30 2011
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.Ro
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -100,7 +101,8 @@ public class TestStoreFileBlockCacheSumm
    scan(ht, FAMILY);
    scan(ht2, FAMILY);
       
-   BlockCache bc = StoreFile.getBlockCache(TEST_UTIL.getConfiguration());
+   BlockCache bc =
+     new CacheConfig(TEST_UTIL.getConfiguration()).getBlockCache();
    List<BlockCacheColumnFamilySummary> bcs = 
      bc.getBlockCacheColumnFamilySummaries(TEST_UTIL.getConfiguration());
    LOG.info("blockCacheSummary: " + bcs);

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1182194&r1=1182193&r2=1182194&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Wed Oct 12 04:15:30 2011
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.FlushRequester;
@@ -199,7 +200,8 @@ public class TestWALReplay {
     HLog wal = createWAL(this.conf);
     HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
     Path f =  new Path(basedir, "hfile");
-    HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(this.fs, f);
+    HFile.Writer writer =
+      HFile.getWriterFactory(conf).createWriter(this.fs, f);
     byte [] family = htd.getFamilies().iterator().next().getName();
     byte [] row = Bytes.toBytes(tableNameStr);
     writer.append(new KeyValue(row, family, family, row));