You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2013/02/07 01:36:27 UTC

svn commit: r1443289 [5/6] - in /hbase/trunk: ./ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/ hbase-common/src/main/java/org/apache/hadoop/hbase/util/ hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/ hbase-common/src/mai...

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataDifferentTimestamps.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataDifferentTimestamps.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataDifferentTimestamps.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataDifferentTimestamps.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,94 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+import org.junit.Assert;
+
+import com.google.common.collect.Lists;
+
+/*
+ * test different timestamps
+ */
+public class TestRowDataDifferentTimestamps extends BaseTestRowData{
+
+	static byte[]
+        Arow = Bytes.toBytes("Arow"),
+        Brow = Bytes.toBytes("Brow"),
+        cf = Bytes.toBytes("fammy"),
+        cq0 = Bytes.toBytes("cq0"),
+        cq1 = Bytes.toBytes("cq1"),
+        v0 = Bytes.toBytes("v0");
+
+	static List<KeyValue> d = Lists.newArrayList();
+	static{
+	  KeyValue kv0 = new KeyValue(Arow, cf, cq0, 0L, v0);
+	  kv0.setMvccVersion(123456789L);
+	  d.add(kv0);
+
+	  KeyValue kv1 = new KeyValue(Arow, cf, cq1, 1L, v0);
+    kv1.setMvccVersion(3L);
+    d.add(kv1);
+
+	  KeyValue kv2 = new KeyValue(Brow, cf, cq0, 12345678L, v0);
+    kv2.setMvccVersion(65537L);
+    d.add(kv2);
+
+		//watch out... Long.MAX_VALUE comes back as 1332221664203, even with other encoders
+//		d.add(new KeyValue(Brow, cf, cq1, Long.MAX_VALUE, v0));
+	  KeyValue kv3 = new KeyValue(Brow, cf, cq1, Long.MAX_VALUE-1, v0);
+    kv3.setMvccVersion(1L);
+    d.add(kv3);
+
+	  KeyValue kv4 = new KeyValue(Brow, cf, cq1, 999999999, v0);
+    //don't set memstoreTS
+	  d.add(kv4);
+
+	  KeyValue kv5 = new KeyValue(Brow, cf, cq1, 12345, v0);
+    kv5.setMvccVersion(0L);
+    d.add(kv5);
+	}
+
+	@Override
+	public List<KeyValue> getInputs() {
+		return d;
+	}
+
+	@Override
+	public void individualBlockMetaAssertions(PrefixTreeBlockMeta blockMeta) {
+	  Assert.assertTrue(blockMeta.getNumMvccVersionBytes() > 0);
+	  Assert.assertEquals(12, blockMeta.getNumValueBytes());
+
+		Assert.assertFalse(blockMeta.isAllSameTimestamp());
+		Assert.assertNotNull(blockMeta.getMinTimestamp());
+		Assert.assertTrue(blockMeta.getTimestampIndexWidth() > 0);
+		Assert.assertTrue(blockMeta.getTimestampDeltaWidth() > 0);
+
+    Assert.assertFalse(blockMeta.isAllSameMvccVersion());
+    Assert.assertNotNull(blockMeta.getMinMvccVersion());
+    Assert.assertTrue(blockMeta.getMvccVersionIndexWidth() > 0);
+    Assert.assertTrue(blockMeta.getMvccVersionDeltaWidth() > 0);
+	}
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataEmpty.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataEmpty.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataEmpty.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataEmpty.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,43 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataEmpty extends BaseTestRowData{
+
+  private static byte[] b = new byte[0];
+
+  static List<KeyValue> d = Lists.newArrayList();
+  static {
+    d.add(new KeyValue(b, b, b, 0L, Type.Put, b));
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return d;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataExerciseFInts.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,114 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hbase.codec.prefixtree.PrefixTreeTestConstants;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+import org.apache.hbase.util.byterange.impl.ByteRangeTreeSet;
+import org.junit.Assert;
+
+import com.google.common.collect.Lists;
+
+/*
+ * test different timestamps
+ *
+ * http://pastebin.com/7ks8kzJ2
+ * http://pastebin.com/MPn03nsK
+ */
+public class TestRowDataExerciseFInts extends BaseTestRowData{
+
+	static List<ByteRange> rows;
+	static{
+		List<String> rowStrings = new ArrayList<String>();
+        rowStrings.add("com.edsBlog/directoryAa/pageAaa");
+        rowStrings.add("com.edsBlog/directoryAa/pageBbb");
+        rowStrings.add("com.edsBlog/directoryAa/pageCcc");
+        rowStrings.add("com.edsBlog/directoryAa/pageDdd");
+        rowStrings.add("com.edsBlog/directoryBb/pageEee");
+        rowStrings.add("com.edsBlog/directoryBb/pageFff");
+        rowStrings.add("com.edsBlog/directoryBb/pageGgg");
+        rowStrings.add("com.edsBlog/directoryBb/pageHhh");
+        rowStrings.add("com.isabellasBlog/directoryAa/pageAaa");
+        rowStrings.add("com.isabellasBlog/directoryAa/pageBbb");
+        rowStrings.add("com.isabellasBlog/directoryAa/pageCcc");
+        rowStrings.add("com.isabellasBlog/directoryAa/pageDdd");
+        rowStrings.add("com.isabellasBlog/directoryBb/pageEee");
+        rowStrings.add("com.isabellasBlog/directoryBb/pageFff");
+        rowStrings.add("com.isabellasBlog/directoryBb/pageGgg");
+        rowStrings.add("com.isabellasBlog/directoryBb/pageHhh");
+        ByteRangeTreeSet ba = new ByteRangeTreeSet();
+        for(String row : rowStrings){
+        	ba.add(new ByteRange(Bytes.toBytes(row)));
+        }
+        rows = ba.compile().getSortedRanges();
+	}
+
+	static List<String> cols = Lists.newArrayList();
+	static{
+		cols.add("Chrome");
+		cols.add("Chromeb");
+		cols.add("Firefox");
+		cols.add("InternetExplorer");
+		cols.add("Opera");
+		cols.add("Safari");
+		cols.add("Z1stBrowserWithHuuuuuuuuuuuugeQualifier");
+		cols.add("Z2ndBrowserWithEvenBiggerQualifierMoreMoreMoreMoreMore");
+		cols.add("Z3rdBrowserWithEvenBiggerQualifierMoreMoreMoreMoreMore");
+		cols.add("Z4thBrowserWithEvenBiggerQualifierMoreMoreMoreMoreMore");
+		cols.add("Z5thBrowserWithEvenBiggerQualifierMoreMoreMoreMoreMore");
+		cols.add("Z6thBrowserWithEvenBiggerQualifierMoreMoreMoreMoreMore");
+		cols.add("Z7thBrowserWithEvenBiggerQualifierMoreMoreMoreMoreMore");
+		cols.add("Z8thBrowserWithEvenBiggerQualifierMoreMoreMoreMoreMore");
+		cols.add("Z9thBrowserWithEvenBiggerQualifierMoreMoreMoreMoreMore");
+	}
+
+  static long ts = 1234567890;
+
+  static int MAX_VALUE = 50;
+
+  static List<KeyValue> kvs = Lists.newArrayList();
+  static {
+    for (ByteRange row : rows) {
+      for (String col : cols) {
+        KeyValue kv = new KeyValue(row.deepCopyToNewArray(), PrefixTreeTestConstants.TEST_CF,
+            Bytes.toBytes(col), ts, KeyValue.Type.Put, Bytes.toBytes("VALUE"));
+        kvs.add(kv);
+      }
+    }
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return kvs;
+  }
+
+  @Override
+  public void individualBlockMetaAssertions(PrefixTreeBlockMeta blockMeta) {
+    Assert.assertTrue(blockMeta.getNextNodeOffsetWidth() > 1);
+    Assert.assertTrue(blockMeta.getQualifierOffsetWidth() > 1);
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataMultiFamilies.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataMultiFamilies.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataMultiFamilies.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataMultiFamilies.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,60 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataMultiFamilies extends BaseTestRowData{
+
+  static byte[] 
+        rowA = Bytes.toBytes("rowA"),
+        rowB = Bytes.toBytes("rowB"),
+        famA = Bytes.toBytes("famA"),
+        famB = Bytes.toBytes("famB"),
+        famBB = Bytes.toBytes("famBB"),
+        q0 = Bytes.toBytes("q0"),
+        q1 = Bytes.toBytes("q1"),//start with a different character
+        vvv = Bytes.toBytes("vvv");
+
+  static long ts = 55L;
+
+  static List<KeyValue> d = Lists.newArrayList();
+  static {
+    d.add(new KeyValue(rowA, famA, q0, ts, vvv));
+    d.add(new KeyValue(rowA, famB, q1, ts, vvv));
+    d.add(new KeyValue(rowA, famBB, q0, ts, vvv));
+    d.add(new KeyValue(rowB, famA, q0, ts, vvv));
+    d.add(new KeyValue(rowB, famA, q1, ts, vvv));
+    d.add(new KeyValue(rowB, famB, q0, ts, vvv));
+    d.add(new KeyValue(rowB, famBB, q0, ts, vvv));
+    d.add(new KeyValue(rowB, famBB, q1, ts, vvv));
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return d;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNub.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNub.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNub.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNub.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,59 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.codec.prefixtree.PrefixTreeTestConstants;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataNub extends BaseTestRowData{
+
+	static byte[]
+        rowA = Bytes.toBytes("rowA"),
+        rowB = Bytes.toBytes("rowB"),//nub
+        rowBB = Bytes.toBytes("rowBB"),
+        cf = PrefixTreeTestConstants.TEST_CF,
+        cq0 = Bytes.toBytes("cq0"),
+        cq1 = Bytes.toBytes("cq1"),
+        v0 = Bytes.toBytes("v0");
+
+	static long
+		ts = 55L;
+
+	static List<KeyValue> d = Lists.newArrayList();
+	static{
+		d.add(new KeyValue(rowA, cf, cq0, ts, v0));
+		d.add(new KeyValue(rowA, cf, cq1, ts, v0));
+		d.add(new KeyValue(rowB, cf, cq0, ts, v0));
+		d.add(new KeyValue(rowB, cf, cq1, ts, v0));
+		d.add(new KeyValue(rowBB, cf, cq0, ts, v0));
+		d.add(new KeyValue(rowBB, cf, cq1, ts, v0));
+	}
+
+	@Override
+	public List<KeyValue> getInputs() {
+		return d;
+	}
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataNumberStrings.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,61 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.cell.CellComparator;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataNumberStrings extends BaseTestRowData{
+
+  static List<KeyValue> d = Lists.newArrayList();
+  static {
+
+  /**
+   * Test a string-encoded list of numbers.  0, 1, 10, 11 will sort as 0, 1, 10, 11 if strings
+   * <p/>
+   * This helped catch a bug with reverse scanning where it was jumping from the last leaf cell to
+   * the previous nub.  It should do 11->10, but it was incorrectly doing 11->1
+   */
+    List<Integer> problematicSeries = Lists.newArrayList(0, 1, 10, 11);//sort this at the end
+    for(Integer i : problematicSeries){
+//    for(int i=0; i < 13; ++i){
+      byte[] row = Bytes.toBytes(""+i);
+      byte[] family = Bytes.toBytes("F");
+      byte[] column = Bytes.toBytes("C");
+      byte[] value = Bytes.toBytes("V");
+
+      d.add(new KeyValue(row, family, column, 0L, Type.Put, value));
+    }
+    Collections.sort(d, new CellComparator());
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return d;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataQualifierByteOrdering.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataQualifierByteOrdering.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataQualifierByteOrdering.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataQualifierByteOrdering.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,58 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataQualifierByteOrdering extends BaseTestRowData{
+
+	static byte[]
+        Arow = Bytes.toBytes("Arow"),
+        Brow = Bytes.toBytes("Brow"),
+        Brow2 = Bytes.toBytes("Brow2"),
+        fam = Bytes.toBytes("HappyFam"),
+        cq0 = Bytes.toBytes("cq0"),
+        cq1 = Bytes.toBytes("cq1tail"),//make sure tail does not come back as liat
+        cq2 = Bytes.toBytes("cq2"),
+        v0 = Bytes.toBytes("v0");
+
+  static long ts = 55L;
+
+  static List<KeyValue> d = Lists.newArrayList();
+  static {
+    d.add(new KeyValue(Arow, fam, cq0, ts, v0));
+    d.add(new KeyValue(Arow, fam, cq1, ts, v0));
+    d.add(new KeyValue(Brow, fam, cq0, ts, v0));
+    d.add(new KeyValue(Brow, fam, cq2, ts, v0));
+    d.add(new KeyValue(Brow2, fam, cq1, ts, v0));
+    d.add(new KeyValue(Brow2, fam, cq2, ts, v0));
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return d;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataRandomKeyValues.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataRandomKeyValues.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataRandomKeyValues.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataRandomKeyValues.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,42 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.test.RedundantKVGenerator;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataRandomKeyValues extends BaseTestRowData {
+
+  static List<KeyValue> d = Lists.newArrayList();
+  static RedundantKVGenerator generator = new RedundantKVGenerator();
+  static {
+    d = generator.generateTestKeyValues(1 << 10);
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return d;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSearcherRowMiss.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,123 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.Cell;
+import org.apache.hbase.cell.CellComparator;
+import org.apache.hbase.cell.CellScannerPosition;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+import org.apache.hbase.codec.prefixtree.scanner.CellSearcher;
+import org.junit.Assert;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataSearcherRowMiss extends BaseTestRowData{
+
+  static byte[]
+      //don't let the rows share any common prefix bytes
+      A = Bytes.toBytes("A"),
+      AA = Bytes.toBytes("AA"),
+      AAA = Bytes.toBytes("AAA"),
+      B = Bytes.toBytes("B"),
+      cf = Bytes.toBytes("fam"),
+      cq = Bytes.toBytes("cq0"),
+      v = Bytes.toBytes("v0");
+
+  static long
+    ts = 55L;
+
+  static List<KeyValue> d = Lists.newArrayList();
+  static{
+    d.add(new KeyValue(A, cf, cq, ts, v));
+    d.add(new KeyValue(AA, cf, cq, ts, v));
+    d.add(new KeyValue(AAA, cf, cq, ts, v));
+    d.add(new KeyValue(B, cf, cq, ts, v));
+  }
+
+	@Override
+	public List<KeyValue> getInputs() {
+		return d;
+	}
+
+	@Override
+	public void individualSearcherAssertions(CellSearcher searcher) {
+	  assertRowOffsetsCorrect();
+
+    searcher.resetToBeforeFirstEntry();
+
+    //test first cell
+    searcher.next();
+    Cell first = searcher.getCurrent();
+    Assert.assertTrue(CellComparator.equals(d.get(0), first));
+
+    //test first cell in second row
+    Assert.assertTrue(searcher.positionAt(d.get(1)));
+    Assert.assertTrue(CellComparator.equals(d.get(1), searcher.getCurrent()));
+
+    testBetween1and2(searcher);
+    testBetween2and3(searcher);
+  }
+
+	/************ private methods, call from above *******************/
+
+	private void assertRowOffsetsCorrect(){
+	  Assert.assertEquals(4, getRowStartIndexes().size());
+	}
+
+	private void testBetween1and2(CellSearcher searcher){
+    CellScannerPosition p;//reuse
+    Cell betweenAAndAAA = new KeyValue(AA, cf, cq, ts-2, v);
+
+    //test exact
+    Assert.assertFalse(searcher.positionAt(betweenAAndAAA));
+
+    //test atOrBefore
+    p = searcher.positionAtOrBefore(betweenAAndAAA);
+    Assert.assertEquals(CellScannerPosition.BEFORE, p);
+    Assert.assertTrue(CellComparator.equals(searcher.getCurrent(), d.get(1)));
+
+    //test atOrAfter
+    p = searcher.positionAtOrAfter(betweenAAndAAA);
+    Assert.assertEquals(CellScannerPosition.AFTER, p);
+    Assert.assertTrue(CellComparator.equals(searcher.getCurrent(), d.get(2)));
+	}
+
+  private void testBetween2and3(CellSearcher searcher){
+    CellScannerPosition p;//reuse
+    Cell betweenAAAndB = new KeyValue(AAA, cf, cq, ts-2, v);
+
+    //test exact
+    Assert.assertFalse(searcher.positionAt(betweenAAAndB));
+
+    //test atOrBefore
+    p = searcher.positionAtOrBefore(betweenAAAndB);
+    Assert.assertEquals(CellScannerPosition.BEFORE, p);
+    Assert.assertTrue(CellComparator.equals(searcher.getCurrent(), d.get(2)));
+
+    //test atOrAfter
+    p = searcher.positionAtOrAfter(betweenAAAndB);
+    Assert.assertEquals(CellScannerPosition.AFTER, p);
+    Assert.assertTrue(CellComparator.equals(searcher.getCurrent(), d.get(3)));
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSimple.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSimple.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSimple.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSimple.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,112 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hbase.Cell;
+import org.apache.hbase.cell.CellComparator;
+import org.apache.hbase.cell.CellScannerPosition;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+import org.apache.hbase.codec.prefixtree.scanner.CellSearcher;
+import org.junit.Assert;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataSimple extends BaseTestRowData {
+
+  static byte[]
+  // don't let the rows share any common prefix bytes
+      rowA = Bytes.toBytes("Arow"),
+      rowB = Bytes.toBytes("Brow"), cf = Bytes.toBytes("fam"),
+      cq0 = Bytes.toBytes("cq0"),
+      cq1 = Bytes.toBytes("cq1tail"),// make sure tail does not come back as liat
+      cq2 = Bytes.toBytes("dcq2"),// start with a different character
+      v0 = Bytes.toBytes("v0");
+
+  static long ts = 55L;
+
+  static List<KeyValue> d = Lists.newArrayList();
+  static {
+    d.add(new KeyValue(rowA, cf, cq0, ts, v0));
+    d.add(new KeyValue(rowA, cf, cq1, ts, v0));
+    d.add(new KeyValue(rowA, cf, cq2, ts, v0));
+    d.add(new KeyValue(rowB, cf, cq0, ts, v0));
+    d.add(new KeyValue(rowB, cf, cq1, ts, v0));
+    d.add(new KeyValue(rowB, cf, cq2, ts, v0));
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return d;
+  }
+
+  @Override
+  public void individualSearcherAssertions(CellSearcher searcher) {
+    CellScannerPosition p;// reuse
+    searcher.resetToBeforeFirstEntry();
+
+    // test first cell
+    searcher.next();
+    Cell first = searcher.getCurrent();
+    Assert.assertTrue(CellComparator.equals(d.get(0), first));
+
+    // test first cell in second row
+    Assert.assertTrue(searcher.positionAt(d.get(3)));
+    Assert.assertTrue(CellComparator.equals(d.get(3), searcher.getCurrent()));
+
+    Cell between4And5 = new KeyValue(rowB, cf, cq1, ts - 2, v0);
+
+    // test exact
+    Assert.assertFalse(searcher.positionAt(between4And5));
+
+    // test atOrBefore
+    p = searcher.positionAtOrBefore(between4And5);
+    Assert.assertEquals(CellScannerPosition.BEFORE, p);
+    Assert.assertTrue(CellComparator.equals(searcher.getCurrent(), d.get(4)));
+
+    // test atOrAfter
+    p = searcher.positionAtOrAfter(between4And5);
+    Assert.assertEquals(CellScannerPosition.AFTER, p);
+    Assert.assertTrue(CellComparator.equals(searcher.getCurrent(), d.get(5)));
+
+    // test when key falls before first key in block
+    Cell beforeFirst = new KeyValue(Bytes.toBytes("A"), cf, cq0, ts, v0);
+    Assert.assertFalse(searcher.positionAt(beforeFirst));
+    p = searcher.positionAtOrBefore(beforeFirst);
+    Assert.assertEquals(CellScannerPosition.BEFORE_FIRST, p);
+    p = searcher.positionAtOrAfter(beforeFirst);
+    Assert.assertEquals(CellScannerPosition.AFTER, p);
+    Assert.assertTrue(CellComparator.equals(searcher.getCurrent(), d.get(0)));
+    Assert.assertEquals(d.get(0), searcher.getCurrent());
+
+    // test when key falls after last key in block
+    Cell afterLast = new KeyValue(Bytes.toBytes("z"), cf, cq0, ts, v0);// must be lower case z
+    Assert.assertFalse(searcher.positionAt(afterLast));
+    p = searcher.positionAtOrAfter(afterLast);
+    Assert.assertEquals(CellScannerPosition.AFTER_LAST, p);
+    p = searcher.positionAtOrBefore(afterLast);
+    Assert.assertEquals(CellScannerPosition.BEFORE, p);
+    Assert.assertTrue(CellComparator.equals(searcher.getCurrent(), CollectionUtils.getLast(d)));
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSingleQualifier.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSingleQualifier.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSingleQualifier.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataSingleQualifier.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,52 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.codec.prefixtree.PrefixTreeTestConstants;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataSingleQualifier extends BaseTestRowData{
+
+	static byte[]
+        rowA = Bytes.toBytes("rowA"),
+        rowB = Bytes.toBytes("rowB"),
+        cf = PrefixTreeTestConstants.TEST_CF,
+        cq0 = Bytes.toBytes("cq0"),
+        v0 = Bytes.toBytes("v0");
+
+  static long ts = 55L;
+
+  static List<KeyValue> d = Lists.newArrayList();
+  static {
+    d.add(new KeyValue(rowA, cf, cq0, ts, v0));
+    d.add(new KeyValue(rowB, cf, cq0, ts, v0));
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return d;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataTrivial.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataTrivial.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataTrivial.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataTrivial.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,73 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.cell.CellScannerPosition;
+import org.apache.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+import org.apache.hbase.codec.prefixtree.scanner.CellSearcher;
+import org.junit.Assert;
+
+import com.google.common.collect.Lists;
+
+public class TestRowDataTrivial extends BaseTestRowData{
+
+	static byte[]
+        rA = Bytes.toBytes("rA"),
+        rB = Bytes.toBytes("rB"),//turn "r" into a branch for the Searcher tests
+        cf = Bytes.toBytes("fam"),
+        cq0 = Bytes.toBytes("q0"),
+        v0 = Bytes.toBytes("v0");
+
+  static long ts = 55L;
+
+  static List<KeyValue> d = Lists.newArrayList();
+  static {
+    d.add(new KeyValue(rA, cf, cq0, ts, v0));
+    d.add(new KeyValue(rB, cf, cq0, ts, v0));
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return d;
+  }
+
+  @Override
+  public void individualBlockMetaAssertions(PrefixTreeBlockMeta blockMeta) {
+    // node[0] -> root[r]
+    // node[1] -> leaf[A], etc
+    Assert.assertEquals(2, blockMeta.getRowTreeDepth());
+  }
+
+  @Override
+  public void individualSearcherAssertions(CellSearcher searcher) {
+    /**
+     * The searcher should get a token mismatch on the "r" branch. Assert that it skips not only rA,
+     * but rB as well.
+     */
+    KeyValue afterLast = KeyValue.createFirstOnRow(Bytes.toBytes("zzz"));
+    CellScannerPosition position = searcher.positionAtOrAfter(afterLast);
+    Assert.assertEquals(CellScannerPosition.AFTER_LAST, position);
+    Assert.assertNull(searcher.getCurrent());
+  }
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataUrls.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataUrls.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataUrls.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataUrls.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,98 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.ByteRange;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.codec.prefixtree.PrefixTreeTestConstants;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+import org.apache.hbase.util.byterange.impl.ByteRangeTreeSet;
+
+import com.google.common.collect.Lists;
+
+/*
+ * test different timestamps
+ * 
+ * http://pastebin.com/7ks8kzJ2
+ * http://pastebin.com/MPn03nsK
+ */
+public class TestRowDataUrls extends BaseTestRowData{
+
+	static List<ByteRange> rows;
+	static{
+    List<String> rowStrings = new ArrayList<String>();
+    rowStrings.add("com.edsBlog/directoryAa/pageAaa");
+    rowStrings.add("com.edsBlog/directoryAa/pageBbb");
+    rowStrings.add("com.edsBlog/directoryAa/pageCcc");
+    rowStrings.add("com.edsBlog/directoryAa/pageDdd");
+    rowStrings.add("com.edsBlog/directoryBb/pageEee");
+    rowStrings.add("com.edsBlog/directoryBb/pageFff");
+    rowStrings.add("com.edsBlog/directoryBb/pageGgg");
+    rowStrings.add("com.edsBlog/directoryBb/pageHhh");
+    rowStrings.add("com.isabellasBlog/directoryAa/pageAaa");
+    rowStrings.add("com.isabellasBlog/directoryAa/pageBbb");
+    rowStrings.add("com.isabellasBlog/directoryAa/pageCcc");
+    rowStrings.add("com.isabellasBlog/directoryAa/pageDdd");
+    rowStrings.add("com.isabellasBlog/directoryBb/pageEee");
+    rowStrings.add("com.isabellasBlog/directoryBb/pageFff");
+    rowStrings.add("com.isabellasBlog/directoryBb/pageGgg");
+    rowStrings.add("com.isabellasBlog/directoryBb/pageHhh");
+    ByteRangeTreeSet ba = new ByteRangeTreeSet();
+    for (String row : rowStrings) {
+      ba.add(new ByteRange(Bytes.toBytes(row)));
+    }
+    rows = ba.compile().getSortedRanges();
+  }
+
+  static List<String> cols = Lists.newArrayList();
+  static {
+    cols.add("Chrome");
+    cols.add("Chromeb");
+    cols.add("Firefox");
+    cols.add("InternetExplorer");
+    cols.add("Opera");
+    cols.add("Safari");
+  }
+
+  static long ts = 1234567890;
+
+  static int MAX_VALUE = 50;
+
+  static List<KeyValue> kvs = Lists.newArrayList();
+  static {
+    for (ByteRange row : rows) {
+      for (String col : cols) {
+        KeyValue kv = new KeyValue(row.deepCopyToNewArray(), PrefixTreeTestConstants.TEST_CF,
+            Bytes.toBytes(col), ts, KeyValue.Type.Put, Bytes.toBytes("VALUE"));
+        kvs.add(kv);
+        // System.out.println("TestRows5:"+kv);
+      }
+    }
+  }
+
+  @Override
+  public List<KeyValue> getInputs() {
+    return kvs;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataUrlsExample.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataUrlsExample.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataUrlsExample.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/row/data/TestRowDataUrlsExample.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,126 @@
+/*
+ * 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.hbase.codec.prefixtree.row.data;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueTestUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.codec.prefixtree.encode.PrefixTreeEncoder;
+import org.apache.hbase.codec.prefixtree.encode.column.ColumnNodeWriter;
+import org.apache.hbase.codec.prefixtree.encode.row.RowNodeWriter;
+import org.apache.hbase.codec.prefixtree.encode.tokenize.TokenizerNode;
+import org.apache.hbase.codec.prefixtree.row.BaseTestRowData;
+
+import com.google.common.collect.Lists;
+
+/*
+ * test different timestamps
+ * 
+ * http://pastebin.com/7ks8kzJ2
+ * http://pastebin.com/MPn03nsK
+ */
+public class TestRowDataUrlsExample extends BaseTestRowData{
+
+  static String TENANT_ID = Integer.toString(95322);
+  static String APP_ID = Integer.toString(12);
+  static List<String> URLS = Lists.newArrayList(
+      "com.dablog/2011/10/04/boating", 
+      "com.dablog/2011/10/09/lasers", 
+      "com.jamiesrecipes", //this nub helped find a bug
+      "com.jamiesrecipes/eggs");
+  static String FAMILY = "hits";
+  static List<String> BROWSERS = Lists.newArrayList(
+      "Chrome", "IE8", "IE9beta");//, "Opera", "Safari");
+	static long TIMESTAMP = 1234567890;
+
+	static int MAX_VALUE = 50;
+
+	static List<KeyValue> kvs = Lists.newArrayList();
+	static{
+		for(String rowKey : URLS){
+			for(String qualifier : BROWSERS){
+			  KeyValue kv = new KeyValue(
+						Bytes.toBytes(rowKey), 
+						Bytes.toBytes(FAMILY), 
+						Bytes.toBytes(qualifier), 
+						TIMESTAMP, 
+						KeyValue.Type.Put, 
+						Bytes.toBytes("VvvV"));
+				kvs.add(kv);
+			}
+		}
+	}
+
+	/**
+	 * Used for generating docs.
+	 */
+	public static void main(String... args) throws IOException{
+    System.out.println("-- inputs --");
+    System.out.println(KeyValueTestUtil.toStringWithPadding(kvs, true));
+		ByteArrayOutputStream os = new ByteArrayOutputStream(1<<20);
+    PrefixTreeEncoder encoder = new PrefixTreeEncoder(os, false);
+
+    for(KeyValue kv : kvs){
+      encoder.write(kv);
+    }
+    encoder.flush();
+
+    System.out.println("-- qualifier SortedPtBuilderNodes --");
+    for(TokenizerNode tokenizer : encoder.getQualifierWriter().getNonLeaves()){
+      System.out.println(tokenizer);
+    }
+    for(TokenizerNode tokenizerNode : encoder.getQualifierWriter().getLeaves()){
+      System.out.println(tokenizerNode);
+    }
+
+    System.out.println("-- qualifier PtColumnNodeWriters --");
+    for(ColumnNodeWriter writer : encoder.getQualifierWriter().getColumnNodeWriters()){
+      System.out.println(writer);
+    }
+
+    System.out.println("-- rowKey SortedPtBuilderNodes --");
+    for(TokenizerNode tokenizerNode : encoder.getRowWriter().getNonLeaves()){
+      System.out.println(tokenizerNode);
+    }
+    for(TokenizerNode tokenizerNode : encoder.getRowWriter().getLeaves()){
+      System.out.println(tokenizerNode);
+    }
+
+    System.out.println("-- row PtRowNodeWriters --");
+    for(RowNodeWriter writer : encoder.getRowWriter().getNonLeafWriters()){
+      System.out.println(writer);
+    }
+    for(RowNodeWriter writer : encoder.getRowWriter().getLeafWriters()){
+      System.out.println(writer);
+    }
+
+    System.out.println("-- concatenated values --");
+    System.out.println(Bytes.toStringBinary(encoder.getValueByteRange().deepCopyToNewArray()));
+	}
+
+	@Override
+	public List<KeyValue> getInputs() {
+		return kvs;
+	}
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/TestTimestampData.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/TestTimestampData.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/TestTimestampData.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/TestTimestampData.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,45 @@
+/*
+ * 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.hbase.codec.prefixtree.timestamp;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hbase.codec.prefixtree.timestamp.data.TestTimestampDataBasic;
+import org.apache.hbase.codec.prefixtree.timestamp.data.TestTimestampDataNumbers;
+import org.apache.hbase.codec.prefixtree.timestamp.data.TestTimestampDataRepeats;
+
+import com.google.common.collect.Lists;
+
+public interface TestTimestampData {
+
+  List<Long> getInputs();
+  long getMinimum();
+  List<Long> getOutputs();
+
+  public static class InMemory {
+    public Collection<Object[]> getAllAsObjectArray() {
+      List<Object[]> all = Lists.newArrayList();
+      all.add(new Object[] { new TestTimestampDataBasic() });
+      all.add(new Object[] { new TestTimestampDataNumbers() });
+      all.add(new Object[] { new TestTimestampDataRepeats() });
+      return all;
+    }
+  }
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/TestTimestampEncoder.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/TestTimestampEncoder.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/TestTimestampEncoder.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/TestTimestampEncoder.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,92 @@
+/*
+ * 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.hbase.codec.prefixtree.timestamp;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.hbase.codec.prefixtree.PrefixTreeBlockMeta;
+import org.apache.hbase.codec.prefixtree.decode.timestamp.TimestampDecoder;
+import org.apache.hbase.codec.prefixtree.encode.other.LongEncoder;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class TestTimestampEncoder {
+
+  @Parameters
+  public static Collection<Object[]> parameters() {
+    return new TestTimestampData.InMemory().getAllAsObjectArray();
+  }
+
+  private TestTimestampData timestamps;
+  private PrefixTreeBlockMeta blockMeta;
+  private LongEncoder encoder;
+  private byte[] bytes;
+  private TimestampDecoder decoder;
+
+  public TestTimestampEncoder(TestTimestampData testTimestamps) throws IOException {
+    this.timestamps = testTimestamps;
+    this.blockMeta = new PrefixTreeBlockMeta();
+    this.blockMeta.setNumMetaBytes(0);
+    this.blockMeta.setNumRowBytes(0);
+    this.blockMeta.setNumQualifierBytes(0);
+    this.encoder = new LongEncoder();
+    for (Long ts : testTimestamps.getInputs()) {
+      encoder.add(ts);
+    }
+    encoder.compile();
+    blockMeta.setTimestampFields(encoder);
+    bytes = encoder.getByteArray();
+    decoder = new TimestampDecoder();
+    decoder.initOnBlock(blockMeta, bytes);
+  }
+
+  @Test
+  public void testCompressorMinimum() {
+    Assert.assertEquals(timestamps.getMinimum(), encoder.getMin());
+  }
+
+  @Test
+  public void testCompressorRoundTrip() {
+    long[] outputs = encoder.getSortedUniqueTimestamps();
+    for (int i = 0; i < timestamps.getOutputs().size(); ++i) {
+      long input = timestamps.getOutputs().get(i);
+      long output = outputs[i];
+      Assert.assertEquals(input, output);
+    }
+  }
+
+  @Test
+  public void testReaderMinimum() {
+    Assert.assertEquals(timestamps.getMinimum(), decoder.getLong(0));
+  }
+
+  @Test
+  public void testReaderRoundTrip() {
+    for (int i = 0; i < timestamps.getOutputs().size(); ++i) {
+      long input = timestamps.getOutputs().get(i);
+      long output = decoder.getLong(i);
+      Assert.assertEquals(input, output);
+    }
+  }
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataBasic.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataBasic.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataBasic.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataBasic.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,54 @@
+/*
+ * 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.hbase.codec.prefixtree.timestamp.data;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hbase.codec.prefixtree.timestamp.TestTimestampData;
+
+public class TestTimestampDataBasic implements TestTimestampData {
+
+  @Override
+  public List<Long> getInputs() {
+    List<Long> d = new ArrayList<Long>();
+    d.add(5L);
+    d.add(3L);
+    d.add(0L);
+    d.add(1L);
+    d.add(3L);
+    return d;
+  }
+
+  @Override
+  public long getMinimum() {
+    return 0L;
+  }
+
+  @Override
+  public List<Long> getOutputs() {
+    List<Long> d = new ArrayList<Long>();
+    d.add(0L);
+    d.add(1L);
+    d.add(3L);
+    d.add(5L);
+    return d;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataNumbers.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataNumbers.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataNumbers.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataNumbers.java Thu Feb  7 00:36:24 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.hbase.codec.prefixtree.timestamp.data;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hbase.codec.prefixtree.timestamp.TestTimestampData;
+
+public class TestTimestampDataNumbers implements TestTimestampData {
+
+  private int shift = 8;
+
+  @Override
+  public List<Long> getInputs() {
+    List<Long> d = new ArrayList<Long>();
+    d.add(5L << shift);
+    d.add(3L << shift);
+    d.add(7L << shift);
+    d.add(1L << shift);
+    d.add(3L << shift);
+    return d;
+  }
+
+  @Override
+  public long getMinimum() {
+    return 1L << shift;
+  }
+
+  @Override
+  public List<Long> getOutputs() {
+    List<Long> d = new ArrayList<Long>();
+    d.add(1L << shift);
+    d.add(3L << shift);
+    d.add(5L << shift);
+    d.add(7L << shift);
+    return d;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataRepeats.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataRepeats.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataRepeats.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/codec/prefixtree/timestamp/data/TestTimestampDataRepeats.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,52 @@
+/*
+ * 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.hbase.codec.prefixtree.timestamp.data;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hbase.codec.prefixtree.timestamp.TestTimestampData;
+
+public class TestTimestampDataRepeats implements TestTimestampData {
+
+  private static long t = 1234567890L;
+
+  @Override
+  public List<Long> getInputs() {
+    List<Long> d = new ArrayList<Long>();
+    d.add(t);
+    d.add(t);
+    d.add(t);
+    d.add(t);
+    d.add(t);
+    return d;
+  }
+
+  @Override
+  public long getMinimum() {
+    return t;
+  }
+
+  @Override
+  public List<Long> getOutputs() {
+    List<Long> d = new ArrayList<Long>();
+    return d;
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/bytes/TestByteRange.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/bytes/TestByteRange.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/bytes/TestByteRange.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/bytes/TestByteRange.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,34 @@
+/*
+ * 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.hbase.util.bytes;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hbase.util.ByteRange;
+import org.junit.Test;
+
+public class TestByteRange {
+
+  @Test
+  public void testConstructor() {
+    ByteRange b = new ByteRange(new byte[] { 0, 1, 2 });
+    Assert.assertEquals(3, b.getLength());
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/comparator/ByteArrayComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/comparator/ByteArrayComparator.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/comparator/ByteArrayComparator.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/comparator/ByteArrayComparator.java Thu Feb  7 00:36:24 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.hbase.util.comparator;
+
+import java.util.Comparator;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class ByteArrayComparator implements Comparator<byte[]> {
+
+  @Override
+  public int compare(byte[] a, byte[] b) {
+    return Bytes.compareTo(a, b);
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/number/NumberFormatter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/number/NumberFormatter.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/number/NumberFormatter.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/number/NumberFormatter.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,33 @@
+/*
+ * 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.hbase.util.number;
+
+import java.text.DecimalFormat;
+
+public class NumberFormatter {
+
+  public static String addCommas(final Number pValue) {
+    if (pValue == null) {
+      return null;
+    }
+    String format = "###,###,###,###,###,###,###,###.#####################";
+    return new DecimalFormat(format).format(pValue);// biggest is 19 digits
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/number/RandomNumberUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/number/RandomNumberUtils.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/number/RandomNumberUtils.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/number/RandomNumberUtils.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,34 @@
+/*
+ * 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.hbase.util.number;
+
+import java.util.Random;
+
+public class RandomNumberUtils {
+
+  public static long nextPositiveLong(Random random) {
+    while (true) {
+      long value = random.nextLong();
+      if (value > 0) {
+        return value;
+      }
+    }
+  }
+
+}

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestFIntTool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestFIntTool.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestFIntTool.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestFIntTool.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,122 @@
+/*
+ * 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.hbase.util.vint;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/********************** tests *************************/
+
+public class TestFIntTool {
+  @Test
+  public void testLeadingZeros() {
+    Assert.assertEquals(64, Long.numberOfLeadingZeros(0));
+    Assert.assertEquals(63, Long.numberOfLeadingZeros(1));
+    Assert.assertEquals(0, Long.numberOfLeadingZeros(Long.MIN_VALUE));
+    Assert.assertEquals(0, Long.numberOfLeadingZeros(-1));
+    Assert.assertEquals(1, Long.numberOfLeadingZeros(Long.MAX_VALUE));
+    Assert.assertEquals(1, Long.numberOfLeadingZeros(Long.MAX_VALUE - 1));
+  }
+
+  @Test
+  public void testMaxValueForNumBytes() {
+    Assert.assertEquals(255, UFIntTool.maxValueForNumBytes(1));
+    Assert.assertEquals(65535, UFIntTool.maxValueForNumBytes(2));
+    Assert.assertEquals(0xffffff, UFIntTool.maxValueForNumBytes(3));
+    Assert.assertEquals(0xffffffffffffffL, UFIntTool.maxValueForNumBytes(7));
+  }
+
+  @Test
+  public void testNumBytes() {
+    Assert.assertEquals(1, UFIntTool.numBytes(0));
+    Assert.assertEquals(1, UFIntTool.numBytes(1));
+    Assert.assertEquals(1, UFIntTool.numBytes(255));
+    Assert.assertEquals(2, UFIntTool.numBytes(256));
+    Assert.assertEquals(2, UFIntTool.numBytes(65535));
+    Assert.assertEquals(3, UFIntTool.numBytes(65536));
+    Assert.assertEquals(4, UFIntTool.numBytes(0xffffffffL));
+    Assert.assertEquals(5, UFIntTool.numBytes(0x100000000L));
+    Assert.assertEquals(4, UFIntTool.numBytes(Integer.MAX_VALUE));
+    Assert.assertEquals(8, UFIntTool.numBytes(Long.MAX_VALUE));
+    Assert.assertEquals(8, UFIntTool.numBytes(Long.MAX_VALUE - 1));
+  }
+
+  @Test
+  public void testGetBytes() {
+    Assert.assertArrayEquals(new byte[] { 0 }, UFIntTool.getBytes(1, 0));
+    Assert.assertArrayEquals(new byte[] { 1 }, UFIntTool.getBytes(1, 1));
+    Assert.assertArrayEquals(new byte[] { -1 }, UFIntTool.getBytes(1, 255));
+    Assert.assertArrayEquals(new byte[] { 1, 0 }, UFIntTool.getBytes(2, 256));
+    Assert.assertArrayEquals(new byte[] { 1, 3 }, UFIntTool.getBytes(2, 256 + 3));
+    Assert.assertArrayEquals(new byte[] { 1, -128 }, UFIntTool.getBytes(2, 256 + 128));
+    Assert.assertArrayEquals(new byte[] { 1, -1 }, UFIntTool.getBytes(2, 256 + 255));
+    Assert.assertArrayEquals(new byte[] { 127, -1, -1, -1 },
+      UFIntTool.getBytes(4, Integer.MAX_VALUE));
+    Assert.assertArrayEquals(new byte[] { 127, -1, -1, -1, -1, -1, -1, -1 },
+      UFIntTool.getBytes(8, Long.MAX_VALUE));
+  }
+
+  @Test
+  public void testFromBytes() {
+    Assert.assertEquals(0, UFIntTool.fromBytes(new byte[] { 0 }));
+    Assert.assertEquals(1, UFIntTool.fromBytes(new byte[] { 1 }));
+    Assert.assertEquals(255, UFIntTool.fromBytes(new byte[] { -1 }));
+    Assert.assertEquals(256, UFIntTool.fromBytes(new byte[] { 1, 0 }));
+    Assert.assertEquals(256 + 3, UFIntTool.fromBytes(new byte[] { 1, 3 }));
+    Assert.assertEquals(256 + 128, UFIntTool.fromBytes(new byte[] { 1, -128 }));
+    Assert.assertEquals(256 + 255, UFIntTool.fromBytes(new byte[] { 1, -1 }));
+    Assert.assertEquals(Integer.MAX_VALUE, UFIntTool.fromBytes(new byte[] { 127, -1, -1, -1 }));
+    Assert.assertEquals(Long.MAX_VALUE,
+      UFIntTool.fromBytes(new byte[] { 127, -1, -1, -1, -1, -1, -1, -1 }));
+  }
+
+  @Test
+  public void testRoundTrips() {
+    long[] values = new long[] { 0, 1, 2, 255, 256, 31123, 65535, 65536, 65537, 0xfffffeL,
+        0xffffffL, 0x1000000L, 0x1000001L, Integer.MAX_VALUE - 1, Integer.MAX_VALUE,
+        (long) Integer.MAX_VALUE + 1, Long.MAX_VALUE - 1, Long.MAX_VALUE };
+    for (int i = 0; i < values.length; ++i) {
+      Assert.assertEquals(values[i], UFIntTool.fromBytes(UFIntTool.getBytes(8, values[i])));
+    }
+  }
+
+  @Test
+  public void testWriteBytes() throws IOException {// copied from testGetBytes
+    Assert.assertArrayEquals(new byte[] { 0 }, bytesViaOutputStream(1, 0));
+    Assert.assertArrayEquals(new byte[] { 1 }, bytesViaOutputStream(1, 1));
+    Assert.assertArrayEquals(new byte[] { -1 }, bytesViaOutputStream(1, 255));
+    Assert.assertArrayEquals(new byte[] { 1, 0 }, bytesViaOutputStream(2, 256));
+    Assert.assertArrayEquals(new byte[] { 1, 3 }, bytesViaOutputStream(2, 256 + 3));
+    Assert.assertArrayEquals(new byte[] { 1, -128 }, bytesViaOutputStream(2, 256 + 128));
+    Assert.assertArrayEquals(new byte[] { 1, -1 }, bytesViaOutputStream(2, 256 + 255));
+    Assert.assertArrayEquals(new byte[] { 127, -1, -1, -1 },
+      bytesViaOutputStream(4, Integer.MAX_VALUE));
+    Assert.assertArrayEquals(new byte[] { 127, -1, -1, -1, -1, -1, -1, -1 },
+      bytesViaOutputStream(8, Long.MAX_VALUE));
+  }
+
+  private byte[] bytesViaOutputStream(int outputWidth, long value) throws IOException {
+    ByteArrayOutputStream os = new ByteArrayOutputStream();
+    UFIntTool.writeBytes(outputWidth, value, os);
+    return os.toByteArray();
+  }
+}
\ No newline at end of file

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestVIntTool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestVIntTool.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestVIntTool.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestVIntTool.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,98 @@
+/*
+ * 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.hbase.util.vint;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Random;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestVIntTool {
+
+  @Test
+  public void testNumBytes() {
+    Assert.assertEquals(1, UVIntTool.numBytes(0));
+    Assert.assertEquals(1, UVIntTool.numBytes(1));
+    Assert.assertEquals(1, UVIntTool.numBytes(100));
+    Assert.assertEquals(1, UVIntTool.numBytes(126));
+    Assert.assertEquals(1, UVIntTool.numBytes(127));
+    Assert.assertEquals(2, UVIntTool.numBytes(128));
+    Assert.assertEquals(2, UVIntTool.numBytes(129));
+    Assert.assertEquals(5, UVIntTool.numBytes(Integer.MAX_VALUE));
+  }
+
+  @Test
+  public void testWriteBytes() throws IOException {
+    Assert.assertArrayEquals(new byte[] { 0 }, bytesViaOutputStream(0));
+    Assert.assertArrayEquals(new byte[] { 1 }, bytesViaOutputStream(1));
+    Assert.assertArrayEquals(new byte[] { 63 }, bytesViaOutputStream(63));
+    Assert.assertArrayEquals(new byte[] { 127 }, bytesViaOutputStream(127));
+    Assert.assertArrayEquals(new byte[] { -128, 1 }, bytesViaOutputStream(128));
+    Assert.assertArrayEquals(new byte[] { -128 + 27, 1 }, bytesViaOutputStream(155));
+    Assert.assertArrayEquals(UVIntTool.MAX_VALUE_BYTES, bytesViaOutputStream(Integer.MAX_VALUE));
+  }
+
+  private byte[] bytesViaOutputStream(int value) throws IOException {
+    ByteArrayOutputStream os = new ByteArrayOutputStream();
+    UVIntTool.writeBytes(value, os);
+    return os.toByteArray();
+  }
+
+  @Test
+  public void testToBytes() {
+    Assert.assertArrayEquals(new byte[] { 0 }, UVIntTool.getBytes(0));
+    Assert.assertArrayEquals(new byte[] { 1 }, UVIntTool.getBytes(1));
+    Assert.assertArrayEquals(new byte[] { 63 }, UVIntTool.getBytes(63));
+    Assert.assertArrayEquals(new byte[] { 127 }, UVIntTool.getBytes(127));
+    Assert.assertArrayEquals(new byte[] { -128, 1 }, UVIntTool.getBytes(128));
+    Assert.assertArrayEquals(new byte[] { -128 + 27, 1 }, UVIntTool.getBytes(155));
+    Assert.assertArrayEquals(UVIntTool.MAX_VALUE_BYTES, UVIntTool.getBytes(Integer.MAX_VALUE));
+  }
+
+  @Test
+  public void testFromBytes() {
+    Assert.assertEquals(Integer.MAX_VALUE, UVIntTool.getInt(UVIntTool.MAX_VALUE_BYTES));
+  }
+
+  @Test
+  public void testRoundTrips() {
+    Random random = new Random();
+    for (int i = 0; i < 10000; ++i) {
+      int value = random.nextInt(Integer.MAX_VALUE);
+      byte[] bytes = UVIntTool.getBytes(value);
+      int roundTripped = UVIntTool.getInt(bytes);
+      Assert.assertEquals(value, roundTripped);
+    }
+  }
+
+  @Test
+  public void testInputStreams() throws IOException {
+    ByteArrayInputStream is;
+    is = new ByteArrayInputStream(new byte[] { 0 });
+    Assert.assertEquals(0, UVIntTool.getInt(is));
+    is = new ByteArrayInputStream(new byte[] { 5 });
+    Assert.assertEquals(5, UVIntTool.getInt(is));
+    is = new ByteArrayInputStream(new byte[] { -128 + 27, 1 });
+    Assert.assertEquals(155, UVIntTool.getInt(is));
+  }
+
+}
\ No newline at end of file

Added: hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestVLongTool.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestVLongTool.java?rev=1443289&view=auto
==============================================================================
--- hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestVLongTool.java (added)
+++ hbase/trunk/hbase-prefix-tree/src/test/java/org/apache/hbase/util/vint/TestVLongTool.java Thu Feb  7 00:36:24 2013
@@ -0,0 +1,105 @@
+/*
+ * 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.hbase.util.vint;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hbase.util.number.RandomNumberUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestVLongTool {
+
+  @Test
+  public void testNumBytes() {
+    Assert.assertEquals(1, UVLongTool.numBytes(0));
+    Assert.assertEquals(1, UVLongTool.numBytes(1));
+    Assert.assertEquals(1, UVLongTool.numBytes(100));
+    Assert.assertEquals(1, UVLongTool.numBytes(126));
+    Assert.assertEquals(1, UVLongTool.numBytes(127));
+    Assert.assertEquals(2, UVLongTool.numBytes(128));
+    Assert.assertEquals(2, UVLongTool.numBytes(129));
+    Assert.assertEquals(9, UVLongTool.numBytes(Long.MAX_VALUE));
+  }
+
+  @Test
+  public void testToBytes() {
+    Assert.assertArrayEquals(new byte[] { 0 }, UVLongTool.getBytes(0));
+    Assert.assertArrayEquals(new byte[] { 1 }, UVLongTool.getBytes(1));
+    Assert.assertArrayEquals(new byte[] { 63 }, UVLongTool.getBytes(63));
+    Assert.assertArrayEquals(new byte[] { 127 }, UVLongTool.getBytes(127));
+    Assert.assertArrayEquals(new byte[] { -128, 1 }, UVLongTool.getBytes(128));
+    Assert.assertArrayEquals(new byte[] { -128 + 27, 1 }, UVLongTool.getBytes(155));
+    Assert.assertArrayEquals(UVLongTool.MAX_VALUE_BYTES, UVLongTool.getBytes(Long.MAX_VALUE));
+  }
+
+  @Test
+  public void testFromBytes() {
+    Assert.assertEquals(Long.MAX_VALUE, UVLongTool.getLong(UVLongTool.MAX_VALUE_BYTES));
+  }
+
+  @Test
+  public void testFromBytesOffset() {
+    Assert.assertEquals(Long.MAX_VALUE, UVLongTool.getLong(UVLongTool.MAX_VALUE_BYTES, 0));
+
+    long ms = 1318966363481L;
+//    System.out.println(ms);
+    byte[] bytes = UVLongTool.getBytes(ms);
+//    System.out.println(Arrays.toString(bytes));
+    long roundTripped = UVLongTool.getLong(bytes, 0);
+    Assert.assertEquals(ms, roundTripped);
+
+    int calculatedNumBytes = UVLongTool.numBytes(ms);
+    int actualNumBytes = bytes.length;
+    Assert.assertEquals(actualNumBytes, calculatedNumBytes);
+
+    byte[] shiftedBytes = new byte[1000];
+    int shift = 33;
+    System.arraycopy(bytes, 0, shiftedBytes, shift, bytes.length);
+    long shiftedRoundTrip = UVLongTool.getLong(shiftedBytes, shift);
+    Assert.assertEquals(ms, shiftedRoundTrip);
+  }
+
+  @Test
+  public void testRoundTrips() {
+    Random random = new Random();
+    for (int i = 0; i < 10000; ++i) {
+      long value = RandomNumberUtils.nextPositiveLong(random);
+      byte[] bytes = UVLongTool.getBytes(value);
+      long roundTripped = UVLongTool.getLong(bytes);
+      Assert.assertEquals(value, roundTripped);
+      int calculatedNumBytes = UVLongTool.numBytes(value);
+      int actualNumBytes = bytes.length;
+      Assert.assertEquals(actualNumBytes, calculatedNumBytes);
+    }
+  }
+
+  @Test
+  public void testInputStreams() throws IOException {
+    ByteArrayInputStream is;
+    is = new ByteArrayInputStream(new byte[] { 0 });
+    Assert.assertEquals(0, UVLongTool.getLong(is));
+    is = new ByteArrayInputStream(new byte[] { 5 });
+    Assert.assertEquals(5, UVLongTool.getLong(is));
+    is = new ByteArrayInputStream(new byte[] { -128 + 27, 1 });
+    Assert.assertEquals(155, UVLongTool.getLong(is));
+  }
+}
\ No newline at end of file

Modified: hbase/trunk/hbase-server/pom.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/pom.xml?rev=1443289&r1=1443288&r2=1443289&view=diff
==============================================================================
--- hbase/trunk/hbase-server/pom.xml (original)
+++ hbase/trunk/hbase-server/pom.xml Thu Feb  7 00:36:24 2013
@@ -284,6 +284,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-prefix-tree</artifactId>
+      <!-- unfortunately, runtime scope causes eclipse to put it in the compile time classpath -->
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-common</artifactId>
       <type>test-jar</type>
     </dependency>

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java?rev=1443289&r1=1443288&r2=1443289&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java Thu Feb  7 00:36:24 2013
@@ -112,9 +112,10 @@ import org.hbase.async.Scanner;
 public class PerformanceEvaluation extends Configured implements Tool {
   protected static final Log LOG = LogFactory.getLog(PerformanceEvaluation.class.getName());
 
-  private static final int ROW_LENGTH = 1000;
+  private static final int DEFAULT_ROW_PREFIX_LENGTH = 16;
+  private static final int VALUE_LENGTH = 1000;
   private static final int ONE_GB = 1024 * 1024 * 1000;
-  private static final int ROWS_PER_GB = ONE_GB / ROW_LENGTH;
+  private static final int ROWS_PER_GB = ONE_GB / VALUE_LENGTH;
 
   public static final byte[] COMPRESSION = Bytes.toBytes("NONE");
   public static final byte[] TABLE_NAME = Bytes.toBytes("TestTable");
@@ -127,6 +128,7 @@ public class PerformanceEvaluation exten
 
   private boolean miniCluster = false;
   private boolean nomapred = false;
+  private int rowPrefixLength = DEFAULT_ROW_PREFIX_LENGTH;
   private int N = 1;
   private int R = ROWS_PER_GB;
   private byte[] tableName = TABLE_NAME;
@@ -537,10 +539,11 @@ public class PerformanceEvaluation exten
     if (this.presplitRegions == 0)
       return new byte [0][];
 
-    byte[][] splits = new byte[this.presplitRegions][];
+    int numSplitPoints = presplitRegions - 1;
+    byte[][] splits = new byte[numSplitPoints][];
     int jump = this.R  / this.presplitRegions;
-    for (int i=0; i <this.presplitRegions; i++) {
-      int rowkey = jump * i;
+    for (int i=0; i < numSplitPoints; i++) {
+      int rowkey = jump * (1 + i);
       splits[i] = format(rowkey);
     }
     return splits;
@@ -931,9 +934,9 @@ public class PerformanceEvaluation exten
       if (row.size() != 1) {
         throw new IOException((row.isEmpty() ? "No" : "Multiple (" + row.size() + ')')
                               + " KeyValue found in row");
-      } else if (row.get(0).value().length != ROW_LENGTH) {
+      } else if (row.get(0).value().length != VALUE_LENGTH) {
         throw new IOException("Invalid value length (found: " + row.get(0).value().length
-                              + ", expected: " + ROW_LENGTH + ") in row \""
+                              + ", expected: " + VALUE_LENGTH + ") in row \""
                               + new String(row.get(0).key()) + '"');
       }
     }
@@ -1420,7 +1423,7 @@ public class PerformanceEvaluation exten
    * number (Does absolute in case number is negative).
    */
   public static byte [] format(final int number) {
-    byte [] b = new byte[10];
+    byte [] b = new byte[DEFAULT_ROW_PREFIX_LENGTH + 10];
     int d = Math.abs(number);
     for (int i = b.length - 1; i >= 0; i--) {
       b[i] = (byte)((d % 10) + '0');
@@ -1436,10 +1439,10 @@ public class PerformanceEvaluation exten
    * @return Generated random value to insert into a table cell.
    */
   public static byte[] generateValue(final Random r) {
-    byte [] b = new byte [ROW_LENGTH];
+    byte [] b = new byte [VALUE_LENGTH];
     int i = 0;
 
-    for(i = 0; i < (ROW_LENGTH-8); i += 8) {
+    for(i = 0; i < (VALUE_LENGTH-8); i += 8) {
       b[i] = (byte) (65 + r.nextInt(26));
       b[i+1] = b[i];
       b[i+2] = b[i];
@@ -1451,7 +1454,7 @@ public class PerformanceEvaluation exten
     }
 
     byte a = (byte) (65 + r.nextInt(26));
-    for(; i < ROW_LENGTH; i++) {
+    for(; i < VALUE_LENGTH; i++) {
       b[i] = a;
     }
     return b;