You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2008/05/16 00:10:50 UTC

svn commit: r656868 [8/10] - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/filter/ src/java/org/apache/hadoop/hbase/hql/ src/java/org/apache/hadoop/hbase/io/ src/j...

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestHBaseCluster.java Thu May 15 15:10:47 2008
@@ -20,17 +20,17 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Iterator;
-import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Scanner;
-import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Test HBase Master and Region servers, client API 
@@ -76,17 +76,19 @@
 
   private static final int FIRST_ROW = 1;
   private static final int NUM_VALS = 1000;
-  private static final Text CONTENTS = new Text("contents:");
-  private static final Text CONTENTS_BASIC = new Text("contents:basic");
+  private static final byte [] CONTENTS = Bytes.toBytes("contents:");
+  private static final byte [] CONTENTS_MINUS_COLON = Bytes.toBytes("contents");
+  private static final byte [] CONTENTS_BASIC = Bytes.toBytes("contents:basic");
   private static final String CONTENTSTR = "contentstr";
-  private static final Text ANCHOR = new Text("anchor:");
+  private static final byte [] ANCHOR = Bytes.toBytes("anchor:");
+  private static final byte [] ANCHOR_MINUS_COLON = Bytes.toBytes("anchor");
   private static final String ANCHORNUM = "anchor:anchornum-";
   private static final String ANCHORSTR = "anchorstr";
 
   private void setup() throws IOException {
     desc = new HTableDescriptor("test");
-    desc.addFamily(new HColumnDescriptor(CONTENTS.toString()));
-    desc.addFamily(new HColumnDescriptor(ANCHOR.toString()));
+    desc.addFamily(new HColumnDescriptor(CONTENTS));
+    desc.addFamily(new HColumnDescriptor(ANCHOR));
     admin = new HBaseAdmin(conf);
     admin.createTable(desc);
     table = new HTable(conf, desc.getName());
@@ -100,11 +102,9 @@
     // Write out a bunch of values
 
     for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
-      BatchUpdate b = new BatchUpdate(new Text("row_" + k));
-      b.put(CONTENTS_BASIC,
-          (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING));
-      b.put(new Text(ANCHORNUM + k),
-          (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING));
+      BatchUpdate b = new BatchUpdate("row_" + k);
+      b.put(CONTENTS_BASIC, Bytes.toBytes(CONTENTSTR + k));
+      b.put(ANCHORNUM + k, Bytes.toBytes(ANCHORSTR + k));
       table.commit(b);
     }
     LOG.info("Write " + NUM_VALS + " rows. Elapsed time: "
@@ -114,9 +114,9 @@
 
     startTime = System.currentTimeMillis();
 
-    Text collabel = null;
+    byte [] collabel = null;
     for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
-      Text rowlabel = new Text("row_" + k);
+      byte [] rowlabel = Bytes.toBytes("row_" + k);
 
       byte bodydata[] = table.get(rowlabel, CONTENTS_BASIC).getValue();
       assertNotNull("no data for row " + rowlabel + "/" + CONTENTS_BASIC,
@@ -127,7 +127,7 @@
           CONTENTS_BASIC + "), expected: '" + teststr + "' got: '" +
           bodystr + "'", teststr.compareTo(bodystr) == 0);
       
-      collabel = new Text(ANCHORNUM + k);
+      collabel = Bytes.toBytes(ANCHORNUM + k);
       bodydata = table.get(rowlabel, collabel).getValue();
       assertNotNull("no data for row " + rowlabel + "/" + collabel, bodydata);
       bodystr = new String(bodydata, HConstants.UTF8_ENCODING);
@@ -142,33 +142,30 @@
   }
   
   private void scanner() throws IOException {
-    Text[] cols = new Text[] {
-        new Text(ANCHORNUM + "[0-9]+"),
-        new Text(CONTENTS_BASIC)
-    };
+    byte [][] cols = new byte [][] {Bytes.toBytes(ANCHORNUM + "[0-9]+"),
+      CONTENTS_BASIC};
     
     long startTime = System.currentTimeMillis();
     
-    Scanner s = table.getScanner(cols, new Text());
+    Scanner s = table.getScanner(cols, HConstants.EMPTY_BYTE_ARRAY);
     try {
 
       int contentsFetched = 0;
       int anchorFetched = 0;
       int k = 0;
       for (RowResult curVals : s) {
-        for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
-          Text col = it.next();
+        for (Iterator<byte []> it = curVals.keySet().iterator(); it.hasNext(); ) {
+          byte [] col = it.next();
           byte val[] = curVals.get(col).getValue();
-          String curval = new String(val, HConstants.UTF8_ENCODING).trim();
-
-          if(col.compareTo(CONTENTS_BASIC) == 0) {
+          String curval = Bytes.toString(val);
+          if (Bytes.compareTo(col, CONTENTS_BASIC) == 0) {
             assertTrue("Error at:" + curVals.getRow() 
                 + ", Value for " + col + " should start with: " + CONTENTSTR
                 + ", but was fetched as: " + curval,
                 curval.startsWith(CONTENTSTR));
             contentsFetched++;
             
-          } else if(col.toString().startsWith(ANCHORNUM)) {
+          } else if (Bytes.toString(col).startsWith(ANCHORNUM)) {
             assertTrue("Error at:" + curVals.getRow()
                 + ", Value for " + col + " should start with: " + ANCHORSTR
                 + ", but was fetched as: " + curval,
@@ -176,13 +173,18 @@
             anchorFetched++;
             
           } else {
-            LOG.info(col);
+            LOG.info(Bytes.toString(col));
           }
         }
         k++;
       }
-      assertEquals("Expected " + NUM_VALS + " " + CONTENTS_BASIC + " values, but fetched " + contentsFetched, NUM_VALS, contentsFetched);
-      assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM + " values, but fetched " + anchorFetched, NUM_VALS, anchorFetched);
+      assertEquals("Expected " + NUM_VALS + " " +
+        Bytes.toString(CONTENTS_BASIC) + " values, but fetched " +
+        contentsFetched,
+        NUM_VALS, contentsFetched);
+      assertEquals("Expected " + NUM_VALS + " " + ANCHORNUM +
+        " values, but fetched " + anchorFetched,
+        NUM_VALS, anchorFetched);
 
       LOG.info("Scanned " + NUM_VALS
           + " rows. Elapsed time: "
@@ -196,10 +198,10 @@
   private void listTables() throws IOException {
     HTableDescriptor[] tables = admin.listTables();
     assertEquals(1, tables.length);
-    assertEquals(desc.getName(), tables[0].getName());
-    Set<Text> families = tables[0].families().keySet();
+    assertTrue(Bytes.equals(desc.getName(), tables[0].getName()));
+    Collection<HColumnDescriptor> families = tables[0].getFamilies();
     assertEquals(2, families.size());
-    assertTrue(families.contains(new Text(CONTENTS)));
-    assertTrue(families.contains(new Text(ANCHOR)));
+    assertTrue(tables[0].hasFamily(CONTENTS));
+    assertTrue(tables[0].hasFamily(ANCHOR));
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestInfoServers.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestInfoServers.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestInfoServers.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestInfoServers.java Thu May 15 15:10:47 2008
@@ -25,7 +25,6 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.client.HTable;
 
 /**
@@ -48,8 +47,7 @@
    */
   public void testInfoServersAreUp() throws Exception {
     // give the cluster time to start up
-    HTable table = new HTable(conf, new Text(".META."));
-    
+    new HTable(conf, ".META.");
     int port = cluster.getMaster().getInfoServer().getPort();
     assertHasExpectedContent(new URL("http://localhost:" + port +
       "/index.html"), "Master");

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java Thu May 15 15:10:47 2008
@@ -21,19 +21,19 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /** tests administrative functions */
 public class TestMasterAdmin extends HBaseClusterTestCase {
   private final Log LOG = LogFactory.getLog(this.getClass().getName());
   
-  private static final Text COLUMN_NAME = new Text("col1:");
+  private static final byte [] COLUMN_NAME = Bytes.toBytes("col1:");
   private static HTableDescriptor testDesc;
   static {
     testDesc = new HTableDescriptor("testadmin1");
-    testDesc.addFamily(new HColumnDescriptor(COLUMN_NAME.toString()));
+    testDesc.addFamily(new HColumnDescriptor(COLUMN_NAME));
   }
   
   private HBaseAdmin admin;
@@ -61,9 +61,9 @@
     }
     assertTrue(exception);
     admin.createTable(testDesc);
-    LOG.info("Table " + testDesc.getName().toString() + " created");
+    LOG.info("Table " + testDesc.getNameAsString() + " created");
     admin.disableTable(testDesc.getName());
-    LOG.info("Table " + testDesc.getName().toString() + " disabled");
+    LOG.info("Table " + testDesc.getNameAsString() + " disabled");
     try {
       @SuppressWarnings("unused")
       HTable table = new HTable(conf, testDesc.getName());
@@ -78,13 +78,13 @@
     admin.addColumn(testDesc.getName(), new HColumnDescriptor("col2:"));
     admin.enableTable(testDesc.getName());
     try {
-      admin.deleteColumn(testDesc.getName(), new Text("col2:"));
+      admin.deleteColumn(testDesc.getName(), Bytes.toBytes("col2:"));
     } catch(TableNotDisabledException e) {
       // Expected
     }
 
     admin.disableTable(testDesc.getName());
-    admin.deleteColumn(testDesc.getName(), new Text("col2:"));
+    admin.deleteColumn(testDesc.getName(), Bytes.toBytes("col2:"));
     admin.deleteTable(testDesc.getName());
   }
 }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestRegionRebalancing.java Thu May 15 15:10:47 2008
@@ -25,7 +25,6 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.Text;
 
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.client.HTable;
@@ -36,6 +35,7 @@
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Test whether region rebalancing works. (HBASE-71)
@@ -48,7 +48,7 @@
   
   final byte[] FIVE_HUNDRED_KBYTES;
   
-  final Text COLUMN_NAME = new Text("col:");
+  final byte [] COLUMN_NAME = Bytes.toBytes("col:");
   
   /** constructor */
   public TestRegionRebalancing() {
@@ -59,7 +59,7 @@
     }
     
     desc = new HTableDescriptor("test");
-    desc.addFamily(new HColumnDescriptor(COLUMN_NAME.toString()));
+    desc.addFamily(new HColumnDescriptor(COLUMN_NAME));
   }
   
   /**
@@ -68,10 +68,10 @@
   @Override
   public void preHBaseClusterSetup() throws IOException {
     // create a 20-region table by writing directly to disk
-    List<Text> startKeys = new ArrayList<Text>();
+    List<byte []> startKeys = new ArrayList<byte []>();
     startKeys.add(null);
     for (int i = 10; i < 29; i++) {
-      startKeys.add(new Text("row_" + i));
+      startKeys.add(Bytes.toBytes("row_" + i));
     }
     startKeys.add(null);
     LOG.info(startKeys.size() + " start keys generated");
@@ -84,9 +84,9 @@
     // Now create the root and meta regions and insert the data regions
     // created above into the meta
     
-    HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo,
+    HRegion root = HRegion.createHRegion(HRegionInfo.ROOT_REGIONINFO,
       testDir, conf);
-    HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo,
+    HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO,
       testDir, conf);
     HRegion.addRegionToMETA(root, meta);
     
@@ -105,7 +105,7 @@
    * region servers to see if the assignment or regions is pretty balanced.
    */
   public void testRebalancing() throws IOException {
-    table = new HTable(conf, new Text("test"));
+    table = new HTable(conf, "test");
     assertEquals("Test table should have 20 regions", 
       20, table.getStartKeys().length);
     
@@ -220,12 +220,12 @@
    * create a region with the specified start and end key and exactly one row
    * inside. 
    */
-  private HRegion createAregion(Text startKey, Text endKey) 
+  private HRegion createAregion(byte [] startKey, byte [] endKey) 
   throws IOException {
     
     HRegion region = createNewHRegion(desc, startKey, endKey);
     
-    Text keyToWrite = startKey == null ? new Text("row_000") : startKey;
+    byte [] keyToWrite = startKey == null ? Bytes.toBytes("row_000") : startKey;
     
     BatchUpdate bu = new BatchUpdate(keyToWrite);
     bu.put(COLUMN_NAME, "test".getBytes());

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestScannerAPI.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestScannerAPI.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestScannerAPI.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestScannerAPI.java Thu May 15 15:10:47 2008
@@ -22,30 +22,30 @@
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.Iterator;
+import java.util.Collection;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Scanner;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
 
 /** test the scanner API at all levels */
 public class TestScannerAPI extends HBaseClusterTestCase {
-  private final Text[] columns = new Text[] {
+  private final byte [][] columns = Bytes.toByteArrays(new Text[] {
     new Text("a:"),
     new Text("b:")
-  };
-  private final Text startRow = new Text("0");
+  });
+  private final byte [] startRow = Bytes.toBytes("0");
 
-  private final TreeMap<Text, SortedMap<Text, byte[]>> values =
-    new TreeMap<Text, SortedMap<Text, byte[]>>();
+  private final TreeMap<byte [], SortedMap<byte [], byte[]>> values =
+    new TreeMap<byte [], SortedMap<byte [], byte[]>>(Bytes.BYTES_COMPARATOR);
   
   /**
    * @throws Exception
@@ -53,12 +53,13 @@
   public TestScannerAPI() throws Exception {
     super();
     try {
-      TreeMap<Text, byte[]> columns = new TreeMap<Text, byte[]>();
-      columns.put(new Text("a:1"), "1".getBytes(HConstants.UTF8_ENCODING));
-      values.put(new Text("1"), columns);
-      columns = new TreeMap<Text, byte[]>();
-      columns.put(new Text("a:2"), "2".getBytes(HConstants.UTF8_ENCODING));
-      columns.put(new Text("b:2"), "2".getBytes(HConstants.UTF8_ENCODING));
+      TreeMap<byte [], byte[]> columns =
+        new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
+      columns.put(Bytes.toBytes("a:1"), Bytes.toBytes("1"));
+      values.put(Bytes.toBytes("1"), columns);
+      columns = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
+      columns.put(Bytes.toBytes("a:2"), Bytes.toBytes("2"));
+      columns.put(Bytes.toBytes("b:2"), Bytes.toBytes("2"));
     } catch (Exception e) {
       e.printStackTrace();
       throw e;
@@ -76,17 +77,17 @@
     HBaseAdmin admin = new HBaseAdmin(conf);
     HTableDescriptor tableDesc = new HTableDescriptor(tableName);
     for (int i = 0; i < columns.length; i++) {
-      tableDesc.addFamily(new HColumnDescriptor(columns[i].toString()));
+      tableDesc.addFamily(new HColumnDescriptor(columns[i]));
     }
     admin.createTable(tableDesc);
 
     // Insert values
     
-    HTable table = new HTable(conf, new Text(getName()));
+    HTable table = new HTable(conf, getName());
 
-    for (Map.Entry<Text, SortedMap<Text, byte[]>> row: values.entrySet()) {
+    for (Map.Entry<byte [], SortedMap<byte [], byte[]>> row: values.entrySet()) {
       BatchUpdate b = new BatchUpdate(row.getKey());
-      for (Map.Entry<Text, byte[]> val: row.getValue().entrySet()) {
+      for (Map.Entry<byte [], byte[]> val: row.getValue().entrySet()) {
         b.put(val.getKey(), val.getValue());
       }
       table.commit(b);
@@ -94,11 +95,11 @@
 
     HRegion region = null;
     try {
-      Map<Text, HRegion> regions =
+      Collection<HRegion> regions =
         cluster.getRegionThreads().get(0).getRegionServer().getOnlineRegions();
-      for (Map.Entry<Text, HRegion> e: regions.entrySet()) {
-        if (!e.getValue().getRegionInfo().isMetaRegion()) {
-          region = e.getValue();
+      for (HRegion r: regions) {
+        if (!r.getRegionInfo().isMetaRegion()) {
+          region = r;
         }
       }
     } catch (Exception e) {
@@ -127,10 +128,10 @@
       for (RowResult r : scanner2) {
         assertTrue("row key", values.containsKey(r.getRow()));
 
-        SortedMap<Text, byte[]> columnValues = values.get(r.getRow());
-        assertEquals(columnValues.size(), r.size());        
-        for (Map.Entry<Text, byte[]> e: columnValues.entrySet()) {
-          Text column = e.getKey();
+        SortedMap<byte [], byte[]> columnValues = values.get(r.getRow());
+        assertEquals(columnValues.size(), r.size());
+        for (Map.Entry<byte [], byte[]> e: columnValues.entrySet()) {
+          byte [] column = e.getKey();
           assertTrue("column", r.containsKey(column));
           assertTrue("value", Arrays.equals(columnValues.get(column),
             r.get(column).getValue()));
@@ -143,15 +144,16 @@
   
   private void verify(ScannerIncommon scanner) throws IOException {
     HStoreKey key = new HStoreKey();
-    SortedMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
+    SortedMap<byte [], byte[]> results =
+      new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
     while (scanner.next(key, results)) {
-      Text row = key.getRow();
+      byte [] row = key.getRow();
       assertTrue("row key", values.containsKey(row));
       
-      SortedMap<Text, byte[]> columnValues = values.get(row);
+      SortedMap<byte [], byte[]> columnValues = values.get(row);
       assertEquals(columnValues.size(), results.size());
-      for (Map.Entry<Text, byte[]> e: columnValues.entrySet()) {
-        Text column = e.getKey();
+      for (Map.Entry<byte [], byte[]> e: columnValues.entrySet()) {
+        byte [] column = e.getKey();
         assertTrue("column", results.containsKey(column));
         assertTrue("value", Arrays.equals(columnValues.get(column),
             results.get(column)));
@@ -159,4 +161,4 @@
       results.clear();
     }
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestSerialization.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestSerialization.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestSerialization.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestSerialization.java Thu May 15 15:10:47 2008
@@ -19,17 +19,18 @@
  */
 package org.apache.hadoop.hbase;
 
-import junit.framework.TestCase;
-
 import org.apache.hadoop.hbase.io.BatchOperation;
 import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.io.HbaseMapWritable;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.Text;
 
 /**
  * Test HBase Writables serializations
  */
-public class TestSerialization extends TestCase {
+public class TestSerialization extends HBaseTestCase {
 
   protected void setUp() throws Exception {
     super.setUp();
@@ -39,6 +40,64 @@
     super.tearDown();
   }
 
+  public void testname() throws Exception {
+    HMsg  m = new HMsg(HMsg.MSG_REGIONSERVER_QUIESCE);
+    byte [] mb = Writables.getBytes(m);
+    HMsg deserializedHMsg = (HMsg)Writables.getWritable(mb, new HMsg());
+    assertTrue(m.getMsg() == deserializedHMsg.getMsg());
+  }
+  
+  public void testTableDescriptor() throws Exception {
+    HTableDescriptor htd = createTableDescriptor(getName());
+    byte [] mb = Writables.getBytes(htd);
+    HTableDescriptor deserializedHtd =
+      (HTableDescriptor)Writables.getWritable(mb, new HTableDescriptor());
+    assertEquals(htd.getNameAsString(), deserializedHtd.getNameAsString());
+  }
+
+  /**
+   * Test RegionInfo serialization
+   * @throws Exception
+   */
+  public void testRowResult() throws Exception {
+    HbaseMapWritable<byte [], Cell> m = new HbaseMapWritable<byte [], Cell>();
+    byte [] b = Bytes.toBytes(getName());
+    m.put(b, new Cell(b, System.currentTimeMillis()));
+    RowResult rr = new RowResult(b, m);
+    byte [] mb = Writables.getBytes(rr);
+    RowResult deserializedRr =
+      (RowResult)Writables.getWritable(mb, new RowResult());
+    assertTrue(Bytes.equals(rr.getRow(), deserializedRr.getRow()));
+    byte [] one = rr.get(b).getValue();
+    byte [] two = deserializedRr.get(b).getValue();
+    assertTrue(Bytes.equals(one, two));
+    Writables.copyWritable(rr, deserializedRr);
+    one = rr.get(b).getValue();
+    two = deserializedRr.get(b).getValue();
+    assertTrue(Bytes.equals(one, two));
+    
+  }
+
+  /**
+   * Test RegionInfo serialization
+   * @throws Exception
+   */
+  public void testRegionInfo() throws Exception {
+    HTableDescriptor htd = new HTableDescriptor(getName());
+    String [] families = new String [] {"info:", "anchor:"};
+    for (int i = 0; i < families.length; i++) {
+      htd.addFamily(new HColumnDescriptor(families[i]));
+    }
+    HRegionInfo hri = new HRegionInfo(htd,
+      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
+    byte [] hrib = Writables.getBytes(hri);
+    HRegionInfo deserializedHri =
+      (HRegionInfo)Writables.getWritable(hrib, new HRegionInfo());
+    assertEquals(hri.getEncodedName(), deserializedHri.getEncodedName());
+    assertEquals(hri.getTableDesc().getFamilies().size(),
+      deserializedHri.getTableDesc().getFamilies().size());
+  }
+  
   /**
    * Test ServerInfo serialization
    * @throws Exception
@@ -57,16 +116,15 @@
    * @throws Exception
    */
   public void testBatchUpdate() throws Exception {
-    final Text testName = new Text(getName());
     // Add row named 'testName'.
-    BatchUpdate bu = new BatchUpdate(testName);
+    BatchUpdate bu = new BatchUpdate(getName());
     // Add a column named same as row.
-    bu.put(testName, testName.getBytes());
+    bu.put(getName(), getName().getBytes());
     byte [] b = Writables.getBytes(bu);
     BatchUpdate bubu =
       (BatchUpdate)Writables.getWritable(b, new BatchUpdate());
     // Assert rows are same.
-    assertTrue(bu.getRow().equals(bubu.getRow()));
+    assertTrue(Bytes.equals(bu.getRow(), bubu.getRow()));
     // Assert has same number of BatchOperations.
     int firstCount = 0;
     for (BatchOperation bo: bubu) {
@@ -76,7 +134,7 @@
     // accumulating BatchOperations on each deserialization.
     BatchUpdate bububu = (BatchUpdate)Writables.getWritable(b, bubu);
     // Assert rows are same again.
-    assertTrue(bu.getRow().equals(bububu.getRow()));
+    assertTrue(Bytes.equals(bu.getRow(), bububu.getRow()));
     int secondCount = 0;
     for (BatchOperation bo: bububu) {
       secondCount++;

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTable.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTable.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestTable.java Thu May 15 15:10:47 2008
@@ -35,28 +35,28 @@
     final HBaseAdmin admin = new HBaseAdmin(conf);
     String msg = null;
     try {
-      admin.createTable(HTableDescriptor.rootTableDesc);
+      admin.createTable(HTableDescriptor.ROOT_TABLEDESC);
     } catch (IllegalArgumentException e) {
       msg = e.toString();
     }
     assertTrue("Unexcepted exception message " + msg, msg != null &&
       msg.startsWith(IllegalArgumentException.class.getName()) &&
-      msg.contains(HTableDescriptor.rootTableDesc.getName().toString()));
+      msg.contains(HTableDescriptor.ROOT_TABLEDESC.getNameAsString()));
     
     msg = null;
     try {
-      admin.createTable(HTableDescriptor.metaTableDesc);
+      admin.createTable(HTableDescriptor.META_TABLEDESC);
     } catch(IllegalArgumentException e) {
       msg = e.toString();
     }
     assertTrue("Unexcepted exception message " + msg, msg != null &&
       msg.startsWith(IllegalArgumentException.class.getName()) &&
-      msg.contains(HTableDescriptor.metaTableDesc.getName().toString()));
+      msg.contains(HTableDescriptor.META_TABLEDESC.getNameAsString()));
     
     // Try doing a duplicate database create.
     msg = null;
     HTableDescriptor desc = new HTableDescriptor(getName());
-    desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
+    desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY));
     admin.createTable(desc);
     assertTrue("First table creation completed", admin.listTables().length == 1);
     boolean gotException = false;
@@ -72,9 +72,8 @@
     
     // Now try and do concurrent creation with a bunch of threads.
     final HTableDescriptor threadDesc =
-      new HTableDescriptor("threaded-" + getName());
-    threadDesc.addFamily(new HColumnDescriptor(HConstants.
-      COLUMN_FAMILY.toString()));
+      new HTableDescriptor("threaded_" + getName());
+    threadDesc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY));
     int count = 10;
     Thread [] threads = new Thread [count];
     final AtomicInteger successes = new AtomicInteger(0);
@@ -123,6 +122,6 @@
     admin.createTable(new HTableDescriptor(getName()));
     // Before fix, below would fail throwing a NoServerForRegionException.
     @SuppressWarnings("unused")
-    HTable table = new HTable(conf, new Text(getName()));
+    HTable table = new HTable(conf, getName());
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestToString.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestToString.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestToString.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TestToString.java Thu May 15 15:10:47 2008
@@ -41,25 +41,25 @@
    * Test HTableDescriptor.toString();
    */
   public void testHTableDescriptor() {
-    HTableDescriptor htd = HTableDescriptor.rootTableDesc;
+    HTableDescriptor htd = HTableDescriptor.ROOT_TABLEDESC;
     System. out.println(htd.toString());
-    assertEquals("Table descriptor", "name: -ROOT-, families: {info:={name: " +
+    assertEquals("Table descriptor", "name: -ROOT-, families: [{name: " +
       "info, max versions: 1, compression: NONE, in memory: false, " +
       "block cache enabled: false, max length: 2147483647, " +
-      "time to live: FOREVER, bloom filter: none}}", htd.toString());
+      "time to live: FOREVER, bloom filter: none}]", htd.toString());
   }
   
   /**
    * Tests HRegionInfo.toString()
    */
   public void testHRegionInfo() {
-    HRegionInfo hri = HRegionInfo.rootRegionInfo;
+    HRegionInfo hri = HRegionInfo.ROOT_REGIONINFO;
     System.out.println(hri.toString());
     assertEquals("HRegionInfo", 
       "regionname: -ROOT-,,0, startKey: <>, endKey: <>, encodedName: 70236052, tableDesc: " +
-      "{name: -ROOT-, families: {info:={name: info, max versions: 1, " +
+      "{name: -ROOT-, families: [{name: info, max versions: 1, " +
       "compression: NONE, in memory: false, block cache enabled: false, " +
-      "max length: 2147483647, time to live: FOREVER, bloom filter: none}}}",
+      "max length: 2147483647, time to live: FOREVER, bloom filter: none}]}",
       hri.toString());
   }
 }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TimestampTestBase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TimestampTestBase.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TimestampTestBase.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/TimestampTestBase.java Thu May 15 15:10:47 2008
@@ -21,22 +21,11 @@
 import java.io.IOException;
 import java.util.TreeMap;
 
-import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
-import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.Scanner;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HStoreKey;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.io.Cell;
-import org.apache.hadoop.hbase.io.BatchUpdate;
-import org.apache.hadoop.hbase.HBaseTestCase;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Tests user specifiable time stamps putting, getting and scanning.  Also
@@ -53,12 +42,9 @@
   
   private static final String COLUMN_NAME = "contents:";
   
-  private static final Text COLUMN = new Text(COLUMN_NAME);
-  private static final Text ROW = new Text("row");
-  
-  // When creating column descriptor, how many versions of a cell to allow.
-  private static final int VERSIONS = 3;
-  
+  private static final byte [] COLUMN = Bytes.toBytes(COLUMN_NAME);
+  private static final byte [] ROW = Bytes.toBytes("row");
+
     /*
    * Run test that delete works according to description in <a
    * href="https://issues.apache.org/jira/browse/HADOOP-1784">hadoop-1784</a>.
@@ -115,7 +101,7 @@
   throws IOException {
     Cell[] cellValues = incommon.get(ROW, COLUMN, 3/*Ask for too much*/);
     assertEquals(1, cellValues.length);
-    long time = Writables.bytesToLong(cellValues[0].getValue());
+    long time = Bytes.toLong(cellValues[0].getValue());
     assertEquals(time, currentTime);
     assertNull(incommon.get(ROW, COLUMN, T1, 3 /*Too many*/));
     assertTrue(assertScanContentTimestamp(incommon, T1) == 0);
@@ -133,19 +119,19 @@
   throws IOException {
     // Assert that 'latest' is what we expect.
     byte [] bytes = incommon.get(ROW, COLUMN).getValue();
-    assertEquals(Writables.bytesToLong(bytes), tss[0]);
+    assertEquals(Bytes.toLong(bytes), tss[0]);
     // Now assert that if we ask for multiple versions, that they come out in
     // order.
     Cell[] cellValues = incommon.get(ROW, COLUMN, tss.length);
     assertEquals(tss.length, cellValues.length);
     for (int i = 0; i < cellValues.length; i++) {
-      long ts = Writables.bytesToLong(cellValues[i].getValue());
+      long ts = Bytes.toLong(cellValues[i].getValue());
       assertEquals(ts, tss[i]);
     }
     // Specify a timestamp get multiple versions.
     cellValues = incommon.get(ROW, COLUMN, tss[0], cellValues.length - 1);
     for (int i = 1; i < cellValues.length; i++) {
-      long ts = Writables.bytesToLong(cellValues[i].getValue());
+      long ts = Bytes.toLong(cellValues[i].getValue());
       assertEquals(ts, tss[i]);
     }
     // Test scanner returns expected version
@@ -191,12 +177,13 @@
     int count = 0;
     try {
       HStoreKey key = new HStoreKey();
-      TreeMap<Text, byte []>value = new TreeMap<Text, byte[]>();
+      TreeMap<byte [], byte []>value =
+        new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
       while (scanner.next(key, value)) {
         assertTrue(key.getTimestamp() <= ts);
         // Content matches the key or HConstants.LATEST_TIMESTAMP.
         // (Key does not match content if we 'put' with LATEST_TIMESTAMP).
-        long l = Writables.bytesToLong(value.get(COLUMN));
+        long l = Bytes.toLong(value.get(COLUMN));
         assertTrue(key.getTimestamp() == l ||
           HConstants.LATEST_TIMESTAMP == l);
         count++;
@@ -210,13 +197,13 @@
   
   public static void put(final Incommon loader, final long ts)
   throws IOException {
-    put(loader, Writables.longToBytes(ts), ts);
+    put(loader, Bytes.toBytes(ts), ts);
   }
   
   public static void put(final Incommon loader)
   throws IOException {
     long ts = HConstants.LATEST_TIMESTAMP;
-    put(loader, Writables.longToBytes(ts), ts);
+    put(loader, Bytes.toBytes(ts), ts);
   }
   
   /*

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestBatchUpdate.java Thu May 15 15:10:47 2008
@@ -30,14 +30,14 @@
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.io.RowResult;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Test batch updates
  */
 public class TestBatchUpdate extends HBaseClusterTestCase {
   private static final String CONTENTS_STR = "contents:";
-  private static final Text CONTENTS = new Text(CONTENTS_STR);
+  private static final byte [] CONTENTS = Bytes.toBytes(CONTENTS_STR);
   private byte[] value;
 
   private HTableDescriptor desc = null;
@@ -68,19 +68,19 @@
    * @throws IOException
    */
   public void testBatchUpdate() throws IOException {
-    BatchUpdate bu = new BatchUpdate(new Text("row1"));
+    BatchUpdate bu = new BatchUpdate("row1");
     bu.put(CONTENTS, value);
     bu.delete(CONTENTS);
     table.commit(bu);
 
-    bu = new BatchUpdate(new Text("row2"));
+    bu = new BatchUpdate("row2");
     bu.put(CONTENTS, value);
     table.commit(bu);
 
-    Text[] columns = { CONTENTS };
-    Scanner scanner = table.getScanner(columns, new Text());
+    byte [][] columns = { CONTENTS };
+    Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW);
     for (RowResult r : scanner) {
-      for(Map.Entry<Text, Cell> e: r.entrySet()) {
+      for(Map.Entry<byte [], Cell> e: r.entrySet()) {
         System.out.println(r.getRow() + ": row: " + e.getKey() + " value: " + 
             new String(e.getValue().getValue(), HConstants.UTF8_ENCODING));
       }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestHTable.java Thu May 15 15:10:47 2008
@@ -21,21 +21,19 @@
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.hadoop.io.Text;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HBaseClusterTestCase;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.io.RowResult;
-import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.Text;
 
 /**
  * Tests HTable
@@ -43,13 +41,13 @@
 public class TestHTable extends HBaseClusterTestCase implements HConstants {
   private static final Log LOG = LogFactory.getLog(TestHTable.class);
   private static final HColumnDescriptor column =
-    new HColumnDescriptor(COLUMN_FAMILY.toString());
+    new HColumnDescriptor(COLUMN_FAMILY);
 
-  private static final Text nosuchTable = new Text("nosuchTable");
-  private static final Text tableAname = new Text("tableA");
-  private static final Text tableBname = new Text("tableB");
+  private static final byte [] nosuchTable = Bytes.toBytes("nosuchTable");
+  private static final byte [] tableAname = Bytes.toBytes("tableA");
+  private static final byte [] tableBname = Bytes.toBytes("tableB");
   
-  private static final Text row = new Text("row");
+  private static final byte [] row = Bytes.toBytes("row");
  
   /**
    * the test
@@ -69,10 +67,10 @@
       fail();
     }
     
-    HTableDescriptor tableAdesc = new HTableDescriptor(tableAname.toString());
+    HTableDescriptor tableAdesc = new HTableDescriptor(tableAname);
     tableAdesc.addFamily(column);
     
-    HTableDescriptor tableBdesc = new HTableDescriptor(tableBname.toString());
+    HTableDescriptor tableBdesc = new HTableDescriptor(tableBname);
     tableBdesc.addFamily(column);
 
     // create a couple of tables
@@ -106,7 +104,7 @@
     try {
       for (RowResult r : s) {
         batchUpdate = new BatchUpdate(r.getRow());
-        for(Map.Entry<Text, Cell> e: r.entrySet()) {
+        for(Map.Entry<byte [], Cell> e: r.entrySet()) {
           batchUpdate.put(e.getKey(), e.getValue().getValue());
         }
         b.commit(batchUpdate);
@@ -153,7 +151,7 @@
   public void testTableNotFoundExceptionWithATable() {
     try {
       HColumnDescriptor column =
-        new HColumnDescriptor(COLUMN_FAMILY.toString());
+        new HColumnDescriptor(COLUMN_FAMILY);
       HBaseAdmin admin = new HBaseAdmin(conf);
       HTableDescriptor testTableADesc =
         new HTableDescriptor("table");

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestListTables.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestListTables.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestListTables.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestListTables.java Thu May 15 15:10:47 2008
@@ -20,12 +20,12 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.HashSet;
+
 import org.apache.hadoop.hbase.HBaseClusterTestCase;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
 
 /**
  * Tests the listTables client API
@@ -33,7 +33,7 @@
 public class TestListTables extends HBaseClusterTestCase {
   HBaseAdmin admin = null;
   
-  private static final HTableDescriptor[] tables = {
+  private static final HTableDescriptor[] TABLES = {
       new HTableDescriptor("table1"),
       new HTableDescriptor("table2"),
       new HTableDescriptor("table3")
@@ -43,15 +43,12 @@
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    
     admin = new HBaseAdmin(conf);
-
     HColumnDescriptor family =
       new HColumnDescriptor(HConstants.COLUMN_FAMILY_STR);
-    
-    for (int i = 0; i < tables.length; i++) {
-      tables[i].addFamily(family);
-      admin.createTable(tables[i]);
+    for (int i = 0; i < TABLES.length; i++) {
+      TABLES[i].addFamily(family);
+      admin.createTable(TABLES[i]);
     }
   }
 
@@ -60,13 +57,15 @@
    * @throws IOException
    */
   public void testListTables() throws IOException {
-    HashSet<HTableDescriptor> result =
-      new HashSet<HTableDescriptor>(Arrays.asList(admin.listTables()));
-    
+    HTableDescriptor [] ts = admin.listTables();
+    HashSet<HTableDescriptor> result = new HashSet<HTableDescriptor>(ts.length);
+    for (int i = 0; i < ts.length; i++) {
+      result.add(ts[i]);
+    }
     int size = result.size();
-    assertEquals(tables.length, size);
-    for (int i = 0; i < tables.length && i < size; i++) {
-      assertTrue(result.contains(tables[i]));
+    assertEquals(TABLES.length, size);
+    for (int i = 0; i < TABLES.length && i < size; i++) {
+      assertTrue(result.contains(TABLES[i]));
     }
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestTimestamp.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestTimestamp.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestTimestamp.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestTimestamp.java Thu May 15 15:10:47 2008
@@ -19,24 +19,14 @@
 package org.apache.hadoop.hbase.client;
 
 import java.io.IOException;
-import java.util.TreeMap;
-
-import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
-import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.Scanner;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HStoreKey;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.io.Cell;
-import org.apache.hadoop.hbase.HBaseClusterTestCase;
-import org.apache.hadoop.hbase.TimestampTestBase;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HBaseClusterTestCase;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TimestampTestBase;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Tests user specifiable time stamps putting, getting and scanning.  Also
@@ -48,7 +38,7 @@
     LogFactory.getLog(TestTimestamp.class.getName());
   
   private static final String COLUMN_NAME = "contents:";
-  private static final Text COLUMN = new Text(COLUMN_NAME);
+  private static final byte [] COLUMN = Bytes.toBytes(COLUMN_NAME);
   // When creating column descriptor, how many versions of a cell to allow.
   private static final int VERSIONS = 3;
   
@@ -90,6 +80,6 @@
     desc.addFamily(new HColumnDescriptor(COLUMN_NAME));
     HBaseAdmin admin = new HBaseAdmin(conf);
     admin.createTable(desc);
-    return new HTable(conf, new Text(getName()));
+    return new HTable(conf, getName());
   }
 }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java Thu May 15 15:10:47 2008
@@ -24,7 +24,7 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.util.Bytes;
 
 import junit.framework.TestCase;
 
@@ -32,9 +32,9 @@
  * Tests the inclusive stop row filter
  */
 public class TestInclusiveStopRowFilter extends TestCase {
-  private final Text STOP_ROW = new Text("stop_row");
-  private final Text GOOD_ROW = new Text("good_row");
-  private final Text PAST_STOP_ROW = new Text("zzzzzz");
+  private final byte [] STOP_ROW = Bytes.toBytes("stop_row");
+  private final byte [] GOOD_ROW = Bytes.toBytes("good_row");
+  private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz");
   
   RowFilterInterface mainFilter;
 

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java Thu May 15 15:10:47 2008
@@ -24,7 +24,8 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.util.Bytes;
+
 
 import junit.framework.TestCase;
 
@@ -82,7 +83,7 @@
   private void testFiltersBeyondPageSize(final RowFilterInterface filter,
     final int pageSize) {
     for (int i = 0; i < (pageSize * 2); i++) {
-      Text row = new Text(Integer.toString(i));
+      byte [] row = Bytes.toBytes(Integer.toString(i));
       boolean filterOut = filter.filterRowKey(row);
       if (!filterOut) {
         assertFalse("Disagrees with 'filter'", filter.filterAllRemaining());

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java Thu May 15 15:10:47 2008
@@ -31,13 +31,13 @@
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.regionserver.HLogEdit;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Tests for regular expression row filter
  */
 public class TestRegExpRowFilter extends TestCase {
-  TreeMap<Text, byte []> colvalues;
+  TreeMap<byte [], byte []> colvalues;
   RowFilterInterface mainFilter;
   final char FIRST_CHAR = 'a';
   final char LAST_CHAR = 'e';
@@ -55,9 +55,9 @@
   @Override
   protected void setUp() throws Exception {
     super.setUp();
-    this.colvalues = new TreeMap<Text, byte[]>();
+    this.colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
     for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
-      colvalues.put(new Text(new String(new char [] {c})), GOOD_BYTES);
+      colvalues.put(Bytes.toBytes(new String(new char [] {c})), GOOD_BYTES);
     }
     this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues);
   }
@@ -113,28 +113,28 @@
  
   private void regexRowTests(RowFilterInterface filter) throws Exception {
     for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
-      Text t = createRow(c);
+      byte [] t = createRow(c);
       assertFalse("Failed with characer " + c, filter.filterRowKey(t));
     }
     String yahooSite = "com.yahoo.www";
     assertTrue("Failed with character " +
-      yahooSite, filter.filterRowKey(new Text(yahooSite)));
+      yahooSite, filter.filterRowKey(Bytes.toBytes(yahooSite)));
   }
   
   private void regexRowColumnTests(RowFilterInterface filter)
     throws UnsupportedEncodingException {
     
     for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
-      Text t = createRow(c);
-      for (Map.Entry<Text, byte []> e: this.colvalues.entrySet()) {
+      byte [] t = createRow(c);
+      for (Map.Entry<byte [], byte []> e: this.colvalues.entrySet()) {
         assertFalse("Failed on " + c,
           filter.filterColumn(t, e.getKey(), e.getValue()));
       }
     }
     // Try a row and column I know will pass.
     char c = 'c';
-    Text r = createRow(c);
-    Text col = new Text(Character.toString(c));
+    byte [] r = createRow(c);
+    byte [] col = Bytes.toBytes(Character.toString(c));
     assertFalse("Failed with character " + c,
       filter.filterColumn(r, col, GOOD_BYTES));
     
@@ -144,24 +144,24 @@
     
     // Do with good bytes but bad column name.  Should not filter out.
     assertFalse("Failed with character " + c,
-      filter.filterColumn(r, new Text("badcolumn"), GOOD_BYTES));
+      filter.filterColumn(r, Bytes.toBytes("badcolumn"), GOOD_BYTES));
     
     // Good column, good bytes but bad row.
     assertTrue("Failed with character " + c,
-      filter.filterColumn(new Text("bad row"), new Text("badcolumn"), GOOD_BYTES));
+      filter.filterColumn(Bytes.toBytes("bad row"),
+        Bytes.toBytes("badcolumn"), GOOD_BYTES));
   }
  
   private void filterNotNullTests(RowFilterInterface filter) throws Exception {
     // Modify the filter to expect certain columns to be null:
     // Expecting a row WITH columnKeys: a-d, WITHOUT columnKey: e
-    ((RegExpRowFilter)filter).setColumnFilter(new Text(new String(new char[] { 
-      LAST_CHAR })), null);
+    ((RegExpRowFilter)filter).setColumnFilter(new byte [] {LAST_CHAR}, null);
     
     char secondToLast = (char)(LAST_CHAR - 1);
     char thirdToLast = (char)(LAST_CHAR - 2);
     
     // Modify the row to be missing an expected columnKey (d)
-    colvalues.remove(new Text(new String(new char[] { secondToLast })));
+    colvalues.remove(new byte [] {(byte)secondToLast});
 
     // Try a row that is missing an expected columnKey.
     // Testing row with columnKeys: a-c
@@ -171,27 +171,26 @@
     // Try a row that has all expected columnKeys, and NO null-expected
     // columnKeys.
     // Testing row with columnKeys: a-d
-    colvalues.put(new Text(new String(new char[] { secondToLast })),
-      GOOD_BYTES);
+    colvalues.put(new byte [] {(byte)secondToLast}, GOOD_BYTES);
     assertFalse("Failed with last columnKey " + secondToLast, filter.
       filterRow(colvalues));
 
     // Try a row that has all expected columnKeys AND a null-expected columnKey.
     // Testing row with columnKeys: a-e
-    colvalues.put(new Text(new String(new char[] { LAST_CHAR })), GOOD_BYTES);
+    colvalues.put(new byte [] {LAST_CHAR}, GOOD_BYTES);
     assertTrue("Failed with last columnKey " + LAST_CHAR, filter.
       filterRow(colvalues));
     
     // Try a row that has all expected columnKeys and a null-expected columnKey 
     // that maps to a null value.
     // Testing row with columnKeys: a-e, e maps to null
-    colvalues.put(new Text(new String(new char[] { LAST_CHAR })), 
+    colvalues.put(new byte [] {LAST_CHAR}, 
       HLogEdit.deleteBytes.get());
     assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.", 
       filter.filterRow(colvalues));
   }
 
-  private Text createRow(final char c) {
-    return new Text(HOST_PREFIX + Character.toString(c));
+  private byte [] createRow(final char c) {
+    return Bytes.toBytes(HOST_PREFIX + Character.toString(c));
   }
 }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterAfterWrite.java Thu May 15 15:10:47 2008
@@ -24,7 +24,6 @@
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.TreeMap;
 
 import junit.framework.Assert;
 
@@ -33,15 +32,14 @@
 import org.apache.hadoop.hbase.HBaseClusterTestCase;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Scanner;
 import org.apache.hadoop.hbase.io.BatchUpdate;
-import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.io.Cell;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /** Test regexp filters HBASE-476 */
 public class TestRowFilterAfterWrite extends HBaseClusterTestCase {
@@ -52,11 +50,11 @@
   static final String TABLE_NAME = "TestTable";
   static final String FAMILY = "C:";
   static final String COLUMN1 = FAMILY + "col1";
-  static final Text TEXT_COLUMN1 = new Text(COLUMN1);
+  static final byte [] TEXT_COLUMN1 = Bytes.toBytes(COLUMN1);
   static final String COLUMN2 = FAMILY + "col2";
-  static final Text TEXT_COLUMN2 = new Text(COLUMN2);
+  static final byte [] TEXT_COLUMN2 = Bytes.toBytes(COLUMN2);
 
-  private static final Text[] columns = {
+  private static final byte [][] columns = {
     TEXT_COLUMN1, TEXT_COLUMN2
   };
 
@@ -120,11 +118,10 @@
     admin.createTable(desc);
 
     // insert some data into the test table
-    HTable table = new HTable(conf, new Text(TABLE_NAME));
+    HTable table = new HTable(conf, TABLE_NAME);
 
     for (int i = 0; i < NUM_ROWS; i++) {
-      BatchUpdate b =
-        new BatchUpdate(new Text("row_" + String.format("%1$05d", i)));
+      BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
 
       b.put(TEXT_COLUMN1, VALUE);
       b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
@@ -139,8 +136,7 @@
     // Do some identity write operations on one column of the data.
     for (int n = 0; n < NUM_REWRITES; n++) {
       for (int i = 0; i < NUM_ROWS; i++) {
-        BatchUpdate b =
-          new BatchUpdate(new Text("row_" + String.format("%1$05d", i)));
+        BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
 
         b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
         table.commit(b);
@@ -155,9 +151,7 @@
 
     // Do another round so to populate the mem cache
     for (int i = 0; i < NUM_ROWS; i++) {
-      BatchUpdate b =
-        new BatchUpdate(new Text("row_" + String.format("%1$05d", i)));
-
+      BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
       b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
       table.commit(b);
     }
@@ -169,7 +163,7 @@
   }
 
   private void scanTable(final String tableName, final boolean printValues) throws IOException {
-    HTable table = new HTable(conf, new Text(tableName));
+    HTable table = new HTable(conf, tableName);
 
     Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW);
     int numFound = doScan(scanner, printValues);
@@ -177,8 +171,8 @@
   }
 
   private void scanTableWithRowFilter(final String tableName, final boolean printValues) throws IOException {
-    HTable table = new HTable(conf, new Text(tableName));
-    Map<Text, byte[]> columnMap = new HashMap<Text, byte[]>();
+    HTable table = new HTable(conf, tableName);
+    Map<byte [], byte[]> columnMap = new HashMap<byte [], byte[]>();
     columnMap.put(TEXT_COLUMN1, VALUE);
     RegExpRowFilter filter = new RegExpRowFilter(null, columnMap);
     Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW, filter);
@@ -194,8 +188,7 @@
         for (RowResult result : scanner) {
           if (printValues) {
             LOG.info("row: " + result.getRow());
-
-            for (Map.Entry<Text, Cell> e : result.entrySet()) {
+            for (Map.Entry<byte [], Cell> e : result.entrySet()) {
               LOG.info(" column: " + e.getKey() + " value: "
                   + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING));
             }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterOnMultipleFamilies.java Thu May 15 15:10:47 2008
@@ -23,7 +23,6 @@
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.TreeMap;
 
 import junit.framework.Assert;
 
@@ -32,15 +31,14 @@
 import org.apache.hadoop.hbase.HBaseClusterTestCase;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Scanner;
 import org.apache.hadoop.hbase.io.BatchUpdate;
-import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.io.Cell;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Test for regexp filters (HBASE-527)
@@ -50,13 +48,11 @@
 
   static final String TABLE_NAME = "TestTable";
   static final String COLUMN1 = "A:col1";
-  static final Text TEXT_COLUMN1 = new Text(COLUMN1);
+  static final byte [] TEXT_COLUMN1 = Bytes.toBytes(COLUMN1);
   static final String COLUMN2 = "B:col2";
-  static final Text TEXT_COLUMN2 = new Text(COLUMN2);
+  static final byte [] TEXT_COLUMN2 = Bytes.toBytes(COLUMN2);
 
-  private static final Text[] columns = {
-    TEXT_COLUMN1, TEXT_COLUMN2
-  };
+  private static final byte [][] columns = {TEXT_COLUMN1, TEXT_COLUMN2};
 
   private static final int NUM_ROWS = 10;
   private static final byte[] VALUE = "HELLO".getBytes();
@@ -72,11 +68,10 @@
     admin.createTable(desc);
 
     // insert some data into the test table
-    HTable table = new HTable(conf, new Text(TABLE_NAME));
+    HTable table = new HTable(conf, TABLE_NAME);
 
     for (int i = 0; i < NUM_ROWS; i++) {
-      BatchUpdate b =
-        new BatchUpdate(new Text("row_" + String.format("%1$05d", i)));
+      BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
       b.put(TEXT_COLUMN1, VALUE);
       b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
       table.commit(b);
@@ -89,16 +84,15 @@
   }
 
   private void scanTable(final String tableName, final boolean printValues) throws IOException {
-    HTable table = new HTable(conf, new Text(tableName));
-
+    HTable table = new HTable(conf, tableName);
     Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW);
     int numFound = doScan(scanner, printValues);
     Assert.assertEquals(NUM_ROWS, numFound);
   }
 
   private void scanTableWithRowFilter(final String tableName, final boolean printValues) throws IOException {
-    HTable table = new HTable(conf, new Text(tableName));
-    Map<Text, byte[]> columnMap = new HashMap<Text, byte[]>();
+    HTable table = new HTable(conf, tableName);
+    Map<byte [], byte[]> columnMap = new HashMap<byte [], byte[]>();
     columnMap.put(TEXT_COLUMN1, VALUE);
     RegExpRowFilter filter = new RegExpRowFilter(null, columnMap);
     Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW, filter);
@@ -115,7 +109,7 @@
           if (printValues) {
             LOG.info("row: " + result.getRow());
 
-            for (Map.Entry<Text, Cell> e : result.entrySet()) {
+            for (Map.Entry<byte [], Cell> e : result.entrySet()) {
               LOG.info(" column: " + e.getKey() + " value: "
                   + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING));
             }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java Thu May 15 15:10:47 2008
@@ -29,7 +29,7 @@
 import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.util.Bytes;
 
 
 import junit.framework.TestCase;
@@ -44,7 +44,7 @@
   static final int MAX_PAGES = 5;
   final char FIRST_CHAR = 'a';
   final char LAST_CHAR = 'e';
-  TreeMap<Text, byte[]> colvalues;
+  TreeMap<byte [], byte[]> colvalues;
   static byte[] GOOD_BYTES = null;
   static byte[] BAD_BYTES = null;
 
@@ -62,15 +62,15 @@
   protected void setUp() throws Exception {
     super.setUp();
     
-    colvalues = new TreeMap<Text, byte[]>();
+    colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
     for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
-      colvalues.put(new Text(new String(new char [] {c})), GOOD_BYTES);
+      colvalues.put(new byte [] {(byte)c}, GOOD_BYTES);
     }
     
     Set<RowFilterInterface> filters = new HashSet<RowFilterInterface>();
     filters.add(new PageRowFilter(MAX_PAGES));
     filters.add(new RegExpRowFilter(".*regex.*", colvalues));
-    filters.add(new WhileMatchRowFilter(new StopRowFilter(new Text("yyy"))));
+    filters.add(new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("yyy"))));
     filters.add(new WhileMatchRowFilter(new RegExpRowFilter(".*match.*")));
     filterMPALL = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL, 
       filters);
@@ -132,7 +132,7 @@
     
     // Accept several more rows such that PageRowFilter will exceed its limit.
     for (int i=0; i<=MAX_PAGES-3; i++)
-      filter.rowProcessed(false, new Text("unimportant_key"));
+      filter.rowProcessed(false, Bytes.toBytes("unimportant_key"));
     
     // A row that should cause the RegExpRowFilter to filter this row, making 
     // all the filters return true and thus the RowFilterSet as well.
@@ -163,15 +163,15 @@
     RFSAssertReset(filter);
     
     // A row that should cause the RegExpRowFilter to fail.
-    boolean filtered = filter.filterColumn(new Text("regex_match"), 
-      new Text(new String(new char[] { FIRST_CHAR })), BAD_BYTES);
+    boolean filtered = filter.filterColumn(Bytes.toBytes("regex_match"), 
+      new byte [] { FIRST_CHAR }, BAD_BYTES);
     assertTrue("Filtering on 'regex_match' and bad column data.", filtered);
-    filterMPALL.rowProcessed(filtered, new Text("regex_match"));
+    filterMPALL.rowProcessed(filtered, Bytes.toBytes("regex_match"));
   }
   
   private void RFSAssertion(RowFilterInterface filter, String toTest, 
     boolean assertTrue) throws Exception {
-    Text testText = new Text(toTest);
+    byte [] testText = Bytes.toBytes(toTest);
     boolean filtered = filter.filterRowKey(testText);
     assertTrue("Filtering on '" + toTest + "'", 
       assertTrue? filtered : !filtered);

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java Thu May 15 15:10:47 2008
@@ -24,7 +24,7 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.util.Bytes;
 
 import junit.framework.TestCase;
 
@@ -32,9 +32,9 @@
  * Tests the stop row filter
  */
 public class TestStopRowFilter extends TestCase {
-  private final Text STOP_ROW = new Text("stop_row");
-  private final Text GOOD_ROW = new Text("good_row");
-  private final Text PAST_STOP_ROW = new Text("zzzzzz");
+  private final byte [] STOP_ROW = Bytes.toBytes("stop_row");
+  private final byte [] GOOD_ROW = Bytes.toBytes("good_row");
+  private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz");
   
   RowFilterInterface mainFilter;
 

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java Thu May 15 15:10:47 2008
@@ -24,9 +24,10 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 
+import org.apache.hadoop.hbase.util.Bytes;
+
 import junit.framework.TestCase;
 
-import org.apache.hadoop.io.Text;
 
 /**
  * Tests for the while-match filter
@@ -41,7 +42,7 @@
   protected void setUp() throws Exception {
     super.setUp();
     wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter(
-        new Text("s")));
+        Bytes.toBytes("s")));
     wmRegExpRowFilter = new WhileMatchRowFilter(new RegExpRowFilter(
     ".*regex.*"));
   }
@@ -90,14 +91,14 @@
     
     // Test cases that should pass the row
     toTest = "apples";
-    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(new Text(toTest)));
-    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(new Text(
+    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
         toTest)));
     
     // Test cases that should fail the row
     toTest = "tuna";
-    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(new Text(toTest)));
-    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(new Text(
+    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
         toTest)));
     
     // The difference in switch
@@ -121,14 +122,14 @@
     
     // Test cases that should pass the row
     toTest = "regex_match";
-    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(new Text(toTest)));
-    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(new Text(
+    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
         toTest)));
     
     // Test cases that should fail the row
     toTest = "not_a_match";
-    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(new Text(toTest)));
-    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(new Text(
+    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
         toTest)));
     
     // The difference in switch
@@ -143,7 +144,7 @@
     
     // Test filter(Text, Text, byte[]) for functionality only (no switch-cases)
     toTest = "asdf_regex_hjkl";
-    assertFalse("filter: '" + toTest + "'", filter.filterColumn(new Text(toTest), 
+    assertFalse("filter: '" + toTest + "'", filter.filterColumn(Bytes.toBytes(toTest), 
       null, null));
   }
 }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java Thu May 15 15:10:47 2008
@@ -29,9 +29,11 @@
 import junit.framework.TestCase;
 
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.filter.RowFilterInterface;
 import org.apache.hadoop.hbase.filter.StopRowFilter;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparator;
 
 public class TestHbaseObjectWritable extends TestCase {
 
@@ -49,6 +51,11 @@
     // Do primitive type
     final int COUNT = 101;
     assertTrue(doType(conf, COUNT, int.class).equals(COUNT));
+    // Do array
+    final byte [] testing = "testing".getBytes();
+    byte [] result = (byte [])doType(conf, testing, testing.getClass());
+    assertTrue(WritableComparator.compareBytes(testing, 0, testing.length,
+       result, 0, result.length) == 0);
     // Do unsupported type.
     boolean exception = false;
     try {
@@ -67,7 +74,7 @@
     obj = doType(conf, new Text(""), Text.class);
     assertTrue(obj instanceof Text);
     // Try type that should get transferred old fashion way.
-    obj = doType(conf, new StopRowFilter(new Text("")),
+    obj = doType(conf, new StopRowFilter(HConstants.EMPTY_BYTE_ARRAY),
         RowFilterInterface.class);
     assertTrue(obj instanceof StopRowFilter);
   }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableIndex.java Thu May 15 15:10:47 2008
@@ -36,13 +36,13 @@
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Scanner;
 import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MultiRegionTable;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
@@ -63,12 +63,12 @@
 
   static final String TABLE_NAME = "moretest";
   static final String INPUT_COLUMN = "contents:";
-  static final Text TEXT_INPUT_COLUMN = new Text(INPUT_COLUMN);
+  static final byte [] TEXT_INPUT_COLUMN = Bytes.toBytes(INPUT_COLUMN);
   static final String OUTPUT_COLUMN = "text:";
-  static final Text TEXT_OUTPUT_COLUMN = new Text(OUTPUT_COLUMN);
+  static final byte [] TEXT_OUTPUT_COLUMN = Bytes.toBytes(OUTPUT_COLUMN);
   static final String ROWKEY_NAME = "key";
   static final String INDEX_DIR = "testindex";
-  private static final Text[] columns = {
+  private static final byte [][] columns = new byte [][] {
     TEXT_INPUT_COLUMN,
     TEXT_OUTPUT_COLUMN
   };
@@ -177,7 +177,7 @@
 
   private void scanTable(boolean printResults)
   throws IOException {
-    HTable table = new HTable(conf, new Text(TABLE_NAME));
+    HTable table = new HTable(conf, TABLE_NAME);
     Scanner scanner = table.getScanner(columns,
         HConstants.EMPTY_START_ROW);
     try {
@@ -185,7 +185,7 @@
         if (printResults) {
           LOG.info("row: " + r.getRow());
         }
-        for (Map.Entry<Text, Cell> e : r.entrySet()) {
+        for (Map.Entry<byte [], Cell> e : r.entrySet()) {
           if (printResults) {
             LOG.info(" column: " + e.getKey() + " value: "
                 + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING));
@@ -201,7 +201,7 @@
     // Force a cache flush for every online region to ensure that when the
     // scanner takes its snapshot, all the updates have made it into the cache.
     for (HRegion r : cluster.getRegionThreads().get(0).getRegionServer().
-        getOnlineRegions().values()) {
+        getOnlineRegions()) {
       HRegionIncommon region = new HRegionIncommon(r);
       region.flushcache();
     }
@@ -228,7 +228,7 @@
         throw new IOException("no index directory found");
       }
 
-      HTable table = new HTable(conf, new Text(TABLE_NAME));
+      HTable table = new HTable(conf, TABLE_NAME);
       scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW);
 
       IndexConfiguration indexConf = new IndexConfiguration();
@@ -240,7 +240,7 @@
 
       int count = 0;
       for (RowResult r : scanner) {
-        String value = r.getRow().toString();
+        String value = Bytes.toString(r.getRow());
         Term term = new Term(rowkeyName, value);
         int hitCount = searcher.search(new TermQuery(term)).length();
         assertEquals("check row " + value, 1, hitCount);
@@ -263,4 +263,4 @@
   public static void main(@SuppressWarnings("unused") String[] args) {
     TestRunner.run(new TestSuite(TestTableIndex.class));
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java Thu May 15 15:10:47 2008
@@ -19,8 +19,8 @@
  */
 package org.apache.hadoop.hbase.mapred;
 
-import java.io.IOException;
 import java.io.File;
+import java.io.IOException;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
@@ -28,19 +28,20 @@
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Scanner;
-import org.apache.hadoop.hbase.io.RowResult;
-import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MultiRegionTable;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Scanner;
 import org.apache.hadoop.hbase.io.BatchUpdate;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.io.RowResult;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
-import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
 
 /**
  * Test Map/Reduce job over HBase tables. The map/reduce process we're testing
@@ -54,13 +55,11 @@
 
   static final String MULTI_REGION_TABLE_NAME = "mrtest";
   static final String INPUT_COLUMN = "contents:";
-  static final Text TEXT_INPUT_COLUMN = new Text(INPUT_COLUMN);
   static final String OUTPUT_COLUMN = "text:";
-  static final Text TEXT_OUTPUT_COLUMN = new Text(OUTPUT_COLUMN);
   
-  private static final Text[] columns = {
-    TEXT_INPUT_COLUMN,
-    TEXT_OUTPUT_COLUMN
+  private static final byte [][] columns = new byte [][] {
+    Bytes.toBytes(INPUT_COLUMN),
+    Bytes.toBytes(OUTPUT_COLUMN)
   };
 
   /** constructor */
@@ -74,22 +73,21 @@
   /**
    * Pass the given key and processed record reduce
    */
-  public static class ProcessContentsMapper extends TableMap<Text, BatchUpdate> {
+  public static class ProcessContentsMapper extends TableMap<ImmutableBytesWritable, BatchUpdate> {
     /**
      * Pass the key, and reversed value to reduce
      */
     @SuppressWarnings("unchecked")
     @Override
-    public void map(Text key, RowResult value,
-      OutputCollector<Text, BatchUpdate> output,
+    public void map(ImmutableBytesWritable key, RowResult value,
+      OutputCollector<ImmutableBytesWritable, BatchUpdate> output,
       @SuppressWarnings("unused") Reporter reporter) 
     throws IOException {
-      if(value.size() != 1) {
+      if (value.size() != 1) {
         throw new IOException("There should only be one input column");
       }
-
-      Text[] keys = value.keySet().toArray(new Text[value.size()]);
-      if(!keys[0].equals(TEXT_INPUT_COLUMN)) {
+      byte [][] keys = value.keySet().toArray(new byte [value.size()][]);
+      if(!Bytes.equals(keys[0], Bytes.toBytes(INPUT_COLUMN))) {
         throw new IOException("Wrong input column. Expected: " + INPUT_COLUMN
             + " but got: " + keys[0]);
       }
@@ -105,10 +103,8 @@
       
       // Now set the value to be collected
 
-      BatchUpdate outval = new BatchUpdate(key);
-      outval.put(TEXT_OUTPUT_COLUMN, 
-        newValue.toString().getBytes(HConstants.UTF8_ENCODING));
-      
+      BatchUpdate outval = new BatchUpdate(key.get());
+      outval.put(OUTPUT_COLUMN, Bytes.toBytes(newValue.toString()));
       output.collect(key, outval);
     }
   }
@@ -118,10 +114,9 @@
    * @throws IOException
    */
   public void testMultiRegionTable() throws IOException {
-    runTestOnTable(new HTable(conf, new Text(MULTI_REGION_TABLE_NAME)));
+    runTestOnTable(new HTable(conf, MULTI_REGION_TABLE_NAME));
   }
 
-
   private void runTestOnTable(HTable table) throws IOException {
     @SuppressWarnings("deprecation")
     MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1);
@@ -132,19 +127,18 @@
       jobConf = new JobConf(conf, TestTableMapReduce.class);
       jobConf.setJobName("process column contents");
       jobConf.setNumReduceTasks(1);
-      
-      TableMap.initJob(table.getTableName().toString(), INPUT_COLUMN, 
-          ProcessContentsMapper.class, Text.class, BatchUpdate.class, jobConf);
-
-      TableReduce.initJob(table.getTableName().toString(),
-          IdentityTableReduce.class, jobConf);
+      TableMap.initJob(Bytes.toString(table.getTableName()), INPUT_COLUMN, 
+        ProcessContentsMapper.class, ImmutableBytesWritable.class,
+        BatchUpdate.class, jobConf);
+      TableReduce.initJob(Bytes.toString(table.getTableName()),
+        IdentityTableReduce.class, jobConf);
             
       LOG.info("Started " + table.getTableName());
       JobClient.runJob(jobConf);
       LOG.info("After map/reduce completion");
 
       // verify map-reduce results
-      verify(table.getTableName().toString());
+      verify(Bytes.toString(table.getTableName()));
     } finally {
       mrCluster.shutdown();
       if (jobConf != null) {
@@ -155,7 +149,7 @@
 
   @SuppressWarnings("null")
   private void verify(String tableName) throws IOException {
-    HTable table = new HTable(conf, new Text(tableName));
+    HTable table = new HTable(conf, tableName);
     boolean verified = false;
     long pause = conf.getLong("hbase.client.pause", 5 * 1000);
     int numRetries = conf.getInt("hbase.client.retries.number", 5);
@@ -200,7 +194,7 @@
         byte[] firstValue = null;
         byte[] secondValue = null;
         int count = 0;
-        for(Map.Entry<Text, Cell> e: r.entrySet()) {
+        for(Map.Entry<byte [], Cell> e: r.entrySet()) {
           if (count == 0) {
             firstValue = e.getValue().getValue();
           }
@@ -215,14 +209,14 @@
         
         String first = "";
         if (firstValue == null) {
-          throw new NullPointerException(r.getRow().toString() +
+          throw new NullPointerException(Bytes.toString(r.getRow()) +
             ": first value is null");
         }
         first = new String(firstValue, HConstants.UTF8_ENCODING);
         
         String second = "";
         if (secondValue == null) {
-          throw new NullPointerException(r.getRow().toString() +
+          throw new NullPointerException(Bytes.toString(r.getRow()) +
             ": second value is null");
         }
         byte[] secondReversed = new byte[secondValue.length];

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/OOMERegionServer.java Thu May 15 15:10:47 2008
@@ -26,7 +26,6 @@
 import org.apache.hadoop.hbase.io.BatchUpdate;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HServerAddress;
-import org.apache.hadoop.io.Text;
 
 /**
  * A region server that will OOME.
@@ -47,7 +46,7 @@
     super(address, conf);
   }
   
-  public void batchUpdate(Text regionName, BatchUpdate b)
+  public void batchUpdate(byte [] regionName, BatchUpdate b)
   throws IOException {
     super.batchUpdate(regionName, b);
     for (int i = 0; i < 30; i++) {

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java Thu May 15 15:10:47 2008
@@ -25,15 +25,15 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.io.MapFile;
-import org.apache.hadoop.io.Text;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Test compactions
@@ -41,11 +41,11 @@
 public class TestCompaction extends HBaseTestCase {
   static final Log LOG = LogFactory.getLog(TestCompaction.class.getName());
   private HRegion r = null;
-  private static final String COLUMN_FAMILY = COLFAMILY_NAME1;
-  private final Text STARTROW;
-  private static final Text COLUMN_FAMILY_TEXT = new Text(COLUMN_FAMILY);
-  private static final Text COLUMN_FAMILY_TEXT_MINUS_COLON =
-    new Text(COLUMN_FAMILY.substring(0, COLUMN_FAMILY.length() - 1));
+  private static final byte [] COLUMN_FAMILY = COLFAMILY_NAME1;
+  private final byte [] STARTROW = Bytes.toBytes(START_KEY);
+  private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY;
+  private static final byte [] COLUMN_FAMILY_TEXT_MINUS_COLON =
+    Bytes.toBytes(Bytes.toString(COLUMN_FAMILY).substring(0, COLUMN_FAMILY.length - 1));
   private static final int COMPACTION_THRESHOLD = MAXVERSIONS;
 
   private MiniDFSCluster cluster;
@@ -53,7 +53,6 @@
   /** constructor */
   public TestCompaction() {
     super();
-    STARTROW = new Text(START_KEY);
     
     // Set cache flush size to 1MB
     conf.setInt("hbase.hregion.memcache.flush.size", 1024*1024);
@@ -99,7 +98,7 @@
     // Default is that there only 3 (MAXVERSIONS) versions allowed per column.
     // Assert > 3 and then after compaction, assert that only 3 versions
     // available.
-    addContent(new HRegionIncommon(r), COLUMN_FAMILY);
+    addContent(new HRegionIncommon(r), Bytes.toString(COLUMN_FAMILY));
     Cell[] cellValues = 
       r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/);
     // Assert that I can get > 5 versions (Should be at least 5 in there).
@@ -114,9 +113,8 @@
     byte [] secondRowBytes = START_KEY.getBytes(HConstants.UTF8_ENCODING);
     // Increment the least significant character so we get to next row.
     secondRowBytes[START_KEY_BYTES.length - 1]++;
-    Text secondRow = new Text(secondRowBytes);
-    cellValues = r.get(secondRow, COLUMN_FAMILY_TEXT, 100/*Too many*/);
-    LOG.info("Count of " + secondRow + ": " + cellValues.length);
+    cellValues = r.get(secondRowBytes, COLUMN_FAMILY_TEXT, 100/*Too many*/);
+    LOG.info("Count of " + Bytes.toString(secondRowBytes) + ": " + cellValues.length);
     // Commented out because fails on an hp+ubuntu single-processor w/ 1G and
     // "Intel(R) Pentium(R) 4 CPU 3.20GHz" though passes on all local
     // machines and even on hudson.  On said machine, its reporting in the
@@ -147,26 +145,26 @@
     cellValues = r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/);
     assertNull(cellValues);
     // Assert the store files do not have the first record 'aaa' keys in them.
-    for (MapFile.Reader reader:
-        this.r.stores.get(COLUMN_FAMILY_TEXT_MINUS_COLON).getReaders()) {
+    for (MapFile.Reader reader: this.r.stores.
+        get(Bytes.mapKey(COLUMN_FAMILY_TEXT_MINUS_COLON)).getReaders()) {
       reader.reset();
       HStoreKey key = new HStoreKey();
       ImmutableBytesWritable val = new ImmutableBytesWritable();
       while(reader.next(key, val)) {
-        assertFalse(key.getRow().equals(STARTROW));
+        assertFalse(Bytes.equals(key.getRow(), STARTROW));
       }
     }
   }
 
   private void createStoreFile(final HRegion region) throws IOException {
     HRegionIncommon loader = new HRegionIncommon(region);
-    addContent(loader, COLUMN_FAMILY);
+    addContent(loader, Bytes.toString(COLUMN_FAMILY));
     loader.flushcache();
   }
 
   private void createSmallerStoreFile(final HRegion region) throws IOException {
     HRegionIncommon loader = new HRegionIncommon(region); 
-    addContent(loader, COLUMN_FAMILY,
+    addContent(loader, Bytes.toString(COLUMN_FAMILY),
         ("bbb" + PUNCTUATION).getBytes(), null);
     loader.flushcache();
   }