You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ra...@apache.org on 2013/11/11 18:08:05 UTC

svn commit: r1540785 [1/2] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/ hbase-common/src/main/java/or...

Author: ramkrishna
Date: Mon Nov 11 17:08:04 2013
New Revision: 1540785

URL: http://svn.apache.org/r1540785
Log:
HBASE-9816-Address review comments in HBASE-8496 (Ram)


Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/ColumnNodeType.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV3.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Mon Nov 11 17:08:04 2013
@@ -511,7 +511,7 @@ public final class ProtobufUtil {
           byte[] tags;
           if (qv.hasTags()) {
             tags = qv.getTags().toByteArray();
-            Object[] array = Tag.createTags(tags, 0, (short)tags.length).toArray();
+            Object[] array = Tag.asList(tags, 0, (short)tags.length).toArray();
             Tag[] tagArray = new Tag[array.length];
             for(int i = 0; i< array.length; i++) {
               tagArray[i] = (Tag)array[i];

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java Mon Nov 11 17:08:04 2013
@@ -376,4 +376,42 @@ public final class CellUtil {
       // Serialization is probably preceded by a length (it is in the KeyValueCodec at least).
       Bytes.SIZEOF_INT;
   }
+  
+  
+  /********************* tags *************************************/
+  /**
+   * Util method to iterate through the tags. Used in testcase
+   * 
+   * @param tags
+   * @param offset
+   * @param length
+   * @return
+   */
+  public static Iterator<Tag> tagsIterator(final byte[] tags, final int offset, final short length) {
+    return new Iterator<Tag>() {
+      private int pos = offset;
+      private int endOffset = offset + length - 1;
+
+      @Override
+      public boolean hasNext() {
+        return this.pos < endOffset;
+      }
+
+      @Override
+      public Tag next() {
+        if (hasNext()) {
+          short curTagLen = Bytes.toShort(tags, this.pos);
+          Tag tag = new Tag(tags, pos, (short) (curTagLen + Bytes.SIZEOF_SHORT));
+          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
+          return tag;
+        }
+        return null;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
 }

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java Mon Nov 11 17:08:04 2013
@@ -31,7 +31,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -80,6 +79,8 @@ import com.google.common.primitives.Long
  */
 @InterfaceAudience.Private
 public class KeyValue implements Cell, HeapSize, Cloneable {
+  private static final ArrayList<Tag> EMPTY_ARRAY_LIST = new ArrayList<Tag>();
+
   static final Log LOG = LogFactory.getLog(KeyValue.class);
 
   /**
@@ -255,38 +256,6 @@ public class KeyValue implements Cell, H
   }
 
   /**
-   * @return an iterator over the tags in this KeyValue.
-   */
-  public Iterator<Tag> tagsIterator() {
-    // Subtract -1 to point to the end of the complete tag byte[]
-    final int endOffset = this.offset + this.length - 1;
-    return new Iterator<Tag>() {
-      private int pos = getTagsOffset();
-
-      @Override
-      public boolean hasNext() {
-        return this.pos < endOffset;
-      }
-
-      @Override
-      public Tag next() {
-        if (hasNext()) {
-          short curTagLen = Bytes.toShort(bytes, this.pos);
-          Tag tag = new Tag(bytes, pos, (short) (curTagLen + Bytes.SIZEOF_SHORT));
-          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
-          return tag;
-        }
-        return null;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    };
-  }
-
-  /**
    * Lowest possible key.
    * Makes a Key with highest possible Timestamp, empty row and column.  No
    * key can be equal or lower than this one in memstore or in store file.
@@ -1618,18 +1587,15 @@ public class KeyValue implements Cell, H
   }
 
   /**
-   * This method may not be right.  But we cannot use the CellUtil.getTagIterator because we don't know
-   * getKeyOffset and getKeyLength
-   * Cannnot use the getKeyOffset and getKeyLength in CellUtil as they are not part of the Cell interface.
-   * Returns any tags embedded in the KeyValue.
+   * Returns any tags embedded in the KeyValue.  Used in testcases.
    * @return The tags
    */
   public List<Tag> getTags() {
     short tagsLength = getTagsLength();
     if (tagsLength == 0) {
-      return new ArrayList<Tag>();
+      return EMPTY_ARRAY_LIST;
     }
-    return Tag.createTags(getBuffer(), getTagsOffset(), tagsLength);
+    return Tag.asList(getBuffer(), getTagsOffset(), tagsLength);
   }
 
   /**

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java Mon Nov 11 17:08:04 2013
@@ -25,11 +25,9 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.util.Bytes;
-
 /**
- * <code>&lt;taglength>&lt;tagtype>&lt;tagbytes></code>. <code>tagtype</code> is
- * one byte and <code>taglength</code> maximum is <code>Short.MAX_SIZE</code>.
- * It includes 1 byte type length and actual tag bytes length.
+ * Tags are part of cells and helps to add metadata about the KVs.
+ * Metadata could be ACLs per cells, visibility labels, etc.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -38,8 +36,8 @@ public class Tag {
   public final static int TAG_LENGTH_SIZE = Bytes.SIZEOF_SHORT;
   public final static int INFRASTRUCTURE_SIZE = TYPE_LENGTH_SIZE + TAG_LENGTH_SIZE;
 
-  private byte type;
-  private byte[] bytes;
+  private final byte type;
+  private final byte[] bytes;
   private int offset = 0;
   private short length = 0;
 
@@ -56,7 +54,9 @@ public class Tag {
    * @param tag
    */
   public Tag(byte tagType, byte[] tag) {
-    // <length of tag - 2 bytes><type code - 1 byte><tag>
+    /** <length of tag - 2 bytes><type code - 1 byte><tag>
+     * taglength maximum is Short.MAX_SIZE.  It includes 1 byte type length and actual tag bytes length.
+     */
     short tagLength = (short) ((tag.length & 0x0000ffff) + TYPE_LENGTH_SIZE);
     length = (short) (TAG_LENGTH_SIZE + tagLength);
     bytes = new byte[length];
@@ -119,14 +119,14 @@ public class Tag {
   /**
    * @return Length of actual tag bytes within the backed buffer
    */
-  public int getTagLength() {
+  int getTagLength() {
     return this.length - INFRASTRUCTURE_SIZE;
   }
 
   /**
    * @return Offset of actual tag bytes within the backed buffer
    */
-  public int getTagOffset() {
+  int getTagOffset() {
     return this.offset + INFRASTRUCTURE_SIZE;
   }
 
@@ -145,7 +145,7 @@ public class Tag {
    * @param length
    * @return List of tags
    */
-  public static List<Tag> createTags(byte[] b, int offset, short length) {
+  public static List<Tag> asList(byte[] b, int offset, short length) {
     List<Tag> tags = new ArrayList<Tag>();
     int pos = offset;
     while (pos < offset + length) {
@@ -169,4 +169,4 @@ public class Tag {
   int getOffset() {
     return this.offset;
   }
-}
\ No newline at end of file
+}

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java Mon Nov 11 17:08:04 2013
@@ -52,7 +52,7 @@ abstract class BufferedDataBlockEncoder 
 
     HFileBlockDefaultDecodingContext decodingCtx =
         (HFileBlockDefaultDecodingContext) blkDecodingCtx;
-    if (decodingCtx.getHFileContext().shouldCompressTags()) {
+    if (decodingCtx.getHFileContext().isCompressTags()) {
       try {
         TagCompressionContext tagCompressionContext = new TagCompressionContext(LRUDictionary.class);
         decodingCtx.setTagCompressionContext(tagCompressionContext);
@@ -162,7 +162,7 @@ abstract class BufferedDataBlockEncoder 
         this.samePrefixComparator = null;
       }
       this.decodingCtx = decodingCtx;
-      if (decodingCtx.getHFileContext().shouldCompressTags()) {
+      if (decodingCtx.getHFileContext().isCompressTags()) {
         try {
           tagCompressionContext = new TagCompressionContext(LRUDictionary.class);
         } catch (Exception e) {
@@ -172,11 +172,11 @@ abstract class BufferedDataBlockEncoder 
     }
     
     protected boolean includesMvcc() {
-      return this.decodingCtx.getHFileContext().shouldIncludeMvcc();
+      return this.decodingCtx.getHFileContext().isIncludesMvcc();
     }
 
     protected boolean includesTags() {
-      return this.decodingCtx.getHFileContext().shouldIncludeTags();
+      return this.decodingCtx.getHFileContext().isIncludesTags();
     }
 
     @Override
@@ -264,7 +264,7 @@ abstract class BufferedDataBlockEncoder 
       return true;
     }
 
-    public void decodeTags() {
+    protected void decodeTags() {
       current.tagsLength = ByteBufferUtils.readCompressedInt(currentBuffer);
       if (tagCompressionContext != null) {
         // Tag compression is been used. uncompress it into tagsBuffer
@@ -373,7 +373,7 @@ abstract class BufferedDataBlockEncoder 
 
   protected final void afterEncodingKeyValue(ByteBuffer in,
       DataOutputStream out, HFileBlockDefaultEncodingContext encodingCtx) throws IOException {
-    if (encodingCtx.getHFileContext().shouldIncludeTags()) {
+    if (encodingCtx.getHFileContext().isIncludesTags()) {
       short tagsLength = in.getShort();
       ByteBufferUtils.putCompressedInt(out, tagsLength);
       // There are some tags to be written
@@ -388,7 +388,7 @@ abstract class BufferedDataBlockEncoder 
         }
       }
     }
-    if (encodingCtx.getHFileContext().shouldIncludeMvcc()) {
+    if (encodingCtx.getHFileContext().isIncludesMvcc()) {
       // Copy memstore timestamp from the byte buffer to the output stream.
       long memstoreTS = -1;
       try {
@@ -403,7 +403,7 @@ abstract class BufferedDataBlockEncoder 
 
   protected final void afterDecodingKeyValue(DataInputStream source,
       ByteBuffer dest, HFileBlockDefaultDecodingContext decodingCtx) throws IOException {
-    if (decodingCtx.getHFileContext().shouldIncludeTags()) {
+    if (decodingCtx.getHFileContext().isIncludesTags()) {
       short tagsLength = (short) ByteBufferUtils.readCompressedInt(source);
       dest.putShort(tagsLength);
       if (tagsLength > 0) {
@@ -417,7 +417,7 @@ abstract class BufferedDataBlockEncoder 
         }
       }
     }
-    if (decodingCtx.getHFileContext().shouldIncludeMvcc()) {
+    if (decodingCtx.getHFileContext().isIncludesMvcc()) {
       long memstoreTS = -1;
       try {
         // Copy memstore timestamp from the data input stream to the byte
@@ -452,7 +452,7 @@ abstract class BufferedDataBlockEncoder 
   public abstract void internalEncodeKeyValues(DataOutputStream out,
       ByteBuffer in, HFileBlockDefaultEncodingContext encodingCtx) throws IOException;
 
-  public abstract ByteBuffer internalDecodeKeyValues(DataInputStream source,
+  protected abstract ByteBuffer internalDecodeKeyValues(DataInputStream source,
       int allocateHeaderLength, int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx)
       throws IOException;
 
@@ -471,7 +471,7 @@ abstract class BufferedDataBlockEncoder 
     DataOutputStream dataOut =
         ((HFileBlockDefaultEncodingContext) encodingCtx)
         .getOutputStreamForEncoder();
-    if (encodingCtx.getHFileContext().shouldCompressTags()) {
+    if (encodingCtx.getHFileContext().isCompressTags()) {
       try {
         TagCompressionContext tagCompressionContext = new TagCompressionContext(LRUDictionary.class);
         encodingCtx.setTagCompressionContext(tagCompressionContext);

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java Mon Nov 11 17:08:04 2013
@@ -88,7 +88,7 @@ public class CopyKeyDataBlockEncoder ext
   }
 
   @Override
-  public ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength,
+  protected ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength,
       int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx) throws IOException {
     int decompressedSize = source.readInt();
     ByteBuffer buffer = ByteBuffer.allocate(decompressedSize +

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DiffKeyDeltaEncoder.java Mon Nov 11 17:08:04 2013
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.RawComparator;
 
 /**
  * Compress using:
@@ -534,7 +533,7 @@ public class DiffKeyDeltaEncoder extends
   }
 
   @Override
-  public ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength,
+  protected ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength,
       int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx) throws IOException {
     int decompressedSize = source.readInt();
     ByteBuffer buffer = ByteBuffer.allocate(decompressedSize +

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java Mon Nov 11 17:08:04 2013
@@ -113,13 +113,13 @@ public class EncodedDataBlock {
         short tagsLen = 0;
         ByteBufferUtils.skip(decompressedData, klen + vlen);
         // Read the tag length in case when steam contain tags
-        if (meta.shouldIncludeTags()) {
+        if (meta.isIncludesTags()) {
           tagsLen = decompressedData.getShort();
           ByteBufferUtils.skip(decompressedData, tagsLen);
         }
         KeyValue kv = new KeyValue(decompressedData.array(), offset,
             (int) KeyValue.getKeyValueDataStructureSize(klen, vlen, tagsLen));
-        if (meta.shouldIncludeMvcc()) {
+        if (meta.isIncludesMvcc()) {
           long mvccVersion = ByteBufferUtils.readVLong(decompressedData);
           kv.setMvccVersion(mvccVersion);
         }

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/FastDiffDeltaEncoder.java Mon Nov 11 17:08:04 2013
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.RawComparator;
 
 /**
  * Encoder similar to {@link DiffKeyDeltaEncoder} but supposedly faster.
@@ -362,7 +361,7 @@ public class FastDiffDeltaEncoder extend
   }
 
   @Override
-  public ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength,
+  protected ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength,
       int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx) throws IOException {
     int decompressedSize = source.readInt();
     ByteBuffer buffer = ByteBuffer.allocate(decompressedSize +

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/PrefixKeyDeltaEncoder.java Mon Nov 11 17:08:04 2013
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.RawComparator;
 
 /**
  * Compress key by storing size of common prefix with previous KeyValue
@@ -92,7 +91,7 @@ public class PrefixKeyDeltaEncoder exten
   }
 
   @Override
-  public ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength,
+  protected ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength,
       int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx) throws IOException {
     int decompressedSize = source.readInt();
     ByteBuffer buffer = ByteBuffer.allocate(decompressedSize +

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContext.java Mon Nov 11 17:08:04 2013
@@ -27,9 +27,8 @@ import org.apache.hadoop.hbase.util.Clas
 
 /**
  * This carries the information on some of the meta data about the HFile. This
- * meta data would be used across the HFileWriter/Readers and the HFileBlocks.
- * This would help to add new information to the HFile.
- * This class is not meant to be immutable.
+ * meta data is used across the HFileWriter/Readers and the HFileBlocks.
+ * This helps to add new information to the HFile.
  */
 @InterfaceAudience.Private
 public class HFileContext implements HeapSize, Cloneable {
@@ -96,29 +95,27 @@ public class HFileContext implements Hea
     return compressAlgo;
   }
 
-  public boolean shouldUseHBaseChecksum() {
+  public boolean isUseHBaseChecksum() {
     return usesHBaseChecksum;
   }
 
-  public boolean shouldIncludeMvcc() {
+  public boolean isIncludesMvcc() {
     return includesMvcc;
   }
 
-  // TODO : This setter should be removed
   public void setIncludesMvcc(boolean includesMvcc) {
     this.includesMvcc = includesMvcc;
   }
 
-  public boolean shouldIncludeTags() {
+  public boolean isIncludesTags() {
     return includesTags;
   }
 
-  // TODO : This setter should be removed?
   public void setIncludesTags(boolean includesTags) {
     this.includesTags = includesTags;
   }
 
-  public boolean shouldCompressTags() {
+  public boolean isCompressTags() {
     return compressTags;
   }
 

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileContextBuilder.java Mon Nov 11 17:08:04 2013
@@ -38,7 +38,7 @@ public class HFileContextBuilder {
   /** Whether tags are to be included in the Read/Write **/
   private boolean includesTags;
   /** Compression algorithm used **/
-  private Algorithm compressAlgo = Algorithm.NONE;
+  private Algorithm compression = Algorithm.NONE;
   /** Whether tags to be compressed or not **/
   private boolean compressTags;
   /** the checksum type **/
@@ -65,8 +65,8 @@ public class HFileContextBuilder {
     return this;
   }
 
-  public HFileContextBuilder withCompressionAlgo(Algorithm compressionAlgo) {
-    this.compressAlgo = compressionAlgo;
+  public HFileContextBuilder withCompression(Algorithm compression) {
+    this.compression = compression;
     return this;
   }
 
@@ -101,7 +101,7 @@ public class HFileContextBuilder {
   }
 
   public HFileContext build() {
-    return new HFileContext(usesHBaseChecksum, includesMvcc, includesTags, compressAlgo,
+    return new HFileContext(usesHBaseChecksum, includesMvcc, includesTags, compression,
         compressTags, checksumType, bytesPerChecksum, blocksize, encodingOnDisk, encodingInCache);
   }
 }

Modified: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java (original)
+++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java Mon Nov 11 17:08:04 2013
@@ -560,8 +560,9 @@ public class TestKeyValue extends TestCa
     }
     assertTrue(meta1Ok);
     assertTrue(meta2Ok);
-
-    Iterator<Tag> tagItr = kv.tagsIterator();
+    Iterator<Tag> tagItr = CellUtil.tagsIterator(kv.getTagsArray(), kv.getTagsOffset(),
+        kv.getTagsLength());
+    //Iterator<Tag> tagItr = kv.tagsIterator();
     assertTrue(tagItr.hasNext());
     Tag next = tagItr.next();
     assertEquals(10, next.getTagLength());
@@ -574,7 +575,7 @@ public class TestKeyValue extends TestCa
     Bytes.equals(next.getValue(), metaValue2);
     assertFalse(tagItr.hasNext());
 
-    tagItr = kv.tagsIterator();
+    tagItr = CellUtil.tagsIterator(kv.getTagsArray(), kv.getTagsOffset(), kv.getTagsLength());
     assertTrue(tagItr.hasNext());
     next = tagItr.next();
     assertEquals(10, next.getTagLength());

Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java Mon Nov 11 17:08:04 2013
@@ -79,8 +79,8 @@ public class PrefixTreeCodec implements 
         = (HFileBlockDefaultEncodingContext) blkEncodingCtx;
     encodingCtx.prepareEncoding();
     DataOutputStream dataOut = encodingCtx.getOutputStreamForEncoder();
-    internalEncodeKeyValues(dataOut, in, encodingCtx.getHFileContext().shouldIncludeMvcc(),
-        encodingCtx.getHFileContext().shouldIncludeTags());
+    internalEncodeKeyValues(dataOut, in, encodingCtx.getHFileContext().isIncludesMvcc(),
+        encodingCtx.getHFileContext().isIncludesTags());
 
     //do i need to check this, or will it always be DataBlockEncoding.PREFIX_TREE?
     if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) {
@@ -130,7 +130,7 @@ public class PrefixTreeCodec implements 
     result.rewind();
     CellSearcher searcher = null;
     try {
-      boolean includesMvcc = decodingCtx.getHFileContext().shouldIncludeMvcc();
+      boolean includesMvcc = decodingCtx.getHFileContext().isIncludesMvcc();
       searcher = DecoderFactory.checkOut(sourceAsBuffer, includesMvcc);
       while (searcher.advance()) {
         KeyValue currentCell = KeyValueUtil.copyToNewKeyValue(searcher.current());
@@ -199,7 +199,7 @@ public class PrefixTreeCodec implements 
           +"table");
     }
 
-    return new PrefixTreeSeeker(decodingCtx.getHFileContext().shouldIncludeMvcc());
+    return new PrefixTreeSeeker(decodingCtx.getHFileContext().isIncludesMvcc());
   }
 
 }

Modified: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/ColumnNodeType.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/ColumnNodeType.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/ColumnNodeType.java (original)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/ColumnNodeType.java Mon Nov 11 17:08:04 2013
@@ -1,7 +1,3 @@
-package org.apache.hadoop.hbase.codec.prefixtree.encode.other;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -19,10 +15,14 @@ import org.apache.hadoop.classification.
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.hadoop.hbase.codec.prefixtree.encode.other;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
 /**
  * Specifies the type of columnnode writer.
  */
 @InterfaceAudience.Private
 public enum ColumnNodeType {
   FAMILY, QUALIFIER, TAGS;
-}
\ No newline at end of file
+}

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java Mon Nov 11 17:08:04 2013
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.fs.HFileS
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
-import org.apache.hadoop.io.RawComparator;
 
 /**
  * Common functionality needed by all versions of {@link HFile} readers.

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java Mon Nov 11 17:08:04 2013
@@ -104,7 +104,7 @@ public class ChecksumUtil {
     // when the minorVersion is 0, thus this is a defensive check for a
     // cannot-happen case. Since this is a cannot-happen case, it is
     // better to return false to indicate a checksum validation failure.
-    if (!block.getHFileContext().shouldUseHBaseChecksum()) {
+    if (!block.getHFileContext().isUseHBaseChecksum()) {
       return false;
     }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java Mon Nov 11 17:08:04 2013
@@ -398,7 +398,7 @@ public class HFileBlock implements Cache
         "uncompressedSizeWithoutHeader");
 
     sanityCheckAssertion(buf.getLong(), prevBlockOffset, "prevBlocKOffset");
-    if (this.fileContext.shouldUseHBaseChecksum()) {
+    if (this.fileContext.isUseHBaseChecksum()) {
       sanityCheckAssertion(buf.get(), this.fileContext.getChecksumType().getCode(), "checksumType");
       sanityCheckAssertion(buf.getInt(), this.fileContext.getBytesPerChecksum(), "bytesPerChecksum");
       sanityCheckAssertion(buf.getInt(), onDiskDataSizeWithHeader, 
@@ -1024,13 +1024,13 @@ public class HFileBlock implements Cache
                                 .withBlockSize(fileContext.getBlocksize())
                                 .withBytesPerCheckSum(0)
                                 .withChecksumType(ChecksumType.NULL) // no checksums in cached data
-                                .withCompressionAlgo(fileContext.getCompression())
+                                .withCompression(fileContext.getCompression())
                                 .withDataBlockEncodingInCache(fileContext.getEncodingInCache())
                                 .withDataBlockEncodingOnDisk(fileContext.getEncodingOnDisk())
-                                .withHBaseCheckSum(fileContext.shouldUseHBaseChecksum())
-                                .withCompressTags(fileContext.shouldCompressTags())
-                                .withIncludesMvcc(fileContext.shouldIncludeMvcc())
-                                .withIncludesTags(fileContext.shouldIncludeTags())
+                                .withHBaseCheckSum(fileContext.isUseHBaseChecksum())
+                                .withCompressTags(fileContext.isCompressTags())
+                                .withIncludesMvcc(fileContext.isIncludesMvcc())
+                                .withIncludesTags(fileContext.isIncludesTags())
                                 .build();
       return new HFileBlock(blockType, getOnDiskSizeWithoutHeader(),
           getUncompressedSizeWithoutHeader(), prevOffset, getUncompressedBufferWithHeader(),
@@ -1135,7 +1135,7 @@ public class HFileBlock implements Cache
       this.hfs = hfs;
       this.path = path;
       this.fileContext = fileContext;
-      this.hdrSize = headerSize(fileContext.shouldUseHBaseChecksum());
+      this.hdrSize = headerSize(fileContext.isUseHBaseChecksum());
     }
 
     @Override
@@ -1277,7 +1277,7 @@ public class HFileBlock implements Cache
       super(fileSize, hfs, path, fileContext);
       this.streamWrapper = stream;
       // Older versions of HBase didn't support checksum.
-      this.streamWrapper.prepareForBlockReader(!fileContext.shouldUseHBaseChecksum());
+      this.streamWrapper.prepareForBlockReader(!fileContext.isUseHBaseChecksum());
       defaultDecodingCtx =
         new HFileBlockDefaultDecodingContext(fileContext);
       encodedBlockDecodingCtx =
@@ -1449,7 +1449,7 @@ public class HFileBlock implements Cache
         // from memory if using compression. Here we have already read the
         // block's header
         try {
-          b = new HFileBlock(headerBuf, this.fileContext.shouldUseHBaseChecksum());
+          b = new HFileBlock(headerBuf, this.fileContext.isUseHBaseChecksum());
         } catch (IOException ex) {
           // Seen in load testing. Provide comprehensive debug info.
           throw new IOException("Failed to read compressed block at "
@@ -1487,7 +1487,7 @@ public class HFileBlock implements Cache
           readAtOffset(is, headerBuf.array(), headerBuf.arrayOffset(),
               hdrSize, false, offset, pread);
         }
-        b = new HFileBlock(headerBuf, this.fileContext.shouldUseHBaseChecksum());
+        b = new HFileBlock(headerBuf, this.fileContext.isUseHBaseChecksum());
         onDiskBlock = new byte[b.getOnDiskSizeWithHeader() + hdrSize];
         System.arraycopy(headerBuf.array(),
               headerBuf.arrayOffset(), onDiskBlock, 0, hdrSize);
@@ -1534,7 +1534,7 @@ public class HFileBlock implements Cache
         // contains the header of next block, so no need to set next
         // block's header in it.
         b = new HFileBlock(ByteBuffer.wrap(onDiskBlock, 0,
-                onDiskSizeWithHeader), this.fileContext.shouldUseHBaseChecksum());
+                onDiskSizeWithHeader), this.fileContext.isUseHBaseChecksum());
       }
 
       b.nextBlockOnDiskSizeWithHeader = nextBlockOnDiskSize;
@@ -1547,8 +1547,8 @@ public class HFileBlock implements Cache
       }
 
       b.offset = offset;
-      b.fileContext.setIncludesTags(this.fileContext.shouldIncludeTags());
-      b.fileContext.setIncludesMvcc(this.fileContext.shouldIncludeMvcc());
+      b.fileContext.setIncludesTags(this.fileContext.isIncludesTags());
+      b.fileContext.setIncludesMvcc(this.fileContext.isIncludesMvcc());
       return b;
     }
 
@@ -1596,7 +1596,7 @@ public class HFileBlock implements Cache
   }
 
   public void serializeExtraInfo(ByteBuffer destination) {
-    destination.put(this.fileContext.shouldUseHBaseChecksum() ? (byte) 1 : (byte) 0);
+    destination.put(this.fileContext.isUseHBaseChecksum() ? (byte) 1 : (byte) 0);
     destination.putLong(this.offset);
     destination.putInt(this.nextBlockOnDiskSizeWithHeader);
     destination.rewind();
@@ -1679,7 +1679,7 @@ public class HFileBlock implements Cache
     // data to validate. Similarly, a zero value in this.bytesPerChecksum
     // indicates that cached blocks do not have checksum data because
     // checksums were already validated when the block was read from disk.
-    if (!fileContext.shouldUseHBaseChecksum() || this.fileContext.getBytesPerChecksum() == 0) {
+    if (!fileContext.isUseHBaseChecksum() || this.fileContext.getBytesPerChecksum() == 0) {
       return 0;
     }
     return (int)ChecksumUtil.numBytes(onDiskDataSizeWithHeader, this.fileContext.getBytesPerChecksum());
@@ -1689,7 +1689,7 @@ public class HFileBlock implements Cache
    * Returns the size of this block header.
    */
   public int headerSize() {
-    return headerSize(this.fileContext.shouldUseHBaseChecksum());
+    return headerSize(this.fileContext.isUseHBaseChecksum());
   }
 
   /**
@@ -1706,7 +1706,7 @@ public class HFileBlock implements Cache
    * Return the appropriate DUMMY_HEADER for the minor version
    */
   public byte[] getDummyHeaderForVersion() {
-    return getDummyHeaderForVersion(this.fileContext.shouldUseHBaseChecksum());
+    return getDummyHeaderForVersion(this.fileContext.isUseHBaseChecksum());
   }
 
   /**

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java Mon Nov 11 17:08:04 2013
@@ -182,7 +182,7 @@ public class HFileReaderV2 extends Abstr
   protected HFileContext createHFileContext(FixedFileTrailer trailer) {
     HFileContext hFileContext  = new HFileContextBuilder()
                                  .withIncludesMvcc(this.includesMemstoreTS)
-                                 .withCompressionAlgo(this.compressAlgo)
+                                 .withCompression(this.compressAlgo)
                                  .withHBaseCheckSum(trailer.getMinorVersion() >= MINOR_VERSION_WITH_CHECKSUM)
                                  .build();
     return hFileContext;
@@ -651,14 +651,14 @@ public class HFileReaderV2 extends Abstr
         return null;
 
       KeyValue ret = new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
-          + blockBuffer.position(), getKvBufSize(), currKeyLen);
+          + blockBuffer.position(), getCellBufSize(), currKeyLen);
       if (this.reader.shouldIncludeMemstoreTS()) {
         ret.setMvccVersion(currMemstoreTS);
       }
       return ret;
     }
 
-    protected int getKvBufSize() {
+    protected int getCellBufSize() {
       return KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen;
     }
 
@@ -707,7 +707,7 @@ public class HFileReaderV2 extends Abstr
       assertSeeked();
 
       try {
-        blockBuffer.position(getNextKVStartPosition());
+        blockBuffer.position(getNextCellStartPosition());
       } catch (IllegalArgumentException e) {
         LOG.error("Current pos = " + blockBuffer.position()
             + "; currKeyLen = " + currKeyLen + "; currValLen = "
@@ -742,7 +742,7 @@ public class HFileReaderV2 extends Abstr
       return true;
     }
 
-    protected int getNextKVStartPosition() {
+    protected int getNextCellStartPosition() {
       return blockBuffer.position() + KEY_VALUE_LEN_SIZE + currKeyLen + currValueLen
           + currMemstoreTSLen;
     }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV3.java Mon Nov 11 17:08:04 2013
@@ -76,7 +76,7 @@ public class HFileReaderV3 extends HFile
     HFileContext hfileContext = new HFileContextBuilder()
                                 .withIncludesMvcc(this.includesMemstoreTS)
                                 .withHBaseCheckSum(true)
-                                .withCompressionAlgo(this.compressAlgo)
+                                .withCompression(this.compressAlgo)
                                 .build();
     return hfileContext;
   }
@@ -120,9 +120,9 @@ public class HFileReaderV3 extends HFile
     }
 
     @Override
-    protected int getKvBufSize() {
-      int kvBufSize = super.getKvBufSize();
-      if (reader.hfileContext.shouldIncludeTags()) {
+    protected int getCellBufSize() {
+      int kvBufSize = super.getCellBufSize();
+      if (reader.hfileContext.isIncludesTags()) {
         kvBufSize += Bytes.SIZEOF_SHORT + currTagsLen;
       }
       return kvBufSize;
@@ -134,9 +134,9 @@ public class HFileReaderV3 extends HFile
     }
 
     @Override
-    protected int getNextKVStartPosition() {
-      int nextKvPos = super.getNextKVStartPosition();
-      if (reader.hfileContext.shouldIncludeTags()) {
+    protected int getNextCellStartPosition() {
+      int nextKvPos = super.getNextCellStartPosition();
+      if (reader.hfileContext.isIncludesTags()) {
         nextKvPos += Bytes.SIZEOF_SHORT + currTagsLen;
       }
       return nextKvPos;
@@ -147,7 +147,7 @@ public class HFileReaderV3 extends HFile
       currKeyLen = blockBuffer.getInt();
       currValueLen = blockBuffer.getInt();
       ByteBufferUtils.skip(blockBuffer, currKeyLen + currValueLen);
-      if (reader.hfileContext.shouldIncludeTags()) {
+      if (reader.hfileContext.isIncludesTags()) {
         currTagsLen = blockBuffer.getShort();
         ByteBufferUtils.skip(blockBuffer, currTagsLen);
       }
@@ -191,7 +191,7 @@ public class HFileReaderV3 extends HFile
         klen = blockBuffer.getInt();
         vlen = blockBuffer.getInt();
         ByteBufferUtils.skip(blockBuffer, klen + vlen);
-        if (reader.hfileContext.shouldIncludeTags()) {
+        if (reader.hfileContext.isIncludesTags()) {
           tlen = blockBuffer.getShort();
           ByteBufferUtils.skip(blockBuffer, tlen);
         }
@@ -247,7 +247,7 @@ public class HFileReaderV3 extends HFile
         // The size of this key/value tuple, including key/value length fields.
         lastKeyValueSize = klen + vlen + memstoreTSLen + KEY_VALUE_LEN_SIZE;
         // include tag length also if tags included with KV
-        if (reader.hfileContext.shouldIncludeTags()) {
+        if (reader.hfileContext.isIncludesTags()) {
           lastKeyValueSize += tlen + Bytes.SIZEOF_SHORT;
         }
         blockBuffer.position(blockBuffer.position() + lastKeyValueSize);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java Mon Nov 11 17:08:04 2013
@@ -310,7 +310,7 @@ public class HFileWriterV2 extends Abstr
       totalValueLength += vlength;
       out.write(key, koffset, klength);
       out.write(value, voffset, vlength);
-      if (this.hFileContext.shouldIncludeMvcc()) {
+      if (this.hFileContext.isIncludesMvcc()) {
         WritableUtils.writeVLong(out, memstoreTS);
       }
     }
@@ -379,7 +379,7 @@ public class HFileWriterV2 extends Abstr
     fsBlockWriter.writeHeaderAndData(outputStream);
     totalUncompressedBytes += fsBlockWriter.getUncompressedSizeWithHeader();
 
-    if (this.hFileContext.shouldIncludeMvcc()) {
+    if (this.hFileContext.isIncludesMvcc()) {
       appendFileInfo(MAX_MEMSTORE_TS_KEY, Bytes.toBytes(maxMemstoreTS));
       appendFileInfo(KEY_VALUE_VERSION, Bytes.toBytes(KEY_VALUE_VER_WITH_MEMSTORE));
     }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV3.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV3.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV3.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV3.java Mon Nov 11 17:08:04 2013
@@ -146,7 +146,7 @@ public class HFileWriterV3 extends HFile
       out.write(key, koffset, klength);
       out.write(value, voffset, vlength);
       // Write the additional tag into the stream
-      if (hFileContext.shouldIncludeTags()) {
+      if (hFileContext.isIncludesTags()) {
         out.writeShort((short) tagsLength);
         if (tagsLength > 0) {
           out.write(tag, tagsOffset, tagsLength);
@@ -155,7 +155,7 @@ public class HFileWriterV3 extends HFile
           }
         }
       }
-      if (this.hFileContext.shouldIncludeMvcc()) {
+      if (this.hFileContext.isIncludesMvcc()) {
         WritableUtils.writeVLong(out, memstoreTS);
       }
     }
@@ -175,12 +175,12 @@ public class HFileWriterV3 extends HFile
   
   protected void finishFileInfo() throws IOException {
     super.finishFileInfo();
-    if (hFileContext.shouldIncludeTags()) {
+    if (hFileContext.isIncludesTags()) {
       // When tags are not being written in this file, MAX_TAGS_LEN is excluded
       // from the FileInfo
       fileInfo.append(FileInfo.MAX_TAGS_LEN, Bytes.toBytes(this.maxTagsLength), false);
       boolean tagsCompressed = (hFileContext.getEncodingOnDisk() != DataBlockEncoding.NONE)
-          && hFileContext.shouldCompressTags();
+          && hFileContext.isCompressTags();
       fileInfo.append(FileInfo.TAGS_COMPRESSED, Bytes.toBytes(tagsCompressed), false);
     }
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java Mon Nov 11 17:08:04 2013
@@ -193,7 +193,7 @@ public class HFileOutputFormat extends F
         Configuration tempConf = new Configuration(conf);
         tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
         HFileContextBuilder contextBuilder = new HFileContextBuilder()
-                                    .withCompressionAlgo(AbstractHFileWriter.compressionByName(compression))
+                                    .withCompression(AbstractHFileWriter.compressionByName(compression))
                                     .withChecksumType(HStore.getChecksumType(conf))
                                     .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf))
                                     .withBlockSize(blockSize);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java Mon Nov 11 17:08:04 2013
@@ -77,7 +77,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -659,7 +658,7 @@ public class LoadIncrementalHFiles exten
       Algorithm compression = familyDescriptor.getCompression();
       BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
       HFileContext hFileContext = new HFileContextBuilder()
-                                  .withCompressionAlgo(compression)
+                                  .withCompression(compression)
                                   .withChecksumType(HStore.getChecksumType(conf))
                                   .withBytesPerCheckSum(HStore.getBytesPerChecksum(conf))
                                   .withBlockSize(blocksize)

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java Mon Nov 11 17:08:04 2013
@@ -827,8 +827,8 @@ public class HStore implements Store {
     HFileContext hFileContext = new HFileContextBuilder()
                                 .withIncludesMvcc(includeMVCCReadpoint)
                                 .withIncludesTags(includesTag)
+                                .withCompression(compression)
                                 .withCompressTags(family.shouldCompressTags())
-                                .withCompressionAlgo(compression)
                                 .withChecksumType(checksumType)
                                 .withBytesPerCheckSum(bytesPerChecksum)
                                 .withBlockSize(blocksize)

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompressionTest.java Mon Nov 11 17:08:04 2013
@@ -114,7 +114,7 @@ public class CompressionTest {
   throws Exception {
     Configuration conf = HBaseConfiguration.create();
     HFileContext context = new HFileContextBuilder()
-                           .withCompressionAlgo(AbstractHFileWriter.compressionByName(codec)).build();
+                           .withCompression(AbstractHFileWriter.compressionByName(codec)).build();
     HFile.Writer writer = HFile.getWriterFactoryNoCache(conf)
         .withPath(fs, path)
         .withFileContext(context)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java Mon Nov 11 17:08:04 2013
@@ -81,7 +81,7 @@ public class TestDataBlockEncoders {
                         .withHBaseCheckSum(false)
                         .withIncludesMvcc(includesMemstoreTS)
                         .withIncludesTags(includesTags)
-                        .withCompressionAlgo(algo).build();
+                        .withCompression(algo).build();
     if (encoder != null) {
       return encoder.newDataBlockEncodingContext(encoding,
           HConstants.HFILEBLOCK_DUMMY_HEADER, meta);
@@ -119,7 +119,7 @@ public class TestDataBlockEncoders {
                         .withHBaseCheckSum(false)
                         .withIncludesMvcc(includesMemstoreTS)
                         .withIncludesTags(includesTags)
-                        .withCompressionAlgo(Compression.Algorithm.NONE).build();
+                        .withCompression(Compression.Algorithm.NONE).build();
     actualDataset = encoder.decodeKeyValues(dis, encoder.newDataBlockDecodingContext(meta));
     dataset.rewind();
     actualDataset.rewind();
@@ -224,7 +224,7 @@ public class TestDataBlockEncoders {
                           .withHBaseCheckSum(false)
                           .withIncludesMvcc(includesMemstoreTS)
                           .withIncludesTags(includesTags)
-                          .withCompressionAlgo(Compression.Algorithm.NONE)
+                          .withCompression(Compression.Algorithm.NONE)
                           .build();
       DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(KeyValue.COMPARATOR,
           encoder.newDataBlockDecodingContext(meta));
@@ -280,7 +280,7 @@ public class TestDataBlockEncoders {
                           .withHBaseCheckSum(false)
                           .withIncludesMvcc(includesMemstoreTS)
                           .withIncludesTags(includesTags)
-                          .withCompressionAlgo(Compression.Algorithm.NONE)
+                          .withCompression(Compression.Algorithm.NONE)
                           .build();
       DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(KeyValue.COMPARATOR,
           encoder.newDataBlockDecodingContext(meta));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java Mon Nov 11 17:08:04 2013
@@ -101,7 +101,7 @@ public class TestPrefixTreeEncoding {
                         .withHBaseCheckSum(false)
                         .withIncludesMvcc(false)
                         .withIncludesTags(includesTag)
-                        .withCompressionAlgo(Algorithm.NONE).build();
+                        .withCompression(Algorithm.NONE).build();
     HFileBlockEncodingContext blkEncodingCtx = new HFileBlockDefaultEncodingContext(
         DataBlockEncoding.PREFIX_TREE, new byte[0], meta);
     encoder.encodeKeyValues(dataBuffer, blkEncodingCtx);
@@ -142,7 +142,7 @@ public class TestPrefixTreeEncoding {
                         .withHBaseCheckSum(false)
                         .withIncludesMvcc(false)
                         .withIncludesTags(includesTag)
-                        .withCompressionAlgo(Algorithm.NONE)
+                        .withCompression(Algorithm.NONE)
                         .build();
     HFileBlockEncodingContext blkEncodingCtx = new HFileBlockDefaultEncodingContext(
         DataBlockEncoding.PREFIX_TREE, new byte[0], meta);
@@ -179,7 +179,7 @@ public class TestPrefixTreeEncoding {
                         .withHBaseCheckSum(false)
                         .withIncludesMvcc(false)
                         .withIncludesTags(includesTag)
-                        .withCompressionAlgo(Algorithm.NONE)
+                        .withCompression(Algorithm.NONE)
                         .build();
     HFileBlockEncodingContext blkEncodingCtx = new HFileBlockDefaultEncodingContext(
         DataBlockEncoding.PREFIX_TREE, new byte[0], meta);
@@ -201,7 +201,7 @@ public class TestPrefixTreeEncoding {
                         .withHBaseCheckSum(false)
                         .withIncludesMvcc(false)
                         .withIncludesTags(includesTag)
-                        .withCompressionAlgo(Algorithm.NONE)
+                        .withCompression(Algorithm.NONE)
                         .build();
     HFileBlockEncodingContext blkEncodingCtx = new HFileBlockDefaultEncodingContext(
         DataBlockEncoding.PREFIX_TREE, new byte[0], meta);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java Mon Nov 11 17:08:04 2013
@@ -344,7 +344,7 @@ public class CacheTestUtils {
                           .withHBaseCheckSum(false)
                           .withIncludesMvcc(includesMemstoreTS)
                           .withIncludesTags(false)
-                          .withCompressionAlgo(Compression.Algorithm.NONE)
+                          .withCompression(Compression.Algorithm.NONE)
                           .withBytesPerCheckSum(0)
                           .withChecksumType(ChecksumType.NULL)
                           .build();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java Mon Nov 11 17:08:04 2013
@@ -307,7 +307,7 @@ public class TestCacheOnWrite {
     }
     Path storeFileParentDir = new Path(TEST_UTIL.getDataTestDir(),
         "test_cache_on_write");
-    HFileContext meta = new HFileContextBuilder().withCompressionAlgo(compress)
+    HFileContext meta = new HFileContextBuilder().withCompression(compress)
         .withBytesPerCheckSum(CKBYTES).withChecksumType(ChecksumType.NULL)
         .withBlockSize(DATA_BLOCK_SIZE).withDataBlockEncodingInCache(encoder.getEncodingInCache())
         .withDataBlockEncodingOnDisk(encoder.getEncodingOnDisk())

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestChecksum.java Mon Nov 11 17:08:04 2013
@@ -90,7 +90,7 @@ public class TestChecksum {
             + algo);
         FSDataOutputStream os = fs.create(path);
         HFileContext meta = new HFileContextBuilder()
-                            .withCompressionAlgo(algo)
+                            .withCompression(algo)
                             .withIncludesMvcc(true)
                             .withIncludesTags(useTags)
                             .withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
@@ -113,7 +113,7 @@ public class TestChecksum {
         // Do a read that purposely introduces checksum verification failures.
         FSDataInputStreamWrapper is = new FSDataInputStreamWrapper(fs, path);
         meta = new HFileContextBuilder()
-              .withCompressionAlgo(algo)
+              .withCompression(algo)
               .withIncludesMvcc(true)
               .withIncludesTags(useTags)
               .withHBaseCheckSum(true)
@@ -197,7 +197,7 @@ public class TestChecksum {
                              algo + bytesPerChecksum);
         FSDataOutputStream os = fs.create(path);
         HFileContext meta = new HFileContextBuilder()
-                            .withCompressionAlgo(algo)
+                            .withCompression(algo)
                             .withIncludesMvcc(true)
                             .withIncludesTags(useTags)
                             .withHBaseCheckSum(true)
@@ -237,7 +237,7 @@ public class TestChecksum {
         FSDataInputStream is = fs.open(path);
         FSDataInputStream nochecksum = hfs.getNoChecksumFs().open(path);
         meta = new HFileContextBuilder()
-               .withCompressionAlgo(algo)
+               .withCompression(algo)
                .withIncludesMvcc(true)
                .withIncludesTags(useTags)
                .withHBaseCheckSum(true)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java Mon Nov 11 17:08:04 2013
@@ -225,7 +225,7 @@ public class TestHFile extends HBaseTest
     FSDataOutputStream fout = createFSOutput(ncTFile);
     HFileContext meta = new HFileContextBuilder()
                         .withBlockSize(minBlockSize)
-                        .withCompressionAlgo(AbstractHFileWriter.compressionByName(codec))
+                        .withCompression(AbstractHFileWriter.compressionByName(codec))
                         .build();
     Writer writer = HFile.getWriterFactory(conf, cacheConf)
         .withOutputStream(fout)
@@ -314,7 +314,7 @@ public class TestHFile extends HBaseTest
     Path mFile = new Path(ROOT_DIR, "meta.hfile");
     FSDataOutputStream fout = createFSOutput(mFile);
     HFileContext meta = new HFileContextBuilder()
-                        .withCompressionAlgo(AbstractHFileWriter.compressionByName(compress))
+                        .withCompression(AbstractHFileWriter.compressionByName(compress))
                         .withBlockSize(minBlockSize).build();
     Writer writer = HFile.getWriterFactory(conf, cacheConf)
         .withOutputStream(fout)
@@ -347,7 +347,7 @@ public class TestHFile extends HBaseTest
         HBaseTestingUtility.COMPRESSION_ALGORITHMS) {
       Path mFile = new Path(ROOT_DIR, "nometa_" + compressAlgo + ".hfile");
       FSDataOutputStream fout = createFSOutput(mFile);
-      HFileContext meta = new HFileContextBuilder().withCompressionAlgo(compressAlgo)
+      HFileContext meta = new HFileContextBuilder().withCompression(compressAlgo)
                           .withBlockSize(minBlockSize).build();
       Writer writer = HFile.getWriterFactory(conf, cacheConf)
           .withOutputStream(fout)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java Mon Nov 11 17:08:04 2013
@@ -219,7 +219,7 @@ public class TestHFileBlock {
       boolean includesMemstoreTS, boolean includesTag) throws IOException {
     final BlockType blockType = BlockType.DATA;
     HFileContext meta = new HFileContextBuilder()
-                        .withCompressionAlgo(algo)
+                        .withCompression(algo)
                         .withIncludesMvcc(includesMemstoreTS)
                         .withIncludesTags(includesTag)
                         .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
@@ -303,7 +303,7 @@ public class TestHFileBlock {
             + algo);
         FSDataOutputStream os = fs.create(path);
         HFileContext meta = new HFileContextBuilder()
-                           .withCompressionAlgo(algo)
+                           .withCompression(algo)
                            .withIncludesMvcc(includesMemstoreTS)
                            .withIncludesTags(includesTag)
                            .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
@@ -326,7 +326,7 @@ public class TestHFileBlock {
         .withHBaseCheckSum(true)
         .withIncludesMvcc(includesMemstoreTS)
         .withIncludesTags(includesTag)
-        .withCompressionAlgo(algo).build();
+        .withCompression(algo).build();
         HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, totalSize, meta);
         HFileBlock b = hbr.readBlockData(0, -1, -1, pread);
         is.close();
@@ -386,7 +386,7 @@ public class TestHFileBlock {
           HFileDataBlockEncoder dataBlockEncoder =
               new HFileDataBlockEncoderImpl(encoding);
           HFileContext meta = new HFileContextBuilder()
-                              .withCompressionAlgo(algo)
+                              .withCompression(algo)
                               .withIncludesMvcc(includesMemstoreTS)
                               .withIncludesTags(includesTag)
                               .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
@@ -409,7 +409,7 @@ public class TestHFileBlock {
           FSDataInputStream is = fs.open(path);
           meta = new HFileContextBuilder()
                 .withHBaseCheckSum(true)
-                .withCompressionAlgo(algo)
+                .withCompression(algo)
                 .withIncludesMvcc(includesMemstoreTS)
                 .withIncludesTags(includesTag)
                 .build();
@@ -462,7 +462,7 @@ public class TestHFileBlock {
     int headerLen = dummyHeader.length;
     byte[] encodedResultWithHeader = null;
     HFileContext meta = new HFileContextBuilder()
-                        .withCompressionAlgo(algo)
+                        .withCompression(algo)
                         .withIncludesMvcc(includesMemstoreTS)
                         .withIncludesTags(useTag)
                         .build();
@@ -559,7 +559,7 @@ public class TestHFileBlock {
                               .withHBaseCheckSum(true)
                               .withIncludesMvcc(includesMemstoreTS)
                               .withIncludesTags(includesTag)
-                              .withCompressionAlgo(algo).build();
+                              .withCompression(algo).build();
           HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, totalSize, meta);
           long curOffset = 0;
           for (int i = 0; i < NUM_TEST_BLOCKS; ++i) {
@@ -742,7 +742,7 @@ public class TestHFileBlock {
                           .withHBaseCheckSum(true)
                           .withIncludesMvcc(includesMemstoreTS)
                           .withIncludesTags(includesTag)
-                          .withCompressionAlgo(compressAlgo)
+                          .withCompression(compressAlgo)
                           .build();
       HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(is, fileSize, meta);
 
@@ -779,7 +779,7 @@ public class TestHFileBlock {
                         .withHBaseCheckSum(true)
                         .withIncludesMvcc(includesMemstoreTS)
                         .withIncludesTags(includesTag)
-                        .withCompressionAlgo(compressAlgo)
+                        .withCompression(compressAlgo)
                         .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
                         .withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
                         .build();
@@ -850,7 +850,7 @@ public class TestHFileBlock {
                           .withIncludesMvcc(includesMemstoreTS)
                           .withIncludesTags(includesTag)
                           .withHBaseCheckSum(false)
-                          .withCompressionAlgo(Algorithm.NONE)
+                          .withCompression(Algorithm.NONE)
                           .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
                           .withChecksumType(ChecksumType.NULL).build();
       HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockCompatibility.java Mon Nov 11 17:08:04 2013
@@ -201,7 +201,7 @@ public class TestHFileBlockCompatibility
                            .withHBaseCheckSum(false)
                            .withIncludesMvcc(includesMemstoreTS)
                            .withIncludesTags(includesTag)
-                           .withCompressionAlgo(algo)
+                           .withCompression(algo)
                            .build();
         HFileBlock.FSReader hbr = new HFileBlock.FSReaderV2(new FSDataInputStreamWrapper(is),
             totalSize, fs, path, meta);
@@ -284,7 +284,7 @@ public class TestHFileBlockCompatibility
                               .withHBaseCheckSum(false)
                               .withIncludesMvcc(includesMemstoreTS)
                               .withIncludesTags(includesTag)
-                              .withCompressionAlgo(algo)
+                              .withCompression(algo)
                               .build();
           HFileBlock.FSReaderV2 hbr = new HFileBlock.FSReaderV2(new FSDataInputStreamWrapper(is),
               totalSize, fs, path, meta);
@@ -425,7 +425,7 @@ public class TestHFileBlockCompatibility
               .withHBaseCheckSum(false)
               .withIncludesMvcc(includesMemstoreTS)
               .withIncludesTags(includesTag)
-              .withCompressionAlgo(compressionAlgorithm)
+              .withCompression(compressionAlgorithm)
               .build();
       defaultBlockEncodingCtx = new HFileBlockDefaultEncodingContext(null, DUMMY_HEADER, meta);
       dataBlockEncodingCtx =

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java Mon Nov 11 17:08:04 2013
@@ -192,7 +192,7 @@ public class TestHFileBlockIndex {
                         .withHBaseCheckSum(true)
                         .withIncludesMvcc(includesMemstoreTS)
                         .withIncludesTags(useTags)
-                        .withCompressionAlgo(compr)
+                        .withCompression(compr)
                         .build();
     HFileBlock.FSReader blockReader = new HFileBlock.FSReaderV2(istream, fs.getFileStatus(path)
         .getLen(), meta);
@@ -246,7 +246,7 @@ public class TestHFileBlockIndex {
                         .withHBaseCheckSum(true)
                         .withIncludesMvcc(includesMemstoreTS)
                         .withIncludesTags(useTags)
-                        .withCompressionAlgo(compr)
+                        .withCompression(compr)
                         .withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
                         .withBytesPerCheckSum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
                         .build();
@@ -520,7 +520,7 @@ public class TestHFileBlockIndex {
       {
         HFileContext meta = new HFileContextBuilder()
                             .withBlockSize(SMALL_BLOCK_SIZE)
-                            .withCompressionAlgo(compr)
+                            .withCompression(compr)
                             .build();
         HFile.Writer writer =
             HFile.getWriterFactory(conf, cacheConf)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileDataBlockEncoder.java Mon Nov 11 17:08:04 2013
@@ -126,7 +126,7 @@ public class TestHFileDataBlockEncoder {
     buf.position(headerSize);
     keyValues.rewind();
     buf.put(keyValues);
-    HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false)
+    HFileContext hfileContext = new HFileContextBuilder().withHBaseCheckSum(false)
                         .withIncludesMvcc(includesMemstoreTS)
                         .withIncludesTags(useTags)
                         .withBlockSize(0)
@@ -134,7 +134,7 @@ public class TestHFileDataBlockEncoder {
                         .build();
     HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf,
         HFileBlock.FILL_HEADER, 0,
-        0, meta);
+        0, hfileContext);
     HFileBlock cacheBlock = blockEncoder
         .diskToCacheFormat(createBlockOnDisk(block, useTags), false);
     assertEquals(headerSize, cacheBlock.getDummyHeaderForVersion().length);
@@ -205,7 +205,7 @@ public class TestHFileDataBlockEncoder {
                         .withIncludesMvcc(includesMemstoreTS)
                         .withIncludesTags(useTag)
                         .withHBaseCheckSum(true)
-                        .withCompressionAlgo(Algorithm.NONE)
+                        .withCompression(Algorithm.NONE)
                         .withBlockSize(0)
                         .withChecksumType(ChecksumType.NULL)
                         .build();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFilePerformance.java Mon Nov 11 17:08:04 2013
@@ -165,7 +165,7 @@ public class TestHFilePerformance extend
 
     if ("HFile".equals(fileType)){
         HFileContext meta = new HFileContextBuilder()
-                            .withCompressionAlgo(AbstractHFileWriter.compressionByName(codecName))
+                            .withCompression(AbstractHFileWriter.compressionByName(codecName))
                             .withBlockSize(minBlockSize).build();
         System.out.println("HFile write method: ");
         HFile.Writer writer = HFile.getWriterFactoryNoCache(conf)
@@ -415,4 +415,4 @@ public class TestHFilePerformance extend
     int ret = ToolRunner.run(HBaseConfiguration.create(), new TestHFilePerformance(), args);
     System.exit(ret);
   }
-}
\ No newline at end of file
+}

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java Mon Nov 11 17:08:04 2013
@@ -129,7 +129,7 @@ public class TestHFileSeek extends TestC
     try {
       HFileContext context = new HFileContextBuilder()
                             .withBlockSize(options.minBlockSize)
-                            .withCompressionAlgo(AbstractHFileWriter.compressionByName(options.compress))
+                            .withCompression(AbstractHFileWriter.compressionByName(options.compress))
                             .build();
       Writer writer = HFile.getWriterFactoryNoCache(conf)
           .withOutputStream(fout)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV2.java Mon Nov 11 17:08:04 2013
@@ -95,7 +95,7 @@ public class TestHFileWriterV2 {
 
     HFileContext context = new HFileContextBuilder()
                            .withBlockSize(4096)
-                           .withCompressionAlgo(compressAlgo)
+                           .withCompression(compressAlgo)
                            .build();
     HFileWriterV2 writer = (HFileWriterV2)
         new HFileWriterV2.WriterFactoryV2(conf, new CacheConfig(conf))
@@ -141,7 +141,7 @@ public class TestHFileWriterV2 {
                         .withHBaseCheckSum(true)
                         .withIncludesMvcc(false)
                         .withIncludesTags(false)
-                        .withCompressionAlgo(compressAlgo)
+                        .withCompression(compressAlgo)
                         .build();
     
     HFileBlock.FSReader blockReader = new HFileBlock.FSReaderV2(fsdis, fileSize, meta);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java Mon Nov 11 17:08:04 2013
@@ -118,7 +118,7 @@ public class TestHFileWriterV3 {
     HFileContext context = new HFileContextBuilder()
                            .withBlockSize(4096)
                            .withIncludesTags(useTags)
-                           .withCompressionAlgo(compressAlgo).build();
+                           .withCompression(compressAlgo).build();
     HFileWriterV3 writer = (HFileWriterV3)
         new HFileWriterV3.WriterFactoryV3(conf, new CacheConfig(conf))
             .withPath(fs, hfilePath)
@@ -170,7 +170,7 @@ public class TestHFileWriterV3 {
     assertEquals(3, trailer.getMajorVersion());
     assertEquals(entryCount, trailer.getEntryCount());
     HFileContext meta = new HFileContextBuilder()
-                        .withCompressionAlgo(compressAlgo)
+                        .withCompression(compressAlgo)
                         .withIncludesMvcc(false)
                         .withIncludesTags(useTags)
                         .withHBaseCheckSum(true).build();

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java Mon Nov 11 17:08:04 2013
@@ -270,7 +270,7 @@ public class TestLoadIncrementalHFiles {
   {
     HFileContext meta = new HFileContextBuilder()
                         .withBlockSize(BLOCKSIZE)
-                        .withCompressionAlgo(COMPRESSION)
+                        .withCompression(COMPRESSION)
                         .build();
     HFile.Writer writer = HFile.getWriterFactory(configuration, new CacheConfig(configuration))
         .withPath(fs, path)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java Mon Nov 11 17:08:04 2013
@@ -183,7 +183,7 @@ public class CreateRandomStoreFile {
           Integer.valueOf(cmdLine.getOptionValue(INDEX_BLOCK_SIZE_OPTION)));
     }
 
-    HFileContext meta = new HFileContextBuilder().withCompressionAlgo(compr)
+    HFileContext meta = new HFileContextBuilder().withCompression(compr)
                         .withBlockSize(blockSize).build();
     StoreFile.Writer sfw = new StoreFile.WriterBuilder(conf,
         new CacheConfig(conf), fs)

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java?rev=1540785&r1=1540784&r2=1540785&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java Mon Nov 11 17:08:04 2013
@@ -216,7 +216,7 @@ public class DataBlockEncodingTool {
       }
       DataBlockEncoder d = encoding.getEncoder();
       HFileContext meta = new HFileContextBuilder()
-                          .withCompressionAlgo(Compression.Algorithm.NONE)
+                          .withCompression(Compression.Algorithm.NONE)
                           .withIncludesMvcc(includesMemstoreTS)
                           .withIncludesTags(useTag).build();
       codecs.add(new EncodedDataBlock(d, encoding, rawKVs, meta ));