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:07:39 UTC

svn commit: r1455672 [1/5] - in /hbase/trunk: 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/java/or...

Author: stack
Date: Tue Mar 12 19:07:34 2013
New Revision: 1455672

URL: http://svn.apache.org/r1455672
Log:
HBASE-HBASE-8054 HBASE-7797 Use consistent package name dregs

Added:
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeBlockMeta.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/ArraySearcherPool.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/DecoderFactory.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayReversibleScanner.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArraySearcher.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnNodeReader.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/column/ColumnReader.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/row/RowNodeReader.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/MvccVersionDecoder.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/timestamp/TimestampDecoder.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderFactory.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/EncoderPool.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/PrefixTreeEncoder.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/ThreadLocalEncoderPool.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnNodeWriter.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/column/ColumnSectionWriter.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/CellTypeEncoder.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/other/LongEncoder.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/row/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/row/RowNodeWriter.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/row/RowSectionWriter.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenDepthComparator.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/Tokenizer.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenizerNode.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenizerRowSearchPosition.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/encode/tokenize/TokenizerRowSearchResult.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/scanner/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/scanner/CellScannerPosition.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/scanner/CellSearcher.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/scanner/ReversibleCellScanner.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/ByteRangeSet.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/impl/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/impl/ByteRangeHashSet.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/byterange/impl/ByteRangeTreeSet.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/vint/
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/vint/UFIntTool.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/vint/UVIntTool.java
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/util/vint/UVLongTool.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/keyvalue/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/keyvalue/TestKeyValueTool.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeTestConstants.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/blockmeta/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/blockmeta/TestBlockMeta.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTokenizer.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTokenizerData.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/TestTreeDepth.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/data/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/data/TestTokenizerDataBasic.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/builder/data/TestTokenizerDataEdgeCase.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/TestColumnBuilder.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/TestColumnData.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataRandom.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/column/data/TestColumnDataSimple.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/BaseTestRowData.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestPrefixTreeSearcher.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestRowData.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/TestRowEncoder.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataComplexQualifiers.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataDeeper.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataDifferentTimestamps.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataEmpty.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataMultiFamilies.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataNub.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataQualifierByteOrdering.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataRandomKeyValues.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataSimple.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataSingleQualifier.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataTrivial.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataUrls.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/row/data/TestRowDataUrlsExample.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/timestamp/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/timestamp/TestTimestampData.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/timestamp/TestTimestampEncoder.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/timestamp/data/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/timestamp/data/TestTimestampDataBasic.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/timestamp/data/TestTimestampDataNumbers.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/codec/prefixtree/timestamp/data/TestTimestampDataRepeats.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/bytes/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/bytes/TestByteRange.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/comparator/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/comparator/ByteArrayComparator.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/number/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/number/NumberFormatter.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/number/RandomNumberUtils.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/vint/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/vint/TestFIntTool.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/vint/TestVIntTool.java
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hadoop/hbase/util/vint/TestVLongTool.java
Removed:
    hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hbase/
    hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/
Modified:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java?rev=1455672&r1=1455671&r2=1455672&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoding.java Tue Mar 12 19:07:34 2013
@@ -41,7 +41,7 @@ public enum DataBlockEncoding {
   FAST_DIFF(4, "org.apache.hadoop.hbase.io.encoding.FastDiffDeltaEncoder"),
   // id 5 is reserved for the COPY_KEY algorithm for benchmarking
   // COPY_KEY(5, "org.apache.hadoop.hbase.io.encoding.CopyKeyDataBlockEncoder"),
-  PREFIX_TREE(6, "org.apache.hbase.codec.prefixtree.PrefixTreeCodec");
+  PREFIX_TREE(6, "org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeCodec");
 
   private final short id;
   private final byte[] idInBytes;

Added: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeBlockMeta.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeBlockMeta.java?rev=1455672&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeBlockMeta.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeBlockMeta.java Tue Mar 12 19:07:34 2013
@@ -0,0 +1,841 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.other.LongEncoder;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.vint.UVIntTool;
+import org.apache.hadoop.hbase.util.vint.UVLongTool;
+
+/**
+ * Information about the block.  Stored at the beginning of the byte[].  Contains things
+ * like minimum timestamp and width of FInts in the row tree.
+ *
+ * Most fields stored in VInts that get decoded on the first access of each new block.
+ */
+@InterfaceAudience.Private
+public class PrefixTreeBlockMeta {
+
+  /******************* static fields ********************/
+
+  public static final int VERSION = 0;
+
+  public static final int MAX_FAMILY_LENGTH = Byte.MAX_VALUE;// hard-coded in KeyValue
+
+  public static final int
+	  NUM_LONGS = 2,
+    NUM_INTS = 22,
+    NUM_SHORTS = 0,//keyValueTypeWidth not persisted
+    NUM_SINGLE_BYTES = 2,
+    MAX_BYTES = Bytes.SIZEOF_LONG * NUM_LONGS
+        + Bytes.SIZEOF_SHORT * NUM_SHORTS
+        + Bytes.SIZEOF_INT * NUM_INTS
+        + NUM_SINGLE_BYTES;
+
+
+  /**************** transient fields *********************/
+
+  protected int arrayOffset;
+  protected int bufferOffset;
+
+
+  /**************** persisted fields **********************/
+
+  // PrefixTree version to allow future format modifications
+  protected int version;
+  protected int numMetaBytes;
+  protected int numKeyValueBytes;
+  protected boolean includesMvccVersion;//probably don't need this explicitly, but only 1 byte
+
+  // split the byte[] into 6 sections for the different data types
+  protected int numRowBytes;
+  protected int numFamilyBytes;
+  protected int numQualifierBytes;
+  protected int numTimestampBytes;
+  protected int numMvccVersionBytes;
+  protected int numValueBytes;
+
+  // number of bytes in each section of fixed width FInts
+  protected int nextNodeOffsetWidth;
+  protected int familyOffsetWidth;
+  protected int qualifierOffsetWidth;
+  protected int timestampIndexWidth;
+  protected int mvccVersionIndexWidth;
+  protected int valueOffsetWidth;
+  protected int valueLengthWidth;
+
+  // used to pre-allocate structures for reading
+  protected int rowTreeDepth;
+  protected int maxRowLength;
+  protected int maxQualifierLength;
+
+  // the timestamp from which the deltas are calculated
+  protected long minTimestamp;
+  protected int timestampDeltaWidth;
+  protected long minMvccVersion;
+  protected int mvccVersionDeltaWidth;
+
+  protected boolean allSameType;
+  protected byte allTypes;
+
+  protected int numUniqueRows;
+  protected int numUniqueFamilies;
+  protected int numUniqueQualifiers;
+
+
+  /***************** constructors ********************/
+
+  public PrefixTreeBlockMeta() {
+  }
+
+  public PrefixTreeBlockMeta(InputStream is) throws IOException{
+    this.version = VERSION;
+    this.arrayOffset = 0;
+    this.bufferOffset = 0;
+    readVariableBytesFromInputStream(is);
+  }
+
+  /**
+   * @param buffer positioned at start of PtBlockMeta
+   */
+  public PrefixTreeBlockMeta(ByteBuffer buffer) {
+    initOnBlock(buffer);
+  }
+
+  public void initOnBlock(ByteBuffer buffer) {
+    arrayOffset = buffer.arrayOffset();
+    bufferOffset = buffer.position();
+    readVariableBytesFromArray(buffer.array(), arrayOffset + bufferOffset);
+  }
+
+
+	/**************** operate on each field **********************/
+
+  public int calculateNumMetaBytes(){
+    int numBytes = 0;
+    numBytes += UVIntTool.numBytes(version);
+    numBytes += UVLongTool.numBytes(numMetaBytes);
+    numBytes += UVIntTool.numBytes(numKeyValueBytes);
+    ++numBytes;//os.write(getIncludesMvccVersion());
+
+    numBytes += UVIntTool.numBytes(numRowBytes);
+    numBytes += UVIntTool.numBytes(numFamilyBytes);
+    numBytes += UVIntTool.numBytes(numQualifierBytes);
+    numBytes += UVIntTool.numBytes(numTimestampBytes);
+    numBytes += UVIntTool.numBytes(numMvccVersionBytes);
+    numBytes += UVIntTool.numBytes(numValueBytes);
+
+    numBytes += UVIntTool.numBytes(nextNodeOffsetWidth);
+    numBytes += UVIntTool.numBytes(familyOffsetWidth);
+    numBytes += UVIntTool.numBytes(qualifierOffsetWidth);
+    numBytes += UVIntTool.numBytes(timestampIndexWidth);
+    numBytes += UVIntTool.numBytes(mvccVersionIndexWidth);
+    numBytes += UVIntTool.numBytes(valueOffsetWidth);
+    numBytes += UVIntTool.numBytes(valueLengthWidth);
+
+    numBytes += UVIntTool.numBytes(rowTreeDepth);
+    numBytes += UVIntTool.numBytes(maxRowLength);
+    numBytes += UVIntTool.numBytes(maxQualifierLength);
+
+    numBytes += UVLongTool.numBytes(minTimestamp);
+    numBytes += UVIntTool.numBytes(timestampDeltaWidth);
+    numBytes += UVLongTool.numBytes(minMvccVersion);
+    numBytes += UVIntTool.numBytes(mvccVersionDeltaWidth);
+    ++numBytes;//os.write(getAllSameTypeByte());
+    ++numBytes;//os.write(allTypes);
+
+    numBytes += UVIntTool.numBytes(numUniqueRows);
+    numBytes += UVIntTool.numBytes(numUniqueFamilies);
+    numBytes += UVIntTool.numBytes(numUniqueQualifiers);
+    return numBytes;
+  }
+
+  public void writeVariableBytesToOutputStream(OutputStream os) throws IOException{
+      UVIntTool.writeBytes(version, os);
+      UVIntTool.writeBytes(numMetaBytes, os);
+      UVIntTool.writeBytes(numKeyValueBytes, os);
+      os.write(getIncludesMvccVersionByte());
+
+      UVIntTool.writeBytes(numRowBytes, os);
+      UVIntTool.writeBytes(numFamilyBytes, os);
+      UVIntTool.writeBytes(numQualifierBytes, os);
+      UVIntTool.writeBytes(numTimestampBytes, os);
+      UVIntTool.writeBytes(numMvccVersionBytes, os);
+      UVIntTool.writeBytes(numValueBytes, os);
+
+      UVIntTool.writeBytes(nextNodeOffsetWidth, os);
+      UVIntTool.writeBytes(familyOffsetWidth, os);
+      UVIntTool.writeBytes(qualifierOffsetWidth, os);
+      UVIntTool.writeBytes(timestampIndexWidth, os);
+      UVIntTool.writeBytes(mvccVersionIndexWidth, os);
+      UVIntTool.writeBytes(valueOffsetWidth, os);
+      UVIntTool.writeBytes(valueLengthWidth, os);
+
+      UVIntTool.writeBytes(rowTreeDepth, os);
+      UVIntTool.writeBytes(maxRowLength, os);
+      UVIntTool.writeBytes(maxQualifierLength, os);
+
+      UVLongTool.writeBytes(minTimestamp, os);
+      UVIntTool.writeBytes(timestampDeltaWidth, os);
+      UVLongTool.writeBytes(minMvccVersion, os);
+      UVIntTool.writeBytes(mvccVersionDeltaWidth, os);
+      os.write(getAllSameTypeByte());
+      os.write(allTypes);
+
+      UVIntTool.writeBytes(numUniqueRows, os);
+      UVIntTool.writeBytes(numUniqueFamilies, os);
+      UVIntTool.writeBytes(numUniqueQualifiers, os);
+  }
+
+  public void readVariableBytesFromInputStream(InputStream is) throws IOException{
+      version = UVIntTool.getInt(is);
+      numMetaBytes = UVIntTool.getInt(is);
+      numKeyValueBytes = UVIntTool.getInt(is);
+      setIncludesMvccVersion((byte) is.read());
+
+      numRowBytes = UVIntTool.getInt(is);
+      numFamilyBytes = UVIntTool.getInt(is);
+      numQualifierBytes = UVIntTool.getInt(is);
+      numTimestampBytes = UVIntTool.getInt(is);
+      numMvccVersionBytes = UVIntTool.getInt(is);
+      numValueBytes = UVIntTool.getInt(is);
+
+      nextNodeOffsetWidth = UVIntTool.getInt(is);
+      familyOffsetWidth = UVIntTool.getInt(is);
+      qualifierOffsetWidth = UVIntTool.getInt(is);
+      timestampIndexWidth = UVIntTool.getInt(is);
+      mvccVersionIndexWidth = UVIntTool.getInt(is);
+      valueOffsetWidth = UVIntTool.getInt(is);
+      valueLengthWidth = UVIntTool.getInt(is);
+
+      rowTreeDepth = UVIntTool.getInt(is);
+      maxRowLength = UVIntTool.getInt(is);
+      maxQualifierLength = UVIntTool.getInt(is);
+
+      minTimestamp = UVLongTool.getLong(is);
+      timestampDeltaWidth = UVIntTool.getInt(is);
+      minMvccVersion = UVLongTool.getLong(is);
+      mvccVersionDeltaWidth = UVIntTool.getInt(is);
+
+      setAllSameType((byte) is.read());
+      allTypes = (byte) is.read();
+
+      numUniqueRows = UVIntTool.getInt(is);
+      numUniqueFamilies = UVIntTool.getInt(is);
+      numUniqueQualifiers = UVIntTool.getInt(is);
+  }
+
+  public void readVariableBytesFromArray(byte[] bytes, int offset) {
+    int position = offset;
+
+    version = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(version);
+    numMetaBytes = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numMetaBytes);
+    numKeyValueBytes = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numKeyValueBytes);
+    setIncludesMvccVersion(bytes[position]);
+    ++position;
+
+    numRowBytes = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numRowBytes);
+    numFamilyBytes = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numFamilyBytes);
+    numQualifierBytes = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numQualifierBytes);
+    numTimestampBytes = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numTimestampBytes);
+    numMvccVersionBytes = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numMvccVersionBytes);
+    numValueBytes = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numValueBytes);
+
+    nextNodeOffsetWidth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(nextNodeOffsetWidth);
+    familyOffsetWidth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(familyOffsetWidth);
+    qualifierOffsetWidth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(qualifierOffsetWidth);
+    timestampIndexWidth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(timestampIndexWidth);
+    mvccVersionIndexWidth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(mvccVersionIndexWidth);
+    valueOffsetWidth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(valueOffsetWidth);
+    valueLengthWidth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(valueLengthWidth);
+
+    rowTreeDepth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(rowTreeDepth);
+    maxRowLength = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(maxRowLength);
+    maxQualifierLength = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(maxQualifierLength);
+
+    minTimestamp = UVLongTool.getLong(bytes, position);
+    position += UVLongTool.numBytes(minTimestamp);
+    timestampDeltaWidth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(timestampDeltaWidth);
+    minMvccVersion = UVLongTool.getLong(bytes, position);
+    position += UVLongTool.numBytes(minMvccVersion);
+    mvccVersionDeltaWidth = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(mvccVersionDeltaWidth);
+
+    setAllSameType(bytes[position]);
+    ++position;
+    allTypes = bytes[position];
+    ++position;
+
+    numUniqueRows = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numUniqueRows);
+    numUniqueFamilies = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numUniqueFamilies);
+    numUniqueQualifiers = UVIntTool.getInt(bytes, position);
+    position += UVIntTool.numBytes(numUniqueQualifiers);
+  }
+
+	//TODO method that can read directly from ByteBuffer instead of InputStream
+
+
+  /*************** methods *************************/
+
+  public int getKeyValueTypeWidth() {
+    return allSameType ? 0 : 1;
+  }
+
+  public byte getIncludesMvccVersionByte() {
+    return includesMvccVersion ? (byte) 1 : (byte) 0;
+  }
+
+  public void setIncludesMvccVersion(byte includesMvccVersionByte) {
+    includesMvccVersion = includesMvccVersionByte != 0;
+  }
+
+  public byte getAllSameTypeByte() {
+    return allSameType ? (byte) 1 : (byte) 0;
+  }
+
+  public void setAllSameType(byte allSameTypeByte) {
+    allSameType = allSameTypeByte != 0;
+  }
+
+  public boolean isAllSameTimestamp() {
+    return timestampIndexWidth == 0;
+  }
+
+  public boolean isAllSameMvccVersion() {
+    return mvccVersionIndexWidth == 0;
+  }
+
+  public void setTimestampFields(LongEncoder encoder){
+    this.minTimestamp = encoder.getMin();
+    this.timestampIndexWidth = encoder.getBytesPerIndex();
+    this.timestampDeltaWidth = encoder.getBytesPerDelta();
+    this.numTimestampBytes = encoder.getTotalCompressedBytes();
+  }
+
+  public void setMvccVersionFields(LongEncoder encoder){
+    this.minMvccVersion = encoder.getMin();
+    this.mvccVersionIndexWidth = encoder.getBytesPerIndex();
+    this.mvccVersionDeltaWidth = encoder.getBytesPerDelta();
+    this.numMvccVersionBytes = encoder.getTotalCompressedBytes();
+  }
+
+
+  /*************** Object methods *************************/
+
+  /**
+   * Generated by Eclipse
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    PrefixTreeBlockMeta other = (PrefixTreeBlockMeta) obj;
+    if (allSameType != other.allSameType)
+      return false;
+    if (allTypes != other.allTypes)
+      return false;
+    if (arrayOffset != other.arrayOffset)
+      return false;
+    if (bufferOffset != other.bufferOffset)
+      return false;
+    if (valueLengthWidth != other.valueLengthWidth)
+      return false;
+    if (valueOffsetWidth != other.valueOffsetWidth)
+      return false;
+    if (familyOffsetWidth != other.familyOffsetWidth)
+      return false;
+    if (includesMvccVersion != other.includesMvccVersion)
+      return false;
+    if (maxQualifierLength != other.maxQualifierLength)
+      return false;
+    if (maxRowLength != other.maxRowLength)
+      return false;
+    if (mvccVersionDeltaWidth != other.mvccVersionDeltaWidth)
+      return false;
+    if (mvccVersionIndexWidth != other.mvccVersionIndexWidth)
+      return false;
+    if (minMvccVersion != other.minMvccVersion)
+      return false;
+    if (minTimestamp != other.minTimestamp)
+      return false;
+    if (nextNodeOffsetWidth != other.nextNodeOffsetWidth)
+      return false;
+    if (numValueBytes != other.numValueBytes)
+      return false;
+    if (numFamilyBytes != other.numFamilyBytes)
+      return false;
+    if (numMvccVersionBytes != other.numMvccVersionBytes)
+      return false;
+    if (numMetaBytes != other.numMetaBytes)
+      return false;
+    if (numQualifierBytes != other.numQualifierBytes)
+      return false;
+    if (numRowBytes != other.numRowBytes)
+      return false;
+    if (numTimestampBytes != other.numTimestampBytes)
+      return false;
+    if (numUniqueFamilies != other.numUniqueFamilies)
+      return false;
+    if (numUniqueQualifiers != other.numUniqueQualifiers)
+      return false;
+    if (numUniqueRows != other.numUniqueRows)
+      return false;
+    if (numKeyValueBytes != other.numKeyValueBytes)
+      return false;
+    if (qualifierOffsetWidth != other.qualifierOffsetWidth)
+      return false;
+    if (rowTreeDepth != other.rowTreeDepth)
+      return false;
+    if (timestampDeltaWidth != other.timestampDeltaWidth)
+      return false;
+    if (timestampIndexWidth != other.timestampIndexWidth)
+      return false;
+    if (version != other.version)
+      return false;
+    return true;
+  }
+
+  /**
+   * Generated by Eclipse
+   */
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + (allSameType ? 1231 : 1237);
+    result = prime * result + allTypes;
+    result = prime * result + arrayOffset;
+    result = prime * result + bufferOffset;
+    result = prime * result + valueLengthWidth;
+    result = prime * result + valueOffsetWidth;
+    result = prime * result + familyOffsetWidth;
+    result = prime * result + (includesMvccVersion ? 1231 : 1237);
+    result = prime * result + maxQualifierLength;
+    result = prime * result + maxRowLength;
+    result = prime * result + mvccVersionDeltaWidth;
+    result = prime * result + mvccVersionIndexWidth;
+    result = prime * result + (int) (minMvccVersion ^ (minMvccVersion >>> 32));
+    result = prime * result + (int) (minTimestamp ^ (minTimestamp >>> 32));
+    result = prime * result + nextNodeOffsetWidth;
+    result = prime * result + numValueBytes;
+    result = prime * result + numFamilyBytes;
+    result = prime * result + numMvccVersionBytes;
+    result = prime * result + numMetaBytes;
+    result = prime * result + numQualifierBytes;
+    result = prime * result + numRowBytes;
+    result = prime * result + numTimestampBytes;
+    result = prime * result + numUniqueFamilies;
+    result = prime * result + numUniqueQualifiers;
+    result = prime * result + numUniqueRows;
+    result = prime * result + numKeyValueBytes;
+    result = prime * result + qualifierOffsetWidth;
+    result = prime * result + rowTreeDepth;
+    result = prime * result + timestampDeltaWidth;
+    result = prime * result + timestampIndexWidth;
+    result = prime * result + version;
+    return result;
+  }
+
+  /**
+   * Generated by Eclipse
+   */
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("PtBlockMeta [arrayOffset=");
+    builder.append(arrayOffset);
+    builder.append(", bufferOffset=");
+    builder.append(bufferOffset);
+    builder.append(", version=");
+    builder.append(version);
+    builder.append(", numMetaBytes=");
+    builder.append(numMetaBytes);
+    builder.append(", numKeyValueBytes=");
+    builder.append(numKeyValueBytes);
+    builder.append(", includesMvccVersion=");
+    builder.append(includesMvccVersion);
+    builder.append(", numRowBytes=");
+    builder.append(numRowBytes);
+    builder.append(", numFamilyBytes=");
+    builder.append(numFamilyBytes);
+    builder.append(", numQualifierBytes=");
+    builder.append(numQualifierBytes);
+    builder.append(", numTimestampBytes=");
+    builder.append(numTimestampBytes);
+    builder.append(", numMvccVersionBytes=");
+    builder.append(numMvccVersionBytes);
+    builder.append(", numValueBytes=");
+    builder.append(numValueBytes);
+    builder.append(", nextNodeOffsetWidth=");
+    builder.append(nextNodeOffsetWidth);
+    builder.append(", familyOffsetWidth=");
+    builder.append(familyOffsetWidth);
+    builder.append(", qualifierOffsetWidth=");
+    builder.append(qualifierOffsetWidth);
+    builder.append(", timestampIndexWidth=");
+    builder.append(timestampIndexWidth);
+    builder.append(", mvccVersionIndexWidth=");
+    builder.append(mvccVersionIndexWidth);
+    builder.append(", valueOffsetWidth=");
+    builder.append(valueOffsetWidth);
+    builder.append(", valueLengthWidth=");
+    builder.append(valueLengthWidth);
+    builder.append(", rowTreeDepth=");
+    builder.append(rowTreeDepth);
+    builder.append(", maxRowLength=");
+    builder.append(maxRowLength);
+    builder.append(", maxQualifierLength=");
+    builder.append(maxQualifierLength);
+    builder.append(", minTimestamp=");
+    builder.append(minTimestamp);
+    builder.append(", timestampDeltaWidth=");
+    builder.append(timestampDeltaWidth);
+    builder.append(", minMvccVersion=");
+    builder.append(minMvccVersion);
+    builder.append(", mvccVersionDeltaWidth=");
+    builder.append(mvccVersionDeltaWidth);
+    builder.append(", allSameType=");
+    builder.append(allSameType);
+    builder.append(", allTypes=");
+    builder.append(allTypes);
+    builder.append(", numUniqueRows=");
+    builder.append(numUniqueRows);
+    builder.append(", numUniqueFamilies=");
+    builder.append(numUniqueFamilies);
+    builder.append(", numUniqueQualifiers=");
+    builder.append(numUniqueQualifiers);
+    builder.append("]");
+    return builder.toString();
+  }
+
+
+  /************** absolute getters *******************/
+
+  public int getAbsoluteMetaOffset() {
+    return arrayOffset + bufferOffset;
+  }
+
+  public int getAbsoluteRowOffset() {
+    return getAbsoluteMetaOffset() + numMetaBytes;
+  }
+
+  public int getAbsoluteFamilyOffset() {
+    return getAbsoluteRowOffset() + numRowBytes;
+  }
+
+  public int getAbsoluteQualifierOffset() {
+    return getAbsoluteFamilyOffset() + numFamilyBytes;
+  }
+
+  public int getAbsoluteTimestampOffset() {
+    return getAbsoluteQualifierOffset() + numQualifierBytes;
+  }
+
+  public int getAbsoluteMvccVersionOffset() {
+    return getAbsoluteTimestampOffset() + numTimestampBytes;
+  }
+
+  public int getAbsoluteValueOffset() {
+    return getAbsoluteMvccVersionOffset() + numMvccVersionBytes;
+  }
+
+
+  /*************** get/set ***************************/
+
+  public int getTimestampDeltaWidth() {
+    return timestampDeltaWidth;
+  }
+
+  public void setTimestampDeltaWidth(int timestampDeltaWidth) {
+    this.timestampDeltaWidth = timestampDeltaWidth;
+  }
+
+  public int getValueOffsetWidth() {
+    return valueOffsetWidth;
+  }
+
+  public void setValueOffsetWidth(int dataOffsetWidth) {
+    this.valueOffsetWidth = dataOffsetWidth;
+  }
+
+  public int getValueLengthWidth() {
+    return valueLengthWidth;
+  }
+
+  public void setValueLengthWidth(int dataLengthWidth) {
+    this.valueLengthWidth = dataLengthWidth;
+  }
+
+  public int getMaxRowLength() {
+    return maxRowLength;
+  }
+
+  public void setMaxRowLength(int maxRowLength) {
+    this.maxRowLength = maxRowLength;
+  }
+
+  public long getMinTimestamp() {
+    return minTimestamp;
+  }
+
+  public void setMinTimestamp(long minTimestamp) {
+    this.minTimestamp = minTimestamp;
+  }
+
+  public byte getAllTypes() {
+    return allTypes;
+  }
+
+  public void setAllTypes(byte allTypes) {
+    this.allTypes = allTypes;
+  }
+
+  public boolean isAllSameType() {
+    return allSameType;
+  }
+
+  public void setAllSameType(boolean allSameType) {
+    this.allSameType = allSameType;
+  }
+
+  public int getNextNodeOffsetWidth() {
+    return nextNodeOffsetWidth;
+  }
+
+  public void setNextNodeOffsetWidth(int nextNodeOffsetWidth) {
+    this.nextNodeOffsetWidth = nextNodeOffsetWidth;
+  }
+
+  public int getNumRowBytes() {
+    return numRowBytes;
+  }
+
+  public void setNumRowBytes(int numRowBytes) {
+    this.numRowBytes = numRowBytes;
+  }
+
+  public int getNumTimestampBytes() {
+    return numTimestampBytes;
+  }
+
+  public void setNumTimestampBytes(int numTimestampBytes) {
+    this.numTimestampBytes = numTimestampBytes;
+  }
+
+  public int getNumValueBytes() {
+    return numValueBytes;
+  }
+
+  public void setNumValueBytes(int numValueBytes) {
+    this.numValueBytes = numValueBytes;
+  }
+
+  public int getNumMetaBytes() {
+    return numMetaBytes;
+  }
+
+  public void setNumMetaBytes(int numMetaBytes) {
+    this.numMetaBytes = numMetaBytes;
+  }
+
+  public int getArrayOffset() {
+    return arrayOffset;
+  }
+
+  public void setArrayOffset(int arrayOffset) {
+    this.arrayOffset = arrayOffset;
+  }
+
+  public int getBufferOffset() {
+    return bufferOffset;
+  }
+
+  public void setBufferOffset(int bufferOffset) {
+    this.bufferOffset = bufferOffset;
+  }
+
+  public int getNumKeyValueBytes() {
+    return numKeyValueBytes;
+  }
+
+  public void setNumKeyValueBytes(int numKeyValueBytes) {
+    this.numKeyValueBytes = numKeyValueBytes;
+  }
+
+  public int getRowTreeDepth() {
+    return rowTreeDepth;
+  }
+
+  public void setRowTreeDepth(int rowTreeDepth) {
+    this.rowTreeDepth = rowTreeDepth;
+  }
+
+  public int getNumMvccVersionBytes() {
+    return numMvccVersionBytes;
+  }
+
+  public void setNumMvccVersionBytes(int numMvccVersionBytes) {
+    this.numMvccVersionBytes = numMvccVersionBytes;
+  }
+
+  public int getMvccVersionDeltaWidth() {
+    return mvccVersionDeltaWidth;
+  }
+
+  public void setMvccVersionDeltaWidth(int mvccVersionDeltaWidth) {
+    this.mvccVersionDeltaWidth = mvccVersionDeltaWidth;
+  }
+
+  public long getMinMvccVersion() {
+    return minMvccVersion;
+  }
+
+  public void setMinMvccVersion(long minMvccVersion) {
+    this.minMvccVersion = minMvccVersion;
+  }
+
+  public int getNumFamilyBytes() {
+    return numFamilyBytes;
+  }
+
+  public void setNumFamilyBytes(int numFamilyBytes) {
+    this.numFamilyBytes = numFamilyBytes;
+  }
+
+  public int getFamilyOffsetWidth() {
+    return familyOffsetWidth;
+  }
+
+  public void setFamilyOffsetWidth(int familyOffsetWidth) {
+    this.familyOffsetWidth = familyOffsetWidth;
+  }
+
+  public int getNumUniqueRows() {
+    return numUniqueRows;
+  }
+
+  public void setNumUniqueRows(int numUniqueRows) {
+    this.numUniqueRows = numUniqueRows;
+  }
+
+  public int getNumUniqueFamilies() {
+    return numUniqueFamilies;
+  }
+
+  public void setNumUniqueFamilies(int numUniqueFamilies) {
+    this.numUniqueFamilies = numUniqueFamilies;
+  }
+
+  public int getNumUniqueQualifiers() {
+    return numUniqueQualifiers;
+  }
+
+  public void setNumUniqueQualifiers(int numUniqueQualifiers) {
+    this.numUniqueQualifiers = numUniqueQualifiers;
+  }
+
+  public int getNumQualifierBytes() {
+    return numQualifierBytes;
+  }
+
+  public void setNumQualifierBytes(int numQualifierBytes) {
+    this.numQualifierBytes = numQualifierBytes;
+  }
+
+  public int getQualifierOffsetWidth() {
+    return qualifierOffsetWidth;
+  }
+
+  public void setQualifierOffsetWidth(int qualifierOffsetWidth) {
+    this.qualifierOffsetWidth = qualifierOffsetWidth;
+  }
+
+  public int getMaxQualifierLength() {
+    return maxQualifierLength;
+  }
+
+  public void setMaxQualifierLength(int maxQualifierLength) {
+    this.maxQualifierLength = maxQualifierLength;
+  }
+
+  public int getTimestampIndexWidth() {
+    return timestampIndexWidth;
+  }
+
+  public void setTimestampIndexWidth(int timestampIndexWidth) {
+    this.timestampIndexWidth = timestampIndexWidth;
+  }
+
+  public int getMvccVersionIndexWidth() {
+    return mvccVersionIndexWidth;
+  }
+
+  public void setMvccVersionIndexWidth(int mvccVersionIndexWidth) {
+    this.mvccVersionIndexWidth = mvccVersionIndexWidth;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  public void setVersion(int version) {
+    this.version = version;
+  }
+
+  public boolean isIncludesMvccVersion() {
+    return includesMvccVersion;
+  }
+
+  public void setIncludesMvccVersion(boolean includesMvccVersion) {
+    this.includesMvccVersion = includesMvccVersion;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java?rev=1455672&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeCodec.java Tue Mar 12 19:07:34 2013
@@ -0,0 +1,209 @@
+/*
+ * 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;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.KeyComparator;
+import org.apache.hadoop.hbase.KeyValue.MetaKeyComparator;
+import org.apache.hadoop.hbase.KeyValue.RootKeyComparator;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.codec.prefixtree.decode.DecoderFactory;
+import org.apache.hadoop.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.EncoderFactory;
+import org.apache.hadoop.hbase.codec.prefixtree.encode.PrefixTreeEncoder;
+import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
+import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+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.hbase.io.hfile.BlockType;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+import org.apache.hadoop.io.RawComparator;
+
+/**
+ * This class is created via reflection in DataBlockEncoding enum. Update the enum if class name or
+ * package changes.
+ * <p/>
+ * PrefixTreeDataBlockEncoder implementation of DataBlockEncoder. This is the primary entry point
+ * for PrefixTree encoding and decoding. Encoding is delegated to instances of
+ * {@link PrefixTreeEncoder}, and decoding is delegated to instances of
+ * {@link org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher}. Encoder and decoder instances are
+ * created and recycled by static PtEncoderFactory and PtDecoderFactory.
+ */
+@InterfaceAudience.Private
+public class PrefixTreeCodec implements DataBlockEncoder{
+
+  /**
+   * no-arg constructor for reflection
+   */
+  public PrefixTreeCodec() {
+  }
+
+  /**
+   * Copied from BufferedDataBlockEncoder. Almost definitely can be improved, but i'm not familiar
+   * enough with the concept of the HFileBlockEncodingContext.
+   */
+  @Override
+  public void encodeKeyValues(ByteBuffer in, boolean includesMvccVersion,
+      HFileBlockEncodingContext blkEncodingCtx) 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();
+    DataOutputStream dataOut = encodingCtx.getOutputStreamForEncoder();
+    internalEncodeKeyValues(dataOut, in, includesMvccVersion);
+
+    //do i need to check this, or will it always be DataBlockEncoding.PREFIX_TREE?
+    if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) {
+      encodingCtx.postEncoding(BlockType.ENCODED_DATA);
+    } else {
+      encodingCtx.postEncoding(BlockType.DATA);
+    }
+  }
+
+  private void internalEncodeKeyValues(DataOutputStream encodedOutputStream,
+      ByteBuffer rawKeyValues, boolean includesMvccVersion) throws IOException {
+    rawKeyValues.rewind();
+    PrefixTreeEncoder builder = EncoderFactory.checkOut(encodedOutputStream, includesMvccVersion);
+
+    try{
+      KeyValue kv;
+      while ((kv = KeyValueUtil.nextShallowCopy(rawKeyValues, includesMvccVersion)) != null) {
+        builder.write(kv);
+      }
+      builder.flush();
+    }finally{
+      EncoderFactory.checkIn(builder);
+    }
+  }
+
+
+  @Override
+  public ByteBuffer decodeKeyValues(DataInputStream source, boolean includesMvccVersion)
+      throws IOException {
+    return decodeKeyValues(source, 0, 0, includesMvccVersion);
+  }
+
+
+  /**
+   * I don't think this method is called during normal HBase operation, so efficiency is not
+   * important.
+   */
+  @Override
+  public ByteBuffer decodeKeyValues(DataInputStream source, int allocateHeaderLength,
+      int skipLastBytes, boolean includesMvccVersion) throws IOException {
+    ByteBuffer sourceAsBuffer = ByteBufferUtils.drainInputStreamToBuffer(source);// waste
+    sourceAsBuffer.mark();
+    PrefixTreeBlockMeta blockMeta = new PrefixTreeBlockMeta(sourceAsBuffer);
+    sourceAsBuffer.rewind();
+    int numV1BytesWithHeader = allocateHeaderLength + blockMeta.getNumKeyValueBytes();
+    byte[] keyValueBytesWithHeader = new byte[numV1BytesWithHeader];
+    ByteBuffer result = ByteBuffer.wrap(keyValueBytesWithHeader);
+    result.rewind();
+    CellSearcher searcher = null;
+    try {
+      searcher = DecoderFactory.checkOut(sourceAsBuffer, includesMvccVersion);
+      while (searcher.advance()) {
+        KeyValue currentCell = KeyValueUtil.copyToNewKeyValue(searcher.current());
+        // needs to be modified for DirectByteBuffers. no existing methods to
+        // write VLongs to byte[]
+        int offset = result.arrayOffset() + result.position();
+        KeyValueUtil.appendToByteArray(currentCell, result.array(), offset);
+        int keyValueLength = KeyValueUtil.length(currentCell);
+        ByteBufferUtils.skip(result, keyValueLength);
+        offset += keyValueLength;
+        if (includesMvccVersion) {
+          ByteBufferUtils.writeVLong(result, currentCell.getMvccVersion());
+        }
+      }
+      result.position(result.limit());//make it appear as if we were appending
+      return result;
+    } finally {
+      DecoderFactory.checkIn(searcher);
+    }
+  }
+
+
+  @Override
+  public ByteBuffer getFirstKeyInBlock(ByteBuffer block) {
+    block.rewind();
+    PrefixTreeArraySearcher searcher = null;
+    try {
+      //should i includeMemstoreTS (second argument)?  i think PrefixKeyDeltaEncoder is, so i will
+      searcher = DecoderFactory.checkOut(block, true);
+      if (!searcher.positionAtFirstCell()) {
+        return null;
+      }
+      return KeyValueUtil.copyKeyToNewByteBuffer(searcher.current());
+    } finally {
+      DecoderFactory.checkIn(searcher);
+    }
+  }
+
+  @Override
+  public HFileBlockEncodingContext newDataBlockEncodingContext(Algorithm compressionAlgorithm,
+      DataBlockEncoding encoding, byte[] header) {
+    if(DataBlockEncoding.PREFIX_TREE != encoding){
+      //i'm not sure why encoding is in the interface.  Each encoder implementation should probably
+      //know it's encoding type
+      throw new IllegalArgumentException("only DataBlockEncoding.PREFIX_TREE supported");
+    }
+    return new HFileBlockDefaultEncodingContext(compressionAlgorithm, encoding, header);
+  }
+
+  @Override
+  public HFileBlockDecodingContext newDataBlockDecodingContext(Algorithm compressionAlgorithm) {
+    return new HFileBlockDefaultDecodingContext(compressionAlgorithm);
+  }
+
+  /**
+   * Is this the correct handling of an illegal comparator?  How to prevent that from getting all
+   * the way to this point.
+   */
+  @Override
+  public EncodedSeeker createSeeker(RawComparator<byte[]> comparator, boolean includesMvccVersion) {
+    if(! (comparator instanceof KeyComparator)){
+      throw new IllegalArgumentException("comparator must be KeyValue.KeyComparator");
+    }
+    if(comparator instanceof MetaKeyComparator){
+      throw new IllegalArgumentException("DataBlockEncoding.PREFIX_TREE not compatible with META "
+          +"table");
+    }
+    if(comparator instanceof RootKeyComparator){
+      throw new IllegalArgumentException("DataBlockEncoding.PREFIX_TREE not compatible with ROOT "
+          +"table");
+    }
+
+    return new PrefixTreeSeeker(includesMvccVersion);
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java?rev=1455672&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java Tue Mar 12 19:07:34 2013
@@ -0,0 +1,215 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.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.codec.prefixtree.decode.DecoderFactory;
+import org.apache.hadoop.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher;
+import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder.EncodedSeeker;
+
+/**
+ * These methods have the same definition as any implementation of the EncodedSeeker.
+ *
+ * In the future, the EncodedSeeker could be modified to work with the Cell interface directly.  It
+ * currently returns a new KeyValue object each time getKeyValue is called.  This is not horrible,
+ * but in order to create a new KeyValue object, we must first allocate a new byte[] and copy in
+ * the data from the PrefixTreeCell.  It is somewhat heavyweight right now.
+ */
+@InterfaceAudience.Private
+public class PrefixTreeSeeker implements EncodedSeeker {
+
+  protected ByteBuffer block;
+  protected boolean includeMvccVersion;
+  protected PrefixTreeArraySearcher ptSearcher;
+
+  public PrefixTreeSeeker(boolean includeMvccVersion) {
+    this.includeMvccVersion = includeMvccVersion;
+  }
+
+  @Override
+  public void setCurrentBuffer(ByteBuffer fullBlockBuffer) {
+    block = fullBlockBuffer;
+    ptSearcher = DecoderFactory.checkOut(block, includeMvccVersion);
+    rewind();
+  }
+
+  /**
+   * Currently unused.
+   * <p/>
+   * TODO performance leak. should reuse the searchers. hbase does not currently have a hook where
+   * this can be called
+   */
+  public void releaseCurrentSearcher(){
+    DecoderFactory.checkIn(ptSearcher);
+  }
+
+
+  @Override
+  public ByteBuffer getKeyDeepCopy() {
+    return KeyValueUtil.copyKeyToNewByteBuffer(ptSearcher.current());
+  }
+
+
+  @Override
+  public ByteBuffer getValueShallowCopy() {
+    return CellUtil.getValueBufferShallowCopy(ptSearcher.current());
+  }
+
+  /**
+   * currently must do deep copy into new array
+   */
+  @Override
+  public ByteBuffer getKeyValueBuffer() {
+    return KeyValueUtil.copyToNewByteBuffer(ptSearcher.current());
+  }
+
+  /**
+   * currently must do deep copy into new array
+   */
+  @Override
+  public KeyValue getKeyValue() {
+    return KeyValueUtil.copyToNewKeyValue(ptSearcher.current());
+  }
+
+  /**
+   * Currently unused.
+   * <p/>
+   * A nice, lightweight reference, though the underlying cell is transient. This method may return
+   * the same reference to the backing PrefixTreeCell repeatedly, while other implementations may
+   * return a different reference for each Cell.
+   * <p/>
+   * The goal will be to transition the upper layers of HBase, like Filters and KeyValueHeap, to
+   * use this method instead of the getKeyValue() methods above.
+   */
+  public Cell get() {
+    return ptSearcher.current();
+  }
+
+  @Override
+  public void rewind() {
+    ptSearcher.positionAtFirstCell();
+  }
+
+  @Override
+  public boolean next() {
+    return ptSearcher.advance();
+  }
+
+//  @Override
+  public boolean advance() {
+    return ptSearcher.advance();
+  }
+
+
+  private static final boolean USE_POSITION_BEFORE = false;
+
+  /**
+   * Seek forward only (should be called reseekToKeyInBlock?).
+   * <p/>
+   * If the exact key is found look at the seekBefore variable and:<br/>
+   * - if true: go to the previous key if it's true<br/>
+   * - if false: stay on the exact key
+   * <p/>
+   * If the exact key is not found, then go to the previous key *if possible*, but remember to
+   * leave the scanner in a valid state if possible.
+   * <p/>
+   * @param keyOnlyBytes KeyValue format of a Cell's key at which to position the seeker
+   * @param offset offset into the keyOnlyBytes array
+   * @param length number of bytes of the keyOnlyBytes array to use
+   * @param forceBeforeOnExactMatch if an exact match is found and seekBefore=true, back up 1 Cell
+   * @return 0 if the seeker is on the exact key<br/>
+   *         1 if the seeker is not on the key for any reason, including seekBefore being true
+   */
+  @Override
+  public int seekToKeyInBlock(byte[] keyOnlyBytes, int offset, int length,
+      boolean forceBeforeOnExactMatch) {
+    if (USE_POSITION_BEFORE) {
+      return seekToOrBeforeUsingPositionAtOrBefore(keyOnlyBytes, offset, length,
+        forceBeforeOnExactMatch);
+    }else{
+      return seekToOrBeforeUsingPositionAtOrAfter(keyOnlyBytes, offset, length,
+        forceBeforeOnExactMatch);
+    }
+  }
+
+
+
+  /*
+   * Support both of these options since the underlying PrefixTree supports both.  Possibly
+   * expand the EncodedSeeker to utilize them both.
+   */
+
+  protected int seekToOrBeforeUsingPositionAtOrBefore(byte[] keyOnlyBytes, int offset, int length,
+      boolean forceBeforeOnExactMatch){
+    // this does a deep copy of the key byte[] because the CellSearcher interface wants a Cell
+    KeyValue kv = KeyValue.createKeyValueFromKey(keyOnlyBytes, offset, length);
+
+    CellScannerPosition position = ptSearcher.seekForwardToOrBefore(kv);
+
+    if(CellScannerPosition.AT == position){
+      if (forceBeforeOnExactMatch) {
+        ptSearcher.previous();
+        return 1;
+      }
+      return 0;
+    }
+
+    return 1;
+  }
+
+
+  protected int seekToOrBeforeUsingPositionAtOrAfter(byte[] keyOnlyBytes, int offset, int length,
+      boolean forceBeforeOnExactMatch){
+    // this does a deep copy of the key byte[] because the CellSearcher interface wants a Cell
+    KeyValue kv = KeyValue.createKeyValueFromKey(keyOnlyBytes, offset, length);
+
+    //should probably switch this to use the seekForwardToOrBefore method
+    CellScannerPosition position = ptSearcher.seekForwardToOrAfter(kv);
+
+    if(CellScannerPosition.AT == position){
+      if (forceBeforeOnExactMatch) {
+        ptSearcher.previous();
+        return 1;
+      }
+      return 0;
+
+    }
+
+    if(CellScannerPosition.AFTER == position){
+      if(!ptSearcher.isBeforeFirst()){
+        ptSearcher.previous();
+      }
+      return 1;
+    }
+
+    if(position == CellScannerPosition.AFTER_LAST){
+      return 1;
+    }
+
+    throw new RuntimeException("unexpected CellScannerPosition:"+position);
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/ArraySearcherPool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/ArraySearcherPool.java?rev=1455672&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/ArraySearcherPool.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/ArraySearcherPool.java Tue Mar 12 19:07:34 2013
@@ -0,0 +1,63 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.Queue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Pools PrefixTreeArraySearcher objects. Each Searcher can consist of hundreds or thousands of
+ * objects and 1 is needed for each HFile during a Get operation. With tens of thousands of
+ * Gets/second, reusing these searchers may save a lot of young gen collections.
+ * <p/>
+ * Alternative implementation would be a ByteBufferSearcherPool (not implemented yet).
+ */
+@InterfaceAudience.Private
+public class ArraySearcherPool {
+
+  /**
+   * One decoder is needed for each storefile for each Get operation so we may need hundreds at the
+   * same time, however, decoding is a CPU bound activity so should limit this to something in the
+   * realm of maximum reasonable active threads.
+   */
+  private static final Integer MAX_POOL_SIZE = 1000;
+
+  protected Queue<PrefixTreeArraySearcher> pool
+    = new LinkedBlockingQueue<PrefixTreeArraySearcher>(MAX_POOL_SIZE);
+
+  public PrefixTreeArraySearcher checkOut(ByteBuffer buffer, boolean includesMvccVersion) {
+    PrefixTreeArraySearcher searcher = pool.poll();//will return null if pool is empty
+    searcher = DecoderFactory.ensureArraySearcherValid(buffer, searcher, includesMvccVersion);
+    return searcher;
+  }
+
+  public void checkIn(PrefixTreeArraySearcher searcher) {
+    searcher.releaseBlockReference();
+    pool.offer(searcher);
+  }
+
+  @Override
+  public String toString() {
+    return ("poolSize:" + pool.size());
+  }
+
+}
\ No newline at end of file

Added: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/DecoderFactory.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/DecoderFactory.java?rev=1455672&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/DecoderFactory.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/DecoderFactory.java Tue Mar 12 19:07:34 2013
@@ -0,0 +1,89 @@
+/*
+ * 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 java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
+
+/**
+ * Static wrapper class for the ArraySearcherPool.
+ */
+@InterfaceAudience.Private
+public class DecoderFactory {
+
+  private static final ArraySearcherPool POOL = new ArraySearcherPool();
+
+  //TODO will need a PrefixTreeSearcher on top of CellSearcher
+  public static PrefixTreeArraySearcher checkOut(final ByteBuffer buffer, 
+      boolean includeMvccVersion) {
+    if (buffer.isDirect()) {
+      throw new IllegalArgumentException("DirectByteBuffers not supported yet");
+      // TODO implement PtByteBufferBlockScanner
+    }
+
+    PrefixTreeArraySearcher searcher = POOL.checkOut(buffer,
+      includeMvccVersion);
+    return searcher;
+  }
+
+  public static void checkIn(CellSearcher pSearcher) {
+    if (pSearcher == null) {
+      return;
+    }
+    if (! (pSearcher instanceof PrefixTreeArraySearcher)) {
+      throw new IllegalArgumentException("Cannot return "+pSearcher.getClass()+" to "
+          +DecoderFactory.class);
+    }
+    PrefixTreeArraySearcher searcher = (PrefixTreeArraySearcher) pSearcher;
+    POOL.checkIn(searcher);
+  }
+
+
+  /**************************** helper ******************************/
+
+  public static PrefixTreeArraySearcher ensureArraySearcherValid(ByteBuffer buffer,
+      PrefixTreeArraySearcher searcher, boolean includeMvccVersion) {
+    if (searcher == null) {
+      PrefixTreeBlockMeta blockMeta = new PrefixTreeBlockMeta(buffer);
+      searcher = new PrefixTreeArraySearcher(blockMeta, blockMeta.getRowTreeDepth(),
+          blockMeta.getMaxRowLength(), blockMeta.getMaxQualifierLength());
+      searcher.initOnBlock(blockMeta, buffer.array(), includeMvccVersion);
+      return searcher;
+    }
+
+    PrefixTreeBlockMeta blockMeta = searcher.getBlockMeta();
+    blockMeta.initOnBlock(buffer);
+    if (!searcher.areBuffersBigEnough()) {
+      int maxRowTreeStackNodes = Math.max(blockMeta.getRowTreeDepth(),
+        searcher.getMaxRowTreeStackNodes());
+      int rowBufferLength = Math.max(blockMeta.getMaxRowLength(), searcher.getRowBufferLength());
+      int qualifierBufferLength = Math.max(blockMeta.getMaxQualifierLength(),
+        searcher.getQualifierBufferLength());
+      searcher = new PrefixTreeArraySearcher(blockMeta, maxRowTreeStackNodes, rowBufferLength,
+          qualifierBufferLength);
+    }
+    //this is where we parse the BlockMeta
+    searcher.initOnBlock(blockMeta, buffer.array(), includeMvccVersion);
+    return searcher;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayReversibleScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayReversibleScanner.java?rev=1455672&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayReversibleScanner.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayReversibleScanner.java Tue Mar 12 19:07:34 2013
@@ -0,0 +1,144 @@
+/*
+ * 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.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.codec.prefixtree.scanner.ReversibleCellScanner;
+
+/**
+ * Methods for going backwards through a PrefixTree block.  This class is split out on its own to
+ * simplify the Scanner superclass and Searcher subclass.
+ */
+@InterfaceAudience.Private
+public class PrefixTreeArrayReversibleScanner extends PrefixTreeArrayScanner implements
+    ReversibleCellScanner {
+
+  /***************** construct ******************************/
+
+  public PrefixTreeArrayReversibleScanner(PrefixTreeBlockMeta blockMeta, int rowTreeDepth,
+      int rowBufferLength, int qualifierBufferLength) {
+    super(blockMeta, rowTreeDepth, rowBufferLength, qualifierBufferLength);
+  }
+
+
+  /***************** Object methods ***************************/
+
+  @Override
+  public boolean equals(Object obj) {
+    //trivial override to confirm intent (findbugs)
+    return super.equals(obj);
+  }
+
+
+  /***************** methods **********************************/
+
+  @Override
+  public boolean previous() {
+    if (afterLast) {
+      afterLast = false;
+      positionAtLastCell();
+      return true;
+    }
+    if (beforeFirst) {
+      return false;
+    }
+    if (isFirstCellInRow()) {
+      previousRowInternal();
+      if (beforeFirst) {
+        return false;
+      }
+      populateLastNonRowFields();
+      return true;
+    }
+    populatePreviousNonRowFields();
+    return true;
+  }
+
+  @Override
+  public boolean previousRow(boolean endOfRow) {
+    previousRowInternal();
+    if(beforeFirst){
+      return false;
+    }
+    if(endOfRow){
+      populateLastNonRowFields();
+    }else{
+      populateFirstNonRowFields();
+    }
+    return true;
+  }
+
+  private boolean previousRowInternal() {
+    if (beforeFirst) {
+      return false;
+    }
+    if (afterLast) {
+      positionAtLastRow();
+      return true;
+    }
+    if (currentRowNode.hasOccurrences()) {
+      discardCurrentRowNode(false);
+      if(currentRowNode==null){
+        return false;
+      }
+    }
+    while (!beforeFirst) {
+      if (isDirectlyAfterNub()) {//we are about to back up to the nub
+        currentRowNode.resetFanIndex();//sets it to -1, which is before the first leaf
+        nubCellsRemain = true;//this positions us on the nub
+        return true;
+      }
+      if (currentRowNode.hasPreviousFanNodes()) {
+        followPreviousFan();
+        descendToLastRowFromCurrentPosition();
+      } else {// keep going up the stack until we find previous fan positions
+        discardCurrentRowNode(false);
+        if(currentRowNode==null){
+          return false;
+        }
+      }
+      if (currentRowNode.hasOccurrences()) {// escape clause
+        return true;// found some values
+      }
+    }
+    return false;// went past the beginning
+  }
+  
+  protected boolean isDirectlyAfterNub() {
+    return currentRowNode.isNub() && currentRowNode.getFanIndex()==0;
+  }
+
+  protected void positionAtLastRow() {
+    reInitFirstNode();
+    descendToLastRowFromCurrentPosition();
+  }
+
+  protected void descendToLastRowFromCurrentPosition() {
+    while (currentRowNode.hasChildren()) {
+      followLastFan();
+    }
+  }
+
+  protected void positionAtLastCell() {
+    positionAtLastRow();
+    populateLastNonRowFields();
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java?rev=1455672&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeArrayScanner.java Tue Mar 12 19:07:34 2013
@@ -0,0 +1,506 @@
+/*
+ * 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.CellScanner;
+import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hadoop.hbase.codec.prefixtree.decode.column.ColumnReader;
+import org.apache.hadoop.hbase.codec.prefixtree.decode.row.RowNodeReader;
+import org.apache.hadoop.hbase.codec.prefixtree.decode.timestamp.MvccVersionDecoder;
+import org.apache.hadoop.hbase.codec.prefixtree.decode.timestamp.TimestampDecoder;
+
+/**
+ * Extends PtCell and manipulates its protected fields.  Could alternatively contain a PtCell and
+ * call get/set methods.
+ *
+ * This is an "Array" scanner to distinguish from a future "ByteBuffer" scanner.  This
+ * implementation requires that the bytes be in a normal java byte[] for performance.  The
+ * alternative ByteBuffer implementation would allow for accessing data in an off-heap ByteBuffer
+ * without copying the whole buffer on-heap.
+ */
+@InterfaceAudience.Private
+public class PrefixTreeArrayScanner extends PrefixTreeCell implements CellScanner {
+
+  /***************** fields ********************************/
+
+  protected PrefixTreeBlockMeta blockMeta;
+
+  protected boolean beforeFirst;
+  protected boolean afterLast;
+
+  protected RowNodeReader[] rowNodes;
+  protected int rowNodeStackIndex;
+
+  protected RowNodeReader currentRowNode;
+  protected ColumnReader familyReader;
+  protected ColumnReader qualifierReader;
+  protected TimestampDecoder timestampDecoder;
+  protected MvccVersionDecoder mvccVersionDecoder;
+
+  protected boolean nubCellsRemain;
+  protected int currentCellIndex;
+
+
+  /*********************** construct ******************************/
+
+  // pass in blockMeta so we can initialize buffers big enough for all cells in the block
+  public PrefixTreeArrayScanner(PrefixTreeBlockMeta blockMeta, int rowTreeDepth, 
+      int rowBufferLength, int qualifierBufferLength) {
+    this.rowNodes = new RowNodeReader[rowTreeDepth];
+    for (int i = 0; i < rowNodes.length; ++i) {
+      rowNodes[i] = new RowNodeReader();
+    }
+    this.rowBuffer = new byte[rowBufferLength];
+    this.familyBuffer = new byte[PrefixTreeBlockMeta.MAX_FAMILY_LENGTH];
+    this.familyReader = new ColumnReader(familyBuffer, true);
+    this.qualifierBuffer = new byte[qualifierBufferLength];
+    this.qualifierReader = new ColumnReader(qualifierBuffer, false);
+    this.timestampDecoder = new TimestampDecoder();
+    this.mvccVersionDecoder = new MvccVersionDecoder();
+  }
+
+
+  /**************** init helpers ***************************************/
+
+  /**
+   * Call when first accessing a block.
+   * @return entirely new scanner if false
+   */
+  public boolean areBuffersBigEnough() {
+    if (rowNodes.length < blockMeta.getRowTreeDepth()) {
+      return false;
+    }
+    if (rowBuffer.length < blockMeta.getMaxRowLength()) {
+      return false;
+    }
+    if (qualifierBuffer.length < blockMeta.getMaxQualifierLength()) {
+      return false;
+    }
+    return true;
+  }
+
+  public void initOnBlock(PrefixTreeBlockMeta blockMeta, byte[] block,
+      boolean includeMvccVersion) {
+    this.block = block;
+    this.blockMeta = blockMeta;
+    this.familyOffset = familyBuffer.length;
+    this.familyReader.initOnBlock(blockMeta, block);
+    this.qualifierOffset = qualifierBuffer.length;
+    this.qualifierReader.initOnBlock(blockMeta, block);
+    this.timestampDecoder.initOnBlock(blockMeta, block);
+    this.mvccVersionDecoder.initOnBlock(blockMeta, block);
+    this.includeMvccVersion = includeMvccVersion;
+    resetToBeforeFirstEntry();
+  }
+
+  // Does this have to be in the CellScanner Interface?  TODO
+  public void resetToBeforeFirstEntry() {
+    beforeFirst = true;
+    afterLast = false;
+    rowNodeStackIndex = -1;
+    currentRowNode = null;
+    rowLength = 0;
+    familyOffset = familyBuffer.length;
+    familyLength = 0;
+    qualifierOffset = blockMeta.getMaxQualifierLength();
+    qualifierLength = 0;
+    nubCellsRemain = false;
+    currentCellIndex = -1;
+    timestamp = -1L;
+    type = DEFAULT_TYPE;
+    absoluteValueOffset = 0;//use 0 vs -1 so the cell is valid when value hasn't been initialized
+    valueLength = 0;// had it at -1, but that causes null Cell to add up to the wrong length
+  }
+
+  /**
+   * Call this before putting the scanner back into a pool so it doesn't hold the last used block
+   * in memory.
+   */
+  public void releaseBlockReference(){
+    block = null;
+  }
+
+
+  /********************** CellScanner **********************/
+
+  @Override
+  public Cell current() {
+    if(isOutOfBounds()){
+      return null;
+    }
+    return (Cell)this;
+  }
+
+  /******************* Object methods ************************/
+
+  @Override
+  public boolean equals(Object obj) {
+    //trivial override to confirm intent (findbugs)
+    return super.equals(obj);
+  }
+  
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  /**
+   * Override PrefixTreeCell.toString() with a check to see if the current cell is valid.
+   */
+  @Override
+  public String toString() {
+    Cell currentCell = current();
+    if(currentCell==null){
+      return "null";
+    }
+    return ((PrefixTreeCell)currentCell).getKeyValueString();
+  }
+
+
+  /******************* advance ***************************/
+
+  public boolean positionAtFirstCell() {
+    reInitFirstNode();
+    return advance();
+  }
+
+  @Override
+  public boolean advance() {
+    if (afterLast) {
+      return false;
+    }
+    if (!hasOccurrences()) {
+      resetToBeforeFirstEntry();
+    }
+    if (beforeFirst || isLastCellInRow()) {
+      nextRow();
+      if (afterLast) {
+        return false;
+      }
+    } else {
+      ++currentCellIndex;
+    }
+
+    populateNonRowFields(currentCellIndex);
+    return true;
+  }
+
+
+  public boolean nextRow() {
+    nextRowInternal();
+    if (afterLast) {
+      return false;
+    }
+    populateNonRowFields(currentCellIndex);
+    return true;
+  }
+
+
+  /**
+   * This method is safe to call when the scanner is not on a fully valid row node, as in the case
+   * of a row token miss in the Searcher
+   * @return true if we are positioned on a valid row, false if past end of block
+   */
+  protected boolean nextRowInternal() {
+    if (afterLast) {
+      return false;
+    }
+    if (beforeFirst) {
+      initFirstNode();
+      if (currentRowNode.hasOccurrences()) {
+        if (currentRowNode.isNub()) {
+          nubCellsRemain = true;
+        }
+        currentCellIndex = 0;
+        return true;
+      }
+    }
+    if (currentRowNode.isLeaf()) {
+      discardCurrentRowNode(true);
+    }
+    while (!afterLast) {
+      if (nubCellsRemain) {
+        nubCellsRemain = false;
+      }
+      if (currentRowNode.hasMoreFanNodes()) {
+        followNextFan();
+        if (currentRowNode.hasOccurrences()) {
+          currentCellIndex = 0;
+          return true;
+        }// found some values
+      } else {
+        discardCurrentRowNode(true);
+      }
+    }
+    return false;// went past the end
+  }
+
+
+  /**************** secondary traversal methods ******************************/
+
+  protected void reInitFirstNode() {
+    resetToBeforeFirstEntry();
+    initFirstNode();
+  }
+
+  protected void initFirstNode() {
+    int offsetIntoUnderlyingStructure = blockMeta.getAbsoluteRowOffset();
+    rowNodeStackIndex = 0;
+    currentRowNode = rowNodes[0];
+    currentRowNode.initOnBlock(blockMeta, block, offsetIntoUnderlyingStructure);
+    appendCurrentTokenToRowBuffer();
+    beforeFirst = false;
+  }
+
+  protected void followFirstFan() {
+    followFan(0);
+  }
+
+  protected void followPreviousFan() {
+    int nextFanPosition = currentRowNode.getFanIndex() - 1;
+    followFan(nextFanPosition);
+  }
+
+  protected void followCurrentFan() {
+    int currentFanPosition = currentRowNode.getFanIndex();
+    followFan(currentFanPosition);
+  }
+
+  protected void followNextFan() {
+    int nextFanPosition = currentRowNode.getFanIndex() + 1;
+    followFan(nextFanPosition);
+  }
+
+  protected void followLastFan() {
+    followFan(currentRowNode.getLastFanIndex());
+  }
+
+  protected void followFan(int fanIndex) {
+    currentRowNode.setFanIndex(fanIndex);
+    appendToRowBuffer(currentRowNode.getFanByte(fanIndex));
+
+    int nextOffsetIntoUnderlyingStructure = currentRowNode.getOffset()
+        + currentRowNode.getNextNodeOffset(fanIndex, blockMeta);
+    ++rowNodeStackIndex;
+
+    currentRowNode = rowNodes[rowNodeStackIndex];
+    currentRowNode.initOnBlock(blockMeta, block, nextOffsetIntoUnderlyingStructure);
+
+    //TODO getToken is spewing garbage
+    appendCurrentTokenToRowBuffer();
+    if (currentRowNode.isNub()) {
+      nubCellsRemain = true;
+    }
+    currentCellIndex = 0;
+  }
+
+  /**
+   * @param forwards which marker to set if we overflow
+   */
+  protected void discardCurrentRowNode(boolean forwards) {
+    RowNodeReader rowNodeBeingPopped = currentRowNode;
+    --rowNodeStackIndex;// pop it off the stack
+    if (rowNodeStackIndex < 0) {
+      currentRowNode = null;
+      if (forwards) {
+        markAfterLast();
+      } else {
+        markBeforeFirst();
+      }
+      return;
+    }
+    popFromRowBuffer(rowNodeBeingPopped);
+    currentRowNode = rowNodes[rowNodeStackIndex];
+  }
+
+  protected void markBeforeFirst() {
+    beforeFirst = true;
+    afterLast = false;
+    currentRowNode = null;
+  }
+
+  protected void markAfterLast() {
+    beforeFirst = false;
+    afterLast = true;
+    currentRowNode = null;
+  }
+
+
+  /***************** helper methods **************************/
+
+  protected void appendCurrentTokenToRowBuffer() {
+    System.arraycopy(block, currentRowNode.getTokenArrayOffset(), rowBuffer, rowLength, 
+      currentRowNode.getTokenLength());
+    rowLength += currentRowNode.getTokenLength();
+  }
+
+  protected void appendToRowBuffer(byte b) {
+    rowBuffer[rowLength] = b;
+    ++rowLength;
+  }
+
+  protected void popFromRowBuffer(RowNodeReader rowNodeBeingPopped) {
+    rowLength -= rowNodeBeingPopped.getTokenLength();
+    --rowLength; // pop the parent's fan byte
+  }
+
+  protected boolean hasOccurrences() {
+    return currentRowNode != null && currentRowNode.hasOccurrences();
+  }
+
+  protected boolean isBranch() {
+    return currentRowNode != null && !currentRowNode.hasOccurrences()
+        && currentRowNode.hasChildren();
+  }
+
+  protected boolean isNub() {
+    return currentRowNode != null && currentRowNode.hasOccurrences()
+        && currentRowNode.hasChildren();
+  }
+
+  protected boolean isLeaf() {
+    return currentRowNode != null && currentRowNode.hasOccurrences()
+        && !currentRowNode.hasChildren();
+  }
+
+  //TODO expose this in a PrefixTreeScanner interface
+  public boolean isBeforeFirst(){
+    return beforeFirst;
+  }
+
+  public boolean isAfterLast(){
+    return afterLast;
+  }
+
+  protected boolean isOutOfBounds(){
+    return beforeFirst || afterLast;
+  }
+
+  protected boolean isFirstCellInRow() {
+    return currentCellIndex == 0;
+  }
+
+  protected boolean isLastCellInRow() {
+    return currentCellIndex == currentRowNode.getLastCellIndex();
+  }
+
+
+  /********************* fill in family/qualifier/ts/type/value ************/
+
+  protected int populateNonRowFieldsAndCompareTo(int cellNum, Cell key) {
+    populateNonRowFields(cellNum);
+    return CellComparator.compareStatic(this, key);
+  }
+
+  protected void populateFirstNonRowFields() {
+    populateNonRowFields(0);
+  }
+
+  protected void populatePreviousNonRowFields() {
+    populateNonRowFields(currentCellIndex - 1);
+  }
+
+  protected void populateLastNonRowFields() {
+    populateNonRowFields(currentRowNode.getLastCellIndex());
+  }
+
+  protected void populateNonRowFields(int cellIndex) {
+    currentCellIndex = cellIndex;
+    populateFamily();
+    populateQualifier();
+    populateTimestamp();
+    populateMvccVersion();
+    populateType();
+    populateValueOffsets();
+  }
+
+  protected void populateFamily() {
+    int familyTreeIndex = currentRowNode.getFamilyOffset(currentCellIndex, blockMeta);
+    familyOffset = familyReader.populateBuffer(familyTreeIndex).getColumnOffset();
+    familyLength = familyReader.getColumnLength();
+  }
+
+  protected void populateQualifier() {
+    int qualifierTreeIndex = currentRowNode.getColumnOffset(currentCellIndex, blockMeta);
+    qualifierOffset = qualifierReader.populateBuffer(qualifierTreeIndex).getColumnOffset();
+    qualifierLength = qualifierReader.getColumnLength();
+  }
+
+  protected void populateTimestamp() {
+    if (blockMeta.isAllSameTimestamp()) {
+      timestamp = blockMeta.getMinTimestamp();
+    } else {
+      int timestampIndex = currentRowNode.getTimestampIndex(currentCellIndex, blockMeta);
+      timestamp = timestampDecoder.getLong(timestampIndex);
+    }
+  }
+
+  protected void populateMvccVersion() {
+    if (blockMeta.isAllSameMvccVersion()) {
+      mvccVersion = blockMeta.getMinMvccVersion();
+    } else {
+      int mvccVersionIndex = currentRowNode.getMvccVersionIndex(currentCellIndex,
+        blockMeta);
+      mvccVersion = mvccVersionDecoder.getMvccVersion(mvccVersionIndex);
+    }
+  }
+
+  protected void populateType() {
+    int typeInt;
+    if (blockMeta.isAllSameType()) {
+      typeInt = blockMeta.getAllTypes();
+    } else {
+      typeInt = currentRowNode.getType(currentCellIndex, blockMeta);
+    }
+    type = PrefixTreeCell.TYPES[typeInt];
+  }
+
+  protected void populateValueOffsets() {
+    int offsetIntoValueSection = currentRowNode.getValueOffset(currentCellIndex, blockMeta);
+    absoluteValueOffset = blockMeta.getAbsoluteValueOffset() + offsetIntoValueSection;
+    valueLength = currentRowNode.getValueLength(currentCellIndex, blockMeta);
+  }
+
+
+  /**************** getters ***************************/
+
+  public byte[] getTreeBytes() {
+    return block;
+  }
+
+  public PrefixTreeBlockMeta getBlockMeta() {
+    return blockMeta;
+  }
+
+  public int getMaxRowTreeStackNodes() {
+    return rowNodes.length;
+  }
+
+  public int getRowBufferLength() {
+    return rowBuffer.length;
+  }
+
+  public int getQualifierBufferLength() {
+    return qualifierBuffer.length;
+  }
+
+}