You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2016/09/08 05:55:36 UTC

hbase git commit: HBASE-16530 Reduce DBE code duplication

Repository: hbase
Updated Branches:
  refs/heads/master e65817ef1 -> fc224ed0e


HBASE-16530 Reduce DBE code duplication

Signed-off-by: Yu Li <li...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/fc224ed0
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/fc224ed0
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/fc224ed0

Branch: refs/heads/master
Commit: fc224ed0eda20f876b3ecd3a858e919afa30ec12
Parents: e65817e
Author: binlijin <bi...@gmail.com>
Authored: Thu Sep 8 13:44:41 2016 +0800
Committer: Yu Li <li...@apache.org>
Committed: Thu Sep 8 13:48:27 2016 +0800

----------------------------------------------------------------------
 .../io/encoding/AbstractDataBlockEncoder.java   | 83 ++++++++++++++++++++
 .../io/encoding/BufferedDataBlockEncoder.java   | 48 ++---------
 .../io/encoding/CopyKeyDataBlockEncoder.java    | 53 +++++++------
 .../hadoop/hbase/io/encoding/NoneEncoder.java   | 68 ++++++++++++++++
 .../hbase/io/encoding/RowIndexCodecV1.java      | 31 +-------
 .../hbase/io/encoding/RowIndexEncoderV1.java    | 35 ++-------
 .../hbase/io/encoding/RowIndexSeekerV1.java     | 18 +----
 .../hbase/io/hfile/NoOpDataBlockEncoder.java    | 59 +++++++-------
 8 files changed, 221 insertions(+), 174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fc224ed0/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/AbstractDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/AbstractDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/AbstractDataBlockEncoder.java
new file mode 100644
index 0000000..8a3dadd
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/AbstractDataBlockEncoder.java
@@ -0,0 +1,83 @@
+/*
+ * 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.encoding;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.hfile.BlockType;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+
+@InterfaceAudience.Private
+public abstract class AbstractDataBlockEncoder implements DataBlockEncoder {
+
+  @Override
+  public HFileBlockEncodingContext newDataBlockEncodingContext(
+      DataBlockEncoding encoding, byte[] header, HFileContext meta) {
+    return new HFileBlockDefaultEncodingContext(encoding, header, meta);
+  }
+
+  @Override
+  public HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta) {
+    return new HFileBlockDefaultDecodingContext(meta);
+  }
+
+  protected void postEncoding(HFileBlockEncodingContext encodingCtx)
+      throws IOException {
+    if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) {
+      encodingCtx.postEncoding(BlockType.ENCODED_DATA);
+    } else {
+      encodingCtx.postEncoding(BlockType.DATA);
+    }
+  }
+
+  protected Cell createFirstKeyCell(ByteBuffer key, int keyLength) {
+    if (key.hasArray()) {
+      return new KeyValue.KeyOnlyKeyValue(key.array(), key.arrayOffset()
+          + key.position(), keyLength);
+    } else {
+      return new ByteBufferedKeyOnlyKeyValue(key, key.position(), keyLength);
+    }
+  }
+
+  protected abstract static class AbstractEncodedSeeker implements
+      EncodedSeeker {
+    protected HFileBlockDecodingContext decodingCtx;
+    protected final CellComparator comparator;
+
+    public AbstractEncodedSeeker(CellComparator comparator,
+        HFileBlockDecodingContext decodingCtx) {
+      this.comparator = comparator;
+      this.decodingCtx = decodingCtx;
+    }
+
+    protected boolean includesMvcc() {
+      return this.decodingCtx.getHFileContext().isIncludesMvcc();
+    }
+
+    protected boolean includesTags() {
+      return this.decodingCtx.getHFileContext().isIncludesTags();
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc224ed0/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
index e3882cd..43ae020 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
@@ -23,7 +23,6 @@ import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.ByteBufferedCell;
-import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -36,8 +35,6 @@ import org.apache.hadoop.hbase.Streamable;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
-import org.apache.hadoop.hbase.io.hfile.BlockType;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.nio.ByteBuff;
@@ -51,7 +48,7 @@ import org.apache.hadoop.io.WritableUtils;
  * Base class for all data block encoders that use a buffer.
  */
 @InterfaceAudience.Private
-abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
+abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
   /**
    * TODO: This datablockencoder is dealing in internals of hfileblocks. Purge reference to HFBs
    */
@@ -682,11 +679,8 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
     }
   }
 
-  protected abstract static class
-      BufferedEncodedSeeker<STATE extends SeekerState>
-      implements EncodedSeeker {
-    protected HFileBlockDecodingContext decodingCtx;
-    protected final CellComparator comparator;
+  protected abstract static class BufferedEncodedSeeker<STATE extends SeekerState>
+      extends AbstractEncodedSeeker {
     protected ByteBuff currentBuffer;
     protected TagCompressionContext tagCompressionContext = null;
     protected  KeyValue.KeyOnlyKeyValue keyOnlyKV = new KeyValue.KeyOnlyKeyValue();
@@ -697,8 +691,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
 
     public BufferedEncodedSeeker(CellComparator comparator,
         HFileBlockDecodingContext decodingCtx) {
-      this.comparator = comparator;
-      this.decodingCtx = decodingCtx;
+      super(comparator, decodingCtx);
       if (decodingCtx.getHFileContext().isCompressTags()) {
         try {
           tagCompressionContext = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
@@ -710,14 +703,6 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
       previous = createSeekerState(); // may not be valid
     }
 
-    protected boolean includesMvcc() {
-      return this.decodingCtx.getHFileContext().isIncludesMvcc();
-    }
-
-    protected boolean includesTags() {
-      return this.decodingCtx.getHFileContext().isIncludesTags();
-    }
-
     @Override
     public int compareKey(CellComparator comparator, Cell key) {
       keyOnlyKV.setKey(current.keyBuffer, 0, current.keyLength);
@@ -1049,17 +1034,6 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
     }
   }
 
-  @Override
-  public HFileBlockEncodingContext newDataBlockEncodingContext(DataBlockEncoding encoding,
-      byte[] header, HFileContext meta) {
-    return new HFileBlockDefaultEncodingContext(encoding, header, meta);
-  }
-
-  @Override
-  public HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta) {
-    return new HFileBlockDefaultDecodingContext(meta);
-  }
-
   protected abstract ByteBuffer internalDecodeKeyValues(DataInputStream source,
       int allocateHeaderLength, int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx)
       throws IOException;
@@ -1139,19 +1113,7 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
     Bytes.putInt(uncompressedBytesWithHeader,
       HConstants.HFILEBLOCK_HEADER_SIZE + DataBlockEncoding.ID_SIZE, state.unencodedDataSizeWritten
         );
-    if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) {
-      encodingCtx.postEncoding(BlockType.ENCODED_DATA);
-    } else {
-      encodingCtx.postEncoding(BlockType.DATA);
-    }
+    postEncoding(encodingCtx);
   }
 
-  protected Cell createFirstKeyCell(ByteBuffer key, int keyLength) {
-    if (key.hasArray()) {
-      return new KeyValue.KeyOnlyKeyValue(key.array(), key.arrayOffset() + key.position(),
-          keyLength);
-    } else {
-      return new ByteBufferedKeyOnlyKeyValue(key, key.position(), keyLength);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc224ed0/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java
index 178f65d..2adff48 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/CopyKeyDataBlockEncoder.java
@@ -23,14 +23,10 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.WritableUtils;
 
 /**
  * Just copy data, do not do any kind of compression. Use for comparison and
@@ -39,31 +35,36 @@ import org.apache.hadoop.io.WritableUtils;
 @InterfaceAudience.Private
 public class CopyKeyDataBlockEncoder extends BufferedDataBlockEncoder {
 
+  private static class CopyKeyEncodingState extends EncodingState {
+    NoneEncoder encoder = null;
+  }
+
   @Override
-  public int internalEncode(Cell cell, HFileBlockDefaultEncodingContext encodingContext,
+  public void startBlockEncoding(HFileBlockEncodingContext blkEncodingCtx,
       DataOutputStream out) throws IOException {
-    int klength = KeyValueUtil.keyLength(cell);
-    int vlength = cell.getValueLength();
-
-    out.writeInt(klength);
-    out.writeInt(vlength);
-    CellUtil.writeFlatKey(cell, out);
-    CellUtil.writeValue(out, cell, vlength);
-    int size = klength + vlength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
-    // Write the additional tag into the stream
-    if (encodingContext.getHFileContext().isIncludesTags()) {
-      int tagsLength = cell.getTagsLength();
-      out.writeShort(tagsLength);
-      if (tagsLength > 0) {
-        CellUtil.writeTags(out, cell, tagsLength);
-      }
-      size += tagsLength + KeyValue.TAGS_LENGTH_SIZE;
-    }
-    if (encodingContext.getHFileContext().isIncludesMvcc()) {
-      WritableUtils.writeVLong(out, cell.getSequenceId());
-      size += WritableUtils.getVIntSize(cell.getSequenceId());
+    if (blkEncodingCtx.getClass() != HFileBlockDefaultEncodingContext.class) {
+      throw new IOException(this.getClass().getName() + " only accepts "
+          + HFileBlockDefaultEncodingContext.class.getName() + " as the "
+          + "encoding context.");
     }
-    return size;
+
+    HFileBlockDefaultEncodingContext encodingCtx = (HFileBlockDefaultEncodingContext) blkEncodingCtx;
+    encodingCtx.prepareEncoding(out);
+
+    NoneEncoder encoder = new NoneEncoder(out, encodingCtx);
+    CopyKeyEncodingState state = new CopyKeyEncodingState();
+    state.encoder = encoder;
+    blkEncodingCtx.setEncodingState(state);
+  }
+
+  @Override
+  public int internalEncode(Cell cell,
+      HFileBlockDefaultEncodingContext encodingContext, DataOutputStream out)
+      throws IOException {
+    CopyKeyEncodingState state = (CopyKeyEncodingState) encodingContext
+        .getEncodingState();
+    NoneEncoder encoder = state.encoder;
+    return encoder.write(cell);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc224ed0/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/NoneEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/NoneEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/NoneEncoder.java
new file mode 100644
index 0000000..4b8d203
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/NoneEncoder.java
@@ -0,0 +1,68 @@
+/*
+ *
+ * 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.encoding;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.io.WritableUtils;
+
+@InterfaceAudience.Private
+public class NoneEncoder {
+
+  private DataOutputStream out;
+  private HFileBlockDefaultEncodingContext encodingCtx;
+
+  public NoneEncoder(DataOutputStream out,
+      HFileBlockDefaultEncodingContext encodingCtx) {
+    this.out = out;
+    this.encodingCtx = encodingCtx;
+  }
+
+  public int write(Cell cell) throws IOException {
+    int klength = KeyValueUtil.keyLength(cell);
+    int vlength = cell.getValueLength();
+
+    out.writeInt(klength);
+    out.writeInt(vlength);
+    CellUtil.writeFlatKey(cell, out);
+    CellUtil.writeValue(out, cell, vlength);
+    int size = klength + vlength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
+    // Write the additional tag into the stream
+    if (encodingCtx.getHFileContext().isIncludesTags()) {
+      int tagsLength = cell.getTagsLength();
+      out.writeShort(tagsLength);
+      if (tagsLength > 0) {
+        CellUtil.writeTags(out, cell, tagsLength);
+      }
+      size += tagsLength + KeyValue.TAGS_LENGTH_SIZE;
+    }
+    if (encodingCtx.getHFileContext().isIncludesMvcc()) {
+      WritableUtils.writeVLong(out, cell.getSequenceId());
+      size += WritableUtils.getVIntSize(cell.getSequenceId());
+    }
+    return size;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc224ed0/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java
index 8877032..d81bb4a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexCodecV1.java
@@ -23,15 +23,12 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
-import org.apache.hadoop.hbase.io.hfile.BlockType;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.nio.SingleByteBuff;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
@@ -51,7 +48,7 @@ import org.apache.hadoop.io.WritableUtils;
  *
 */
 @InterfaceAudience.Private
-public class RowIndexCodecV1 implements DataBlockEncoder {
+public class RowIndexCodecV1 extends AbstractDataBlockEncoder {
 
   private static class RowIndexEncodingState extends EncodingState {
     RowIndexEncoderV1 encoder = null;
@@ -92,11 +89,7 @@ public class RowIndexCodecV1 implements DataBlockEncoder {
         .getEncodingState();
     RowIndexEncoderV1 encoder = state.encoder;
     encoder.flush();
-    if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) {
-      encodingCtx.postEncoding(BlockType.ENCODED_DATA);
-    } else {
-      encodingCtx.postEncoding(BlockType.DATA);
-    }
+    postEncoding(encodingCtx);
   }
 
   @Override
@@ -139,17 +132,6 @@ public class RowIndexCodecV1 implements DataBlockEncoder {
   }
 
   @Override
-  public HFileBlockEncodingContext newDataBlockEncodingContext(
-      DataBlockEncoding encoding, byte[] header, HFileContext meta) {
-    return new HFileBlockDefaultEncodingContext(encoding, header, meta);
-  }
-
-  @Override
-  public HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta) {
-    return new HFileBlockDefaultDecodingContext(meta);
-  }
-
-  @Override
   public Cell getFirstKeyCellInBlock(ByteBuff block) {
     block.mark();
     int keyLength = block.getInt();
@@ -165,13 +147,4 @@ public class RowIndexCodecV1 implements DataBlockEncoder {
     return new RowIndexSeekerV1(comparator, decodingCtx);
   }
 
-  protected Cell createFirstKeyCell(ByteBuffer key, int keyLength) {
-    if (key.hasArray()) {
-      return new KeyValue.KeyOnlyKeyValue(key.array(), key.arrayOffset()
-          + key.position(), keyLength);
-    } else {
-      return new ByteBufferedKeyOnlyKeyValue(key, key.position(), keyLength);
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc224ed0/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java
index 7c69aea..0946812 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexEncoderV1.java
@@ -17,12 +17,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
-import org.apache.hadoop.io.WritableUtils;
 
 @InterfaceAudience.Private
 public class RowIndexEncoderV1 {
@@ -32,14 +28,15 @@ public class RowIndexEncoderV1 {
   private Cell lastCell = null;
 
   private DataOutputStream out;
-  private HFileBlockDefaultEncodingContext encodingCtx;
+  private NoneEncoder encoder;
   private int startOffset = -1;
   private ByteArrayOutputStream rowsOffsetBAOS = new ByteArrayOutputStream(
       64 * 4);
 
-  public RowIndexEncoderV1(DataOutputStream out, HFileBlockDefaultEncodingContext encodingCtx) {
+  public RowIndexEncoderV1(DataOutputStream out,
+      HFileBlockDefaultEncodingContext encodingCtx) {
     this.out = out;
-    this.encodingCtx = encodingCtx;
+    this.encoder = new NoneEncoder(out, encodingCtx);
   }
 
   public int write(Cell cell) throws IOException {
@@ -50,30 +47,8 @@ public class RowIndexEncoderV1 {
       }
       rowsOffsetBAOS.writeInt(out.size() - startOffset);
     }
-    int klength = KeyValueUtil.keyLength(cell);
-    int vlength = cell.getValueLength();
-    out.writeInt(klength);
-    out.writeInt(vlength);
-    CellUtil.writeFlatKey(cell, out);
-    // Write the value part
-    CellUtil.writeValue(out, cell, vlength);
-    int encodedKvSize = klength + vlength
-        + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
-    // Write the additional tag into the stream
-    if (encodingCtx.getHFileContext().isIncludesTags()) {
-      int tagsLength = cell.getTagsLength();
-      out.writeShort(tagsLength);
-      if (tagsLength > 0) {
-        CellUtil.writeTags(out, cell, tagsLength);
-      }
-      encodedKvSize += tagsLength + KeyValue.TAGS_LENGTH_SIZE;
-    }
-    if (encodingCtx.getHFileContext().isIncludesMvcc()) {
-      WritableUtils.writeVLong(out, cell.getSequenceId());
-      encodedKvSize += WritableUtils.getVIntSize(cell.getSequenceId());
-    }
     lastCell = cell;
-    return encodedKvSize;
+    return encoder.write(cell);
   }
 
   protected boolean checkRow(final Cell cell) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc224ed0/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
index 3bdc580..389ce01 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
@@ -29,17 +29,14 @@ import org.apache.hadoop.hbase.OffheapKeyValue;
 import org.apache.hadoop.hbase.SizeCachedKeyValue;
 import org.apache.hadoop.hbase.SizeCachedNoTagsKeyValue;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder.EncodedSeeker;
+import org.apache.hadoop.hbase.io.encoding.AbstractDataBlockEncoder.AbstractEncodedSeeker;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ObjectIntPair;
 
 @InterfaceAudience.Private
-public class RowIndexSeekerV1 implements EncodedSeeker {
-
-  private HFileBlockDecodingContext decodingCtx;
-  private final CellComparator comparator;
+public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
 
   // A temp pair object which will be reused by ByteBuff#asSubByteBuffer calls. This avoids too
   // many object creations.
@@ -54,8 +51,7 @@ public class RowIndexSeekerV1 implements EncodedSeeker {
 
   public RowIndexSeekerV1(CellComparator comparator,
       HFileBlockDecodingContext decodingCtx) {
-    this.comparator = comparator;
-    this.decodingCtx = decodingCtx;
+    super(comparator, decodingCtx);
   }
 
   @Override
@@ -293,14 +289,6 @@ public class RowIndexSeekerV1 implements EncodedSeeker {
         current.keyLength);
   }
 
-  protected boolean includesMvcc() {
-    return this.decodingCtx.getHFileContext().isIncludesMvcc();
-  }
-
-  protected boolean includesTags() {
-    return this.decodingCtx.getHFileContext().isIncludesTags();
-  }
-
   protected void decodeTags() {
     current.tagsLength = currentBuffer.getShortAfterPosition(0);
     currentBuffer.skip(Bytes.SIZEOF_SHORT);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fc224ed0/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java
index f5e2b61..1d7cfbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/NoOpDataBlockEncoder.java
@@ -19,17 +19,15 @@ package org.apache.hadoop.hbase.io.hfile;
 import java.io.DataOutputStream;
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.encoding.EncodingState;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
-import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.hbase.io.encoding.NoneEncoder;
 
 /**
  * Does not perform any kind of encoding/decoding.
@@ -40,35 +38,21 @@ public class NoOpDataBlockEncoder implements HFileDataBlockEncoder {
   public static final NoOpDataBlockEncoder INSTANCE =
       new NoOpDataBlockEncoder();
 
+  private static class NoneEncodingState extends EncodingState {
+    NoneEncoder encoder = null;
+  }
+
   /** Cannot be instantiated. Use {@link #INSTANCE} instead. */
   private NoOpDataBlockEncoder() {
   }
 
   @Override
-  public int encode(Cell cell, HFileBlockEncodingContext encodingCtx, DataOutputStream out)
-      throws IOException {
-    int klength = KeyValueUtil.keyLength(cell);
-    int vlength = cell.getValueLength();
-
-    out.writeInt(klength);
-    out.writeInt(vlength);
-    CellUtil.writeFlatKey(cell, out);
-    CellUtil.writeValue(out, cell, vlength);
-    int encodedKvSize = klength + vlength + KeyValue.KEYVALUE_INFRASTRUCTURE_SIZE;
-    // Write the additional tag into the stream
-    if (encodingCtx.getHFileContext().isIncludesTags()) {
-      int tagsLength = cell.getTagsLength();
-      out.writeShort(tagsLength);
-      if (tagsLength > 0) {
-        CellUtil.writeTags(out, cell, tagsLength);
-      }
-      encodedKvSize += tagsLength + KeyValue.TAGS_LENGTH_SIZE;
-    }
-    if (encodingCtx.getHFileContext().isIncludesMvcc()) {
-      WritableUtils.writeVLong(out, cell.getSequenceId());
-      encodedKvSize += WritableUtils.getVIntSize(cell.getSequenceId());
-    }
-    return encodedKvSize;
+  public int encode(Cell cell, HFileBlockEncodingContext encodingCtx,
+      DataOutputStream out) throws IOException {
+    NoneEncodingState state = (NoneEncodingState) encodingCtx
+        .getEncodingState();
+    NoneEncoder encoder = state.encoder;
+    return encoder.write(cell);
   }
 
   @Override
@@ -107,8 +91,21 @@ public class NoOpDataBlockEncoder implements HFileDataBlockEncoder {
   }
 
   @Override
-  public void startBlockEncoding(HFileBlockEncodingContext encodingCtx, DataOutputStream out)
-      throws IOException {
+  public void startBlockEncoding(HFileBlockEncodingContext blkEncodingCtx,
+      DataOutputStream out) throws IOException {
+    if (blkEncodingCtx.getClass() != HFileBlockDefaultEncodingContext.class) {
+      throw new IOException(this.getClass().getName() + " only accepts "
+          + HFileBlockDefaultEncodingContext.class.getName() + " as the "
+          + "encoding context.");
+    }
+
+    HFileBlockDefaultEncodingContext encodingCtx = (HFileBlockDefaultEncodingContext) blkEncodingCtx;
+    encodingCtx.prepareEncoding(out);
+
+    NoneEncoder encoder = new NoneEncoder(out, encodingCtx);
+    NoneEncodingState state = new NoneEncodingState();
+    state.encoder = encoder;
+    blkEncodingCtx.setEncodingState(state);
   }
 
   @Override