You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/03/14 22:11:48 UTC

[22/26] git commit: ACCUMULO-2468 Add unit tests for o.a.a.core.data

ACCUMULO-2468 Add unit tests for o.a.a.core.data

Add unit tests for several classes. Convert from JUnit 3 to 4 where
appropriate. Fix bugs where new unit tests uncovered them. Replaced
assertTrue with assertEquals where possible.


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

Branch: refs/heads/ACCUMULO-2061
Commit: e68d748b02c9d11f57767addb0daec905e10cd31
Parents: a3172ac
Author: Mike Drob <md...@cloudera.com>
Authored: Wed Mar 12 14:50:14 2014 -0400
Committer: Mike Drob <md...@cloudera.com>
Committed: Fri Mar 14 10:23:38 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/data/ArrayByteSequence.java   |   7 +-
 .../apache/accumulo/core/data/KeyExtent.java    |   3 +-
 .../core/data/ArrayByteSequenceTest.java        | 107 +++++++++
 .../accumulo/core/data/ByteSequenceTest.java    |  41 ++++
 .../apache/accumulo/core/data/ColumnTest.java   |  70 ++++--
 .../accumulo/core/data/KeyExtentTest.java       | 240 ++++++++++++++-----
 .../org/apache/accumulo/core/data/KeyTest.java  |  33 +++
 7 files changed, 416 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e68d748b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
index ff56c31..ca769cb 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
@@ -52,12 +52,13 @@ public class ArrayByteSequence extends ByteSequence implements Serializable {
   }
   
   public ArrayByteSequence(ByteBuffer buffer) {
+    this.length = buffer.remaining();
+
     if (buffer.hasArray()) {
       this.data = buffer.array();
-      this.offset = buffer.arrayOffset();
-      this.length = buffer.limit();
+      this.offset = buffer.position();
     } else {
-      this.data = new byte[buffer.remaining()];
+      this.data = new byte[length];
       this.offset = 0;
       buffer.get(data);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e68d748b/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java b/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
index dda78fb..4c9978f 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java
@@ -301,9 +301,10 @@ public class KeyExtent implements WritableComparable<KeyExtent> {
   /**
    * Empty start or end rows tell the method there are no start or end rows, and to use all the keyextents that are before the end row if no start row etc.
    * 
+   * @deprecated this method not intended for public use and is likely to be removed in a future version.
    * @return all the key extents that the rows cover
    */
-
+  @Deprecated
   public static Collection<KeyExtent> getKeyExtentsForRange(Text startRow, Text endRow, Set<KeyExtent> kes) {
     if (kes == null)
       return Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e68d748b/core/src/test/java/org/apache/accumulo/core/data/ArrayByteSequenceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/data/ArrayByteSequenceTest.java b/core/src/test/java/org/apache/accumulo/core/data/ArrayByteSequenceTest.java
new file mode 100644
index 0000000..70c40ed
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/data/ArrayByteSequenceTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.accumulo.core.data;
+
+import static org.junit.Assert.assertEquals;
+
+import java.nio.ByteBuffer;
+
+import org.apache.accumulo.core.Constants;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ArrayByteSequenceTest {
+
+  ArrayByteSequence abs;
+  byte[] data;
+
+  @Before
+  public void setUp() {
+    data = new byte[] {'s', 'm', 'i', 'l', 'e', 's'};
+    abs = new ArrayByteSequence(data);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidByteBufferBounds0() {
+    abs = new ArrayByteSequence(data, -1, 0);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidByteBufferBounds1() {
+    abs = new ArrayByteSequence(data, data.length + 1, 0);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidByteBufferBounds2() {
+    abs = new ArrayByteSequence(data, 0, -1);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidByteBufferBounds3() {
+    abs = new ArrayByteSequence(data, 6, 2);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidByteAt0() {
+    abs.byteAt(-1);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidByteAt1() {
+    abs.byteAt(data.length);
+  }
+
+  @Test
+  public void testSubSequence() {
+    assertEquals(0, abs.subSequence(0, 0).length());
+    assertEquals("mile", abs.subSequence(1, 5).toString());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidSubsequence0() {
+    abs.subSequence(5, 1);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidSubsequence1() {
+    abs.subSequence(-1, 1);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidSubsequence3() {
+    abs.subSequence(0, 10);
+  }
+
+  @Test
+  public void testFromByteBuffer() {
+    ByteBuffer bb = ByteBuffer.wrap(data, 1, 4);
+    abs = new ArrayByteSequence(bb);
+
+    assertEquals("mile", abs.toString());
+
+    bb = bb.asReadOnlyBuffer();
+    abs = new ArrayByteSequence(bb);
+
+    assertEquals("mile", abs.toString());
+  }
+
+  @Test
+  public void testToString() {
+    assertEquals("String conversion should round trip correctly", "", new ArrayByteSequence("").toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e68d748b/core/src/test/java/org/apache/accumulo/core/data/ByteSequenceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/data/ByteSequenceTest.java b/core/src/test/java/org/apache/accumulo/core/data/ByteSequenceTest.java
new file mode 100644
index 0000000..2cb60ab
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/data/ByteSequenceTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.accumulo.core.data;
+
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+public class ByteSequenceTest {
+
+  @Test
+  public void testCompareBytes() {
+    ByteSequence a = new ArrayByteSequence("a");
+    ByteSequence b = new ArrayByteSequence("b");
+    ByteSequence abc = new ArrayByteSequence("abc");
+
+    assertLessThan(a, b);
+    assertLessThan(a, abc);
+    assertLessThan(abc, b);
+  }
+
+  private void assertLessThan(ByteSequence lhs, ByteSequence rhs) {
+    int result = ByteSequence.compareBytes(lhs, rhs);
+    assertTrue(result < 0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e68d748b/core/src/test/java/org/apache/accumulo/core/data/ColumnTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/data/ColumnTest.java b/core/src/test/java/org/apache/accumulo/core/data/ColumnTest.java
index f040942..9071248 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/ColumnTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/ColumnTest.java
@@ -16,11 +16,37 @@
  */
 package org.apache.accumulo.core.data;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
 
-public class ColumnTest extends TestCase {
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.accumulo.core.data.thrift.TColumn;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class ColumnTest {
+  static Column col[];
+
+  @BeforeClass
+  public static void setup() {
+    col = new Column[5];
+    col[0] = new Column("colfam".getBytes(), "colq".getBytes(), "colv".getBytes());
+    col[1] = new Column("colfam".getBytes(), "colq".getBytes(), "colv".getBytes());
+    col[2] = new Column(new byte[0], new byte[0], new byte[0]);
+    col[3] = new Column(null, null, null);
+    col[4] = new Column("colfam".getBytes(), "cq".getBytes(), "cv".getBytes());
+  }
+
+
+  @Test
   public void testEquals() {
-    Column[] col = createColumns();
     for (int i = 0; i < col.length; i++) {
       for (int j = 0; j < col.length; j++) {
         if (i == j || (i == 0 && j == 1) || (i == 1 && j == 0))
@@ -31,31 +57,43 @@ public class ColumnTest extends TestCase {
     }
   }
   
+  @Test
   public void testCompare() {
-    Column[] col = createColumns();
     for (int i = 0; i < col.length; i++) {
       for (int j = 0; j < col.length; j++) {
         if (i == j || (i == 0 && j == 1) || (i == 1 && j == 0))
-          assertTrue(col[i].compareTo(col[j]) == 0);
+          assertEquals(0, col[i].compareTo(col[j]));
         else
-          assertFalse(col[i].compareTo(col[j]) == 0);
+          assertNotEquals(0, col[i].compareTo(col[j]));
       }
     }
   }
-  
+
+  @Test
   public void testEqualsCompare() {
-    Column[] col = createColumns();
     for (int i = 0; i < col.length; i++)
       for (int j = 0; j < col.length; j++)
-        assertTrue((col[i].compareTo(col[j]) == 0) == col[i].equals(col[j]));
+        assertEquals(col[i].equals(col[j]), col[i].compareTo(col[j]) == 0);
   }
   
-  public Column[] createColumns() {
-    Column col[] = new Column[4];
-    col[0] = new Column("colfam".getBytes(), "colq".getBytes(), "colv".getBytes());
-    col[1] = new Column("colfam".getBytes(), "colq".getBytes(), "colv".getBytes());
-    col[2] = new Column(new byte[0], new byte[0], new byte[0]);
-    col[3] = new Column(null, null, null);
-    return col;
+  @Test
+  public void testWriteReadFields() throws IOException {
+    for (Column c : col) {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      c.write(new DataOutputStream(baos));
+
+      Column other = new Column();
+      other.readFields(new DataInputStream(new ByteArrayInputStream(baos.toByteArray())));
+
+      assertEquals(c, other);
+    }
+  }
+
+  @Test
+  public void testThriftRoundTrip() {
+    for (Column c : col) {
+      TColumn tc = c.toThrift();
+      assertEquals(c, new Column(tc));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e68d748b/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java b/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
index 068ec89..ce4ad98 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
@@ -16,86 +16,111 @@
  */
 package org.apache.accumulo.core.data;
 
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
 
-public class KeyExtentTest extends TestCase {
+public class KeyExtentTest {
   KeyExtent nke(String t, String er, String per) {
     return new KeyExtent(new Text(t), er == null ? null : new Text(er), per == null ? null : new Text(per));
   }
   
+  KeyExtent ke;
+  TreeSet<KeyExtent> set0;
+
+  @Before
+  public void setup() {
+    set0 = new TreeSet<KeyExtent>();
+  }
+
+  @Test
   public void testDecodingMetadataRow() {
     Text flattenedExtent = new Text("foo;bar");
     
-    KeyExtent ke = new KeyExtent(flattenedExtent, (Text) null);
+    ke = new KeyExtent(flattenedExtent, (Text) null);
     
-    assertTrue(ke.getEndRow().equals(new Text("bar")));
-    assertTrue(ke.getTableId().equals(new Text("foo")));
-    assertTrue(ke.getPrevEndRow() == null);
+    assertEquals(new Text("bar"), ke.getEndRow());
+    assertEquals(new Text("foo"), ke.getTableId());
+    assertNull(ke.getPrevEndRow());
     
     flattenedExtent = new Text("foo<");
     
     ke = new KeyExtent(flattenedExtent, (Text) null);
     
-    assertTrue(ke.getEndRow() == null);
-    assertTrue(ke.getTableId().equals(new Text("foo")));
-    assertTrue(ke.getPrevEndRow() == null);
+    assertNull(ke.getEndRow());
+    assertEquals(new Text("foo"), ke.getTableId());
+    assertNull(ke.getPrevEndRow());
     
     flattenedExtent = new Text("foo;bar;");
     
     ke = new KeyExtent(flattenedExtent, (Text) null);
     
-    assertTrue(ke.getEndRow().equals(new Text("bar;")));
-    assertTrue(ke.getTableId().equals(new Text("foo")));
-    assertTrue(ke.getPrevEndRow() == null);
+    assertEquals(new Text("bar;"), ke.getEndRow());
+    assertEquals(new Text("foo"), ke.getTableId());
+    assertNull(ke.getPrevEndRow());
     
   }
   
+  @Test
   public void testFindContainingExtents() {
-    TreeSet<KeyExtent> set0 = new TreeSet<KeyExtent>();
-    
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, null), set0) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "1", "0"), set0) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "1", null), set0) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, "0"), set0) == null);
+    assertNull(KeyExtent.findContainingExtent(nke("t", null, null), set0));
+    assertNull(KeyExtent.findContainingExtent(nke("t", "1", "0"), set0));
+    assertNull(KeyExtent.findContainingExtent(nke("t", "1", null), set0));
+    assertNull(KeyExtent.findContainingExtent(nke("t", null, "0"), set0));
     
     TreeSet<KeyExtent> set1 = new TreeSet<KeyExtent>();
     
     set1.add(nke("t", null, null));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, null), set1).equals(nke("t", null, null)));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "1", "0"), set1).equals(nke("t", null, null)));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "1", null), set1).equals(nke("t", null, null)));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, "0"), set1).equals(nke("t", null, null)));
+    assertEquals(nke("t", null, null), KeyExtent.findContainingExtent(nke("t", null, null), set1));
+    assertEquals(nke("t", null, null), KeyExtent.findContainingExtent(nke("t", "1", "0"), set1));
+    assertEquals(nke("t", null, null), KeyExtent.findContainingExtent(nke("t", "1", null), set1));
+    assertEquals(nke("t", null, null), KeyExtent.findContainingExtent(nke("t", null, "0"), set1));
     
     TreeSet<KeyExtent> set2 = new TreeSet<KeyExtent>();
     
     set2.add(nke("t", "g", null));
     set2.add(nke("t", null, "g"));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, null), set2) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "c", "a"), set2).equals(nke("t", "g", null)));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "c", null), set2).equals(nke("t", "g", null)));
+    assertNull(KeyExtent.findContainingExtent(nke("t", null, null), set2));
+    assertEquals(nke("t", "g", null), KeyExtent.findContainingExtent(nke("t", "c", "a"), set2));
+    assertEquals(nke("t", "g", null), KeyExtent.findContainingExtent(nke("t", "c", null), set2));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "g", "a"), set2).equals(nke("t", "g", null)));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "g", null), set2).equals(nke("t", "g", null)));
+    assertEquals(nke("t", "g", null), KeyExtent.findContainingExtent(nke("t", "g", "a"), set2));
+    assertEquals(nke("t", "g", null), KeyExtent.findContainingExtent(nke("t", "g", null), set2));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "h", "a"), set2) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "h", null), set2) == null);
+    assertNull(KeyExtent.findContainingExtent(nke("t", "h", "a"), set2));
+    assertNull(KeyExtent.findContainingExtent(nke("t", "h", null), set2));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "z", "f"), set2) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, "f"), set2) == null);
+    assertNull(KeyExtent.findContainingExtent(nke("t", "z", "f"), set2));
+    assertNull(KeyExtent.findContainingExtent(nke("t", null, "f"), set2));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "z", "g"), set2).equals(nke("t", null, "g")));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, "g"), set2).equals(nke("t", null, "g")));
+    assertEquals(nke("t", null, "g"), KeyExtent.findContainingExtent(nke("t", "z", "g"), set2));
+    assertEquals(nke("t", null, "g"), KeyExtent.findContainingExtent(nke("t", null, "g"), set2));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "z", "h"), set2).equals(nke("t", null, "g")));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, "h"), set2).equals(nke("t", null, "g")));
+    assertEquals(nke("t", null, "g"), KeyExtent.findContainingExtent(nke("t", "z", "h"), set2));
+    assertEquals(nke("t", null, "g"), KeyExtent.findContainingExtent(nke("t", null, "h"), set2));
     
     TreeSet<KeyExtent> set3 = new TreeSet<KeyExtent>();
     
@@ -103,25 +128,25 @@ public class KeyExtentTest extends TestCase {
     set3.add(nke("t", "s", "g"));
     set3.add(nke("t", null, "s"));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, null), set3) == null);
+    assertNull(KeyExtent.findContainingExtent(nke("t", null, null), set3));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "g", null), set3).equals(nke("t", "g", null)));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "s", "g"), set3).equals(nke("t", "s", "g")));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, "s"), set3).equals(nke("t", null, "s")));
+    assertEquals(nke("t", "g", null), KeyExtent.findContainingExtent(nke("t", "g", null), set3));
+    assertEquals(nke("t", "s", "g"), KeyExtent.findContainingExtent(nke("t", "s", "g"), set3));
+    assertEquals(nke("t", null, "s"), KeyExtent.findContainingExtent(nke("t", null, "s"), set3));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "t", "g"), set3) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "t", "f"), set3) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "s", "f"), set3) == null);
+    assertNull(KeyExtent.findContainingExtent(nke("t", "t", "g"), set3));
+    assertNull(KeyExtent.findContainingExtent(nke("t", "t", "f"), set3));
+    assertNull(KeyExtent.findContainingExtent(nke("t", "s", "f"), set3));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "r", "h"), set3).equals(nke("t", "s", "g")));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "s", "h"), set3).equals(nke("t", "s", "g")));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "r", "g"), set3).equals(nke("t", "s", "g")));
+    assertEquals(nke("t", "s", "g"), KeyExtent.findContainingExtent(nke("t", "r", "h"), set3));
+    assertEquals(nke("t", "s", "g"), KeyExtent.findContainingExtent(nke("t", "s", "h"), set3));
+    assertEquals(nke("t", "s", "g"), KeyExtent.findContainingExtent(nke("t", "r", "g"), set3));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, "t"), set3).equals(nke("t", null, "s")));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, "r"), set3) == null);
+    assertEquals(nke("t", null, "s"), KeyExtent.findContainingExtent(nke("t", null, "t"), set3));
+    assertNull(KeyExtent.findContainingExtent(nke("t", null, "r"), set3));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "f", null), set3).equals(nke("t", "g", null)));
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "h", null), set3) == null);
+    assertEquals(nke("t", "g", null), KeyExtent.findContainingExtent(nke("t", "f", null), set3));
+    assertNull(KeyExtent.findContainingExtent(nke("t", "h", null), set3));
     
     TreeSet<KeyExtent> set4 = new TreeSet<KeyExtent>();
     
@@ -132,24 +157,24 @@ public class KeyExtentTest extends TestCase {
     set4.add(nke("t2", "s", "g"));
     set4.add(nke("t2", null, "s"));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", null, null), set4) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("z", null, null), set4) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t11", null, null), set4) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t1", null, null), set4) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t2", null, null), set4) == null);
+    assertNull(KeyExtent.findContainingExtent(nke("t", null, null), set4));
+    assertNull(KeyExtent.findContainingExtent(nke("z", null, null), set4));
+    assertNull(KeyExtent.findContainingExtent(nke("t11", null, null), set4));
+    assertNull(KeyExtent.findContainingExtent(nke("t1", null, null), set4));
+    assertNull(KeyExtent.findContainingExtent(nke("t2", null, null), set4));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t", "g", null), set4) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("z", "g", null), set4) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t11", "g", null), set4) == null);
-    assertTrue(KeyExtent.findContainingExtent(nke("t1", "g", null), set4) == null);
+    assertNull(KeyExtent.findContainingExtent(nke("t", "g", null), set4));
+    assertNull(KeyExtent.findContainingExtent(nke("z", "g", null), set4));
+    assertNull(KeyExtent.findContainingExtent(nke("t11", "g", null), set4));
+    assertNull(KeyExtent.findContainingExtent(nke("t1", "g", null), set4));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t2", "g", null), set4).equals(nke("t2", "g", null)));
-    assertTrue(KeyExtent.findContainingExtent(nke("t2", "s", "g"), set4).equals(nke("t2", "s", "g")));
-    assertTrue(KeyExtent.findContainingExtent(nke("t2", null, "s"), set4).equals(nke("t2", null, "s")));
+    assertEquals(nke("t2", "g", null), KeyExtent.findContainingExtent(nke("t2", "g", null), set4));
+    assertEquals(nke("t2", "s", "g"), KeyExtent.findContainingExtent(nke("t2", "s", "g"), set4));
+    assertEquals(nke("t2", null, "s"), KeyExtent.findContainingExtent(nke("t2", null, "s"), set4));
     
-    assertTrue(KeyExtent.findContainingExtent(nke("t1", "d", null), set4).equals(nke("t1", "d", null)));
-    assertTrue(KeyExtent.findContainingExtent(nke("t1", "q", "d"), set4).equals(nke("t1", "q", "d")));
-    assertTrue(KeyExtent.findContainingExtent(nke("t1", null, "q"), set4).equals(nke("t1", null, "q")));
+    assertEquals(nke("t1", "d", null), KeyExtent.findContainingExtent(nke("t1", "d", null), set4));
+    assertEquals(nke("t1", "q", "d"), KeyExtent.findContainingExtent(nke("t1", "q", "d"), set4));
+    assertEquals(nke("t1", null, "q"), KeyExtent.findContainingExtent(nke("t1", null, "q"), set4));
     
   }
   
@@ -157,6 +182,7 @@ public class KeyExtentTest extends TestCase {
     return !KeyExtent.findOverlapping(extent, extents).isEmpty();
   }
   
+  @Test
   public void testOverlaps() {
     SortedMap<KeyExtent,Object> set0 = new TreeMap<KeyExtent,Object>();
     set0.put(nke("a", null, null), null);
@@ -237,4 +263,88 @@ public class KeyExtentTest extends TestCase {
     assertFalse(nke("t", null, "b").overlaps(nke("t", "b", "a")));
     assertFalse(nke("t", null, "b").overlaps(nke("t", "b", null)));
   }
+
+  @Test
+  public void testWriteReadFields() throws Exception {
+    ke = nke("t", "e", "b");
+    assertEquals(ke, writeAndReadFields(ke));
+
+    ke = nke("t", "e", null);
+    assertEquals(ke, writeAndReadFields(ke));
+
+    ke = nke("t", null, "b");
+    assertEquals(ke, writeAndReadFields(ke));
+
+    ke = nke("t", null, null);
+    assertEquals(ke, writeAndReadFields(ke));
+  }
+
+  private KeyExtent writeAndReadFields(KeyExtent in) throws IOException {
+    KeyExtent out = new KeyExtent();
+
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    in.write(new DataOutputStream(baos));
+
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    out.readFields(new DataInputStream(bais));
+
+    return out;
+  }
+
+  @Test
+  public void testKeyExtentsForSimpleRange() {
+    Collection<KeyExtent> results;
+
+    results = KeyExtent.getKeyExtentsForRange(null, null, null);
+    assertTrue("Non-empty set returned from no extents", results.isEmpty());
+
+    results = KeyExtent.getKeyExtentsForRange(null, null, Collections.<KeyExtent> emptySet());
+    assertTrue("Non-empty set returned from no extents", results.isEmpty());
+
+    KeyExtent t = nke("t", null, null);
+    results = KeyExtent.getKeyExtentsForRange(null, null, Collections.<KeyExtent> singleton(t));
+    assertEquals("Single tablet should always be returned", 1, results.size());
+    assertEquals(t, results.iterator().next());
+  }
+
+  @Test
+  public void testKeyExtentsForRange() {
+    KeyExtent b = nke("t", "b", null);
+    KeyExtent e = nke("t", "e", "b");
+    KeyExtent h = nke("t", "h", "e");
+    KeyExtent m = nke("t", "m", "h");
+    KeyExtent z = nke("t", null, "m");
+
+    set0.addAll(Arrays.asList(b, e, h, m, z));
+
+    Collection<KeyExtent> results;
+
+    results = KeyExtent.getKeyExtentsForRange(null, null, set0);
+    assertThat("infinite range should return full set", results.size(), is(5));
+    assertThat("infinite range should return full set", results, hasItems(b, e, h, m, z));
+
+    results = KeyExtent.getKeyExtentsForRange(new Text("a"), new Text("z"), set0);
+    assertThat("full overlap should return full set", results.size(), is(5));
+    assertThat("full overlap should return full set", results, hasItems(b, e, h, m, z));
+
+    results = KeyExtent.getKeyExtentsForRange(null, new Text("f"), set0);
+    assertThat("end row should return head set", results.size(), is(3));
+    assertThat("end row should return head set", results, hasItems(b, e, h));
+
+    results = KeyExtent.getKeyExtentsForRange(new Text("f"), null, set0);
+    assertThat("start row should return tail set", results.size(), is(3));
+    assertThat("start row should return tail set", results, hasItems(h, m, z));
+
+    results = KeyExtent.getKeyExtentsForRange(new Text("f"), new Text("g"), set0);
+    assertThat("slice should return correct subset", results.size(), is(1));
+    assertThat("slice should return correct subset", results, hasItem(h));
+  }
+
+  @Test
+  public void testDecodeEncode() {
+    assertNull(KeyExtent.decodePrevEndRow(KeyExtent.encodePrevEndRow(null)));
+
+    Text x = new Text();
+    assertEquals(x, KeyExtent.decodePrevEndRow(KeyExtent.encodePrevEndRow(x)));
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e68d748b/core/src/test/java/org/apache/accumulo/core/data/KeyTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/data/KeyTest.java b/core/src/test/java/org/apache/accumulo/core/data/KeyTest.java
index 56442a1..cc737db 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/KeyTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/KeyTest.java
@@ -21,7 +21,12 @@ import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
 import org.apache.accumulo.core.data.thrift.TKey;
+import org.apache.accumulo.core.data.thrift.TKeyValue;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
@@ -133,4 +138,32 @@ public class KeyTest {
     tk.setRow((byte[]) null);
     new Key(tk);
   }
+
+  @Test
+  public void testCompressDecompress() {
+    List<KeyValue> kvs = new ArrayList<KeyValue>();
+    kvs.add(new KeyValue(new Key(), new byte[] {}));
+    kvs.add(new KeyValue(new Key("r"), new byte[] {}));
+    kvs.add(new KeyValue(new Key("r", "cf"), new byte[] {}));
+    kvs.add(new KeyValue(new Key("r2", "cf"), new byte[] {}));
+    kvs.add(new KeyValue(new Key("r", "cf", "cq"), new byte[] {}));
+    kvs.add(new KeyValue(new Key("r2", "cf2", "cq"), new byte[] {}));
+    kvs.add(new KeyValue(new Key("r", "cf", "cq", "cv"), new byte[] {}));
+    kvs.add(new KeyValue(new Key("r2", "cf2", "cq2", "cv"), new byte[] {}));
+    kvs.add(new KeyValue(new Key("r2", "cf2", "cq2", "cv"), new byte[] {}));
+    kvs.add(new KeyValue(new Key(), new byte[] {}));
+
+    List<TKeyValue> tkvs = Key.compress(kvs);
+    Key.decompress(tkvs);
+
+    assertEquals(kvs.size(), tkvs.size());
+    Iterator<KeyValue> kvi = kvs.iterator();
+    Iterator<TKeyValue> tkvi = tkvs.iterator();
+
+    while (kvi.hasNext()) {
+      KeyValue kv = kvi.next();
+      TKeyValue tkv = tkvi.next();
+      assertEquals(kv.getKey(), new Key(tkv.getKey()));
+    }
+  }
 }