You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/03/12 20:11:02 UTC

svn commit: r1455673 [2/5] - in /hbase/branches/0.95: hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/ hbase-prefix-tree/src/main/java/org/apache/hadoop/ hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/ hbase-prefix-tree/src/main...

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,405 @@
+/*
+ * 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.codec.prefixtree.decode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
+import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
+
+import com.google.common.primitives.UnsignedBytes;
+
+/**
+ * Searcher extends the capabilities of the Scanner + ReversibleScanner to add the ability to
+ * position itself on a requested Cell without scanning through cells before it. The PrefixTree is
+ * set up to be a Trie of rows, so finding a particular row is extremely cheap.
+ * <p/>
+ * Once it finds the row, it does a binary search through the cells inside the row, which is not as
+ * fast as the trie search, but faster than iterating through every cell like existing block
+ * formats
+ * do. For this reason, this implementation is targeted towards schemas where rows are narrow
+ * enough
+ * to have several or many per block, and where you are generally looking for the entire row or
+ * the
+ * first cell. It will still be fast for wide rows or point queries, but could be improved upon.
+ */
+@InterfaceAudience.Private
+public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner implements
+    CellSearcher {
+
+  /*************** construct ******************************/
+
+  public PrefixTreeArraySearcher(PrefixTreeBlockMeta blockMeta, int rowTreeDepth,
+      int rowBufferLength, int qualifierBufferLength) {
+    super(blockMeta, rowTreeDepth, rowBufferLength, qualifierBufferLength);
+  }
+
+
+  /********************* CellSearcher methods *******************/
+
+  @Override
+  public boolean positionAt(Cell key) {
+    return CellScannerPosition.AT == positionAtOrAfter(key);
+  }
+
+  @Override
+  public CellScannerPosition positionAtOrBefore(Cell key) {
+    reInitFirstNode();
+    int fanIndex = -1;
+
+    while(true){
+      //detect row mismatch.  break loop if mismatch
+      int currentNodeDepth = rowLength;
+      int rowTokenComparison = compareToCurrentToken(key);
+      if(rowTokenComparison != 0){
+        return fixRowTokenMissReverse(rowTokenComparison);
+      }
+
+      //exact row found, move on to qualifier & ts
+      if(rowMatchesAfterCurrentPosition(key)){
+        return positionAtQualifierTimestamp(key, true);
+      }
+
+      //detect dead end (no fan to descend into)
+      if(!currentRowNode.hasFan()){
+        if(hasOccurrences()){//must be leaf or nub
+          populateLastNonRowFields();
+          return CellScannerPosition.BEFORE;
+        }else{
+          //TODO i don't think this case is exercised by any tests
+          return fixRowFanMissReverse(0);
+        }
+      }
+
+      //keep hunting for the rest of the row
+      byte searchForByte = CellUtil.getRowByte(key, currentNodeDepth);
+      fanIndex = currentRowNode.whichFanNode(searchForByte);
+      if(fanIndex < 0){//no matching row.  return early
+        int insertionPoint = -fanIndex;
+        return fixRowFanMissReverse(insertionPoint);
+      }
+      //found a match, so dig deeper into the tree
+      followFan(fanIndex);
+    }
+  }
+
+  /**
+   * Identical workflow as positionAtOrBefore, but split them to avoid having ~10 extra
+   * if-statements. Priority on readability and debugability.
+   */
+  @Override
+  public CellScannerPosition positionAtOrAfter(Cell key) {
+    reInitFirstNode();
+    int fanIndex = -1;
+
+    while(true){
+      //detect row mismatch.  break loop if mismatch
+      int currentNodeDepth = rowLength;
+      int rowTokenComparison = compareToCurrentToken(key);
+      if(rowTokenComparison != 0){
+        return fixRowTokenMissForward(rowTokenComparison);
+      }
+
+      //exact row found, move on to qualifier & ts
+      if(rowMatchesAfterCurrentPosition(key)){
+        return positionAtQualifierTimestamp(key, false);
+      }
+
+      //detect dead end (no fan to descend into)
+      if(!currentRowNode.hasFan()){
+        if(hasOccurrences()){
+          populateFirstNonRowFields();
+          return CellScannerPosition.AFTER;
+        }else{
+          //TODO i don't think this case is exercised by any tests
+          return fixRowFanMissForward(0);
+        }
+      }
+
+      //keep hunting for the rest of the row
+      byte searchForByte = CellUtil.getRowByte(key, currentNodeDepth);
+      fanIndex = currentRowNode.whichFanNode(searchForByte);
+      if(fanIndex < 0){//no matching row.  return early
+        int insertionPoint = -fanIndex;
+        return fixRowFanMissForward(insertionPoint);
+      }
+      //found a match, so dig deeper into the tree
+      followFan(fanIndex);
+    }
+  }
+
+  @Override
+  public boolean seekForwardTo(Cell key) {
+    if(currentPositionIsAfter(key)){
+      //our position is after the requested key, so can't do anything
+      return false;
+    }
+    return positionAt(key);
+  }
+
+  @Override
+  public CellScannerPosition seekForwardToOrBefore(Cell key) {
+    //Do we even need this check or should upper layers avoid this situation.  It's relatively
+    //expensive compared to the rest of the seek operation.
+    if(currentPositionIsAfter(key)){
+      //our position is after the requested key, so can't do anything
+      return CellScannerPosition.AFTER;
+    }
+
+    return positionAtOrBefore(key);
+  }
+
+  @Override
+  public CellScannerPosition seekForwardToOrAfter(Cell key) {
+    //Do we even need this check or should upper layers avoid this situation.  It's relatively
+    //expensive compared to the rest of the seek operation.
+    if(currentPositionIsAfter(key)){
+      //our position is after the requested key, so can't do anything
+      return CellScannerPosition.AFTER;
+    }
+
+    return positionAtOrAfter(key);
+  }
+
+  /**
+   * The content of the buffers doesn't matter here, only that afterLast=true and beforeFirst=false
+   */
+  @Override
+  public void positionAfterLastCell() {
+    resetToBeforeFirstEntry();
+    beforeFirst = false;
+    afterLast = true;
+  }
+
+
+  /***************** Object methods ***************************/
+
+  @Override
+  public boolean equals(Object obj) {
+    //trivial override to confirm intent (findbugs)
+    return super.equals(obj);
+  }
+
+
+  /****************** internal methods ************************/
+
+  protected boolean currentPositionIsAfter(Cell cell){
+    return compareTo(cell) > 0;
+  }
+
+  protected CellScannerPosition positionAtQualifierTimestamp(Cell key, boolean beforeOnMiss) {
+    int minIndex = 0;
+    int maxIndex = currentRowNode.getLastCellIndex();
+    int diff;
+    while (true) {
+      int midIndex = (maxIndex + minIndex) / 2;//don't worry about overflow
+      diff = populateNonRowFieldsAndCompareTo(midIndex, key);
+
+      if (diff == 0) {// found exact match
+        return CellScannerPosition.AT;
+      } else if (minIndex == maxIndex) {// even termination case
+        break;
+      } else if ((minIndex + 1) == maxIndex) {// odd termination case
+        diff = populateNonRowFieldsAndCompareTo(maxIndex, key);
+        if(diff > 0){
+          diff = populateNonRowFieldsAndCompareTo(minIndex, key);
+        }
+        break;
+      } else if (diff < 0) {// keep going forward
+        minIndex = currentCellIndex;
+      } else {// went past it, back up
+        maxIndex = currentCellIndex;
+      }
+    }
+
+    if (diff == 0) {
+      return CellScannerPosition.AT;
+
+    } else if (diff < 0) {// we are before key
+      if (beforeOnMiss) {
+        return CellScannerPosition.BEFORE;
+      }
+      if (advance()) {
+        return CellScannerPosition.AFTER;
+      }
+      return CellScannerPosition.AFTER_LAST;
+
+    } else {// we are after key
+      if (!beforeOnMiss) {
+        return CellScannerPosition.AFTER;
+      }
+      if (previous()) {
+        return CellScannerPosition.BEFORE;
+      }
+      return CellScannerPosition.BEFORE_FIRST;
+    }
+  }
+
+  /**
+   * compare this.row to key.row but starting at the current rowLength
+   * @param key Cell being searched for
+   * @return true if row buffer contents match key.row
+   */
+  protected boolean rowMatchesAfterCurrentPosition(Cell key) {
+    if (!currentRowNode.hasOccurrences()) {
+      return false;
+    }
+    int thatRowLength = key.getRowLength();
+    if (rowLength != thatRowLength) {
+      return false;
+    }
+    return true;
+  }
+
+  // TODO move part of this to Cell comparator?
+  /**
+   * Compare only the bytes within the window of the current token
+   * @param key
+   * @return return -1 if key is lessThan (before) this, 0 if equal, and 1 if key is after
+   */
+  protected int compareToCurrentToken(Cell key) {
+    int startIndex = rowLength - currentRowNode.getTokenLength();
+    int endIndexExclusive = startIndex + currentRowNode.getTokenLength();
+    for (int i = startIndex; i < endIndexExclusive; ++i) {
+      if (i >= key.getRowLength()) {// key was shorter, so it's first
+        return -1;
+      }
+      byte keyByte = CellUtil.getRowByte(key, i);
+      byte thisByte = rowBuffer[i];
+      if (keyByte == thisByte) {
+        continue;
+      }
+      return UnsignedBytes.compare(keyByte, thisByte);
+    }
+    return 0;
+  }
+
+  protected void followLastFansUntilExhausted(){
+    while(currentRowNode.hasFan()){
+      followLastFan();
+    }
+  }
+
+
+	/****************** complete seek when token mismatch ******************/
+
+  /**
+   * @param searcherIsAfterInputKey <0: input key is before the searcher's position<br/>
+   *          >0: input key is after the searcher's position
+   */
+  protected CellScannerPosition fixRowTokenMissReverse(int searcherIsAfterInputKey) {
+    if (searcherIsAfterInputKey < 0) {//searcher position is after the input key, so back up
+      boolean foundPreviousRow = previousRow(true);
+      if(foundPreviousRow){
+        populateLastNonRowFields();
+        return CellScannerPosition.BEFORE;
+      }else{
+        return CellScannerPosition.BEFORE_FIRST;
+      }
+
+    }else{//searcher position is before the input key
+      if(currentRowNode.hasOccurrences()){
+        populateFirstNonRowFields();
+        return CellScannerPosition.BEFORE;
+      }
+      boolean foundNextRow = nextRow();
+      if(foundNextRow){
+        return CellScannerPosition.AFTER;
+      }else{
+        return CellScannerPosition.AFTER_LAST;
+      }
+    }
+  }
+
+  /**
+   * @param searcherIsAfterInputKey <0: input key is before the searcher's position<br/>
+   *                   >0: input key is after the searcher's position
+   */
+  protected CellScannerPosition fixRowTokenMissForward(int searcherIsAfterInputKey) {
+    if (searcherIsAfterInputKey < 0) {//searcher position is after the input key
+      if(currentRowNode.hasOccurrences()){
+        populateFirstNonRowFields();
+        return CellScannerPosition.AFTER;
+      }
+      boolean foundNextRow = nextRow();
+      if(foundNextRow){
+        return CellScannerPosition.AFTER;
+      }else{
+        return CellScannerPosition.AFTER_LAST;
+      }
+
+    }else{//searcher position is before the input key, so go forward
+      discardCurrentRowNode(true);
+      boolean foundNextRow = nextRow();
+      if(foundNextRow){
+        return CellScannerPosition.AFTER;
+      }else{
+        return CellScannerPosition.AFTER_LAST;
+      }
+    }
+  }
+
+
+  /****************** complete seek when fan mismatch ******************/
+
+  protected CellScannerPosition fixRowFanMissReverse(int fanInsertionPoint){
+    if(fanInsertionPoint == 0){//we need to back up a row
+      boolean foundPreviousRow = previousRow(true);//true -> position on last cell in row
+      if(foundPreviousRow){
+        populateLastNonRowFields();
+        return CellScannerPosition.BEFORE;
+      }
+      return CellScannerPosition.BEFORE_FIRST;
+    }
+
+    //follow the previous fan, but then descend recursively forward
+    followFan(fanInsertionPoint - 1);
+    followLastFansUntilExhausted();
+    populateLastNonRowFields();
+    return CellScannerPosition.BEFORE;
+  }
+
+  protected CellScannerPosition fixRowFanMissForward(int fanInsertionPoint){
+    if(fanInsertionPoint >= currentRowNode.getFanOut()){
+      discardCurrentRowNode(true);
+      if (!nextRow()) {
+        return CellScannerPosition.AFTER_LAST;
+      } else {
+        return CellScannerPosition.AFTER;
+      }
+    }
+
+    followFan(fanInsertionPoint);
+    if(hasOccurrences()){
+      populateFirstNonRowFields();
+      return CellScannerPosition.AFTER;
+    }
+
+    if(nextRowInternal()){
+      populateFirstNonRowFields();
+      return CellScannerPosition.AFTER;
+
+    }else{
+      return CellScannerPosition.AFTER_LAST;
+    }
+  }
+
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,197 @@
+/*
+ * 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.codec.prefixtree.decode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+
+/**
+ * As the PrefixTreeArrayScanner moves through the tree bytes, it changes the values in the fields
+ * of this class so that Cell logic can be applied, but without allocating new memory for every Cell
+ * iterated through.
+ */
+@InterfaceAudience.Private
+public class PrefixTreeCell implements Cell, Comparable<Cell> {
+
+  /********************** static **********************/
+
+  public static final KeyValue.Type[] TYPES = new KeyValue.Type[256];
+  static {
+    for (KeyValue.Type type : KeyValue.Type.values()) {
+      TYPES[type.getCode() & 0xff] = type;
+    }
+  }
+
+  //Same as KeyValue constructor.  Only used to avoid NPE's when full cell hasn't been initialized.
+  public static final KeyValue.Type DEFAULT_TYPE = KeyValue.Type.Put;
+
+  /******************** fields ************************/
+
+  protected byte[] block;
+  //we could also avoid setting the mvccVersion in the scanner/searcher, but this is simpler
+  protected boolean includeMvccVersion;
+
+  protected byte[] rowBuffer;
+  protected int rowLength;
+
+  protected byte[] familyBuffer;
+  protected int familyOffset;
+  protected int familyLength;
+
+  protected byte[] qualifierBuffer;// aligned to the end of the array
+  protected int qualifierOffset;
+  protected int qualifierLength;
+
+  protected Long timestamp;
+  protected Long mvccVersion;
+
+  protected KeyValue.Type type;
+
+  protected int absoluteValueOffset;
+  protected int valueLength;
+
+
+  /********************** Cell methods ******************/
+
+  /**
+   * For debugging.  Currently creates new KeyValue to utilize its toString() method.
+   */
+  @Override
+  public String toString() {
+    return getKeyValueString();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof Cell)) {
+      return false;
+    }
+    //Temporary hack to maintain backwards compatibility with KeyValue.equals
+    return CellComparator.equalsIgnoreMvccVersion(this, (Cell)obj);
+
+    //TODO return CellComparator.equals(this, (Cell)obj);//see HBASE-6907
+  }
+
+  @Override
+  public int hashCode(){
+    //Temporary hack to maintain backwards compatibility with KeyValue.hashCode
+    //I don't think this is used in any hot code paths
+    return KeyValueUtil.copyToNewKeyValue(this).hashCode();
+
+    //TODO return CellComparator.hashCode(this);//see HBASE-6907
+  }
+
+  @Override
+  public int compareTo(Cell other) {
+    return CellComparator.compareStatic(this, other);
+  }
+
+  @Override
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public long getMvccVersion() {
+    if (!includeMvccVersion) {
+      return 0L;
+    }
+    return mvccVersion;
+  }
+
+  @Override
+  public int getValueLength() {
+    return valueLength;
+  }
+
+  @Override
+  public byte[] getRowArray() {
+    return rowBuffer;
+  }
+
+  @Override
+  public int getRowOffset() {
+    return 0;
+  }
+
+  @Override
+  public short getRowLength() {
+    return (short) rowLength;
+  }
+
+  @Override
+  public byte[] getFamilyArray() {
+    return familyBuffer;
+  }
+
+  @Override
+  public int getFamilyOffset() {
+    return familyOffset;
+  }
+
+  @Override
+  public byte getFamilyLength() {
+    return (byte) familyLength;
+  }
+
+  @Override
+  public byte[] getQualifierArray() {
+    return qualifierBuffer;
+  }
+
+  @Override
+  public int getQualifierOffset() {
+    return qualifierOffset;
+  }
+
+  @Override
+  public int getQualifierLength() {
+    return qualifierLength;
+  }
+
+  @Override
+  public byte[] getValueArray() {
+    return block;
+  }
+
+  @Override
+  public int getValueOffset() {
+    return absoluteValueOffset;
+  }
+
+  @Override
+  public byte getTypeByte() {
+    return type.getCode();
+  }
+
+
+  /************************* helper methods *************************/
+
+  /**
+   * Need this separate method so we can call it from subclasses' toString() methods
+   */
+  protected String getKeyValueString(){
+    KeyValue kv = KeyValueUtil.copyToNewKeyValue(this);
+    return kv.toString();
+  }
+
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnNodeReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnNodeReader.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnNodeReader.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnNodeReader.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,104 @@
+/*
+ * 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.codec.prefixtree.decode.column;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.util.vint.UFIntTool;
+import org.apache.hadoop.hbase.util.vint.UVIntTool;
+
+@InterfaceAudience.Private
+public class ColumnNodeReader {
+
+  /**************** fields ************************/
+
+  protected PrefixTreeBlockMeta blockMeta;
+  protected byte[] block;
+
+  protected byte[] columnBuffer;
+  protected boolean familyVsQualifier;
+
+  protected int offsetIntoBlock;
+
+  protected int tokenOffsetIntoBlock;
+  protected int tokenLength;
+  protected int parentStartPosition;
+
+
+  /************** construct *************************/
+
+  public ColumnNodeReader(byte[] columnBuffer, boolean familyVsQualifier) {
+    this.columnBuffer = columnBuffer;
+    this.familyVsQualifier = familyVsQualifier;
+  }
+
+  public void initOnBlock(PrefixTreeBlockMeta blockMeta, byte[] block) {
+    this.blockMeta = blockMeta;
+    this.block = block;
+  }
+
+
+  /************* methods *****************************/
+
+  public void positionAt(int offsetIntoBlock) {
+    this.offsetIntoBlock = offsetIntoBlock;
+    tokenLength = UVIntTool.getInt(block, offsetIntoBlock);
+    tokenOffsetIntoBlock = offsetIntoBlock + UVIntTool.numBytes(tokenLength);
+    int parentStartPositionIndex = tokenOffsetIntoBlock + tokenLength;
+    int offsetWidth;
+    if (familyVsQualifier) {
+      offsetWidth = blockMeta.getFamilyOffsetWidth();
+    } else {
+      offsetWidth = blockMeta.getQualifierOffsetWidth();
+    }
+    parentStartPosition = (int) UFIntTool.fromBytes(block, parentStartPositionIndex, offsetWidth);
+  }
+
+  public void prependTokenToBuffer(int bufferStartIndex) {
+    System.arraycopy(block, tokenOffsetIntoBlock, columnBuffer, bufferStartIndex, tokenLength);
+  }
+
+  public boolean isRoot() {
+    if (familyVsQualifier) {
+      return offsetIntoBlock == blockMeta.getAbsoluteFamilyOffset();
+    } else {
+      return offsetIntoBlock == blockMeta.getAbsoluteQualifierOffset();
+    }
+  }
+
+
+  /************** standard methods *********************/
+
+  @Override
+  public String toString() {
+    return super.toString() + "[" + offsetIntoBlock + "]";
+  }
+
+
+  /****************** get/set ****************************/
+
+  public int getTokenLength() {
+    return tokenLength;
+  }
+
+  public int getParentStartPosition() {
+    return parentStartPosition;
+  }
+
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnReader.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnReader.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnReader.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,104 @@
+/*
+ * 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.codec.prefixtree.decode.column;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+
+/**
+ * Position one of these appropriately in the data block and you can call its methods to retrieve
+ * the family or qualifier at the current position.
+ */
+@InterfaceAudience.Private
+public class ColumnReader {
+
+  /****************** fields *************************/
+
+  protected PrefixTreeBlockMeta blockMeta;
+
+  protected byte[] columnBuffer;
+  protected int columnOffset;
+  protected int columnLength;
+  protected boolean familyVsQualifier;
+
+  protected ColumnNodeReader columnNodeReader;
+
+
+  /******************** construct *******************/
+
+  public ColumnReader(byte[] columnBuffer, boolean familyVsQualifier) {
+    this.columnBuffer = columnBuffer;
+    this.familyVsQualifier = familyVsQualifier;
+    this.columnNodeReader = new ColumnNodeReader(columnBuffer, familyVsQualifier);
+  }
+
+  public void initOnBlock(PrefixTreeBlockMeta blockMeta, byte[] block) {
+    this.blockMeta = blockMeta;
+    clearColumnBuffer();
+    columnNodeReader.initOnBlock(blockMeta, block);
+  }
+
+
+  /********************* methods *******************/
+
+  public ColumnReader populateBuffer(int offsetIntoColumnData) {
+    clearColumnBuffer();
+    int nextRelativeOffset = offsetIntoColumnData;
+    while (true) {
+      int absoluteOffset;
+      if (familyVsQualifier) {
+        absoluteOffset = blockMeta.getAbsoluteFamilyOffset() + nextRelativeOffset;
+      } else {
+        absoluteOffset = blockMeta.getAbsoluteQualifierOffset() + nextRelativeOffset;
+      }
+      columnNodeReader.positionAt(absoluteOffset);
+      columnOffset -= columnNodeReader.getTokenLength();
+      columnLength += columnNodeReader.getTokenLength();
+      columnNodeReader.prependTokenToBuffer(columnOffset);
+      if (columnNodeReader.isRoot()) {
+        return this;
+      }
+      nextRelativeOffset = columnNodeReader.getParentStartPosition();
+    }
+  }
+
+  public byte[] copyBufferToNewArray() {// for testing
+    byte[] out = new byte[columnLength];
+    System.arraycopy(columnBuffer, columnOffset, out, 0, out.length);
+    return out;
+  }
+
+  public int getColumnLength() {
+    return columnLength;
+  }
+
+  public void clearColumnBuffer() {
+    columnOffset = columnBuffer.length;
+    columnLength = 0;
+  }
+
+
+  /****************************** get/set *************************************/
+
+  public int getColumnOffset() {
+    return columnOffset;
+  }
+
+}
+

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,267 @@
+/*
+ * 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.codec.prefixtree.decode.row;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.vint.UFIntTool;
+import org.apache.hadoop.hbase.util.vint.UVIntTool;
+
+/**
+ * Position one of these appropriately in the data block and you can call its methods to retrieve
+ * information necessary to decode the cells in the row.
+ */
+@InterfaceAudience.Private
+public class RowNodeReader {
+
+  /************* fields ***********************************/
+
+  protected byte[] block;
+  protected int offset;
+  protected int fanIndex;
+
+  protected int numCells;
+
+  protected int tokenOffset;
+  protected int tokenLength;
+  protected int fanOffset;
+  protected int fanOut;
+
+  protected int familyOffsetsOffset;
+  protected int qualifierOffsetsOffset;
+  protected int timestampIndexesOffset;
+  protected int mvccVersionIndexesOffset;
+  protected int operationTypesOffset;
+  protected int valueOffsetsOffset;
+  protected int valueLengthsOffset;
+  protected int nextNodeOffsetsOffset;
+
+
+  /******************* construct **************************/
+
+  public void initOnBlock(PrefixTreeBlockMeta blockMeta, byte[] block, int offset) {
+     this.block = block;
+
+    this.offset = offset;
+    resetFanIndex();
+
+    this.tokenLength = UVIntTool.getInt(block, offset);
+    this.tokenOffset = offset + UVIntTool.numBytes(tokenLength);
+
+    this.fanOut = UVIntTool.getInt(block, tokenOffset + tokenLength);
+    this.fanOffset = tokenOffset + tokenLength + UVIntTool.numBytes(fanOut);
+
+    this.numCells = UVIntTool.getInt(block, fanOffset + fanOut);
+
+    this.familyOffsetsOffset = fanOffset + fanOut + UVIntTool.numBytes(numCells);
+    this.qualifierOffsetsOffset = familyOffsetsOffset + numCells * blockMeta.getFamilyOffsetWidth();
+    this.timestampIndexesOffset = qualifierOffsetsOffset + numCells
+        * blockMeta.getQualifierOffsetWidth();
+    this.mvccVersionIndexesOffset = timestampIndexesOffset + numCells
+        * blockMeta.getTimestampIndexWidth();
+    this.operationTypesOffset = mvccVersionIndexesOffset + numCells
+        * blockMeta.getMvccVersionIndexWidth();
+    this.valueOffsetsOffset = operationTypesOffset + numCells * blockMeta.getKeyValueTypeWidth();
+    this.valueLengthsOffset = valueOffsetsOffset + numCells * blockMeta.getValueOffsetWidth();
+    this.nextNodeOffsetsOffset = valueLengthsOffset + numCells * blockMeta.getValueLengthWidth();
+  }
+
+
+  /******************** methods ****************************/
+
+  public boolean isLeaf() {
+    return fanOut == 0;
+  }
+
+  public boolean isNub() {
+    return fanOut > 0 && numCells > 0;
+  }
+
+  public boolean isBranch() {
+    return fanOut > 0 && numCells == 0;
+  }
+
+  public boolean hasOccurrences() {
+    return numCells > 0;
+  }
+
+  public int getTokenArrayOffset(){
+    return tokenOffset;
+  }
+
+  public int getTokenLength() {
+    return tokenLength;
+  }
+
+  public byte getFanByte(int i) {
+    return block[fanOffset + i];
+  }
+  
+  /**
+   * for debugging
+   */
+  protected String getFanByteReadable(int i){
+    return Bytes.toStringBinary(block, fanOffset + i, 1);
+  }
+
+  public int getFamilyOffset(int index, PrefixTreeBlockMeta blockMeta) {
+    int fIntWidth = blockMeta.getFamilyOffsetWidth();
+    int startIndex = familyOffsetsOffset + fIntWidth * index;
+    return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth);
+  }
+
+  public int getColumnOffset(int index, PrefixTreeBlockMeta blockMeta) {
+    int fIntWidth = blockMeta.getQualifierOffsetWidth();
+    int startIndex = qualifierOffsetsOffset + fIntWidth * index;
+    return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth);
+  }
+
+  public int getTimestampIndex(int index, PrefixTreeBlockMeta blockMeta) {
+    int fIntWidth = blockMeta.getTimestampIndexWidth();
+    int startIndex = timestampIndexesOffset + fIntWidth * index;
+    return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth);
+  }
+
+  public int getMvccVersionIndex(int index, PrefixTreeBlockMeta blockMeta) {
+    int fIntWidth = blockMeta.getMvccVersionIndexWidth();
+    int startIndex = mvccVersionIndexesOffset + fIntWidth * index;
+    return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth);
+  }
+
+  public int getType(int index, PrefixTreeBlockMeta blockMeta) {
+    if (blockMeta.isAllSameType()) {
+      return blockMeta.getAllTypes();
+    }
+    return block[operationTypesOffset + index];
+  }
+
+  public int getValueOffset(int index, PrefixTreeBlockMeta blockMeta) {
+    int fIntWidth = blockMeta.getValueOffsetWidth();
+    int startIndex = valueOffsetsOffset + fIntWidth * index;
+    int offset = (int) UFIntTool.fromBytes(block, startIndex, fIntWidth);
+    return offset;
+  }
+
+  public int getValueLength(int index, PrefixTreeBlockMeta blockMeta) {
+    int fIntWidth = blockMeta.getValueLengthWidth();
+    int startIndex = valueLengthsOffset + fIntWidth * index;
+    int length = (int) UFIntTool.fromBytes(block, startIndex, fIntWidth);
+    return length;
+  }
+
+  public int getNextNodeOffset(int index, PrefixTreeBlockMeta blockMeta) {
+    int fIntWidth = blockMeta.getNextNodeOffsetWidth();
+    int startIndex = nextNodeOffsetsOffset + fIntWidth * index;
+    return (int) UFIntTool.fromBytes(block, startIndex, fIntWidth);
+  }
+
+  public String getBranchNubLeafIndicator() {
+    if (isNub()) {
+      return "N";
+    }
+    return isBranch() ? "B" : "L";
+  }
+
+  public boolean hasChildren() {
+    return fanOut > 0;
+  }
+
+  public int getLastFanIndex() {
+    return fanOut - 1;
+  }
+
+  public int getLastCellIndex() {
+    return numCells - 1;
+  }
+
+  public int getNumCells() {
+    return numCells;
+  }
+
+  public int getFanOut() {
+    return fanOut;
+  }
+
+  public byte[] getToken() {
+    // TODO pass in reusable ByteRange
+    return new ByteRange(block, tokenOffset, tokenLength).deepCopyToNewArray();
+  }
+
+  public int getOffset() {
+    return offset;
+  }
+
+  public int whichFanNode(byte searchForByte) {
+    if( ! hasFan()){
+      throw new IllegalStateException("This row node has no fan, so can't search it");
+    }
+    int fanIndexInBlock = Bytes.unsignedBinarySearch(block, fanOffset, fanOffset + fanOut,
+      searchForByte);
+    if (fanIndexInBlock >= 0) {// found it, but need to adjust for position of fan in overall block
+      return fanIndexInBlock - fanOffset;
+    }
+    return fanIndexInBlock + fanOffset + 1;// didn't find it, so compensate in reverse
+  }
+
+  public void resetFanIndex() {
+    fanIndex = -1;// just the way the logic currently works
+  }
+
+  public int getFanIndex() {
+    return fanIndex;
+  }
+
+  public void setFanIndex(int fanIndex) {
+    this.fanIndex = fanIndex;
+  }
+
+  public boolean hasFan(){
+    return fanOut > 0;
+  }
+
+  public boolean hasPreviousFanNodes() {
+    return fanOut > 0 && fanIndex > 0;
+  }
+
+  public boolean hasMoreFanNodes() {
+    return fanIndex < getLastFanIndex();
+  }
+
+  public boolean isOnLastFanNode() {
+    return !hasMoreFanNodes();
+  }
+
+
+  /*************** standard methods **************************/
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("fan:" + Bytes.toStringBinary(block, fanOffset, fanOut));
+    sb.append(",token:" + Bytes.toStringBinary(block, tokenOffset, tokenLength));
+    sb.append(",numCells:" + numCells);
+    sb.append(",fanIndex:"+fanIndex);
+    if(fanIndex>=0){
+      sb.append("("+getFanByteReadable(fanIndex)+")");
+    }
+    return sb.toString();
+  }
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/MvccVersionDecoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/MvccVersionDecoder.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/MvccVersionDecoder.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/MvccVersionDecoder.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,57 @@
+/*
+ * 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.codec.prefixtree.decode.timestamp;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.util.vint.UFIntTool;
+
+/**
+ * Given a block and its blockMeta, this will decode the MvccVersion for the i-th Cell in the block.
+ */
+@InterfaceAudience.Private
+public class MvccVersionDecoder {
+
+  protected PrefixTreeBlockMeta blockMeta;
+  protected byte[] block;
+
+
+  /************** construct ***********************/
+
+  public MvccVersionDecoder() {
+  }
+
+  public void initOnBlock(PrefixTreeBlockMeta blockMeta, byte[] block) {
+    this.block = block;
+    this.blockMeta = blockMeta;
+  }
+
+
+  /************** methods *************************/
+
+  public long getMvccVersion(int index) {
+    if (blockMeta.getMvccVersionIndexWidth() == 0) {//all mvccVersions in the block were identical
+      return blockMeta.getMinMvccVersion();
+    }
+    int startIndex = blockMeta.getAbsoluteMvccVersionOffset()
+        + blockMeta.getMvccVersionDeltaWidth() * index;
+    long delta = UFIntTool.fromBytes(block, startIndex, blockMeta.getMvccVersionDeltaWidth());
+    return blockMeta.getMinMvccVersion() + delta;
+  }
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/TimestampDecoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/TimestampDecoder.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/TimestampDecoder.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/TimestampDecoder.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,57 @@
+/*
+ * 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.codec.prefixtree.decode.timestamp;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.util.vint.UFIntTool;
+
+/**
+ * Given a block and its blockMeta, this will decode the timestamp for the i-th Cell in the block.
+ */
+@InterfaceAudience.Private
+public class TimestampDecoder {
+
+  protected PrefixTreeBlockMeta blockMeta;
+  protected byte[] block;
+
+
+  /************** construct ***********************/
+
+  public TimestampDecoder() {
+  }
+
+  public void initOnBlock(PrefixTreeBlockMeta blockMeta, byte[] block) {
+    this.block = block;
+    this.blockMeta = blockMeta;
+  }
+
+
+  /************** methods *************************/
+
+  public long getLong(int index) {
+    if (blockMeta.getTimestampIndexWidth() == 0) {//all timestamps in the block were identical
+      return blockMeta.getMinTimestamp();
+    }
+    int startIndex = blockMeta.getAbsoluteTimestampOffset() + blockMeta.getTimestampDeltaWidth()
+        * index;
+    long delta = UFIntTool.fromBytes(block, startIndex, blockMeta.getTimestampDeltaWidth());
+    return blockMeta.getMinTimestamp() + delta;
+  }
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,56 @@
+/*
+ * 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.codec.prefixtree.encode;
+
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Retrieve PrefixTreeEncoders from this factory which handles pooling them and preparing the
+ * ones retrieved from the pool for usage.
+ */
+@InterfaceAudience.Private
+public class EncoderFactory {
+
+  private static final EncoderPool POOL = new ThreadLocalEncoderPool();
+
+
+  public static PrefixTreeEncoder checkOut(OutputStream outputStream, boolean includeMvccVersion) {
+    return POOL.checkOut(outputStream, includeMvccVersion);
+  }
+
+  public static void checkIn(PrefixTreeEncoder encoder) {
+    POOL.checkIn(encoder);
+  }
+
+
+  /**************************** helper ******************************/
+
+  protected static PrefixTreeEncoder prepareEncoder(PrefixTreeEncoder encoder,
+      OutputStream outputStream, boolean includeMvccVersion) {
+    PrefixTreeEncoder ret = encoder;
+    if (encoder == null) {
+      ret = new PrefixTreeEncoder(outputStream, includeMvccVersion);
+    }
+    ret.reset(outputStream, includeMvccVersion);
+    return ret;
+  }
+
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPool.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPool.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPool.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPool.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,32 @@
+/*
+ * 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.codec.prefixtree.encode;
+
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface EncoderPool {
+
+  PrefixTreeEncoder checkOut(OutputStream outputStream, boolean includeMvccVersion);
+  void checkIn(PrefixTreeEncoder encoder);
+
+}
\ No newline at end of file

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,494 @@
+/*
+ * 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.codec.prefixtree.encode;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.column.ColumnSectionWriter;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.other.CellTypeEncoder;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.other.LongEncoder;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.row.RowSectionWriter;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.Tokenizer;
+import org.apache.hadoop.hbase.io.CellOutputStream;
+import org.apache.hadoop.hbase.util.ArrayUtils;
+import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.byterange.ByteRangeSet;
+import org.apache.hadoop.hbase.util.byterange.impl.ByteRangeHashSet;
+import org.apache.hadoop.hbase.util.byterange.impl.ByteRangeTreeSet;
+import org.apache.hadoop.hbase.util.vint.UFIntTool;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * This is the primary class for converting a CellOutputStream into an encoded byte[]. As Cells are
+ * added they are completely copied into the various encoding structures. This is important because
+ * usually the cells being fed in during compactions will be transient.<br/>
+ * <br/>
+ * Usage:<br/>
+ * 1) constructor<br/>
+ * 4) append cells in sorted order: write(Cell cell)<br/>
+ * 5) flush()<br/>
+ */
+@InterfaceAudience.Private
+public class PrefixTreeEncoder implements CellOutputStream {
+
+  /**************** static ************************/
+
+  protected static final Log LOG = LogFactory.getLog(PrefixTreeEncoder.class);
+
+  //future-proof where HBase supports multiple families in a data block.
+  public static final boolean MULITPLE_FAMILIES_POSSIBLE = false;
+
+  private static final boolean USE_HASH_COLUMN_SORTER = true;
+  private static final int INITIAL_PER_CELL_ARRAY_SIZES = 256;
+  private static final int VALUE_BUFFER_INIT_SIZE = 64 * 1024;
+
+
+  /**************** fields *************************/
+
+  protected long numResets = 0L;
+
+  protected OutputStream outputStream;
+
+  /*
+   * Cannot change during a single block's encoding. If false, then substitute incoming Cell's
+   * mvccVersion with zero and write out the block as usual.
+   */
+  protected boolean includeMvccVersion;
+
+  /*
+   * reusable ByteRanges used for communicating with the sorters/compilers
+   */
+  protected ByteRange rowRange;
+  protected ByteRange familyRange;
+  protected ByteRange qualifierRange;
+
+  /*
+   * incoming Cell fields are copied into these arrays
+   */
+  protected long[] timestamps;
+  protected long[] mvccVersions;
+  protected byte[] typeBytes;
+  protected int[] valueOffsets;
+  protected byte[] values;
+
+  protected PrefixTreeBlockMeta blockMeta;
+
+  /*
+   * Sub-encoders for the simple long/byte fields of a Cell.  Add to these as each cell arrives and
+   * compile before flushing.
+   */
+  protected LongEncoder timestampEncoder;
+  protected LongEncoder mvccVersionEncoder;
+  protected CellTypeEncoder cellTypeEncoder;
+
+  /*
+   * Structures used for collecting families and qualifiers, de-duplicating them, and sorting them
+   * so they can be passed to the tokenizers. Unlike row keys where we can detect duplicates by
+   * comparing only with the previous row key, families and qualifiers can arrive in unsorted order
+   * in blocks spanning multiple rows. We must collect them all into a set to de-duplicate them.
+   */
+  protected ByteRangeSet familyDeduplicator;
+  protected ByteRangeSet qualifierDeduplicator;
+
+  /*
+   * Feed sorted byte[]s into these tokenizers which will convert the byte[]s to an in-memory
+   * trie structure with nodes connected by memory pointers (not serializable yet).
+   */
+  protected Tokenizer rowTokenizer;
+  protected Tokenizer familyTokenizer;
+  protected Tokenizer qualifierTokenizer;
+
+  /*
+   * Writers take an in-memory trie, sort the nodes, calculate offsets and lengths, and write
+   * all information to an output stream of bytes that can be stored on disk.
+   */
+  protected RowSectionWriter rowWriter;
+  protected ColumnSectionWriter familyWriter;
+  protected ColumnSectionWriter qualifierWriter;
+
+  /*
+   * Integers used for counting cells and bytes.  We keep track of the size of the Cells as if they
+   * were full KeyValues because some parts of HBase like to know the "unencoded size".
+   */
+  protected int totalCells = 0;
+  protected int totalUnencodedBytes = 0;//numBytes if the cells were KeyValues
+  protected int totalValueBytes = 0;
+  protected int maxValueLength = 0;
+  protected int totalBytes = 0;//
+
+
+  /***************** construct ***********************/
+
+  public PrefixTreeEncoder(OutputStream outputStream, boolean includeMvccVersion) {
+    // used during cell accumulation
+    this.blockMeta = new PrefixTreeBlockMeta();
+    this.rowRange = new ByteRange();
+    this.familyRange = new ByteRange();
+    this.qualifierRange = new ByteRange();
+    this.timestamps = new long[INITIAL_PER_CELL_ARRAY_SIZES];
+    this.mvccVersions = new long[INITIAL_PER_CELL_ARRAY_SIZES];
+    this.typeBytes = new byte[INITIAL_PER_CELL_ARRAY_SIZES];
+    this.valueOffsets = new int[INITIAL_PER_CELL_ARRAY_SIZES];
+    this.values = new byte[VALUE_BUFFER_INIT_SIZE];
+
+    // used during compilation
+    this.familyDeduplicator = USE_HASH_COLUMN_SORTER ? new ByteRangeHashSet()
+        : new ByteRangeTreeSet();
+    this.qualifierDeduplicator = USE_HASH_COLUMN_SORTER ? new ByteRangeHashSet()
+        : new ByteRangeTreeSet();
+    this.timestampEncoder = new LongEncoder();
+    this.mvccVersionEncoder = new LongEncoder();
+    this.cellTypeEncoder = new CellTypeEncoder();
+    this.rowTokenizer = new Tokenizer();
+    this.familyTokenizer = new Tokenizer();
+    this.qualifierTokenizer = new Tokenizer();
+    this.rowWriter = new RowSectionWriter();
+    this.familyWriter = new ColumnSectionWriter();
+    this.qualifierWriter = new ColumnSectionWriter();
+
+    reset(outputStream, includeMvccVersion);
+  }
+
+  public void reset(OutputStream outputStream, boolean includeMvccVersion) {
+    ++numResets;
+    this.includeMvccVersion = includeMvccVersion;
+    this.outputStream = outputStream;
+    valueOffsets[0] = 0;
+
+    familyDeduplicator.reset();
+    qualifierDeduplicator.reset();
+    rowTokenizer.reset();
+    timestampEncoder.reset();
+    mvccVersionEncoder.reset();
+    cellTypeEncoder.reset();
+    familyTokenizer.reset();
+    qualifierTokenizer.reset();
+    rowWriter.reset();
+    familyWriter.reset();
+    qualifierWriter.reset();
+
+    totalCells = 0;
+    totalUnencodedBytes = 0;
+    totalValueBytes = 0;
+    maxValueLength = 0;
+    totalBytes = 0;
+  }
+
+  /**
+   * Check that the arrays used to hold cell fragments are large enough for the cell that is being
+   * added. Since the PrefixTreeEncoder is cached between uses, these arrays may grow during the
+   * first few block encodings but should stabilize quickly.
+   */
+  protected void ensurePerCellCapacities() {
+    int currentCapacity = valueOffsets.length;
+    int neededCapacity = totalCells + 2;// some things write one index ahead. +2 to be safe
+    if (neededCapacity < currentCapacity) {
+      return;
+    }
+
+    int padding = neededCapacity;//this will double the array size
+    timestamps = ArrayUtils.growIfNecessary(timestamps, neededCapacity, padding);
+    mvccVersions = ArrayUtils.growIfNecessary(mvccVersions, neededCapacity, padding);
+    typeBytes = ArrayUtils.growIfNecessary(typeBytes, neededCapacity, padding);
+    valueOffsets = ArrayUtils.growIfNecessary(valueOffsets, neededCapacity, padding);
+  }
+
+  /******************** CellOutputStream methods *************************/
+
+  /**
+   * Note: Unused until support is added to the scanner/heap
+   * <p/>
+   * The following method are optimized versions of write(Cell cell). The result should be
+   * identical, however the implementation may be able to execute them much more efficiently because
+   * it does not need to compare the unchanged fields with the previous cell's.
+   * <p/>
+   * Consider the benefits during compaction when paired with a CellScanner that is also aware of
+   * row boundaries. The CellScanner can easily use these methods instead of blindly passing Cells
+   * to the write(Cell cell) method.
+   * <p/>
+   * The savings of skipping duplicate row detection are significant with long row keys. A
+   * DataBlockEncoder may store a row key once in combination with a count of how many cells are in
+   * the row. With a 100 byte row key, we can replace 100 byte comparisons with a single increment
+   * of the counter, and that is for every cell in the row.
+   */
+
+  /**
+   * Add a Cell to the output stream but repeat the previous row. 
+   */
+  //@Override
+  public void writeWithRepeatRow(Cell cell) {
+    ensurePerCellCapacities();//can we optimize away some of this?
+
+    //save a relatively expensive row comparison, incrementing the row's counter instead
+    rowTokenizer.incrementNumOccurrencesOfLatestValue();
+    addFamilyPart(cell);
+    addQualifierPart(cell);
+    addAfterRowFamilyQualifier(cell);
+  }
+
+
+  @Override
+  public void write(Cell cell) {
+    ensurePerCellCapacities();
+
+    rowTokenizer.addSorted(CellUtil.fillRowRange(cell, rowRange));
+    addFamilyPart(cell);
+    addQualifierPart(cell);
+    addAfterRowFamilyQualifier(cell);
+  }
+
+
+  /***************** internal add methods ************************/
+
+  private void addAfterRowFamilyQualifier(Cell cell){
+    // timestamps
+    timestamps[totalCells] = cell.getTimestamp();
+    timestampEncoder.add(cell.getTimestamp());
+
+    // memstore timestamps
+    if (includeMvccVersion) {
+      mvccVersions[totalCells] = cell.getMvccVersion();
+      mvccVersionEncoder.add(cell.getMvccVersion());
+      totalUnencodedBytes += WritableUtils.getVIntSize(cell.getMvccVersion());
+    }else{
+      //must overwrite in case there was a previous version in this array slot
+      mvccVersions[totalCells] = 0L;
+      if(totalCells == 0){//only need to do this for the first cell added
+        mvccVersionEncoder.add(0L);
+      }
+      //totalUncompressedBytes += 0;//mvccVersion takes zero bytes when disabled
+    }
+
+    // types
+    typeBytes[totalCells] = cell.getTypeByte();
+    cellTypeEncoder.add(cell.getTypeByte());
+
+    // values
+    totalValueBytes += cell.getValueLength();
+    // double the array each time we run out of space
+    values = ArrayUtils.growIfNecessary(values, totalValueBytes, 2 * totalValueBytes);
+    CellUtil.copyValueTo(cell, values, valueOffsets[totalCells]);
+    if (cell.getValueLength() > maxValueLength) {
+      maxValueLength = cell.getValueLength();
+    }
+    valueOffsets[totalCells + 1] = totalValueBytes;
+
+    // general
+    totalUnencodedBytes += KeyValueUtil.length(cell);
+    ++totalCells;
+  }
+
+  private void addFamilyPart(Cell cell) {
+    if (MULITPLE_FAMILIES_POSSIBLE || totalCells == 0) {
+      CellUtil.fillFamilyRange(cell, familyRange);
+      familyDeduplicator.add(familyRange);
+    }
+  }
+
+  private void addQualifierPart(Cell cell) {
+    CellUtil.fillQualifierRange(cell, qualifierRange);
+    qualifierDeduplicator.add(qualifierRange);
+  }
+
+
+  /****************** compiling/flushing ********************/
+
+  /**
+   * Expensive method.  The second half of the encoding work happens here.
+   *
+   * Take all the separate accumulated data structures and turn them into a single stream of bytes
+   * which is written to the outputStream.
+   */
+  @Override
+  public void flush() throws IOException {
+    compile();
+
+    // do the actual flushing to the output stream.  Order matters.
+    blockMeta.writeVariableBytesToOutputStream(outputStream);
+    rowWriter.writeBytes(outputStream);
+    familyWriter.writeBytes(outputStream);
+    qualifierWriter.writeBytes(outputStream);
+    timestampEncoder.writeBytes(outputStream);
+    mvccVersionEncoder.writeBytes(outputStream);
+    //CellType bytes are in the row nodes.  there is no additional type section
+    outputStream.write(values, 0, totalValueBytes);
+  }
+
+  /**
+   * Now that all the cells have been added, do the work to reduce them to a series of byte[]
+   * fragments that are ready to be written to the output stream.
+   */
+  protected void compile(){
+    blockMeta.setNumKeyValueBytes(totalUnencodedBytes);
+    int lastValueOffset = valueOffsets[totalCells];
+    blockMeta.setValueOffsetWidth(UFIntTool.numBytes(lastValueOffset));
+    blockMeta.setValueLengthWidth(UFIntTool.numBytes(maxValueLength));
+    blockMeta.setNumValueBytes(totalValueBytes);
+    totalBytes += totalValueBytes;
+
+    //these compile methods will add to totalBytes
+    compileTypes();
+    compileMvccVersions();
+    compileTimestamps();
+    compileQualifiers();
+    compileFamilies();
+    compileRows();
+
+    int numMetaBytes = blockMeta.calculateNumMetaBytes();
+    blockMeta.setNumMetaBytes(numMetaBytes);
+    totalBytes += numMetaBytes;
+  }
+
+  /**
+   * The following "compile" methods do any intermediate work necessary to transform the cell
+   * fragments collected during the writing phase into structures that are ready to write to the
+   * outputStream.
+   * <p/>
+   * The family and qualifier treatment is almost identical, as is timestamp and mvccVersion.
+   */
+
+  protected void compileTypes() {
+    blockMeta.setAllSameType(cellTypeEncoder.areAllSameType());
+    if(cellTypeEncoder.areAllSameType()){
+      blockMeta.setAllTypes(cellTypeEncoder.getOnlyType());
+    }
+  }
+
+  protected void compileMvccVersions() {
+    mvccVersionEncoder.compile();
+    blockMeta.setMvccVersionFields(mvccVersionEncoder);
+    int numMvccVersionBytes = mvccVersionEncoder.getOutputArrayLength();
+    totalBytes += numMvccVersionBytes;
+  }
+
+  protected void compileTimestamps() {
+    timestampEncoder.compile();
+    blockMeta.setTimestampFields(timestampEncoder);
+    int numTimestampBytes = timestampEncoder.getOutputArrayLength();
+    totalBytes += numTimestampBytes;
+  }
+
+  protected void compileQualifiers() {
+    blockMeta.setNumUniqueQualifiers(qualifierDeduplicator.size());
+    qualifierDeduplicator.compile();
+    qualifierTokenizer.addAll(qualifierDeduplicator.getSortedRanges());
+    qualifierWriter.reconstruct(blockMeta, qualifierTokenizer, false);
+    qualifierWriter.compile();
+    int numQualifierBytes = qualifierWriter.getNumBytes();
+    blockMeta.setNumQualifierBytes(numQualifierBytes);
+    totalBytes += numQualifierBytes;
+  }
+
+  protected void compileFamilies() {
+    blockMeta.setNumUniqueFamilies(familyDeduplicator.size());
+    familyDeduplicator.compile();
+    familyTokenizer.addAll(familyDeduplicator.getSortedRanges());
+    familyWriter.reconstruct(blockMeta, familyTokenizer, true);
+    familyWriter.compile();
+    int numFamilyBytes = familyWriter.getNumBytes();
+    blockMeta.setNumFamilyBytes(numFamilyBytes);
+    totalBytes += numFamilyBytes;
+  }
+
+  protected void compileRows() {
+    rowWriter.reconstruct(this);
+    rowWriter.compile();
+    int numRowBytes = rowWriter.getNumBytes();
+    blockMeta.setNumRowBytes(numRowBytes);
+    blockMeta.setRowTreeDepth(rowTokenizer.getTreeDepth());
+    totalBytes += numRowBytes;
+  }
+
+  /********************* convenience getters ********************************/
+
+  public long getValueOffset(int index) {
+    return valueOffsets[index];
+  }
+
+  public int getValueLength(int index) {
+    return (int) (valueOffsets[index + 1] - valueOffsets[index]);
+  }
+
+  /************************* get/set *************************************/
+
+  public PrefixTreeBlockMeta getBlockMeta() {
+    return blockMeta;
+  }
+
+  public Tokenizer getRowTokenizer() {
+    return rowTokenizer;
+  }
+
+  public LongEncoder getTimestampEncoder() {
+    return timestampEncoder;
+  }
+
+  public int getTotalBytes() {
+    return totalBytes;
+  }
+
+  public long[] getTimestamps() {
+    return timestamps;
+  }
+
+  public long[] getMvccVersions() {
+    return mvccVersions;
+  }
+
+  public byte[] getTypeBytes() {
+    return typeBytes;
+  }
+
+  public LongEncoder getMvccVersionEncoder() {
+    return mvccVersionEncoder;
+  }
+
+  public ByteRangeSet getFamilySorter() {
+    return familyDeduplicator;
+  }
+
+  public ByteRangeSet getQualifierSorter() {
+    return qualifierDeduplicator;
+  }
+
+  public ColumnSectionWriter getFamilyWriter() {
+    return familyWriter;
+  }
+
+  public ColumnSectionWriter getQualifierWriter() {
+    return qualifierWriter;
+  }
+
+  public RowSectionWriter getRowWriter() {
+    return rowWriter;
+  }
+
+  public ByteRange getValueByteRange() {
+    return new ByteRange(values, 0, totalValueBytes);
+  }
+
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/ThreadLocalEncoderPool.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/ThreadLocalEncoderPool.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/ThreadLocalEncoderPool.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/ThreadLocalEncoderPool.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,64 @@
+/*
+ * 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.codec.prefixtree.encode;
+
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+
+/**
+ * Pool to enable reusing the Encoder objects which can consist of thousands of smaller objects and
+ * would be more garbage than the data in the block.  A new encoder is needed for each block in
+ * a flush, compaction, RPC response, etc.
+ *
+ * It is not a pool in the traditional sense, but implements the semantics of a traditional pool
+ * via ThreadLocals to avoid sharing between threads.  Sharing between threads would not be
+ * very expensive given that it's accessed per-block, but this is just as easy.
+ *
+ * This pool implementation assumes there is a one-to-one mapping between a single thread and a
+ * single flush or compaction.
+ */
+@InterfaceAudience.Private
+public class ThreadLocalEncoderPool implements EncoderPool{
+
+  private static final ThreadLocal<PrefixTreeEncoder> ENCODER
+      = new ThreadLocal<PrefixTreeEncoder>();
+
+  /**
+   * Get the encoder attached to the current ThreadLocal, or create a new one and attach it to the
+   * current thread.
+   */
+  @Override
+  public PrefixTreeEncoder checkOut(OutputStream os, boolean includeMvccVersion) {
+    PrefixTreeEncoder builder = ENCODER.get();
+    builder = EncoderFactory.prepareEncoder(builder, os, includeMvccVersion);
+    ENCODER.set(builder);
+    return builder;
+  }
+
+  @Override
+  public void checkIn(PrefixTreeEncoder encoder) {
+    // attached to thread on checkOut, so shouldn't need to do anything here
+
+    // do we need to worry about detaching encoders from compaction threads or are the same threads
+    // used over and over
+  }
+
+}
\ No newline at end of file

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnNodeWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnNodeWriter.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnNodeWriter.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnNodeWriter.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,131 @@
+/*
+ * 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.codec.prefixtree.encode.column;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.TokenizerNode;
+import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Strings;
+import org.apache.hadoop.hbase.util.vint.UFIntTool;
+import org.apache.hadoop.hbase.util.vint.UVIntTool;
+
+/**
+ * Column nodes can be either family nodes or qualifier nodes, as both sections encode similarly.
+ * The family and qualifier sections of the data block are made of 1 or more of these nodes.
+ * <p/>
+ * Each node is composed of 3 sections:<br/>
+ * <li>tokenLength: UVInt (normally 1 byte) indicating the number of token bytes
+ * <li>token[]: the actual token bytes
+ * <li>parentStartPosition: the offset of the next node from the start of the family or qualifier
+ * section
+ */
+@InterfaceAudience.Private
+public class ColumnNodeWriter{
+
+  /************* fields ****************************/
+
+  protected TokenizerNode builderNode;
+  protected PrefixTreeBlockMeta blockMeta;
+
+  protected boolean familyVsQualifier;
+
+  protected int tokenLength;
+  protected byte[] token;
+  protected int parentStartPosition;
+
+
+  /*************** construct **************************/
+
+  public ColumnNodeWriter(PrefixTreeBlockMeta blockMeta, TokenizerNode builderNode,
+      boolean familyVsQualifier) {
+    this.blockMeta = blockMeta;
+    this.builderNode = builderNode;
+    this.familyVsQualifier = familyVsQualifier;
+    calculateTokenLength();
+  }
+
+
+  /************* methods *******************************/
+
+  public boolean isRoot() {
+    return parentStartPosition == 0;
+  }
+
+  private void calculateTokenLength() {
+    tokenLength = builderNode.getTokenLength();
+    token = new byte[tokenLength];
+  }
+
+  /**
+   * This method is called before blockMeta.qualifierOffsetWidth is known, so we pass in a
+   * placeholder.
+   * @param offsetWidthPlaceholder the placeholder
+   * @return node width
+   */
+  public int getWidthUsingPlaceholderForOffsetWidth(int offsetWidthPlaceholder) {
+    int width = 0;
+    width += UVIntTool.numBytes(tokenLength);
+    width += token.length;
+    width += offsetWidthPlaceholder;
+    return width;
+  }
+
+  public void writeBytes(OutputStream os) throws IOException {
+    int parentOffsetWidth;
+    if (familyVsQualifier) {
+      parentOffsetWidth = blockMeta.getFamilyOffsetWidth();
+    } else {
+      parentOffsetWidth = blockMeta.getQualifierOffsetWidth();
+    }
+    UVIntTool.writeBytes(tokenLength, os);
+    os.write(token);
+    UFIntTool.writeBytes(parentOffsetWidth, parentStartPosition, os);
+  }
+
+  public void setTokenBytes(ByteRange source) {
+    source.deepCopySubRangeTo(0, tokenLength, token, 0);
+  }
+
+
+  /****************** standard methods ************************/
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(Strings.padFront(builderNode.getOutputArrayOffset() + "", ' ', 3) + ",");
+    sb.append("[");
+    sb.append(Bytes.toString(token));
+    sb.append("]->");
+    sb.append(parentStartPosition);
+    return sb.toString();
+  }
+
+
+  /************************** get/set ***********************/
+
+  public void setParentStartPosition(int parentStartPosition) {
+    this.parentStartPosition = parentStartPosition;
+  }
+
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnSectionWriter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnSectionWriter.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnSectionWriter.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnSectionWriter.java Tue Mar 12 19:10:59 2013
@@ -0,0 +1,201 @@
+/*
+ * 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.codec.prefixtree.encode.column;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.Tokenizer;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.tokenize.TokenizerNode;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hadoop.hbase.util.vint.UFIntTool;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Takes the tokenized family or qualifier data and flattens it into a stream of bytes. The family
+ * section is written after the row section, and qualifier section after family section.
+ * <p/>
+ * The family and qualifier tries, or "column tries", are structured differently than the row trie.
+ * The trie cannot be reassembled without external data about the offsets of the leaf nodes, and
+ * these external pointers are stored in the nubs and leaves of the row trie. For each cell in a
+ * row, the row trie contains a list of offsets into the column sections (along with pointers to
+ * timestamps and other per-cell fields). These offsets point to the last column node/token that
+ * comprises the column name. To assemble the column name, the trie is traversed in reverse (right
+ * to left), with the rightmost tokens pointing to the start of their "parent" node which is the
+ * node to the left.
+ * <p/>
+ * This choice was made to reduce the size of the column trie by storing the minimum amount of
+ * offset data. As a result, to find a specific qualifier within a row, you must do a binary search
+ * of the column nodes, reassembling each one as you search. Future versions of the PrefixTree might
+ * encode the columns in both a forward and reverse trie, which would convert binary searches into
+ * more efficient trie searches which would be beneficial for wide rows.
+ */
+@InterfaceAudience.Private
+public class ColumnSectionWriter {
+
+  public static final int EXPECTED_NUBS_PLUS_LEAVES = 100;
+
+  /****************** fields ****************************/
+
+  private PrefixTreeBlockMeta blockMeta;
+
+  private boolean familyVsQualifier;
+  private Tokenizer tokenizer;
+  private int numBytes = 0;
+  private ArrayList<TokenizerNode> nonLeaves;
+  private ArrayList<TokenizerNode> leaves;
+  private ArrayList<TokenizerNode> allNodes;
+  private ArrayList<ColumnNodeWriter> columnNodeWriters;
+  private List<Integer> outputArrayOffsets;
+
+
+	/*********************** construct *********************/
+
+  public ColumnSectionWriter() {
+    this.nonLeaves = Lists.newArrayList();
+    this.leaves = Lists.newArrayList();
+    this.outputArrayOffsets = Lists.newArrayList();
+  }
+
+  public ColumnSectionWriter(PrefixTreeBlockMeta blockMeta, Tokenizer builder,
+      boolean familyVsQualifier) {
+    this();// init collections
+    reconstruct(blockMeta, builder, familyVsQualifier);
+  }
+
+  public void reconstruct(PrefixTreeBlockMeta blockMeta, Tokenizer builder,
+      boolean familyVsQualifier) {
+    this.blockMeta = blockMeta;
+    this.tokenizer = builder;
+    this.familyVsQualifier = familyVsQualifier;
+  }
+
+  public void reset() {
+    numBytes = 0;
+    nonLeaves.clear();
+    leaves.clear();
+    outputArrayOffsets.clear();
+  }
+
+
+	/****************** methods *******************************/
+
+  public ColumnSectionWriter compile() {
+    if (familyVsQualifier) {
+      // do nothing. max family length fixed at Byte.MAX_VALUE
+    } else {
+      blockMeta.setMaxQualifierLength(tokenizer.getMaxElementLength());
+    }
+
+    tokenizer.setNodeFirstInsertionIndexes();
+
+    tokenizer.appendNodes(nonLeaves, true, false);
+
+    tokenizer.appendNodes(leaves, false, true);
+
+    allNodes = Lists.newArrayListWithCapacity(nonLeaves.size() + leaves.size());
+    allNodes.addAll(nonLeaves);
+    allNodes.addAll(leaves);
+
+    columnNodeWriters = Lists.newArrayListWithCapacity(CollectionUtils.nullSafeSize(allNodes));
+    for (int i = 0; i < allNodes.size(); ++i) {
+      TokenizerNode node = allNodes.get(i);
+      columnNodeWriters.add(new ColumnNodeWriter(blockMeta, node, familyVsQualifier));
+    }
+
+    // leaf widths are known at this point, so add them up
+    int totalBytesWithoutOffsets = 0;
+    for (int i = allNodes.size() - 1; i >= 0; --i) {
+      ColumnNodeWriter columnNodeWriter = columnNodeWriters.get(i);
+      // leaves store all but their first token byte
+      totalBytesWithoutOffsets += columnNodeWriter.getWidthUsingPlaceholderForOffsetWidth(0);
+    }
+
+    // figure out how wide our offset FInts are
+    int parentOffsetWidth = 0;
+    while (true) {
+      ++parentOffsetWidth;
+      int numBytesFinder = totalBytesWithoutOffsets + parentOffsetWidth * allNodes.size();
+      if (numBytesFinder < UFIntTool.maxValueForNumBytes(parentOffsetWidth)) {
+        numBytes = numBytesFinder;
+        break;
+      }// it fits
+    }
+    if (familyVsQualifier) {
+      blockMeta.setFamilyOffsetWidth(parentOffsetWidth);
+    } else {
+      blockMeta.setQualifierOffsetWidth(parentOffsetWidth);
+    }
+
+    int forwardIndex = 0;
+    for (int i = 0; i < allNodes.size(); ++i) {
+      TokenizerNode node = allNodes.get(i);
+      ColumnNodeWriter columnNodeWriter = columnNodeWriters.get(i);
+      int fullNodeWidth = columnNodeWriter
+          .getWidthUsingPlaceholderForOffsetWidth(parentOffsetWidth);
+      node.setOutputArrayOffset(forwardIndex);
+      columnNodeWriter.setTokenBytes(node.getToken());
+      if (node.isRoot()) {
+        columnNodeWriter.setParentStartPosition(0);
+      } else {
+        columnNodeWriter.setParentStartPosition(node.getParent().getOutputArrayOffset());
+      }
+      forwardIndex += fullNodeWidth;
+    }
+
+    tokenizer.appendOutputArrayOffsets(outputArrayOffsets);
+
+    return this;
+  }
+
+  public void writeBytes(OutputStream os) throws IOException {
+    for (ColumnNodeWriter columnNodeWriter : columnNodeWriters) {
+      columnNodeWriter.writeBytes(os);
+    }
+  }
+
+
+  /************* get/set **************************/
+
+  public ArrayList<ColumnNodeWriter> getColumnNodeWriters() {
+    return columnNodeWriters;
+  }
+
+  public int getNumBytes() {
+    return numBytes;
+  }
+
+  public int getOutputArrayOffset(int sortedIndex) {
+    return outputArrayOffsets.get(sortedIndex);
+  }
+
+  public ArrayList<TokenizerNode> getNonLeaves() {
+    return nonLeaves;
+  }
+
+  public ArrayList<TokenizerNode> getLeaves() {
+    return leaves;
+  }
+
+}

Added: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/CellTypeEncoder.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/CellTypeEncoder.java?rev=1455673&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/CellTypeEncoder.java (added)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/CellTypeEncoder.java Tue Mar 12 19:10:59 2013
@@ -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.codec.prefixtree.encode.other;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Detect if every KV has the same KeyValue.Type, in which case we don't need to store it for each
+ * KV.  If(allSameType) during conversion to byte[], then we can store the "onlyType" in blockMeta,
+ * therefore not repeating it for each cell and saving 1 byte per cell.
+ */
+@InterfaceAudience.Private
+public class CellTypeEncoder {
+
+  /************* fields *********************/
+
+  protected boolean pendingFirstType = true;
+  protected boolean allSameType = true;
+  protected byte onlyType;
+
+
+  /************* construct *********************/
+
+  public void reset() {
+    pendingFirstType = true;
+    allSameType = true;
+  }
+
+
+  /************* methods *************************/
+
+  public void add(byte type) {
+    if (pendingFirstType) {
+      onlyType = type;
+      pendingFirstType = false;
+    } else if (onlyType != type) {
+      allSameType = false;
+    }
+  }
+
+
+  /**************** get/set **************************/
+
+  public boolean areAllSameType() {
+    return allSameType;
+  }
+
+  public byte getOnlyType() {
+    return onlyType;
+  }
+
+}