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 04:20:19 UTC

svn commit: r1181559 - in /hbase/branches/0.89/src: main/java/org/apache/hadoop/hbase/io/hfile/ main/java/org/apache/hadoop/hbase/mapreduce/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/util/ main/java/org/apache/ha...

Author: nspiegelberg
Date: Tue Oct 11 02:20:18 2011
New Revision: 1181559

URL: http://svn.apache.org/viewvc?rev=1181559&view=rev
Log:
Unit test changes, HFile reader/writer API call changes, and utility functions for HFile format version 2

Summary:
This is the sixth diff for the HFile version 2 / compound Bloom filters /
multi-level block indexes project. It bundles the following types of changes:
- API calls to create HFile readers/writes now have to go through the
appropriate factory methods.
- Unit test changes to enable testing both HFile format version 1 and 2. The
set of tests to be made parameterized was selected in an ad-hoc way during
development. Certain unit tests had to be converted to JUnit 4, but still extend
HBaseTestCase, to minimize code changes. Now that the default HFile format for
this feature has been flipped to version 2, we might be able to avoid a lot of
these unit test changes by running two rounds of test with the default HFile
format version specified through e.g. a system property. Also,
testHeapSizeForBlockIndex was moved from TestHFile to block index's own unit
test, TestHFileBlockIndex.
- New utility functions required for other parts of HFile v2 functionality
(e.g. changes to Bytes and TestBytes).
- A fix for a typo in Hash.java javadoc.

Test Plan:
Unit tests. Load testing with HBase test. Will push to dark launch along with
other HFile format v2 changes.

Reviewed By: nspiegelberg
Reviewers: kannan, liyintang, nspiegelberg, kranganathan, gqchen, aaiyer
CC: hbase@lists, , mbautin, nspiegelberg
Revert Plan:
HFile format v2 changes are backwards-compatible but not reverse-compatible.

Differential Revision: 252110

Added:
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
Modified:
    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/mapreduce/HFileOutputFormat.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.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/Store.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Hash.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.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/TestHFilePerformance.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/HBaseTest.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java

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=1181559&r1=1181558&r2=1181559&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 02:20:18 2011
@@ -55,22 +55,22 @@ public class CachedBlock implements Heap
   };
 
   private final String blockName;
-  private final ByteBuffer buf;
+  private final HeapSize buf;
   private volatile long accessTime;
   private long size;
   private BlockPriority priority;
 
-  public CachedBlock(String blockName, ByteBuffer buf, long accessTime) {
+  public CachedBlock(String blockName, HeapSize buf, long accessTime) {
     this(blockName, buf, accessTime, false);
   }
 
-  public CachedBlock(String blockName, ByteBuffer buf, long accessTime,
+  public CachedBlock(String blockName, HeapSize buf, long accessTime,
       boolean inMemory) {
     this.blockName = blockName;
     this.buf = buf;
     this.accessTime = accessTime;
-    this.size = ClassSize.align(blockName.length()) +
-    ClassSize.align(buf.capacity()) + PER_BLOCK_OVERHEAD;
+    this.size = ClassSize.align(blockName.length())
+        + ClassSize.align(buf.heapSize()) + PER_BLOCK_OVERHEAD;
     if(inMemory) {
       this.priority = BlockPriority.MEMORY;
     } else {
@@ -97,7 +97,7 @@ public class CachedBlock implements Heap
     return this.accessTime < that.accessTime ? 1 : -1;
   }
 
-  public ByteBuffer getBuffer() {
+  public HeapSize getBuffer() {
     return this.buf;
   }
 

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java Tue Oct 11 02:20:18 2011
@@ -76,7 +76,7 @@ public class HFileOutputFormat extends F
     // Get the path of the temporary output file
     final Path outputPath = FileOutputFormat.getOutputPath(context);
     final Path outputdir = new FileOutputCommitter(outputPath, context).getWorkPath();
-    Configuration conf = context.getConfiguration();
+    final Configuration conf = context.getConfiguration();
     final FileSystem fs = outputdir.getFileSystem(conf);
     // These configs. are from hbase-*.xml
     final long maxsize = conf.getLong("hbase.hregion.max.filesize",
@@ -131,7 +131,7 @@ public class HFileOutputFormat extends F
 
         // create a new HLog writer, if necessary
         if (wl == null || wl.writer == null) {
-          wl = getNewWriter(family);
+          wl = getNewWriter(family, conf);
         }
 
         // we now have the proper HLog writer. full steam ahead
@@ -161,12 +161,13 @@ public class HFileOutputFormat extends F
        * @return A WriterLength, containing a new HFile.Writer.
        * @throws IOException
        */
-      private WriterLength getNewWriter(byte[] family) throws IOException {
+      private WriterLength getNewWriter(byte[] family, Configuration conf)
+          throws IOException {
         WriterLength wl = new WriterLength();
         Path familydir = new Path(outputdir, Bytes.toString(family));
         String compression = compressionMap.get(family);
         compression = compression == null ? defaultCompression : compression;
-        wl.writer = new HFile.Writer(fs,
+        wl.writer = HFile.getWriterFactory(conf).createWriter(fs,
           StoreFile.getUniqueFile(fs, familydir), blocksize,
           bytesPerChecksum, compression, KeyValue.KEY_COMPARATOR);
         this.writers.put(family, wl);

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java Tue Oct 11 02:20:18 2011
@@ -184,7 +184,7 @@ public class LoadIncrementalHFiles exten
   throws IOException {
     final Path hfilePath = item.hfilePath;
     final FileSystem fs = hfilePath.getFileSystem(getConf());
-    HFile.Reader hfr = new HFile.Reader(fs, hfilePath, null, false, false);
+    HFile.Reader hfr = HFile.createReader(fs, hfilePath, null, false, false);
     final byte[] first, last;
     try {
       hfr.loadFileInfo();
@@ -276,7 +276,7 @@ public class LoadIncrementalHFiles exten
 
       halfWriter = new StoreFile.Writer(
           fs, outFile, blocksize, compression, conf, KeyValue.COMPARATOR,
-          bloomFilterType, 0, false);
+          bloomFilterType, 0);
       HFileScanner scanner = halfReader.getScanner(false, false, false);
       scanner.seekTo();
       do {

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=1181559&r1=1181558&r2=1181559&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 02:20:18 2011
@@ -270,6 +270,13 @@ public class HRegion implements HeapSize
     oldVal.addAndGet(amount);
   }
 
+  static long getNumericMetric(String key) {
+    AtomicLong m = numericMetrics.get(key);
+    if (m == null)
+      return 0;
+    return m.get();
+  }
+
   public static void incrTimeVaryingMetric(String key, long amount) {
     Pair<AtomicLong, AtomicInteger> oldVal = timeVaryingMetrics.get(key);
     if (oldVal == null) {

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=1181559&r1=1181558&r2=1181559&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 02:20:18 2011
@@ -24,10 +24,8 @@ import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.NavigableSet;
-import java.util.Set;
 import java.util.SortedSet;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -60,7 +58,6 @@ import com.google.common.collect.Collect
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 
 /**
  * A Store holds a column family in a Region.  Its a memstore and a set of zero
@@ -327,7 +324,7 @@ public class Store implements HeapSize {
     try {
       LOG.info("Validating hfile at " + srcPath + " for inclusion in "
           + "store " + this + " region " + this.region);
-      reader = new HFile.Reader(srcPath.getFileSystem(conf),
+      reader = HFile.createReader(srcPath.getFileSystem(conf),
           srcPath, null, false, false);
       reader.loadFileInfo();
 
@@ -519,8 +516,7 @@ public class Store implements HeapSize {
   throws IOException {
     return StoreFile.createWriter(this.fs, region.getTmpDir(), this.blocksize,
         this.compression, this.comparator, this.conf,
-        this.family.getBloomFilterType(), maxKeyCount,
-        conf.getBoolean("hbase.rs.cacheblocksonwrite", false));
+        this.family.getBloomFilterType(), maxKeyCount);
   }
 
   /*
@@ -571,7 +567,7 @@ public class Store implements HeapSize {
    */
   void deleteChangedReaderObserver(ChangedReadersObserver o) {
     if (!this.changedReaderObservers.remove(o)) {
-      LOG.warn("Not in set" + o);
+      LOG.warn("Not in set " + o);
     }
   }
 
@@ -1030,7 +1026,8 @@ public class Store implements HeapSize {
         LOG.info("Compacting: " + file +
           "; keyCount = " + keyCount +
           "; Bloom Type = " + r.getBloomFilterType().toString() +
-          "; Size = " + StringUtils.humanReadableInt(r.length()) );
+          "; Size = " + StringUtils.humanReadableInt(r.length()) +
+          "; HFile v" + r.getHFileVersion());
       }
     }
     LOG.info("Estimated total keyCount for output of compaction = " + maxKeyCount);
@@ -1520,6 +1517,37 @@ public class Store implements HeapSize {
   }
 
   /**
+   * Returns the total size of all index blocks in the data block indexes,
+   * including the root level, intermediate levels, and the leaf level for
+   * multi-level indexes, or just the root level for single-level indexes.
+   *
+   * @return the total size of block indexes in the store
+   */
+  long getTotalStaticIndexSize() {
+    long size = 0;
+    for (StoreFile s : storefiles) {
+      size += s.getReader().getUncompressedDataIndexSize();
+    }
+    return size;
+  }
+
+  /**
+   * Returns the total byte size of all Bloom filter bit arrays. For compound
+   * Bloom filters even the Bloom blocks currently not loaded into the block
+   * cache are counted.
+   *
+   * @return the total size of all Bloom filters in the store
+   */
+  long getTotalStaticBloomSize() {
+    long size = 0;
+    for (StoreFile s : storefiles) {
+      StoreFile.Reader r = s.getReader();
+      size += r.getTotalBloomSize();
+    }
+    return size;
+  }
+
+  /**
    * @return The priority that this store should have in the compaction queue
    */
   public int getCompactPriority() {

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Bytes.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Bytes.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Bytes.java Tue Oct 11 02:20:18 2011
@@ -298,17 +298,22 @@ public class Bytes {
    * @see #toStringBinary(byte[], int, int)
    */
   public static String toStringBinary(final byte [] b) {
+    if (b == null)
+      return "null";
     return toStringBinary(b, 0, b.length);
   }
 
   /**
-   * The same as {@link #toStringBinary(byte[])}, but returns a string "null"
-   * if given a null argument.
+   * Converts the given byte buffer, from its array offset to its limit, to
+   * a string. The position and the mark are ignored.
+   *
+   * @param buf a byte buffer
+   * @return a string representation of the buffer's binary contents
    */
-  public static String toStringBinarySafe(final byte [] b) {
-    if (b == null)
+  public static String toStringBinary(ByteBuffer buf) {
+    if (buf == null)
       return "null";
-    return toStringBinary(b, 0, b.length);
+    return toStringBinary(buf.array(), buf.arrayOffset(), buf.limit());
   }
 
   /**
@@ -1150,12 +1155,18 @@ public class Bytes {
 
   /**
    * Binary search for keys in indexes.
+   *
    * @param arr array of byte arrays to search for
    * @param key the key you want to find
    * @param offset the offset in the key you want to find
    * @param length the length of the key
    * @param comparator a comparator to compare.
-   * @return index of key
+   * @return zero-based index of the key, if the key is present in the array.
+   *         Otherwise, a value -(i + 1) such that the key is between arr[i -
+   *         1] and arr[i] non-inclusively, where i is in [0, i], if we define
+   *         arr[-1] = -Inf and arr[N] = Inf for an N-element array. The above
+   *         means that this function can return 2N + 1 different values
+   *         ranging from -(N + 1) to N - 1.
    */
   public static int binarySearch(byte [][]arr, byte []key, int offset,
       int length, RawComparator<byte []> comparator) {
@@ -1265,4 +1276,34 @@ public class Bytes {
     return value;
   }
 
+  /**
+   * Writes a string as a fixed-size field, padded with zeros.
+   */
+  public static void writeStringFixedSize(final DataOutput out, String s,
+      int size) throws IOException {
+    byte[] b = toBytes(s);
+    if (b.length > size) {
+      throw new IOException("Trying to write " + b.length + " bytes (" +
+          toStringBinary(b) + ") into a field of length " + size);
+    }
+
+    out.writeBytes(s);
+    for (int i = 0; i < size - s.length(); ++i)
+      out.writeByte(0);
+  }
+
+  /**
+   * Reads a fixed-size field and interprets it as a string padded with zeros.
+   */
+  public static String readStringFixedSize(final DataInput in, int size)
+      throws IOException {
+    byte[] b = new byte[size];
+    in.readFully(b);
+    int n = b.length;
+    while (n > 0 && b[n - 1] == 0)
+      --n;
+
+    return toString(b, 0, n);
+  }
+
 }

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java Tue Oct 11 02:20:18 2011
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.util;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 
@@ -61,16 +62,18 @@ public class CompressionTest {
 
   public static void main(String[] args) {
     if (args.length != 2) usage();
+    Configuration conf = HBaseConfiguration.create();
     try {
       DistributedFileSystem dfs = openConnection(args[0]);
       dfs.delete(path, false);
-      HFile.Writer writer = new HFile.Writer(dfs, path,
-        HFile.DEFAULT_BLOCKSIZE, HFile.DEFAULT_BYTES_PER_CHECKSUM, args[1], null);
+      HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(dfs,
+          path, HFile.DEFAULT_BLOCKSIZE, HFile.DEFAULT_BYTES_PER_CHECKSUM,
+          args[1], null);
       writer.append(Bytes.toBytes("testkey"), Bytes.toBytes("testval"));
       writer.appendFileInfo(Bytes.toBytes("infokey"), Bytes.toBytes("infoval"));
       writer.close();
 
-      HFile.Reader reader = new HFile.Reader(dfs, path, null, false, false);
+      HFile.Reader reader = HFile.createReader(dfs, path, null, false, false);
       reader.loadFileInfo();
       byte[] key = reader.getFirstKey();
       boolean rc = Bytes.toString(key).equals("testkey");

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Hash.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Hash.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Hash.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/util/Hash.java Tue Oct 11 02:20:18 2011
@@ -122,8 +122,8 @@ public abstract class Hash {
   }
 
   /**
-   * Calculate a hash using bytes from 0 to <code>length</code>, and
-   * the provided seed value
+   * Calculate a hash using bytes from <code>offset</code> to <code>offset +
+   * length</code>, and the provided seed value.
    * @param bytes input bytes
    * @param offset the offset into the array to start consideration
    * @param length length of the valid bytes after offset to consider

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java Tue Oct 11 02:20:18 2011
@@ -472,8 +472,8 @@ public class RecoverableZooKeeper {
                 return path;
               }
               LOG.error("Node " + path + " already exists with " +
-                  Bytes.toStringBinarySafe(currentData) + ", could not write " +
-                  Bytes.toStringBinarySafe(data));
+                  Bytes.toStringBinary(currentData) + ", could not write " +
+                  Bytes.toStringBinary(data));
               throw e;
             }
             LOG.error("Node " + path + " already exists and this is not a " +

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Tue Oct 11 02:20:18 2011
@@ -31,7 +31,6 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
-import java.util.Set;
 import java.util.UUID;
 
 import org.apache.commons.logging.Log;
@@ -50,6 +49,7 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -101,6 +101,19 @@ public class HBaseTestingUtility {
    */
   public static final String DEFAULT_TEST_DIRECTORY = "target/build/data";
 
+  /** Compression algorithms to use in parameterized JUnit 4 tests */
+  public static final List<Object[]> COMPRESSION_ALGORITHMS_PARAMETERIZED =
+    Arrays.asList(new Object[][] {
+      { Compression.Algorithm.NONE },
+      { Compression.Algorithm.GZ }
+    });
+
+  /** Compression algorithms to use in testing */
+  public static final Compression.Algorithm[] COMPRESSION_ALGORITHMS =
+      new Compression.Algorithm[] {
+        Compression.Algorithm.NONE, Compression.Algorithm.GZ
+      };
+
   public HBaseTestingUtility() {
     this(HBaseConfiguration.create());
   }
@@ -117,6 +130,18 @@ public class HBaseTestingUtility {
   }
 
   /**
+   * Makes sure the test directory is set up so that {@link #getTestDir()}
+   * returns a valid directory. Useful in unit tests that do not run a
+   * mini-cluster.
+   */
+  public void initTestDir() {
+    if (System.getProperty(TEST_DIRECTORY_KEY) == null) {
+      clusterTestBuildDir = setupClusterTestBuildDir();
+      System.setProperty(TEST_DIRECTORY_KEY, clusterTestBuildDir.getPath());
+    }
+  }
+
+  /**
    * @return Where to write test data on local filesystem; usually
    * {@link #DEFAULT_TEST_DIRECTORY}
    * @see #setupClusterTestBuildDir()

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/HFilePerformanceEvaluation.java Tue Oct 11 02:20:18 2011
@@ -188,9 +188,9 @@ public class HFilePerformanceEvaluation 
 
     @Override
     void setUp() throws Exception {
-      writer = new HFile.Writer(this.fs, this.mf, RFILE_BLOCKSIZE,
-		HFile.DEFAULT_BYTES_PER_CHECKSUM, (Compression.Algorithm) null, null,
-		null);
+      writer = HFile.getWriterFactory(conf).createWriter(this.fs, this.mf,
+          RFILE_BLOCKSIZE, HFile.DEFAULT_BYTES_PER_CHECKSUM,
+          (Compression.Algorithm) null, null);
     }
 
     @Override
@@ -226,7 +226,7 @@ public class HFilePerformanceEvaluation 
 
     @Override
     void setUp() throws Exception {
-      reader = new HFile.Reader(this.fs, this.mf, null, false, false);
+      reader = HFile.createReader(this.fs, this.mf, null, false, false);
       this.reader.loadFileInfo();
     }
 

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java Tue Oct 11 02:20:18 2011
@@ -20,6 +20,7 @@
 
 package org.apache.hadoop.hbase.io;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -60,9 +61,10 @@ public class TestHalfStoreFileReader {
     String root_dir = HBaseTestingUtility.getTestDir("TestHalfStoreFile").toString();
     Path p = new Path(root_dir, "test");
 
-    FileSystem fs = FileSystem.get(test_util.getConfiguration());
+    Configuration conf = test_util.getConfiguration();
+    FileSystem fs = FileSystem.get(conf);
 
-    HFile.Writer w = new HFile.Writer(fs, p, 1024,
+    HFile.Writer w = HFile.getWriterFactory(conf).createWriter(fs, p, 1024,
       HFile.DEFAULT_BYTES_PER_CHECKSUM, "none", KeyValue.KEY_COMPARATOR);
 
     // write some things.
@@ -72,7 +74,7 @@ public class TestHalfStoreFileReader {
     }
     w.close();
 
-    HFile.Reader r = new HFile.Reader(fs, p, null, false, false);
+    HFile.Reader r = HFile.createReader(fs, p, null, false, false);
     r.loadFileInfo();
     byte [] midkey = r.midkey();
     KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/RandomSeek.java Tue Oct 11 02:20:18 2011
@@ -68,9 +68,9 @@ public class RandomSeek {
     long start = System.currentTimeMillis();
     SimpleBlockCache cache = new SimpleBlockCache();
     //LruBlockCache cache = new LruBlockCache();
-    Reader reader = new HFile.Reader(lfs, path, cache, false, false);
+    Reader reader = HFile.createReader(lfs, path, cache, false, false);
     reader.loadFileInfo();
-    System.out.println(reader.trailer);
+    System.out.println(reader.getTrailer());
     long end = System.currentTimeMillis();
 
     System.out.println("Index read time: " + (end - start));

Added: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java?rev=1181559&view=auto
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java (added)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java Tue Oct 11 02:20:18 2011
@@ -0,0 +1,210 @@
+/*
+ * 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.io.hfile;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.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.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileBlock;
+import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex;
+import org.apache.hadoop.hbase.io.hfile.HFileReaderV2;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.io.hfile.TestHFileWriterV2;
+import org.apache.hadoop.hbase.regionserver.CreateRandomStoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.util.BloomFilterFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import static org.junit.Assert.*;
+
+/**
+ * Tests {@link HFile} cache-on-write functionality for the following block
+ * types: data blocks, non-root index blocks, and Bloom filter blocks.
+ */
+@RunWith(Parameterized.class)
+public class TestCacheOnWrite {
+
+  private static final Log LOG = LogFactory.getLog(TestCacheOnWrite.class);
+
+  private static final HBaseTestingUtility TEST_UTIL =
+    new HBaseTestingUtility();
+  private Configuration conf;
+  private FileSystem fs;
+  private Random rand = new Random(12983177L);
+  private Path storeFilePath;
+  private Compression.Algorithm compress;
+  private CacheOnWriteType cowType;
+  private BlockCache blockCache;
+  private String testName;
+
+  private static final int DATA_BLOCK_SIZE = 2048;
+  private static final int NUM_KV = 25000;
+  private static final int INDEX_BLOCK_SIZE = 512;
+  private static final int BLOOM_BLOCK_SIZE = 4096;
+
+  private static enum CacheOnWriteType {
+    DATA_BLOCKS(BlockType.DATA, HFile.CACHE_BLOCKS_ON_WRITE_KEY),
+    BLOOM_BLOCKS(BlockType.BLOOM_CHUNK,
+        BloomFilterFactory.IO_STOREFILE_BLOOM_CACHE_ON_WRITE),
+    INDEX_BLOCKS(BlockType.LEAF_INDEX,
+        HFileBlockIndex.CACHE_INDEX_BLOCKS_ON_WRITE_KEY);
+
+    private final String confKey;
+    private final BlockType inlineBlockType;
+
+    private CacheOnWriteType(BlockType inlineBlockType, String confKey) {
+      this.inlineBlockType = inlineBlockType;
+      this.confKey = confKey;
+    }
+
+    public boolean shouldBeCached(BlockType blockType) {
+      return blockType == inlineBlockType
+          || blockType == BlockType.INTERMEDIATE_INDEX
+          && inlineBlockType == BlockType.LEAF_INDEX;
+    }
+
+    public void modifyConf(Configuration conf) {
+      for (CacheOnWriteType cowType : CacheOnWriteType.values())
+        conf.setBoolean(cowType.confKey, cowType == this);
+    }
+
+  }
+
+  public TestCacheOnWrite(CacheOnWriteType cowType,
+      Compression.Algorithm compress) {
+    this.cowType = cowType;
+    this.compress = compress;
+    testName = "[cacheOnWrite=" + cowType + ", compress=" + compress + "]";
+  }
+
+  @Parameters
+  public static Collection<Object[]> getParameters() {
+    List<Object[]> cowTypes = new ArrayList<Object[]>();
+    for (CacheOnWriteType cowType : CacheOnWriteType.values())
+      for (Compression.Algorithm compress :
+           HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
+        cowTypes.add(new Object[] { cowType, compress });
+      }
+    return cowTypes;
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    conf = TEST_UTIL.getConfiguration();
+    conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
+    conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, INDEX_BLOCK_SIZE);
+    conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
+        BLOOM_BLOCK_SIZE);
+    cowType.modifyConf(conf);
+    fs = FileSystem.get(conf);
+    blockCache = StoreFile.getBlockCache(conf);
+  }
+
+  @After
+  public void tearDown() {
+    blockCache.evictBlocksByPrefix("");
+  }
+
+  @Test
+  public void testCacheOnWrite() throws IOException {
+    writeStoreFile();
+    readStoreFile();
+  }
+
+  private void readStoreFile() throws IOException {
+    HFileReaderV2 reader = (HFileReaderV2) HFile.createReader(fs,
+        storeFilePath, null, false, false);
+    LOG.info("HFile information: " + reader);
+    HFileScanner scanner = reader.getScanner(false, false);
+    assertTrue(testName, scanner.seekTo());
+
+    long offset = 0;
+    HFileBlock prevBlock = null;
+    EnumMap<BlockType, Integer> blockCountByType =
+        new EnumMap<BlockType, Integer>(BlockType.class);
+
+    while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
+      HFileBlock block = reader.readBlockData(offset, prevBlock == null ? -1
+          : prevBlock.getNextBlockOnDiskSizeWithHeader(), -1, false);
+      String blockCacheKey = HFile.getBlockCacheKey(reader.getName(), offset);
+      boolean isCached = blockCache.getBlock(blockCacheKey) != null;
+      boolean shouldBeCached = cowType.shouldBeCached(block.getBlockType());
+      assertEquals(testName + " " + block, shouldBeCached, isCached);
+      prevBlock = block;
+      offset += block.getOnDiskSizeWithHeader();
+      BlockType bt = block.getBlockType();
+      Integer count = blockCountByType.get(bt);
+      blockCountByType.put(bt, (count == null ? 0 : count) + 1);
+    }
+
+    LOG.info("Block count by type: " + blockCountByType);
+    assertEquals(
+        "{DATA=1367, LEAF_INDEX=172, BLOOM_CHUNK=9, INTERMEDIATE_INDEX=24}",
+        blockCountByType.toString());
+
+    reader.close();
+  }
+
+  public void writeStoreFile() throws IOException {
+    Path storeFileParentDir = new Path(HBaseTestingUtility.getTestDir(),
+        "test_cache_on_write");
+    StoreFile.Writer sfw = StoreFile.createWriter(fs, storeFileParentDir,
+        DATA_BLOCK_SIZE, compress, KeyValue.COMPARATOR, conf,
+        StoreFile.BloomType.ROWCOL, NUM_KV);
+
+    final int rowLen = 32;
+    for (int i = 0; i < NUM_KV; ++i) {
+      byte[] k = TestHFileWriterV2.randomOrderedKey(rand, i);
+      byte[] v = TestHFileWriterV2.randomValue(rand);
+      int cfLen = rand.nextInt(k.length - rowLen + 1);
+      KeyValue kv = new KeyValue(
+          k, 0, rowLen,
+          k, rowLen, cfLen,
+          k, rowLen + cfLen, k.length - rowLen - cfLen,
+          rand.nextLong(),
+          CreateRandomStoreFile.generateKeyType(rand),
+          v, 0, v.length);
+      sfw.append(kv);
+    }
+
+    sfw.close();
+    storeFilePath = sfw.getPath();
+  }
+
+}

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=1181559&r1=1181558&r2=1181559&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 02:20:18 2011
@@ -20,6 +20,9 @@
 package org.apache.hadoop.hbase.io.hfile;
 
 import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.io.HeapSize;
+
 import junit.framework.TestCase;
 
 public class TestCachedBlockQueue extends TestCase {
@@ -124,9 +127,13 @@ public class TestCachedBlockQueue extend
 
   private static class CachedBlock extends org.apache.hadoop.hbase.io.hfile.CachedBlock
   {
-    public CachedBlock(long heapSize, String name, long accessTime) {
+    public CachedBlock(final long heapSize, String name, long accessTime) {
       super(name,
-          ByteBuffer.allocate((int)(heapSize - CachedBlock.PER_BLOCK_OVERHEAD)),
+          new HeapSize(){
+            @Override
+            public long heapSize() {
+              return ((int)(heapSize - CachedBlock.PER_BLOCK_OVERHEAD));
+            }},
           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=1181559&r1=1181558&r2=1181559&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 02:20:18 2011
@@ -33,11 +33,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
-import org.apache.hadoop.hbase.io.hfile.HFile.BlockIndex;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.Writable;
 
 /**
@@ -63,9 +61,9 @@ public class TestHFile extends HBaseTest
    */
   public void testEmptyHFile() throws IOException {
     Path f = new Path(ROOT_DIR, getName());
-    Writer w = new Writer(this.fs, f);
+    Writer w = HFile.getWriterFactory(conf).createWriter(this.fs, f);
     w.close();
-    Reader r = new Reader(fs, f, null, false, false);
+    Reader r = HFile.createReader(fs, f, null, false, false);
     r.loadFileInfo();
     assertNull(r.getFirstKey());
     assertNull(r.getLastKey());
@@ -134,13 +132,13 @@ public class TestHFile extends HBaseTest
   void basicWithSomeCodec(String codec) throws IOException {
     Path ncTFile = new Path(ROOT_DIR, "basic.hfile");
     FSDataOutputStream fout = createFSOutput(ncTFile);
-    Writer writer = new Writer(fout, minBlockSize,
-      Compression.getCompressionAlgorithmByName(codec), null);
+    Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+        minBlockSize, Compression.getCompressionAlgorithmByName(codec), null);
     LOG.info(writer);
     writeRecords(writer);
     fout.close();
     FSDataInputStream fin = fs.open(ncTFile);
-    Reader reader = new Reader(ncTFile, fs.open(ncTFile),
+    Reader reader = HFile.createReader(ncTFile, fs.open(ncTFile),
       fs.getFileStatus(ncTFile).getLen(), null, false, false);
     // Load up the index.
     reader.loadFileInfo();
@@ -209,13 +207,14 @@ public class TestHFile extends HBaseTest
   private void metablocks(final String compress) throws Exception {
     Path mFile = new Path(ROOT_DIR, "meta.hfile");
     FSDataOutputStream fout = createFSOutput(mFile);
-    Writer writer = new Writer(fout, minBlockSize,
-      Compression.getCompressionAlgorithmByName(compress), null);
+    Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+        minBlockSize, Compression.getCompressionAlgorithmByName(compress),
+        null);
     someTestingWithMetaBlock(writer);
     writer.close();
     fout.close();
     FSDataInputStream fin = fs.open(mFile);
-    Reader reader = new Reader(mFile, fs.open(mFile),
+    Reader reader = HFile.createReader(mFile, fs.open(mFile),
         this.fs.getFileStatus(mFile).getLen(), null, false, false);
     reader.loadFileInfo();
     // No data -- this should return false.
@@ -233,33 +232,35 @@ public class TestHFile extends HBaseTest
   }
 
   public void testNullMetaBlocks() throws Exception {
-    Path mFile = new Path(ROOT_DIR, "nometa.hfile");
-    FSDataOutputStream fout = createFSOutput(mFile);
-    Writer writer = new Writer(fout, minBlockSize,
-        Compression.Algorithm.NONE, null);
-    writer.append("foo".getBytes(), "value".getBytes());
-    writer.close();
-    fout.close();
-    Reader reader = new Reader(fs, mFile, null, false, false);
-    reader.loadFileInfo();
-    assertNull(reader.getMetaBlock("non-existant", false));
+    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,
+          minBlockSize, compressAlgo, null);
+      writer.append("foo".getBytes(), "value".getBytes());
+      writer.close();
+      fout.close();
+      Reader reader = HFile.createReader(fs, mFile, null, false, false);
+      reader.loadFileInfo();
+      assertNull(reader.getMetaBlock("non-existant", false));
+    }
   }
 
   /**
    * Make sure the orginals for our compression libs doesn't change on us.
    */
   public void testCompressionOrdinance() {
-    //assertTrue(Compression.Algorithm.LZO.ordinal() == 0);
+    assertTrue(Compression.Algorithm.LZO.ordinal() == 0);
     assertTrue(Compression.Algorithm.GZ.ordinal() == 1);
     assertTrue(Compression.Algorithm.NONE.ordinal() == 2);
   }
 
-
   public void testComparator() throws IOException {
     Path mFile = new Path(ROOT_DIR, "meta.tfile");
     FSDataOutputStream fout = createFSOutput(mFile);
-    Writer writer = new Writer(fout, minBlockSize, (Compression.Algorithm) null,
-      new KeyComparator() {
+    Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+      minBlockSize, (Compression.Algorithm) null, new KeyComparator() {
         @Override
         public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2,
             int l2) {
@@ -277,27 +278,4 @@ public class TestHFile extends HBaseTest
     writer.close();
   }
 
-  /**
-   * Checks if the HeapSize calculator is within reason
-   */
-  @SuppressWarnings("unchecked")
-  public void testHeapSizeForBlockIndex() throws IOException{
-    Class cl = null;
-    long expected = 0L;
-    long actual = 0L;
-
-    cl = BlockIndex.class;
-    expected = ClassSize.estimateBase(cl, false);
-    BlockIndex bi = new BlockIndex(Bytes.BYTES_RAWCOMPARATOR);
-    actual = bi.heapSize();
-    //Since the arrays in BlockIndex(byte [][] blockKeys, long [] blockOffsets,
-    //int [] blockDataSizes) are all null they are not going to show up in the
-    //HeapSize calculation, so need to remove those array costs from ecpected.
-    expected -= ClassSize.align(3 * ClassSize.ARRAY);
-    if(expected != actual) {
-      ClassSize.estimateBase(cl, true);
-      assertEquals(expected, actual);
-    }
-  }
-
 }

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java Tue Oct 11 02:20:18 2011
@@ -157,8 +157,8 @@ public class TestHFilePerformance extend
 
     if ("HFile".equals(fileType)){
         System.out.println("HFile write method: ");
-        HFile.Writer writer =
-          new HFile.Writer(fout, minBlockSize, codecName, null);
+        HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+             minBlockSize, codecName, null);
 
         // Writing value in one shot.
         for (long l=0 ; l<rows ; l++ ) {
@@ -236,7 +236,7 @@ public class TestHFilePerformance extend
     FSDataInputStream fin = fs.open(path);
 
     if ("HFile".equals(fileType)){
-        HFile.Reader reader = new HFile.Reader(path, fs.open(path),
+        HFile.Reader reader = HFile.createReader(path, fs.open(path),
           fs.getFileStatus(path).getLen(), null, false, false);
         reader.loadFileInfo();
         switch (method) {

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java Tue Oct 11 02:20:18 2011
@@ -110,8 +110,8 @@ public class TestHFileSeek extends TestC
     long totalBytes = 0;
     FSDataOutputStream fout = createFSOutput(path, fs);
     try {
-      Writer writer =
-          new Writer(fout, options.minBlockSize, options.compress, null);
+      Writer writer = HFile.getWriterFactory(conf).createWriter(fout,
+          options.minBlockSize, options.compress, null);
       try {
         BytesWritable key = new BytesWritable();
         BytesWritable val = new BytesWritable();
@@ -155,8 +155,8 @@ public class TestHFileSeek extends TestC
     int miss = 0;
     long totalBytes = 0;
     FSDataInputStream fsdis = fs.open(path);
-    Reader reader = new Reader(path, fsdis, fs.getFileStatus(path).getLen(),
-        null, false, false);
+    Reader reader = HFile.createReader(path, fsdis,
+        fs.getFileStatus(path).getLen(), null, false, false);
     reader.loadFileInfo();
     KeySampler kSampler =
         new KeySampler(rng, reader.getFirstKey(), reader.getLastKey(),

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java Tue Oct 11 02:20:18 2011
@@ -42,7 +42,8 @@ public class TestReseekTo {
 
     Path ncTFile = new Path(HBaseTestingUtility.getTestDir(), "basic.hfile");
     FSDataOutputStream fout = TEST_UTIL.getTestFileSystem().create(ncTFile);
-    HFile.Writer writer = new HFile.Writer(fout, 4000, "none", null);
+    HFile.Writer writer = HFile.getWriterFactory(
+        TEST_UTIL.getConfiguration()).createWriter(fout, 4000, "none", null);
     int numberOfKeys = 1000;
 
     String valueString = "Value";
@@ -59,7 +60,7 @@ public class TestReseekTo {
     writer.close();
     fout.close();
 
-    HFile.Reader reader = new HFile.Reader(TEST_UTIL.getTestFileSystem(),
+    HFile.Reader reader = HFile.createReader(TEST_UTIL.getTestFileSystem(),
         ncTFile, null, false, false);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, true);

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java Tue Oct 11 02:20:18 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 = new HFile.Writer(fout, blocksize, "none", null);
+    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)
     writer.append(toKV("c"));
@@ -58,9 +59,10 @@ public class TestSeekTo extends HBaseTes
     fout.close();
     return ncTFile;
   }
+
   public void testSeekBefore() throws Exception {
     Path p = makeNewFile();
-    HFile.Reader reader = new HFile.Reader(fs, p, null, false, false);
+    HFile.Reader reader = HFile.createReader(fs, p, null, false, false);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, true);
     assertEquals(false, scanner.seekBefore(toKV("a").getKey()));
@@ -93,9 +95,9 @@ public class TestSeekTo extends HBaseTes
 
   public void testSeekTo() throws Exception {
     Path p = makeNewFile();
-    HFile.Reader reader = new HFile.Reader(fs, p, null, false, false);
+    HFile.Reader reader = HFile.createReader(fs, p, null, false, false);
     reader.loadFileInfo();
-    assertEquals(2, reader.blockIndex.count);
+    assertEquals(2, reader.getDataBlockIndexReader().getRootBlockCount());
     HFileScanner scanner = reader.getScanner(false, true);
     // lies before the start of the file.
     assertEquals(-1, scanner.seekTo(toKV("a").getKey()));
@@ -113,30 +115,32 @@ public class TestSeekTo extends HBaseTes
 
   public void testBlockContainingKey() throws Exception {
     Path p = makeNewFile();
-    HFile.Reader reader = new HFile.Reader(fs, p, null, false, false);
+    HFile.Reader reader = HFile.createReader(fs, p, null, false, false);
     reader.loadFileInfo();
-    System.out.println(reader.blockIndex.toString());
+    HFileBlockIndex.BlockIndexReader blockIndexReader =
+      reader.getDataBlockIndexReader();
+    System.out.println(blockIndexReader.toString());
     int klen = toKV("a").getKey().length;
     // falls before the start of the file.
-    assertEquals(-1, reader.blockIndex.blockContainingKey(toKV("a").getKey(),
-        0, klen));
-    assertEquals(0, reader.blockIndex.blockContainingKey(toKV("c").getKey(), 0,
-        klen));
-    assertEquals(0, reader.blockIndex.blockContainingKey(toKV("d").getKey(), 0,
-        klen));
-    assertEquals(0, reader.blockIndex.blockContainingKey(toKV("e").getKey(), 0,
-        klen));
-    assertEquals(0, reader.blockIndex.blockContainingKey(toKV("g").getKey(), 0,
-        klen));
-    assertEquals(0, reader.blockIndex.blockContainingKey(toKV("h").getKey(), 0,
-        klen));
-    assertEquals(1, reader.blockIndex.blockContainingKey(toKV("i").getKey(), 0,
-        klen));
-    assertEquals(1, reader.blockIndex.blockContainingKey(toKV("j").getKey(), 0,
-        klen));
-    assertEquals(1, reader.blockIndex.blockContainingKey(toKV("k").getKey(), 0,
-        klen));
-    assertEquals(1, reader.blockIndex.blockContainingKey(toKV("l").getKey(), 0,
-        klen));
-  }
+    assertEquals(-1, blockIndexReader.rootBlockContainingKey(
+        toKV("a").getKey(), 0, klen));
+    assertEquals(0, blockIndexReader.rootBlockContainingKey(
+        toKV("c").getKey(), 0, klen));
+    assertEquals(0, blockIndexReader.rootBlockContainingKey(
+        toKV("d").getKey(), 0, klen));
+    assertEquals(0, blockIndexReader.rootBlockContainingKey(
+        toKV("e").getKey(), 0, klen));
+    assertEquals(0, blockIndexReader.rootBlockContainingKey(
+        toKV("g").getKey(), 0, klen));
+    assertEquals(0, blockIndexReader.rootBlockContainingKey(
+        toKV("h").getKey(), 0, klen));
+    assertEquals(1, blockIndexReader.rootBlockContainingKey(
+        toKV("i").getKey(), 0, klen));
+    assertEquals(1, blockIndexReader.rootBlockContainingKey(
+        toKV("j").getKey(), 0, klen));
+    assertEquals(1, blockIndexReader.rootBlockContainingKey(
+        toKV("k").getKey(), 0, klen));
+    assertEquals(1, blockIndexReader.rootBlockContainingKey(
+        toKV("l").getKey(), 0, klen));
+ }
 }
\ No newline at end of file

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/HBaseTest.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/HBaseTest.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/HBaseTest.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/manual/HBaseTest.java Tue Oct 11 02:20:18 2011
@@ -183,7 +183,7 @@ public class HBaseTest
   private static final String OPT_USAGE_BLOOM = " Bloom filter type, one of " +
       Arrays.toString(StoreFile.BloomType.values());
 
-  private static final String OPT_USAGE_COMPRESSION = " Compression type, " +
+  static final String OPT_USAGE_COMPRESSION = " Compression type, " +
       "one of " + Arrays.toString(Compression.Algorithm.values());
 
   public static void main(String[] args) {

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java Tue Oct 11 02:20:18 2011
@@ -478,6 +478,9 @@ public class TestHFileOutputFormat  {
 
     try {
       // partial map red setup to get an operational writer for testing
+      // We turn off the sequence file compression, because DefaultCodec
+      // pollutes the GZip codec pool with an incompatible compressor.
+      conf.set("io.seqfile.compression.type", "NONE");
       Job job = new Job(conf, "testLocalMRIncrementalLoad");
       setupRandomGeneratorMapper(job);
       HFileOutputFormat.configureIncrementalLoad(job, table);
@@ -508,7 +511,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 = new HFile.Reader(fileSystem, dataFilePath, null, false, true);
+            Reader reader = HFile.createReader(fileSystem, dataFilePath, null,
+                false, true);
             reader.loadFileInfo();
             assertEquals("Incorrect compression used for column family " + familyStr
                          + "(reader: " + reader + ")",

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java Tue Oct 11 02:20:18 2011
@@ -99,8 +99,8 @@ public class TestLoadIncrementalHFiles {
     for (byte[][] range : hfileRanges) {
       byte[] from = range[0];
       byte[] to = range[1];
-      createHFile(fs, new Path(familyDir, "hfile_" + hfileIdx++),
-          FAMILY, QUALIFIER, from, to, 1000);
+      createHFile(util.getConfiguration(), fs, new Path(familyDir, "hfile_"
+          + hfileIdx++), FAMILY, QUALIFIER, from, to, 1000);
     }
     int expectedRows = hfileIdx * 1000;
 
@@ -130,7 +130,7 @@ public class TestLoadIncrementalHFiles {
     FileSystem fs = util.getTestFileSystem();
     Path testIn = new Path(dir, "testhfile");
     HColumnDescriptor familyDesc = new HColumnDescriptor(FAMILY);
-    createHFile(fs, testIn, FAMILY, QUALIFIER,
+    createHFile(util.getConfiguration(), fs, testIn, FAMILY, QUALIFIER,
         Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), 1000);
 
     Path bottomOut = new Path(dir, "bottom.out");
@@ -149,7 +149,7 @@ public class TestLoadIncrementalHFiles {
 
   private int verifyHFile(Path p) throws IOException {
     Configuration conf = util.getConfiguration();
-    HFile.Reader reader = new HFile.Reader(
+    HFile.Reader reader = HFile.createReader(
         p.getFileSystem(conf), p, null, false, false);
     reader.loadFileInfo();
     HFileScanner scanner = reader.getScanner(false, false);
@@ -169,12 +169,13 @@ public class TestLoadIncrementalHFiles {
    * TODO put me in an HFileTestUtil or something?
    */
   static void createHFile(
+      Configuration conf,
       FileSystem fs, Path path,
       byte[] family, byte[] qualifier,
       byte[] startKey, byte[] endKey, int numRows) throws IOException
   {
-    HFile.Writer writer = new HFile.Writer(fs, path, BLOCKSIZE,
-		HFile.DEFAULT_BYTES_PER_CHECKSUM, COMPRESSION,
+    HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(fs, path,
+        BLOCKSIZE, HFile.DEFAULT_BYTES_PER_CHECKSUM, COMPRESSION,
         KeyValue.KEY_COMPARATOR);
     long now = System.currentTimeMillis();
     try {

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java Tue Oct 11 02:20:18 2011
@@ -37,7 +37,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.hfile.Compression;
+import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.io.BytesWritable;
 
 /**
@@ -55,10 +57,14 @@ public class CreateRandomStoreFile {
       LogFactory.getLog(CreateRandomStoreFile.class);
   private static final String OUTPUT_DIR_OPTION = "o";
   private static final String NUM_KV_OPTION = "n";
+  private static final String HFILE_VERSION_OPTION = "h";
   private static final String KEY_SIZE_OPTION = "k";
   private static final String VALUE_SIZE_OPTION = "v";
   private static final String COMPRESSION_OPTION = "c";
-  private static final String BLOOM_FILTER_OPTION = "b";
+  private static final String BLOOM_FILTER_OPTION = "bf";
+  private static final String BLOCK_SIZE_OPTION = "bs";
+  private static final String BLOOM_BLOCK_SIZE_OPTION = "bfbs";
+  private static final String INDEX_BLOCK_SIZE_OPTION = "ibs";
 
   /** The exit code this command-line tool returns on failure */
   private static final int EXIT_FAILURE = 1;
@@ -87,12 +93,20 @@ public class CreateRandomStoreFile {
     options.addOption(KEY_SIZE_OPTION, "key_size", true, "Average key size");
     options.addOption(VALUE_SIZE_OPTION, "value_size", true,
         "Average value size");
+    options.addOption(HFILE_VERSION_OPTION, "hfile_version", true,
+        "HFile version to create");
     options.addOption(COMPRESSION_OPTION, "compression", true,
         " Compression type, one of "
             + Arrays.toString(Compression.Algorithm.values()));
     options.addOption(BLOOM_FILTER_OPTION, "bloom_filter", true,
         "Bloom filter type, one of "
             + Arrays.toString(StoreFile.BloomType.values()));
+    options.addOption(BLOCK_SIZE_OPTION, "block_size", true,
+        "HFile block size");
+    options.addOption(BLOOM_BLOCK_SIZE_OPTION, "bloom_block_size", true,
+        "Compound Bloom filters block size");
+    options.addOption(INDEX_BLOCK_SIZE_OPTION, "index_block_size", true,
+        "Index block size");
 
     if (args.length == 0) {
       HelpFormatter formatter = new HelpFormatter();
@@ -153,9 +167,22 @@ public class CreateRandomStoreFile {
           BLOOM_FILTER_OPTION));
     }
 
-    StoreFile.Writer sfw = StoreFile.createWriter(fs, outputDir,
-        conf.getInt("hfile.min.blocksize.size", 65536), compr,
-        KeyValue.COMPARATOR, conf, bloomType, numKV, false);
+    int blockSize = conf.getInt("hfile.min.blocksize.size", 65536);
+    if (cmdLine.hasOption(BLOCK_SIZE_OPTION))
+      blockSize = Integer.valueOf(cmdLine.getOptionValue(BLOCK_SIZE_OPTION));
+
+    if (cmdLine.hasOption(BLOOM_BLOCK_SIZE_OPTION)) {
+      conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
+          Integer.valueOf(cmdLine.getOptionValue(BLOOM_BLOCK_SIZE_OPTION)));
+    }
+
+    if (cmdLine.hasOption(INDEX_BLOCK_SIZE_OPTION)) {
+      conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY,
+          Integer.valueOf(cmdLine.getOptionValue(INDEX_BLOCK_SIZE_OPTION)));
+    }
+
+    StoreFile.Writer sfw = StoreFile.createWriter(fs, outputDir, blockSize,
+        compr, KeyValue.COMPARATOR, conf, bloomType, numKV);
 
     rand = new Random();
     LOG.info("Writing " + numKV + " key/value pairs");
@@ -208,11 +235,11 @@ public class CreateRandomStoreFile {
         k, rowLen, cfLen,
         k, rowLen + cfLen, k.length - rowLen - cfLen,
         rand.nextLong(),
-        generateKeyType(),
+        generateKeyType(rand),
         v, 0, v.length);
   }
 
-  private KeyValue.Type generateKeyType() {
+  public static KeyValue.Type generateKeyType(Random rand) {
     if (rand.nextBoolean()) {
       // Let's make half of KVs puts.
       return KeyValue.Type.Put;

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java Tue Oct 11 02:20:18 2011
@@ -70,7 +70,8 @@ public class TestFSErrorsExposed {
         HBaseTestingUtility.getTestDir("internalScannerExposesErrors"),
         "regionname"), "familyname");
     FaultyFileSystem fs = new FaultyFileSystem(util.getTestFileSystem());
-    StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2*1024);
+    StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2*1024,
+        util.getConfiguration());
     TestStoreFile.writeStoreFile(
         writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
 
@@ -111,7 +112,8 @@ public class TestFSErrorsExposed {
         HBaseTestingUtility.getTestDir("internalScannerExposesErrors"),
         "regionname"), "familyname");
     FaultyFileSystem fs = new FaultyFileSystem(util.getTestFileSystem());
-    StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2 * 1024);
+    StoreFile.Writer writer = StoreFile.createWriter(fs, hfilePath, 2 * 1024,
+        util.getConfiguration());
     TestStoreFile.writeStoreFile(
         writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
 

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java Tue Oct 11 02:20:18 2011
@@ -203,7 +203,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);
+        StoreFile.DEFAULT_BLOCKSIZE_SMALL, c);
     w.appendMetadata(seqid + 1, false);
     w.close();
     this.store.close();

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Tue Oct 11 02:20:18 2011
@@ -185,7 +185,7 @@ public class TestWALReplay {
     HLog wal = createWAL(this.conf);
     HRegion region = HRegion.openHRegion(hri, basedir, wal, this.conf);
     Path f =  new Path(basedir, "hfile");
-    HFile.Writer writer = new HFile.Writer(this.fs, f);
+    HFile.Writer writer = HFile.getWriterFactory(conf).createWriter(this.fs, f);
     byte [] family = hri.getTableDesc().getFamilies().iterator().next().getName();
     byte [] row = Bytes.toBytes(tableNameStr);
     writer.append(new KeyValue(row, family, family, row));

Modified: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java?rev=1181559&r1=1181558&r2=1181559&view=diff
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java (original)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/util/TestBytes.java Tue Oct 11 02:20:18 2011
@@ -19,6 +19,10 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 
@@ -134,6 +138,8 @@ public class TestBytes extends TestCase 
     byte [] key2 = {4,9};
     byte [] key2_2 = {4};
     byte [] key3 = {5,11};
+    byte [] key4 = {0};
+    byte [] key5 = {2};
 
     assertEquals(1, Bytes.binarySearch(arr, key1, 0, 1,
       Bytes.BYTES_RAWCOMPARATOR));
@@ -149,6 +155,20 @@ public class TestBytes extends TestCase 
       Bytes.BYTES_RAWCOMPARATOR));
     assertEquals(5, Bytes.binarySearch(arr, key3, 1, 1,
       Bytes.BYTES_RAWCOMPARATOR));
+    assertEquals(-1,
+      Bytes.binarySearch(arr, key4, 0, 1, Bytes.BYTES_RAWCOMPARATOR));
+    assertEquals(-2,
+      Bytes.binarySearch(arr, key5, 0, 1, Bytes.BYTES_RAWCOMPARATOR));
+
+    // Search for values to the left and to the right of each item in the array.
+    for (int i = 0; i < arr.length; ++i) {
+      assertEquals(-(i + 1), Bytes.binarySearch(arr,
+          new byte[] { (byte) (arr[i][0] - 1) }, 0, 1,
+          Bytes.BYTES_RAWCOMPARATOR));
+      assertEquals(-(i + 2), Bytes.binarySearch(arr,
+          new byte[] { (byte) (arr[i][0] + 1) }, 0, 1,
+          Bytes.BYTES_RAWCOMPARATOR));
+    }
   }
 
   public void testStartsWith() {
@@ -194,4 +214,30 @@ public class TestBytes extends TestCase 
 
     return (Bytes.toLong(testValue) + amount) == incrementResult;
   }
+
+  public void testFixedSizeString() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    Bytes.writeStringFixedSize(dos, "Hello", 5);
+    Bytes.writeStringFixedSize(dos, "World", 18);
+    Bytes.writeStringFixedSize(dos, "", 9);
+
+    try {
+      // Use a long dash which is three bytes in UTF-8. If encoding happens
+      // using ISO-8859-1, this will fail.
+      Bytes.writeStringFixedSize(dos, "Too\u2013Long", 9);
+      fail("Exception expected");
+    } catch (IOException ex) {
+      assertEquals(
+          "Trying to write 10 bytes (Too\\xE2\\x80\\x93Long) into a field of " +
+          "length 9", ex.getMessage());
+    }
+
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    assertEquals("Hello", Bytes.readStringFixedSize(dis, 5));
+    assertEquals("World", Bytes.readStringFixedSize(dis, 18));
+    assertEquals("", Bytes.readStringFixedSize(dis, 9));
+  }
+
 }