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 [9/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/regionserver/TestDeleteAll.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteAll.java Thu May 15 15:10:47 2008
@@ -19,17 +19,14 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.io.IOException;
-import java.util.Map;
-
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.io.Text;
-import org.apache.commons.logging.*;
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Test the functionality of deleteAll.
@@ -85,15 +82,15 @@
     boolean flush)
   throws Exception{
     // insert a few versions worth of data for a row
-    Text row = new Text("test_row");
+    byte [] row = Bytes.toBytes("test_row");
     long t0 = System.currentTimeMillis();
     long t1 = t0 - 15000;
     long t2 = t1 - 15000;
 
-    Text colA = new Text(COLUMNS[0].toString() + "a");
-    Text colB = new Text(COLUMNS[0].toString() + "b");
-    Text colC = new Text(COLUMNS[0].toString() + "c");
-    Text colD = new Text(COLUMNS[0].toString());
+    byte [] colA = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a");
+    byte [] colB = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b");
+    byte [] colC = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c");
+    byte [] colD = Bytes.toBytes(Bytes.toString(COLUMNS[0]));
 
     BatchUpdate batchUpdate = new BatchUpdate(row, t0);
     batchUpdate.put(colA, cellData(0, flush).getBytes());

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteFamily.java Thu May 15 15:10:47 2008
@@ -19,19 +19,15 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import java.io.IOException;
-import java.util.Map;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.dfs.MiniDFSCluster;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.HBaseTestCase;
-
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.io.Cell;
 import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Test the functionality of deleteFamily.
@@ -82,14 +78,14 @@
     boolean flush)
   throws Exception{
     // insert a few versions worth of data for a row
-    Text row = new Text("test_row");
+    byte [] row = Bytes.toBytes("test_row");
     long t0 = System.currentTimeMillis();
     long t1 = t0 - 15000;
     long t2 = t1 - 15000;
 
-    Text colA = new Text(COLUMNS[0].toString() + "a");
-    Text colB = new Text(COLUMNS[0].toString() + "b");
-    Text colC = new Text(COLUMNS[1].toString() + "c");
+    byte [] colA = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a");
+    byte [] colB = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b");
+    byte [] colC = Bytes.toBytes(Bytes.toString(COLUMNS[1]) + "c");
 
     BatchUpdate batchUpdate = null;
     batchUpdate = new BatchUpdate(row, t0);

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet.java Thu May 15 15:10:47 2008
@@ -29,6 +29,7 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.HBaseTestCase;
 
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
 
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -43,9 +44,9 @@
 public class TestGet extends HBaseTestCase {
   private static final Log LOG = LogFactory.getLog(TestGet.class.getName());
   
-  private static final Text CONTENTS = new Text("contents:");
-  private static final Text ROW_KEY =
-    new Text(HRegionInfo.rootRegionInfo.getRegionName());
+  private static final byte [] CONTENTS = Bytes.toBytes("contents:");
+  private static final byte [] ROW_KEY =
+    HRegionInfo.ROOT_REGIONINFO.getRegionName();
   private static final String SERVER_ADDRESS = "foo.bar.com:1234";
 
 
@@ -61,12 +62,12 @@
     assertNull(value);
     
     // Find out what getFull returns
-    Map<Text, Cell> values = r.getFull(ROW_KEY);
+    Map<byte [], Cell> values = r.getFull(ROW_KEY);
     
     // assertEquals(4, values.keySet().size());
-    for(Iterator<Text> i = values.keySet().iterator(); i.hasNext(); ) {
-      Text column = i.next();
-      if (column.equals(HConstants.COL_SERVER)) {
+    for(Iterator<byte []> i = values.keySet().iterator(); i.hasNext(); ) {
+      byte [] column = i.next();
+      if (Bytes.equals(column, HConstants.COL_SERVER)) {
         String server = Writables.cellToString(values.get(column));
         assertEquals(expectedServer, server);
         LOG.info(server);
@@ -92,8 +93,8 @@
         cluster.getFileSystem().getHomeDirectory().toString());
       
       HTableDescriptor desc = new HTableDescriptor("test");
-      desc.addFamily(new HColumnDescriptor(CONTENTS.toString()));
-      desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
+      desc.addFamily(new HColumnDescriptor(CONTENTS));
+      desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY));
       
       region = createNewHRegion(desc, null, null);
       HRegionIncommon r = new HRegionIncommon(region);
@@ -102,17 +103,17 @@
 
       BatchUpdate batchUpdate = null;
       batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis());
-      batchUpdate.put(CONTENTS, Writables.getBytes(CONTENTS));
+      batchUpdate.put(CONTENTS, CONTENTS);
       batchUpdate.put(HConstants.COL_REGIONINFO, 
-          Writables.getBytes(HRegionInfo.rootRegionInfo));
+          Writables.getBytes(HRegionInfo.ROOT_REGIONINFO));
       r.commit(batchUpdate);
       
       batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis());
       batchUpdate.put(HConstants.COL_SERVER, 
-        Writables.stringToBytes(new HServerAddress(SERVER_ADDRESS).toString()));
-      batchUpdate.put(HConstants.COL_STARTCODE, Writables.longToBytes(12345));
-      batchUpdate.put(new Text(HConstants.COLUMN_FAMILY + "region"), 
-        "region".getBytes(HConstants.UTF8_ENCODING));
+        Bytes.toBytes(new HServerAddress(SERVER_ADDRESS).toString()));
+      batchUpdate.put(HConstants.COL_STARTCODE, Bytes.toBytes(12345));
+      batchUpdate.put(new Text(Bytes.toString(HConstants.COLUMN_FAMILY) +
+        "region"), Bytes.toBytes("region"));
       r.commit(batchUpdate);
       
       // Verify that get works the same from memcache as when reading from disk
@@ -133,12 +134,14 @@
       // Update one family member and add a new one
       
       batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis());
-      batchUpdate.put(new Text(HConstants.COLUMN_FAMILY + "region"),
+      batchUpdate.put(new Text(Bytes.toString(HConstants.COLUMN_FAMILY) +
+        "region"),
         "region2".getBytes(HConstants.UTF8_ENCODING));
       String otherServerName = "bar.foo.com:4321";
       batchUpdate.put(HConstants.COL_SERVER, 
-        Writables.stringToBytes(new HServerAddress(otherServerName).toString()));
-      batchUpdate.put(new Text(HConstants.COLUMN_FAMILY + "junk"),
+        Bytes.toBytes(new HServerAddress(otherServerName).toString()));
+      batchUpdate.put(new Text(Bytes.toString(HConstants.COLUMN_FAMILY) +
+        "junk"),
         "junk".getBytes(HConstants.UTF8_ENCODING));
       r.commit(batchUpdate);
 

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestGet2.java Thu May 15 15:10:47 2008
@@ -24,17 +24,18 @@
 import java.util.HashSet;
 import java.util.TreeMap;
 import java.util.Set;
+import java.util.TreeSet;
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.hbase.filter.StopRowFilter;
 import org.apache.hadoop.hbase.filter.WhileMatchRowFilter;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.HBaseTestCase;
 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.BatchUpdate;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * {@link TestGet} is a medley of tests of get all done up as a single test.
@@ -63,22 +64,23 @@
       HTableDescriptor htd = createTableDescriptor(getName());
       region = createNewHRegion(htd, null, null);
       for (int i = 0; i < COLUMNS.length; i++) {
-        addContent(region, COLUMNS[i].toString());
+        addContent(region, COLUMNS[i]);
       }
       // Find two rows to use doing getFull.
-      final Text arbitraryStartRow = new Text("b");
-      Text actualStartRow = null;
-      final Text arbitraryStopRow = new Text("c");
-      Text actualStopRow = null;
-      Text [] columns = new Text [] {new Text(COLFAMILY_NAME1)};
+      final byte [] arbitraryStartRow = Bytes.toBytes("b");
+      byte [] actualStartRow = null;
+      final byte [] arbitraryStopRow = Bytes.toBytes("c");
+      byte [] actualStopRow = null;
+      byte [][] columns = {COLFAMILY_NAME1};
       scanner = region.getScanner(columns,
           arbitraryStartRow, HConstants.LATEST_TIMESTAMP,
           new WhileMatchRowFilter(new StopRowFilter(arbitraryStopRow)));
       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)) { 
         if (actualStartRow == null) {
-          actualStartRow = new Text(key.getRow());
+          actualStartRow = key.getRow();
         } else {
           actualStopRow = key.getRow();
         }
@@ -120,7 +122,7 @@
       long right_now = System.currentTimeMillis();
       long one_second_ago = right_now - 1000;
       
-      Text t = new Text("test_row");
+      String t = "test_row";
       BatchUpdate batchUpdate = new BatchUpdate(t, one_second_ago);
       batchUpdate.put(COLUMNS[0], "old text".getBytes());
       region_incommon.commit(batchUpdate);
@@ -129,14 +131,16 @@
       batchUpdate.put(COLUMNS[0], "new text".getBytes());
       region_incommon.commit(batchUpdate);
 
-      assertCellEquals(region, t, COLUMNS[0], right_now, "new text");
-      assertCellEquals(region, t, COLUMNS[0], one_second_ago, "old text");
+      assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0],
+        right_now, "new text");
+      assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0],
+        one_second_ago, "old text");
       
       // Force a flush so store files come into play.
       region_incommon.flushcache();
 
-      assertCellEquals(region, t, COLUMNS[0], right_now, "new text");
-      assertCellEquals(region, t, COLUMNS[0], one_second_ago, "old text");
+      assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], right_now, "new text");
+      assertCellEquals(region, Bytes.toBytes(t), COLUMNS[0], one_second_ago, "old text");
 
     } finally {
       if (region != null) {
@@ -166,11 +170,11 @@
       region_incommon = new HRegionIncommon(region);
      
       // set up some test data
-      Text t10 = new Text("010");
-      Text t20 = new Text("020");
-      Text t30 = new Text("030");
-      Text t35 = new Text("035");
-      Text t40 = new Text("040");
+      String t10 = "010";
+      String t20 = "020";
+      String t30 = "030";
+      String t35 = "035";
+      String t40 = "040";
       
       batchUpdate = new BatchUpdate(t10);
       batchUpdate.put(COLUMNS[0], "t10 bytes".getBytes());
@@ -197,42 +201,42 @@
       region.batchUpdate(batchUpdate);
       
       // try finding "015"
-      Text t15 = new Text("015");
-      Map<Text, Cell> results = 
-        region.getClosestRowBefore(t15);
+      String t15 = "015";
+      Map<byte [], Cell> results = 
+        region.getClosestRowBefore(Bytes.toBytes(t15));
       assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t10 bytes");
 
       // try "020", we should get that row exactly
-      results = region.getClosestRowBefore(t20);
+      results = region.getClosestRowBefore(Bytes.toBytes(t20));
       assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t20 bytes");
       
       // try "038", should skip deleted "035" and get "030"
-      Text t38 = new Text("038");
-      results = region.getClosestRowBefore(t38);
+      String t38 = "038";
+      results = region.getClosestRowBefore(Bytes.toBytes(t38));
       assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
       
       // try "050", should get stuff from "040"
-      Text t50 = new Text("050");
-      results = region.getClosestRowBefore(t50);
+      String t50 = "050";
+      results = region.getClosestRowBefore(Bytes.toBytes(t50));
       assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t40 bytes");
 
       // force a flush
       region.flushcache();
 
       // try finding "015"
-      results = region.getClosestRowBefore(t15);
+      results = region.getClosestRowBefore(Bytes.toBytes(t15));
       assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t10 bytes");
 
       // try "020", we should get that row exactly
-      results = region.getClosestRowBefore(t20);
+      results = region.getClosestRowBefore(Bytes.toBytes(t20));
       assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t20 bytes");
 
       // try "038", should skip deleted "035" and get "030"
-      results = region.getClosestRowBefore(t38);
+      results = region.getClosestRowBefore(Bytes.toBytes(t38));
       assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t30 bytes");
 
       // try "050", should get stuff from "040"
-      results = region.getClosestRowBefore(t50);
+      results = region.getClosestRowBefore(Bytes.toBytes(t50));
       assertEquals(new String(results.get(COLUMNS[0]).getValue()), "t40 bytes");
     } finally {
       if (region != null) {
@@ -258,7 +262,7 @@
       region_incommon = new HRegionIncommon(region);
       
       // write a row with a bunch of columns
-      Text row = new Text("some_row");
+      byte [] row = Bytes.toBytes("some_row");
       BatchUpdate bu = new BatchUpdate(row);
       bu.put(COLUMNS[0], "column 0".getBytes());
       bu.put(COLUMNS[1], "column 1".getBytes());
@@ -283,11 +287,11 @@
     }    
   }
   
-  private void assertSpecifiedColumns(final HRegion region, final Text row) 
+  private void assertSpecifiedColumns(final HRegion region, final byte [] row) 
   throws IOException {
-    HashSet<Text> all = new HashSet<Text>();
-    HashSet<Text> one = new HashSet<Text>();
-    HashSet<Text> none = new HashSet<Text>();
+    TreeSet<byte []> all = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+    TreeSet<byte []> one = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+    TreeSet<byte []> none = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
     
     all.add(COLUMNS[0]);
     all.add(COLUMNS[1]);
@@ -295,7 +299,7 @@
     one.add(COLUMNS[0]);
 
     // make sure we get all of them with standard getFull
-    Map<Text, Cell> result = region.getFull(row, null, 
+    Map<byte [], Cell> result = region.getFull(row, null, 
       HConstants.LATEST_TIMESTAMP);
     assertEquals(new String(result.get(COLUMNS[0]).getValue()), "column 0");
     assertEquals(new String(result.get(COLUMNS[1]).getValue()), "column 1");
@@ -324,7 +328,7 @@
     HRegion region = null;
     HRegionIncommon region_incommon = null;
     BatchUpdate batchUpdate = null;
-    Map<Text, Cell> results = null;
+    Map<byte [], Cell> results = null;
     
     try {
       HTableDescriptor htd = createTableDescriptor(getName());
@@ -334,7 +338,7 @@
       //
       // Test ordering issue
       //
-      Text row = new Text("row1");
+      byte [] row = Bytes.toBytes("row1");
      
       // write some data
       batchUpdate = new BatchUpdate(row);
@@ -345,7 +349,7 @@
       region.flushcache();
       
       // assert that getFull gives us the older value
-      results = region.getFull(row, (Set<Text>)null, LATEST_TIMESTAMP);
+      results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP);
       assertEquals("olderValue", new String(results.get(COLUMNS[0]).getValue()));
       
       // write a new value for the cell
@@ -357,16 +361,16 @@
       region.flushcache();
       
       // assert that getFull gives us the later value
-      results = region.getFull(row, (Set<Text>)null, LATEST_TIMESTAMP);
+      results = region.getFull(row, (Set<byte []>)null, LATEST_TIMESTAMP);
       assertEquals("newerValue", new String(results.get(COLUMNS[0]).getValue()));
      
       //
       // Test the delete masking issue
       //
-      Text row2 = new Text("row2");
-      Text cell1 = new Text(COLUMNS[0].toString() + "a");
-      Text cell2 = new Text(COLUMNS[0].toString() + "b");
-      Text cell3 = new Text(COLUMNS[0].toString() + "c");
+      byte [] row2 = Bytes.toBytes("row2");
+      byte [] cell1 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "a");
+      byte [] cell2 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "b");
+      byte [] cell3 = Bytes.toBytes(Bytes.toString(COLUMNS[0]) + "c");
       
       // write some data at two columns
       batchUpdate = new BatchUpdate(row2);
@@ -378,7 +382,7 @@
       region.flushcache();
       
       // assert i get both columns
-      results = region.getFull(row2, (Set<Text>)null, LATEST_TIMESTAMP);
+      results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP);
       assertEquals("Should have two columns in the results map", 2, results.size());
       assertEquals("column0 value", new String(results.get(cell1).getValue()));
       assertEquals("column1 value", new String(results.get(cell2).getValue()));
@@ -393,7 +397,7 @@
       region.flushcache(); 
       
       // assert i get the second column only
-      results = region.getFull(row2, (Set<Text>)null, LATEST_TIMESTAMP);
+      results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP);
       assertEquals("Should have one column in the results map", 1, results.size());
       assertNull("column0 value", results.get(cell1));
       assertEquals("column1 new value", new String(results.get(cell2).getValue()));
@@ -407,7 +411,7 @@
       region.batchUpdate(batchUpdate);
       
       // assert i get the third column only
-      results = region.getFull(row2, (Set<Text>)null, LATEST_TIMESTAMP);
+      results = region.getFull(row2, (Set<byte []>)null, LATEST_TIMESTAMP);
       assertEquals("Should have one column in the results map", 1, results.size());
       assertNull("column0 value", results.get(cell1));
       assertNull("column1 value", results.get(cell2));
@@ -425,18 +429,18 @@
     }  
   }
   
-  private void assertColumnsPresent(final HRegion r, final Text row)
+  private void assertColumnsPresent(final HRegion r, final byte [] row)
   throws IOException {
-    Map<Text, Cell> result = r.getFull(row, null, HConstants.LATEST_TIMESTAMP);
+    Map<byte [], Cell> result = r.getFull(row, null, HConstants.LATEST_TIMESTAMP);
     int columnCount = 0;
-    for (Map.Entry<Text, Cell> e: result.entrySet()) {
+    for (Map.Entry<byte [], Cell> e: result.entrySet()) {
       columnCount++;
-      String column = e.getKey().toString();
+      byte [] column = e.getKey();
       boolean legitColumn = false;
       for (int i = 0; i < COLUMNS.length; i++) {
         // Assert value is same as row.  This is 'nature' of the data added.
-        assertTrue(row.equals(new Text(e.getValue().getValue())));
-        if (COLUMNS[i].equals(new Text(column))) {
+        assertTrue(Bytes.equals(row, e.getValue().getValue()));
+        if (Bytes.equals(COLUMNS[i], column)) {
           legitColumn = true;
           break;
         }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java Thu May 15 15:10:47 2008
@@ -25,12 +25,12 @@
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.SequenceFile.Reader;
 
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /** JUnit test case for HLog */
 public class TestHLog extends HBaseTestCase implements HConstants {
@@ -67,8 +67,8 @@
    * @throws IOException
    */
   public void testSplit() throws IOException {
-    final Text tableName = new Text(getName());
-    final Text rowName = tableName;
+    final byte [] tableName = Bytes.toBytes(getName());
+    final byte [] rowName = tableName;
     HLog log = new HLog(this.fs, this.dir, this.conf, null);
     // Add edits for three regions.
     try {
@@ -76,11 +76,10 @@
         for (int i = 0; i < 3; i++) {
           for (int j = 0; j < 3; j++) {
             TreeMap<HStoreKey, byte[]> edit = new TreeMap<HStoreKey, byte[]>();
-            Text column = new Text(Integer.toString(j));
-            edit.put(
-                new HStoreKey(rowName, column, System.currentTimeMillis()),
-                column.getBytes());
-            log.append(new Text(Integer.toString(i)), tableName, edit);
+            byte [] column = Bytes.toBytes(Integer.toString(j));
+            edit.put(new HStoreKey(rowName, column, System.currentTimeMillis()),
+              column);
+            log.append(Bytes.toBytes(Integer.toString(i)), tableName, edit);
           }
         }
         log.rollWriter();
@@ -99,9 +98,9 @@
    */
   public void testAppend() throws IOException {
     final int COL_COUNT = 10;
-    final Text regionName = new Text("regionname");
-    final Text tableName = new Text("tablename");
-    final Text row = new Text("row");
+    final byte [] regionName = Bytes.toBytes("regionname");
+    final byte [] tableName = Bytes.toBytes("tablename");
+    final byte [] row = Bytes.toBytes("row");
     Reader reader = null;
     HLog log = new HLog(fs, dir, this.conf, null);
     try {
@@ -110,7 +109,7 @@
       long timestamp = System.currentTimeMillis();
       TreeMap<HStoreKey, byte []> cols = new TreeMap<HStoreKey, byte []>();
       for (int i = 0; i < COL_COUNT; i++) {
-        cols.put(new HStoreKey(row, new Text(Integer.toString(i)), timestamp),
+        cols.put(new HStoreKey(row, Bytes.toBytes(Integer.toString(i)), timestamp),
             new byte[] { (byte)(i + '0') });
       }
       log.append(regionName, tableName, cols);
@@ -125,18 +124,18 @@
       HLogEdit val = new HLogEdit();
       for (int i = 0; i < COL_COUNT; i++) {
         reader.next(key, val);
-        assertEquals(regionName, key.getRegionName());
-        assertEquals(tableName, key.getTablename());
-        assertEquals(row, key.getRow());
+        assertTrue(Bytes.equals(regionName, key.getRegionName()));
+        assertTrue(Bytes.equals(tableName, key.getTablename()));
+        assertTrue(Bytes.equals(row, key.getRow()));
         assertEquals((byte)(i + '0'), val.getVal()[0]);
         System.out.println(key + " " + val);
       }
       while (reader.next(key, val)) {
         // Assert only one more row... the meta flushed row.
-        assertEquals(regionName, key.getRegionName());
-        assertEquals(tableName, key.getTablename());
-        assertEquals(HLog.METAROW, key.getRow());
-        assertEquals(HLog.METACOLUMN, val.getColumn());
+        assertTrue(Bytes.equals(regionName, key.getRegionName()));
+        assertTrue(Bytes.equals(tableName, key.getTablename()));
+        assertTrue(Bytes.equals(HLog.METAROW, key.getRow()));
+        assertTrue(Bytes.equals(HLog.METACOLUMN, val.getColumn()));
         assertEquals(0, HLogEdit.completeCacheFlush.compareTo(val.getVal()));
         System.out.println(key + " " + val);
       }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHMemcache.java Thu May 15 15:10:47 2008
@@ -29,11 +29,10 @@
 
 import junit.framework.TestCase;
 
-import org.apache.hadoop.io.Text;
-
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /** memcache test case */
 public class TestHMemcache extends TestCase {
@@ -48,7 +47,7 @@
 
   private static final int FIRST_ROW = 1;
   private static final int NUM_VALS = 1000;
-  private static final Text CONTENTS_BASIC = new Text("contents:basic");
+  private static final byte [] CONTENTS_BASIC = Bytes.toBytes("contents:basic");
   private static final String CONTENTSTR = "contentstr";
   private static final String ANCHORNUM = "anchor:anchornum-";
   private static final String ANCHORSTR = "anchorstr";
@@ -65,13 +64,13 @@
    */
   public void testMemcache() throws UnsupportedEncodingException {
     for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
-      Text row = new Text("row_" + k);
+      byte [] row = Bytes.toBytes("row_" + k);
       HStoreKey key =
         new HStoreKey(row, CONTENTS_BASIC, System.currentTimeMillis());
       hmemcache.add(key, (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING));
       
       key =
-        new HStoreKey(row, new Text(ANCHORNUM + k), System.currentTimeMillis());
+        new HStoreKey(row, Bytes.toBytes(ANCHORNUM + k), System.currentTimeMillis());
       hmemcache.add(key, (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING));
     }
 
@@ -79,7 +78,7 @@
 
     for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
       List<Cell> results;
-      Text row = new Text("row_" + k);
+      byte [] row = Bytes.toBytes("row_" + k);
       HStoreKey key = new HStoreKey(row, CONTENTS_BASIC, Long.MAX_VALUE);
       results = hmemcache.get(key, 1);
       assertNotNull("no data for " + key.toString(), results);
@@ -91,7 +90,7 @@
           "), expected: '" + teststr + "' got: '" +
           bodystr + "'", teststr.compareTo(bodystr) == 0);
       
-      key = new HStoreKey(row, new Text(ANCHORNUM + k), Long.MAX_VALUE);
+      key = new HStoreKey(row, Bytes.toBytes(ANCHORNUM + k), Long.MAX_VALUE);
       results = hmemcache.get(key, 1);
       assertNotNull("no data for " + key.toString(), results);
       assertEquals(1, results.size());
@@ -104,12 +103,12 @@
     }
   }
 
-  private Text getRowName(final int index) {
-    return new Text("row" + Integer.toString(index));
+  private byte [] getRowName(final int index) {
+    return Bytes.toBytes("row" + Integer.toString(index));
   }
 
-  private Text getColumnName(final int rowIndex, final int colIndex) {
-    return new Text(COLUMN_FAMILY + ":" + Integer.toString(rowIndex) + ";" +
+  private byte [] getColumnName(final int rowIndex, final int colIndex) {
+    return Bytes.toBytes(COLUMN_FAMILY + ":" + Integer.toString(rowIndex) + ";" +
         Integer.toString(colIndex));
   }
 
@@ -117,15 +116,12 @@
    * Adds {@link #ROW_COUNT} rows and {@link #COLUMNS_COUNT}
    * @param hmc Instance to add rows to.
    */
-  private void addRows(final Memcache hmc)
-    throws UnsupportedEncodingException {
-    
+  private void addRows(final Memcache hmc) {
     for (int i = 0; i < ROW_COUNT; i++) {
       long timestamp = System.currentTimeMillis();
       for (int ii = 0; ii < COLUMNS_COUNT; ii++) {
-        Text k = getColumnName(i, ii);
-        hmc.add(new HStoreKey(getRowName(i), k, timestamp),
-            k.toString().getBytes(HConstants.UTF8_ENCODING));
+        byte [] k = getColumnName(i, ii);
+        hmc.add(new HStoreKey(getRowName(i), k, timestamp), k);
       }
     }
   }
@@ -155,27 +151,26 @@
     }
   }
   
-  private void isExpectedRowWithoutTimestamps(final int rowIndex, TreeMap<Text, byte[]> row)
-    throws UnsupportedEncodingException {
+  private void isExpectedRowWithoutTimestamps(final int rowIndex, TreeMap<byte [], byte[]> row) {
     int i = 0;
-    for (Text colname: row.keySet()) {
-      String expectedColname = getColumnName(rowIndex, i++).toString();
-      String colnameStr = colname.toString();
+    for (byte [] colname: row.keySet()) {
+      String expectedColname = Bytes.toString(getColumnName(rowIndex, i++));
+      String colnameStr = Bytes.toString(colname);
       assertEquals("Column name", colnameStr, expectedColname);
       // Value is column name as bytes.  Usually result is
       // 100 bytes in size at least. This is the default size
       // for BytesWriteable.  For comparison, comvert bytes to
       // String and trim to remove trailing null bytes.
       byte [] value = row.get(colname);
-      String colvalueStr = new String(value, HConstants.UTF8_ENCODING).trim();
+      String colvalueStr = Bytes.toString(value).trim();
       assertEquals("Content", colnameStr, colvalueStr);
     }
   }
 
-  private void isExpectedRow(final int rowIndex, TreeMap<Text, Cell> row)
-  throws UnsupportedEncodingException {
-    TreeMap<Text, byte[]> converted = new TreeMap<Text, byte[]>();
-    for (Map.Entry<Text, Cell> entry : row.entrySet()) {
+  private void isExpectedRow(final int rowIndex, TreeMap<byte [], Cell> row) {
+    TreeMap<byte [], byte[]> converted =
+      new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
+    for (Map.Entry<byte [], Cell> entry : row.entrySet()) {
       converted.put(entry.getKey(), 
         entry.getValue() == null ? null : entry.getValue().getValue());
     }
@@ -183,49 +178,49 @@
   }
   
   /** Test getFull from memcache
-   * @throws UnsupportedEncodingException
    */
-  public void testGetFull() throws UnsupportedEncodingException {
+  public void testGetFull() {
     addRows(this.hmemcache);
     for (int i = 0; i < ROW_COUNT; i++) {
       HStoreKey hsk = new HStoreKey(getRowName(i));
-      TreeMap<Text, Cell> all = new TreeMap<Text, Cell>();
-      TreeMap<Text, Long> deletes = new TreeMap<Text, Long>();
+      TreeMap<byte [], Cell> all =
+        new TreeMap<byte [], Cell>(Bytes.BYTES_COMPARATOR);
+      TreeMap<byte [], Long> deletes =
+        new TreeMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
       this.hmemcache.getFull(hsk, null, deletes, all);
       isExpectedRow(i, all);
     }
   }
 
   /** Test getNextRow from memcache
-   * @throws UnsupportedEncodingException 
    */
-  public void testGetNextRow() throws UnsupportedEncodingException {
+  public void testGetNextRow() {
     addRows(this.hmemcache);
-    Text closestToEmpty = this.hmemcache.getNextRow(HConstants.EMPTY_TEXT);
-    assertEquals(closestToEmpty, getRowName(0));
+    byte [] closestToEmpty =
+      this.hmemcache.getNextRow(HConstants.EMPTY_BYTE_ARRAY);
+    assertTrue(Bytes.equals(closestToEmpty, getRowName(0)));
     for (int i = 0; i < ROW_COUNT; i++) {
-      Text nr = this.hmemcache.getNextRow(getRowName(i));
+      byte [] nr = this.hmemcache.getNextRow(getRowName(i));
       if (i + 1 == ROW_COUNT) {
         assertEquals(nr, null);
       } else {
-        assertEquals(nr, getRowName(i + 1));
+        assertTrue(Bytes.equals(nr, getRowName(i + 1)));
       }
     }
   }
 
   /** Test getClosest from memcache
-   * @throws UnsupportedEncodingException 
    */
-  public void testGetClosest() throws UnsupportedEncodingException {
+  public void testGetClosest() {
     addRows(this.hmemcache);
-    Text closestToEmpty = this.hmemcache.getNextRow(HConstants.EMPTY_TEXT);
-    assertEquals(closestToEmpty, getRowName(0));
+    byte [] closestToEmpty = this.hmemcache.getNextRow(HConstants.EMPTY_BYTE_ARRAY);
+    assertTrue(Bytes.equals(closestToEmpty, getRowName(0)));
     for (int i = 0; i < ROW_COUNT; i++) {
-      Text nr = this.hmemcache.getNextRow(getRowName(i));
+      byte [] nr = this.hmemcache.getNextRow(getRowName(i));
       if (i + 1 == ROW_COUNT) {
         assertEquals(nr, null);
       } else {
-        assertEquals(nr, getRowName(i + 1));
+        assertTrue(Bytes.equals(nr, getRowName(i + 1)));
       }
     }
   }
@@ -237,23 +232,25 @@
   public void testScanner() throws IOException {
     addRows(this.hmemcache);
     long timestamp = System.currentTimeMillis();
-    Text [] cols = new Text[COLUMNS_COUNT * ROW_COUNT];
+    byte [][] cols = new byte[COLUMNS_COUNT * ROW_COUNT][];
     for (int i = 0; i < ROW_COUNT; i++) {
       for (int ii = 0; ii < COLUMNS_COUNT; ii++) {
         cols[(ii + (i * COLUMNS_COUNT))] = getColumnName(i, ii);
       }
     }
     InternalScanner scanner =
-      this.hmemcache.getScanner(timestamp, cols, new Text());
+      this.hmemcache.getScanner(timestamp, cols, HConstants.EMPTY_START_ROW);
     HStoreKey key = new HStoreKey();
-    TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
+    TreeMap<byte [], byte []> results =
+      new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
     for (int i = 0; scanner.next(key, results); i++) {
       assertTrue("Row name",
-          key.toString().startsWith(getRowName(i).toString()));
+          key.toString().startsWith(Bytes.toString(getRowName(i))));
       assertEquals("Count of columns", COLUMNS_COUNT,
           results.size());
-      TreeMap<Text, byte []> row = new TreeMap<Text, byte []>();
-      for(Map.Entry<Text, byte []> e: results.entrySet() ) {
+      TreeMap<byte [], byte []> row =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
+      for(Map.Entry<byte [], byte []> e: results.entrySet() ) {
         row.put(e.getKey(), e.getValue());
       }
       isExpectedRowWithoutTimestamps(i, row);
@@ -265,11 +262,11 @@
   /** For HBASE-528 */
   public void testGetRowKeyAtOrBefore() {
     // set up some test data
-    Text t10 = new Text("010");
-    Text t20 = new Text("020");
-    Text t30 = new Text("030");
-    Text t35 = new Text("035");
-    Text t40 = new Text("040");
+    byte [] t10 = Bytes.toBytes("010");
+    byte [] t20 = Bytes.toBytes("020");
+    byte [] t30 = Bytes.toBytes("030");
+    byte [] t35 = Bytes.toBytes("035");
+    byte [] t40 = Bytes.toBytes("040");
     
     hmemcache.add(getHSKForRow(t10), "t10 bytes".getBytes());
     hmemcache.add(getHSKForRow(t20), "t20 bytes".getBytes());
@@ -282,7 +279,7 @@
     
     // try finding "015"
     results = new TreeMap<HStoreKey, Long>();
-    Text t15 = new Text("015");
+    byte [] t15 = Bytes.toBytes("015");
     hmemcache.getRowKeyAtOrBefore(t15, results);
     assertEquals(t10, results.lastKey().getRow());
     
@@ -293,18 +290,18 @@
   
     // try "038", should skip the deleted "035" and give "030"
     results = new TreeMap<HStoreKey, Long>();
-    Text t38 = new Text("038");
+    byte [] t38 = Bytes.toBytes("038");
     hmemcache.getRowKeyAtOrBefore(t38, results);
     assertEquals(t30, results.lastKey().getRow());
   
     // try "050", should get stuff from "040"
     results = new TreeMap<HStoreKey, Long>();
-    Text t50 = new Text("050");
+    byte [] t50 = Bytes.toBytes("050");
     hmemcache.getRowKeyAtOrBefore(t50, results);
     assertEquals(t40, results.lastKey().getRow());
   }
   
-  private HStoreKey getHSKForRow(Text row) {
-    return new HStoreKey(row, new Text("test_col:"), HConstants.LATEST_TIMESTAMP);
+  private HStoreKey getHSKForRow(byte [] row) {
+    return new HStoreKey(row, Bytes.toBytes("test_col:"), HConstants.LATEST_TIMESTAMP);
   }
 }
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java Thu May 15 15:10:47 2008
@@ -27,7 +27,6 @@
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.log4j.Logger;
@@ -35,6 +34,7 @@
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Basic stand-alone testing of HRegion.
@@ -72,13 +72,13 @@
   private static final int FIRST_ROW = 1;
   private static final int N_ROWS = 1000000;
   private static final int NUM_VALS = 1000;
-  private static final Text CONTENTS_BASIC = new Text("contents:basic");
+  private static final byte [] CONTENTS_BASIC = Bytes.toBytes("contents:basic");
   private static final String CONTENTSTR = "contentstr";
   private static final String ANCHORNUM = "anchor:anchornum-";
   private static final String ANCHORSTR = "anchorstr";
-  private static final Text CONTENTS_BODY = new Text("contents:body");
-  private static final Text CONTENTS_FIRSTCOL = new Text("contents:firstcol");
-  private static final Text ANCHOR_SECONDCOL = new Text("anchor:secondcol");
+  private static final byte [] CONTENTS_BODY = Bytes.toBytes("contents:body");
+  private static final byte [] CONTENTS_FIRSTCOL = Bytes.toBytes("contents:firstcol");
+  private static final byte [] ANCHOR_SECONDCOL = Bytes.toBytes("anchor:secondcol");
   
   private MiniDFSCluster cluster = null;
   private HLog log = null;
@@ -126,10 +126,10 @@
 
     for (int k = FIRST_ROW; k <= NUM_VALS; k++) {
       BatchUpdate batchUpdate = 
-        new BatchUpdate(new Text("row_" + k), System.currentTimeMillis());
+        new BatchUpdate(Bytes.toBytes("row_" + k), System.currentTimeMillis());
       batchUpdate.put(CONTENTS_BASIC,
           (CONTENTSTR + k).getBytes(HConstants.UTF8_ENCODING));
-      batchUpdate.put(new Text(ANCHORNUM + k),
+      batchUpdate.put(Bytes.toBytes(ANCHORNUM + k),
           (ANCHORSTR + k).getBytes(HConstants.UTF8_ENCODING));
       region.commit(batchUpdate);
     }
@@ -149,9 +149,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 = region.get(rowlabel, CONTENTS_BASIC).getValue();
       assertNotNull(bodydata);
@@ -160,7 +160,7 @@
       assertEquals("Incorrect value for key: (" + rowlabel + "," + CONTENTS_BASIC
           + "), expected: '" + teststr + "' got: '" + bodystr + "'",
           bodystr, teststr);
-      collabel = new Text(ANCHORNUM + k);
+      collabel = Bytes.toBytes(ANCHORNUM + k);
       bodydata = region.get(rowlabel, collabel).getValue();
       bodystr = new String(bodydata, HConstants.UTF8_ENCODING).trim();
       teststr = ANCHORSTR + k;
@@ -180,9 +180,9 @@
     boolean exceptionThrown = false;
     exceptionThrown = false;
     try {
-      BatchUpdate batchUpdate = new BatchUpdate(new Text("Some old key"));
+      BatchUpdate batchUpdate = new BatchUpdate(Bytes.toBytes("Some old key"));
       String unregisteredColName = "FamilyGroup:FamilyLabel";
-      batchUpdate.put(new Text(unregisteredColName),
+      batchUpdate.put(Bytes.toBytes(unregisteredColName),
         unregisteredColName.getBytes(HConstants.UTF8_ENCODING));
       region.commit(batchUpdate);
     } catch (IOException e) {
@@ -205,11 +205,11 @@
       threads.add(new Thread(Integer.toString(i)) {
         @Override
         public void run() {
-          long [] lockids = new long[lockCount];
+          Integer [] lockids = new Integer[lockCount];
           // Get locks.
           for (int i = 0; i < lockCount; i++) {
             try {
-              Text rowid = new Text(Integer.toString(i));
+              byte [] rowid = Bytes.toBytes(Integer.toString(i));
               lockids[i] = r.obtainRowLock(rowid);
               rowid.equals(r.getRowFromLock(lockids[i]));
               LOG.debug(getName() + " locked " + rowid.toString());
@@ -222,7 +222,7 @@
           
           // Abort outstanding locks.
           for (int i = lockCount - 1; i >= 0; i--) {
-            r.releaseRowLock(r.getRowFromLock(lockids[i]));
+            r.releaseRowLock(lockids[i]);
             LOG.debug(getName() + " unlocked " + i);
           }
           LOG.debug(getName() + " released " +
@@ -252,7 +252,7 @@
   // Test scanners. Writes contents:firstcol and anchor:secondcol
   
   private void scan() throws IOException {
-    Text cols[] = new Text[] {
+    byte [] cols[] = {
         CONTENTS_FIRSTCOL,
         ANCHOR_SECONDCOL
     };
@@ -271,7 +271,7 @@
       String kLabel = String.format("%1$03d", k);
 
       BatchUpdate batchUpdate = 
-        new BatchUpdate(new Text("row_vals1_" + kLabel), 
+        new BatchUpdate(Bytes.toBytes("row_vals1_" + kLabel), 
           System.currentTimeMillis());
       batchUpdate.put(cols[0], vals1[k].getBytes(HConstants.UTF8_ENCODING));
       batchUpdate.put(cols[1], vals1[k].getBytes(HConstants.UTF8_ENCODING));
@@ -287,20 +287,21 @@
     startTime = System.currentTimeMillis();
 
     InternalScanner s =
-      r.getScanner(cols, new Text(), System.currentTimeMillis(), null);
+      r.getScanner(cols, HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null);
     int numFetched = 0;
     try {
       HStoreKey curKey = new HStoreKey();
-      TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
+      TreeMap<byte [], byte []> curVals =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
       int k = 0;
       while(s.next(curKey, curVals)) {
-        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);
           int curval =
             Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim());
           for(int j = 0; j < cols.length; j++) {
-            if(col.compareTo(cols[j]) == 0) {
+            if (Bytes.compareTo(col, cols[j]) == 0) {
               assertEquals("Error at:" + curKey.getRow() + "/"
                   + curKey.getTimestamp()
                   + ", Value for " + col + " should be: " + k
@@ -334,20 +335,22 @@
     
     startTime = System.currentTimeMillis();
     
-    s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null);
+    s = r.getScanner(cols, HConstants.EMPTY_START_ROW,
+      System.currentTimeMillis(), null);
     numFetched = 0;
     try {
       HStoreKey curKey = new HStoreKey();
-      TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
+      TreeMap<byte [], byte []> curVals =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
       int k = 0;
       while(s.next(curKey, curVals)) {
-        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);
           int curval =
             Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim());
           for(int j = 0; j < cols.length; j++) {
-            if(col.compareTo(cols[j]) == 0) {
+            if (Bytes.compareTo(col, cols[j]) == 0) {
               assertEquals("Error at:" + curKey.getRow() + "/"
                   + curKey.getTimestamp()
                   + ", Value for " + col + " should be: " + k
@@ -376,7 +379,7 @@
       String kLabel = String.format("%1$03d", k);
       
       BatchUpdate batchUpdate = 
-        new BatchUpdate(new Text("row_vals1_" + kLabel), 
+        new BatchUpdate(Bytes.toBytes("row_vals1_" + kLabel), 
           System.currentTimeMillis());
       batchUpdate.put(cols[0], vals1[k].getBytes(HConstants.UTF8_ENCODING));
       batchUpdate.put(cols[1], vals1[k].getBytes(HConstants.UTF8_ENCODING));
@@ -391,20 +394,22 @@
     
     startTime = System.currentTimeMillis();
 
-    s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null);
+    s = r.getScanner(cols, HConstants.EMPTY_START_ROW,
+        System.currentTimeMillis(), null);
     numFetched = 0;
     try {
       HStoreKey curKey = new HStoreKey();
-      TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
+      TreeMap<byte [], byte []> curVals =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
       int k = 0;
       while(s.next(curKey, curVals)) {
-        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);
           int curval =
             Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim());
           for(int j = 0; j < cols.length; j++) {
-            if(col.compareTo(cols[j]) == 0) {
+            if(Bytes.compareTo(col, cols[j]) == 0) {
               assertEquals("Error at:" + curKey.getRow() + "/"
                   + curKey.getTimestamp()
                   + ", Value for " + col + " should be: " + k
@@ -438,20 +443,21 @@
     
     startTime = System.currentTimeMillis();
     
-    s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null);
+    s = r.getScanner(cols, HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null);
     numFetched = 0;
     try {
       HStoreKey curKey = new HStoreKey();
-      TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
+      TreeMap<byte [], byte []> curVals =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
       int k = 0;
       while(s.next(curKey, curVals)) {
-        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);
           int curval =
             Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim());
           for (int j = 0; j < cols.length; j++) {
-            if (col.compareTo(cols[j]) == 0) {
+            if (Bytes.compareTo(col, cols[j]) == 0) {
               assertEquals("Value for " + col + " should be: " + k
                   + ", but was fetched as: " + curval, curval, k);
               numFetched++;
@@ -474,22 +480,23 @@
 
     startTime = System.currentTimeMillis();
     
-    s = r.getScanner(cols, new Text("row_vals1_500"),
+    s = r.getScanner(cols, Bytes.toBytes("row_vals1_500"),
         System.currentTimeMillis(), null);
     
     numFetched = 0;
     try {
       HStoreKey curKey = new HStoreKey();
-      TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
+      TreeMap<byte [], byte []> curVals =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
       int k = 500;
       while(s.next(curKey, curVals)) {
-        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);
           int curval =
             Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim());
           for (int j = 0; j < cols.length; j++) {
-            if (col.compareTo(cols[j]) == 0) {
+            if (Bytes.compareTo(col, cols[j]) == 0) {
               assertEquals("Value for " + col + " should be: " + k
                   + ", but was fetched as: " + curval, curval, k);
               numFetched++;
@@ -534,7 +541,7 @@
 
       // Write to the HRegion
       BatchUpdate batchUpdate = 
-        new BatchUpdate(new Text("row_" + k), System.currentTimeMillis());
+        new BatchUpdate(Bytes.toBytes("row_" + k), System.currentTimeMillis());
       batchUpdate.put(CONTENTS_BODY,
           buf1.toString().getBytes(HConstants.UTF8_ENCODING));
       region.commit(batchUpdate);
@@ -581,7 +588,7 @@
   // NOTE: This test depends on testBatchWrite succeeding
   private void splitAndMerge() throws IOException {
     Path oldRegionPath = r.getRegionDir();
-    Text midKey = r.compactStores();
+    byte [] midKey = r.compactStores();
     assertNotNull(midKey);
     long startTime = System.currentTimeMillis();
     HRegion subregions[] = r.splitRegion(this, midKey);
@@ -615,14 +622,14 @@
   /**
    * {@inheritDoc}
    */
-  public void closing(@SuppressWarnings("unused") final Text regionName) {
+  public void closing(@SuppressWarnings("unused") final byte [] regionName) {
     // We don't use this here. It is only for the HRegionServer
   }
   
   /**
    * {@inheritDoc}
    */
-  public void closed(@SuppressWarnings("unused") final Text regionName) {
+  public void closed(@SuppressWarnings("unused") final byte [] regionName) {
     // We don't use this here. It is only for the HRegionServer
   }
   
@@ -632,39 +639,41 @@
 
     // First verify the data written by testBasic()
 
-    Text[] cols = new Text[] {
-        new Text(ANCHORNUM + "[0-9]+"),
-        new Text(CONTENTS_BASIC)
+    byte [][] cols = {
+        Bytes.toBytes(ANCHORNUM + "[0-9]+"),
+        CONTENTS_BASIC
     };
     
     long startTime = System.currentTimeMillis();
     
     InternalScanner s =
-      r.getScanner(cols, new Text(), System.currentTimeMillis(), null);
+      r.getScanner(cols, HConstants.EMPTY_START_ROW,
+          System.currentTimeMillis(), null);
 
     try {
 
       int contentsFetched = 0;
       int anchorFetched = 0;
       HStoreKey curKey = new HStoreKey();
-      TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
+      TreeMap<byte [], byte []> curVals =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
       int k = 0;
       while(s.next(curKey, curVals)) {
-        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);
-          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:" + curKey.getRow() + "/" + curKey.getTimestamp()
                 + ", 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:" + curKey.getRow() + "/" + curKey.getTimestamp()
-                + ", Value for " + col + " should start with: " + ANCHORSTR
+                + ", Value for " + Bytes.toString(col) +
+                " should start with: " + ANCHORSTR
                 + ", but was fetched as: " + curval,
                 curval.startsWith(ANCHORSTR));
             anchorFetched++;
@@ -689,28 +698,27 @@
     
     // Verify testScan data
     
-    cols = new Text[] {
-        CONTENTS_FIRSTCOL,
-        ANCHOR_SECONDCOL
-    };
+    cols = new byte [][] {CONTENTS_FIRSTCOL, ANCHOR_SECONDCOL};
     
     startTime = System.currentTimeMillis();
 
-    s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null);
+    s = r.getScanner(cols, HConstants.EMPTY_START_ROW,
+      System.currentTimeMillis(), null);
     try {
       int numFetched = 0;
       HStoreKey curKey = new HStoreKey();
-      TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
+      TreeMap<byte [], byte []> curVals =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
       int k = 0;
       while(s.next(curKey, curVals)) {
-        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);
           int curval =
             Integer.parseInt(new String(val, HConstants.UTF8_ENCODING).trim());
 
           for (int j = 0; j < cols.length; j++) {
-            if (col.compareTo(cols[j]) == 0) {
+            if (Bytes.compareTo(col, cols[j]) == 0) {
               assertEquals("Value for " + col + " should be: " + k
                   + ", but was fetched as: " + curval, curval, k);
               numFetched++;
@@ -730,56 +738,23 @@
       s.close();
     }
     
-    // Verify testBatchWrite data
-
-//    if(StaticTestEnvironment.debugging) {
-//      startTime = System.currentTimeMillis();
-//      s = r.getScanner(new Text[] { CONTENTS_BODY }, new Text(),
-//          System.currentTimeMillis(), null);
-//      
-//      try {
-//        int numFetched = 0;
-//        HStoreKey curKey = new HStoreKey();
-//        TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
-//        int k = 0;
-//        while(s.next(curKey, curVals)) {
-//          for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
-//            Text col = it.next();
-//            byte [] val = curVals.get(col);
-//            assertTrue(col.compareTo(CONTENTS_BODY) == 0);
-//            assertNotNull(val);
-//            numFetched++;
-//          }
-//          curVals.clear();
-//          k++;
-//        }
-//        assertEquals("Inserted " + N_ROWS + " values, but fetched " + numFetched, N_ROWS, numFetched);
-//
-//        LOG.info("Scanned " + N_ROWS
-//            + " rows from disk. Elapsed time: "
-//            + ((System.currentTimeMillis() - startTime) / 1000.0));
-//        
-//      } finally {
-//        s.close();
-//      }
-//    }
-    
     // Test a scanner which only specifies the column family name
     
-    cols = new Text[] {
-        new Text("anchor:")
+    cols = new byte [][] {
+        Bytes.toBytes("anchor:")
     };
     
     startTime = System.currentTimeMillis();
     
-    s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null);
+    s = r.getScanner(cols, HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null);
 
     try {
       int fetched = 0;
       HStoreKey curKey = new HStoreKey();
-      TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
+      TreeMap<byte [], byte []> curVals =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
       while(s.next(curKey, curVals)) {
-        for(Iterator<Text> it = curVals.keySet().iterator(); it.hasNext(); ) {
+        for(Iterator<byte []> it = curVals.keySet().iterator(); it.hasNext(); ) {
           it.next();
           fetched++;
         }

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java?rev=656868&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java Thu May 15 15:10:47 2008
@@ -0,0 +1,37 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.hbase.HBaseTestCase;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class TestHRegionInfo extends HBaseTestCase {
+  public void testCreateHRegionInfoName() throws Exception {
+    String tableName = "tablename";
+    final byte [] tn = Bytes.toBytes(tableName);
+    String startKey = "startkey";
+    final byte [] sk = Bytes.toBytes(startKey);
+    String id = "id";
+    byte [] name = HRegionInfo.createRegionName(tn, sk, id);
+    String nameStr = Bytes.toString(name);
+    assertEquals(nameStr, tableName + "," + startKey + "," + id);
+  }
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java Thu May 15 15:10:47 2008
@@ -26,9 +26,10 @@
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JenkinsHash;
 import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.hbase.HBaseTestCase;
 
@@ -84,9 +85,9 @@
     try {
       for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) {
         byte[] b = new byte[] {(byte)d};
-        Text t = new Text(new String(b, HConstants.UTF8_ENCODING));
+        byte [] t = Bytes.toBytes(new String(b, HConstants.UTF8_ENCODING));
         writer.append(new HStoreKey(t, t, System.currentTimeMillis()),
-            new ImmutableBytesWritable(t.getBytes()));
+            new ImmutableBytesWritable(t));
       }
     } finally {
       writer.close();
@@ -106,9 +107,9 @@
       for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) {
         for (char e = FIRST_CHAR; e <= LAST_CHAR; e++) {
           byte[] b = new byte[] { (byte) d, (byte) e };
-          Text t = new Text(new String(b, HConstants.UTF8_ENCODING));
+          byte [] t = Bytes.toBytes(new String(b, HConstants.UTF8_ENCODING));
           writer.append(new HStoreKey(t, t, System.currentTimeMillis()),
-            new ImmutableBytesWritable(t.getBytes()));
+            new ImmutableBytesWritable(t));
         }
       }
     } finally {
@@ -124,8 +125,9 @@
   public void testReference()
   throws IOException {
     // Make a store file and write data to it.
-    HStoreFile hsf = new HStoreFile(this.conf, this.fs, this.dir, getName(),
-        new Text("colfamily"), 1234567890L, null);
+    HStoreFile hsf = new HStoreFile(this.conf, this.fs, this.dir,
+      JenkinsHash.hash(Bytes.toBytes(getName())),
+      Bytes.toBytes("colfamily"), 1234567890L, null);
     MapFile.Writer writer =
       hsf.getWriter(this.fs, SequenceFile.CompressionType.NONE, null);
     writeStoreFile(writer);
@@ -136,14 +138,15 @@
     HStoreKey midkey = new HStoreKey(((HStoreKey)reader.midKey()).getRow());
     HStoreKey hsk = new HStoreKey();
     reader.finalKey(hsk);
-    Text finalKey = hsk.getRow();
+    byte [] finalKey = hsk.getRow();
     // Make a reference for the bottom half of the just written file.
     HStoreFile.Reference reference =
       new HStoreFile.Reference(hsf.getEncodedRegionName(), hsf.getFileId(),
           midkey, HStoreFile.Range.top);
     HStoreFile refHsf = new HStoreFile(this.conf, this.fs, 
-        new Path(DIR, getName()), getName() + "_reference", hsf.getColFamily(),
-        456, reference);
+        new Path(DIR, getName()),
+        JenkinsHash.hash(Bytes.toBytes(getName() + "_reference")),
+        hsf.getColFamily(), 456, reference);
     // Assert that reference files are written and that we can write and
     // read the info reference file at least.
     refHsf.writeReferenceFiles(this.fs);
@@ -165,11 +168,11 @@
     boolean first = true;
     while(halfReader.next(key, value)) {
       if (first) {
-        assertEquals(key.getRow().toString(), midkey.getRow().toString());
+        assertTrue(Bytes.equals(key.getRow(), midkey.getRow()));
         first = false;
       }
     }
-    assertEquals(key.getRow().toString(), finalKey.toString());
+    assertTrue(Bytes.equals(key.getRow(), finalKey));
   }
 
   /**
@@ -194,7 +197,7 @@
     // I know keys are a-z.  Let the midkey we want to use be 'd'.  See if
     // HalfMapFiles work even if size of file is < than default MapFile
     // interval.
-    checkHalfMapFile(p, new HStoreKey(new Text("d")));
+    checkHalfMapFile(p, new HStoreKey("d"));
   }
   
   private WritableComparable getMidkey(final Path p) throws IOException {
@@ -245,21 +248,20 @@
         assertTrue(key.compareTo(midkey) >= 0);
         if (first) {
           first = false;
-          assertEquals(((HStoreKey)midkey).getRow().toString(),
-            key.getRow().toString());
+          assertTrue(Bytes.equals(((HStoreKey)midkey).getRow(),
+            key.getRow()));
           LOG.info("First in top: " + key.toString());
         }
       }
       LOG.info("Last in top: " + key.toString());
       top.getClosest(midkey, value);
       // Assert value is same as key.
-      assertEquals(new String(value.get(), HConstants.UTF8_ENCODING),
-        ((HStoreKey) midkey).getRow().toString());
+      assertTrue(Bytes.equals(value.get(), ((HStoreKey) midkey).getRow()));
 
       // Next test using a midkey that does not exist in the file.
       // First, do a key that is < than first key. Ensure splits behave
       // properly.
-      WritableComparable badkey = new HStoreKey(new Text("   "));
+      WritableComparable badkey = new HStoreKey("   ");
       bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(),
           this.conf, HStoreFile.Range.bottom, badkey);
       // When badkey is < than the bottom, should return no values.
@@ -273,20 +275,20 @@
         if (first) {
           first = false;
           LOG.info("First top when key < bottom: " + key.toString());
-          String tmp = key.getRow().toString();
+          String tmp = Bytes.toString(key.getRow());
           for (int i = 0; i < tmp.length(); i++) {
             assertTrue(tmp.charAt(i) == 'a');
           }
         }
       }
       LOG.info("Last top when key < bottom: " + key.toString());
-      String tmp = key.getRow().toString();
+      String tmp = Bytes.toString(key.getRow());
       for (int i = 0; i < tmp.length(); i++) {
         assertTrue(tmp.charAt(i) == 'z');
       }
 
       // Test when badkey is > than last key in file ('||' > 'zz').
-      badkey = new HStoreKey(new Text("|||"));
+      badkey = new HStoreKey("|||");
       bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(),
           this.conf, HStoreFile.Range.bottom, badkey);
       first = true;
@@ -294,14 +296,14 @@
         if (first) {
           first = false;
           LOG.info("First bottom when key > top: " + key.toString());
-          tmp = key.getRow().toString();
+          tmp = Bytes.toString(key.getRow());
           for (int i = 0; i < tmp.length(); i++) {
             assertTrue(tmp.charAt(i) == 'a');
           }
         }
       }
       LOG.info("Last bottom when key > top: " + key.toString());
-      tmp = key.getRow().toString();
+      tmp = Bytes.toString(key.getRow());
       for (int i = 0; i < tmp.length(); i++) {
         assertTrue(tmp.charAt(i) == 'z');
       }
@@ -338,7 +340,7 @@
         // Test using a midkey that does not exist in the file.
         // First, do a key that is < than first key.  Ensure splits behave
         // properly.
-        HStoreKey midkey = new HStoreKey(new Text("   "));
+        HStoreKey midkey = new HStoreKey("   ");
         bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(),
           this.conf, HStoreFile.Range.bottom, midkey);
         // When midkey is < than the bottom, should return no values.
@@ -352,14 +354,14 @@
           if (first) {
             first = false;
             LOG.info("First top when key < bottom: " + key.toString());
-            assertEquals("aa", key.getRow().toString());
+            assertEquals("aa", Bytes.toString(key.getRow()));
           }
         }
         LOG.info("Last top when key < bottom: " + key.toString());
-        assertEquals("zz", key.getRow().toString());
+        assertEquals("zz", Bytes.toString(key.getRow()));
         
         // Test when midkey is > than last key in file ('||' > 'zz').
-        midkey = new HStoreKey(new Text("|||"));
+        midkey = new HStoreKey("|||");
         bottom = new HStoreFile.HalfMapFileReader(this.fs, p.toString(),
           this.conf, HStoreFile.Range.bottom, midkey);
         first = true;
@@ -367,11 +369,11 @@
           if (first) {
             first = false;
             LOG.info("First bottom when key > top: " + key.toString());
-            assertEquals("aa", key.getRow().toString());
+            assertEquals("aa", Bytes.toString(key.getRow()));
           }
         }
         LOG.info("Last bottom when key > top: " + key.toString());
-        assertEquals("zz", key.getRow().toString());
+        assertEquals("zz", Bytes.toString(key.getRow()));
         // Now look at top.  Should not return any values.
         top = new HStoreFile.HalfMapFileReader(this.fs, p.toString(),
           this.conf, HStoreFile.Range.top, midkey);
@@ -389,4 +391,4 @@
       this.fs.delete(p);
     }
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java Thu May 15 15:10:47 2008
@@ -33,6 +33,7 @@
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Test log deletion as logs are rolled.
@@ -62,7 +63,7 @@
       while (v.length() < 1000) {
         v.append(className);
       }
-      value = v.toString().getBytes(HConstants.UTF8_ENCODING);
+      value = Bytes.toBytes(v.toString());
       
     } catch (Exception e) {
       LOG.fatal("error in constructor", e);
@@ -107,10 +108,10 @@
     
     // Create the test table and open it
     HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
+    desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY));
     HBaseAdmin admin = new HBaseAdmin(conf);
     admin.createTable(desc);
-    HTable table = new HTable(conf, new Text(tableName));
+    HTable table = new HTable(conf, tableName);
 
     for (int i = 1; i <= 256; i++) {    // 256 writes should cause 8 log rolls
       BatchUpdate b =
@@ -144,7 +145,7 @@
       // flush all regions
       
       List<HRegion> regions =
-        new ArrayList<HRegion>(server.getOnlineRegions().values());
+        new ArrayList<HRegion>(server.getOnlineRegions());
       for (HRegion r: regions) {
         r.flushcache();
       }

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestRegionServerExit.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestRegionServerExit.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestRegionServerExit.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestRegionServerExit.java Thu May 15 15:10:47 2008
@@ -22,24 +22,20 @@
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
-import java.util.TreeMap;
 
 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.io.RowResult;
-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.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.HColumnDescriptor;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
+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.util.Bytes;
 
 /**
  * Tests region server failover when a region server exits both cleanly and
@@ -67,7 +63,7 @@
     new HTable(conf, HConstants.META_TABLE_NAME);
     // Create table and add a row.
     final String tableName = getName();
-    Text row = createTableAndAddRow(tableName);
+    byte [] row = createTableAndAddRow(tableName);
     // Start up a new region server to take over serving of root and meta
     // after we shut down the current meta/root host.
     this.cluster.startRegionServer();
@@ -88,7 +84,7 @@
     new HTable(this.conf, HConstants.META_TABLE_NAME);
     // Create table and add a row.
     final String tableName = getName();
-    Text row = createTableAndAddRow(tableName);
+    byte [] row = createTableAndAddRow(tableName);
     // Start up a new region server to take over serving of root and meta
     // after we shut down the current meta/root host.
     this.cluster.startRegionServer();
@@ -100,17 +96,17 @@
     threadDumpingJoin(t);
   }
   
-  private Text createTableAndAddRow(final String tableName) throws IOException {
+  private byte [] createTableAndAddRow(final String tableName)
+  throws IOException {
     HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY.toString()));
+    desc.addFamily(new HColumnDescriptor(HConstants.COLUMN_FAMILY));
     HBaseAdmin admin = new HBaseAdmin(conf);
     admin.createTable(desc);
     // put some values in the table
-    this.table = new HTable(conf, new Text(tableName));
-    final Text row = new Text("row1");
+    this.table = new HTable(conf, tableName);
+    byte [] row = Bytes.toBytes("row1");
     BatchUpdate b = new BatchUpdate(row);
-    b.put(HConstants.COLUMN_FAMILY,
-        tableName.getBytes(HConstants.UTF8_ENCODING));
+    b.put(HConstants.COLUMN_FAMILY, Bytes.toBytes(tableName));
     table.commit(b);
     return row;
   }
@@ -129,9 +125,10 @@
     int server = -1;
     for (int i = 0; i < regionThreads.size() && server == -1; i++) {
       HRegionServer s = regionThreads.get(i).getRegionServer();
-      Collection<HRegion> regions = s.getOnlineRegions().values();
+      Collection<HRegion> regions = s.getOnlineRegions();
       for (HRegion r : regions) {
-        if (r.getTableDesc().getName().equals(HConstants.META_TABLE_NAME)) {
+        if (Bytes.equals(r.getTableDesc().getName(),
+            HConstants.META_TABLE_NAME)) {
           server = i;
         }
       }
@@ -159,7 +156,7 @@
    * @return Verification thread.  Caller needs to calls start on it.
    */
   private Thread startVerificationThread(final String tableName,
-      final Text row) {
+      final byte [] row) {
     Runnable runnable = new Runnable() {
       public void run() {
         try {
@@ -167,7 +164,8 @@
           // meta server comes back up.
           HTable t = new HTable(conf, HConstants.META_TABLE_NAME);
           Scanner s =
-            t.getScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text());
+            t.getScanner(HConstants.COLUMN_FAMILY_ARRAY,
+              HConstants.EMPTY_START_ROW);
           s.close();
           
         } catch (IOException e) {
@@ -179,17 +177,15 @@
           // Verify that the client can find the data after the region has moved
           // to a different server
           scanner =
-            table.getScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text());
+            table.getScanner(HConstants.COLUMN_FAMILY_ARRAY,
+               HConstants.EMPTY_START_ROW);
           LOG.info("Obtained scanner " + scanner);
-          HStoreKey key = new HStoreKey();
-          TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
           for (RowResult r : scanner) {
-            assertTrue(r.getRow().equals(row));
+            assertTrue(Bytes.equals(r.getRow(), row));
             assertEquals(1, r.size());
             byte[] bytes = r.get(HConstants.COLUMN_FAMILY).getValue();
             assertNotNull(bytes);
-            assertTrue(tableName.equals(new String(bytes,
-                HConstants.UTF8_ENCODING)));
+            assertTrue(tableName.equals(Bytes.toString(bytes)));
           }
           LOG.info("Success!");
         } catch (Exception e) {

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java Thu May 15 15:10:47 2008
@@ -25,6 +25,7 @@
 import java.util.TreeMap;
 
 import org.apache.hadoop.dfs.MiniDFSCluster;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.hbase.HBaseTestCase;
@@ -39,19 +40,19 @@
  * Test of a long-lived scanner validating as we go.
  */
 public class TestScanner extends HBaseTestCase {
-  private static final Text FIRST_ROW = new Text();
-  private static final Text[] COLS = {
+  private static final byte [] FIRST_ROW = HConstants.EMPTY_START_ROW;
+  private static final byte [][] COLS = {
       HConstants.COLUMN_FAMILY
   };
-  private static final Text[] EXPLICIT_COLS = {
+  private static final byte [][] EXPLICIT_COLS = {
     HConstants.COL_REGIONINFO,
     HConstants.COL_SERVER,
     HConstants.COL_STARTCODE
   };
   
-  private static final Text ROW_KEY =
-    new Text(HRegionInfo.rootRegionInfo.getRegionName());
-  private static final HRegionInfo REGION_INFO = HRegionInfo.rootRegionInfo;
+  private static final byte [] ROW_KEY =
+    HRegionInfo.ROOT_REGIONINFO.getRegionName();
+  private static final HRegionInfo REGION_INFO = HRegionInfo.ROOT_REGIONINFO;
   
   private static final long START_CODE = Long.MAX_VALUE;
 
@@ -76,9 +77,9 @@
       (HRegionInfo) Writables.getWritable(regionBytes, new HRegionInfo());
     
     assertEquals(REGION_INFO.getRegionId(), info.getRegionId());
-    assertEquals(0, info.getStartKey().getLength());
-    assertEquals(0, info.getEndKey().getLength());
-    assertEquals(0, info.getRegionName().compareTo(REGION_INFO.getRegionName()));
+    assertEquals(0, info.getStartKey().length);
+    assertEquals(0, info.getEndKey().length);
+    assertEquals(0, Bytes.compareTo(info.getRegionName(), REGION_INFO.getRegionName()));
     assertEquals(0, info.getTableDesc().compareTo(REGION_INFO.getTableDesc()));
   }
   
@@ -87,10 +88,11 @@
       throws IOException {
     
     InternalScanner scanner = null;
-    TreeMap<Text, byte []> results = new TreeMap<Text, byte []>();
+    TreeMap<byte [], byte []> results =
+      new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
     HStoreKey key = new HStoreKey();
 
-    Text[][] scanColumns = {
+    byte [][][] scanColumns = {
         COLS,
         EXPLICIT_COLS
     };
@@ -100,7 +102,7 @@
         scanner = r.getScanner(scanColumns[i], FIRST_ROW,
             System.currentTimeMillis(), null);
         
-        while(scanner.next(key, results)) {
+        while (scanner.next(key, results)) {
           assertTrue(results.containsKey(HConstants.COL_REGIONINFO));
           byte [] val = results.get(HConstants.COL_REGIONINFO); 
           validateRegionInfo(val);
@@ -109,7 +111,7 @@
             val = results.get(HConstants.COL_STARTCODE);
             assertNotNull(val);
             assertFalse(val.length == 0);
-            long startCode = Writables.bytesToLong(val);
+            long startCode = Bytes.toLong(val);
             assertEquals(START_CODE, startCode);
           }
           
@@ -118,7 +120,7 @@
             val = results.get(HConstants.COL_SERVER);
             assertNotNull(val);
             assertFalse(val.length == 0);
-            String server = Writables.bytesToString(val);
+            String server = Bytes.toString(val);
             assertEquals(0, server.compareTo(serverName));
           }
           results.clear();
@@ -154,7 +156,7 @@
 
       ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
       DataOutputStream s = new DataOutputStream(byteStream);
-      HRegionInfo.rootRegionInfo.write(s);
+      HRegionInfo.ROOT_REGIONINFO.write(s);
       batchUpdate.put(HConstants.COL_REGIONINFO, byteStream.toByteArray());
       region.commit(batchUpdate);
 
@@ -181,11 +183,9 @@
 
       batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis());
 
-      batchUpdate.put(HConstants.COL_SERVER, 
-        Writables.stringToBytes(address.toString()));
+      batchUpdate.put(HConstants.COL_SERVER,  Bytes.toBytes(address.toString()));
 
-      batchUpdate.put(HConstants.COL_STARTCODE,
-        Writables.longToBytes(START_CODE));
+      batchUpdate.put(HConstants.COL_STARTCODE, Bytes.toBytes(START_CODE));
 
       region.commit(batchUpdate);
       
@@ -222,7 +222,7 @@
       batchUpdate = new BatchUpdate(ROW_KEY, System.currentTimeMillis());
 
       batchUpdate.put(HConstants.COL_SERVER, 
-        Writables.stringToBytes(address.toString()));
+        Bytes.toBytes(address.toString()));
 
       region.commit(batchUpdate);
       

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestSplit.java Thu May 15 15:10:47 2008
@@ -24,7 +24,6 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.hadoop.hbase.HStoreKey;
@@ -33,6 +32,7 @@
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * {@Link TestHRegion} does a split but this TestCase adds testing of fast
@@ -87,7 +87,7 @@
   private void basicSplit(final HRegion region) throws Exception {
     addContent(region, COLFAMILY_NAME3);
     region.flushcache();
-    Text midkey = region.compactStores();
+    byte [] midkey = region.compactStores();
     assertNotNull(midkey);
     HRegion [] regions = split(region, midkey);
     try {
@@ -99,10 +99,11 @@
       }
       // Assert can get rows out of new regions. Should be able to get first
       // row from first region and the midkey from second region.
-      assertGet(regions[0], COLFAMILY_NAME3, new Text(START_KEY));
+      assertGet(regions[0], COLFAMILY_NAME3, Bytes.toBytes(START_KEY));
       assertGet(regions[1], COLFAMILY_NAME3, midkey);
       // Test I can get scanner and that it starts at right place.
-      assertScan(regions[0], COLFAMILY_NAME3, new Text(START_KEY));
+      assertScan(regions[0], COLFAMILY_NAME3,
+          Bytes.toBytes(START_KEY));
       assertScan(regions[1], COLFAMILY_NAME3, midkey);
       // Now prove can't split regions that have references.
       for (int i = 0; i < regions.length; i++) {
@@ -116,7 +117,7 @@
         regions[i].flushcache();
       }
 
-      Text[] midkeys = new Text[regions.length];
+      byte [][] midkeys = new byte [regions.length][];
       // To make regions splitable force compaction.
       for (int i = 0; i < regions.length; i++) {
         midkeys[i] = regions[i].compactStores();
@@ -130,7 +131,7 @@
         if (midkeys[i] != null) {
           rs = split(regions[i], midkeys[i]);
           for (int j = 0; j < rs.length; j++) {
-            sortedMap.put(rs[j].getRegionName().toString(),
+            sortedMap.put(Bytes.toString(rs[j].getRegionName()),
               openClosedRegion(rs[j]));
           }
         }
@@ -139,10 +140,9 @@
       // The splits should have been even. Test I can get some arbitrary row out
       // of each.
       int interval = (LAST_CHAR - FIRST_CHAR) / 3;
-      byte[] b = START_KEY.getBytes(HConstants.UTF8_ENCODING);
+      byte[] b = Bytes.toBytes(START_KEY);
       for (HRegion r : sortedMap.values()) {
-        assertGet(r, COLFAMILY_NAME3, new Text(new String(b,
-            HConstants.UTF8_ENCODING)));
+        assertGet(r, COLFAMILY_NAME3, b);
         b[0] += interval;
       }
     } finally {
@@ -156,15 +156,14 @@
     }
   }
   
-  private void assertGet(final HRegion r, final String family, final Text k)
+  private void assertGet(final HRegion r, final byte [] family, final byte [] k)
   throws IOException {
     // Now I have k, get values out and assert they are as expected.
-    Cell[] results = r.get(k, new Text(family),
-      Integer.MAX_VALUE);
+    Cell[] results = r.get(k, family, Integer.MAX_VALUE);
     for (int j = 0; j < results.length; j++) {
-      Text tmp = new Text(results[j].getValue());
+      byte [] tmp = results[j].getValue();
       // Row should be equal to value every time.
-      assertEquals(k.toString(), tmp.toString());
+      assertTrue(Bytes.equals(k, tmp));
     }
   }
   
@@ -175,23 +174,24 @@
    * @param firstValue
    * @throws IOException
    */
-  private void assertScan(final HRegion r, final String column,
-      final Text firstValue)
+  private void assertScan(final HRegion r, final byte [] column,
+      final byte [] firstValue)
   throws IOException {
-    Text [] cols = new Text[] {new Text(column)};
+    byte [][] cols = {column};
     InternalScanner s = r.getScanner(cols,
       HConstants.EMPTY_START_ROW, System.currentTimeMillis(), null);
     try {
       HStoreKey curKey = new HStoreKey();
-      TreeMap<Text, byte []> curVals = new TreeMap<Text, byte []>();
+      TreeMap<byte [], byte []> curVals =
+        new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
       boolean first = true;
       OUTER_LOOP: while(s.next(curKey, curVals)) {
-        for(Text col: curVals.keySet()) {
+        for(byte [] col: curVals.keySet()) {
           byte [] val = curVals.get(col);
-          Text curval = new Text(val);
+          byte [] curval = val;
           if (first) {
             first = false;
-            assertTrue(curval.compareTo(firstValue) == 0);
+            assertTrue(Bytes.compareTo(curval, firstValue) == 0);
           } else {
             // Not asserting anything.  Might as well break.
             break OUTER_LOOP;
@@ -203,7 +203,7 @@
     }
   }
   
-  private HRegion [] split(final HRegion r, final Text midKey)
+  private HRegion [] split(final HRegion r, final byte [] midKey)
   throws IOException {
     // Assert can get mid key from passed region.
     assertGet(r, COLFAMILY_NAME3, midKey);

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java?rev=656868&r1=656867&r2=656868&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestTimestamp.java Thu May 15 15:10:47 2008
@@ -20,19 +20,15 @@
 
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+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.HConstants;
-import org.apache.hadoop.hbase.HStoreKey;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-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.HColumnDescriptor.CompressionType;
+import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Tests user specifiable time stamps putting, getting and scanning.  Also
@@ -44,7 +40,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);
   private static final int VERSIONS = 3;
   
   /**