You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/04/02 23:10:54 UTC

svn commit: r1584189 [2/2] - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/ main/java/org/apache/hadoop/hbase/client/ main/java/org/apache/hadoop/hbase/coprocessor/ main/java/org/apache/hadoop/hbase/mapreduce/ main/java/org/apache/h...

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Wed Apr  2 21:10:53 2014
@@ -116,6 +116,7 @@ import org.apache.hadoop.hbase.util.Inje
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.RuntimeHaltAbortStrategy;
 import org.apache.hadoop.hbase.util.Sleeper;
+import org.apache.hadoop.hbase.util.StringBytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.util.Writables;
@@ -1746,7 +1747,7 @@ public class HMaster extends HasThread i
           }
       };
 
-      MetaScanner.metaScan(conf, visitor, tableName);
+      MetaScanner.metaScan(conf, visitor, new StringBytes(tableName));
     }
     else {
       List<MetaRegion> metaRegions = regionManager.getListOfOnlineMetaRegions();
@@ -1806,7 +1807,7 @@ public class HMaster extends HasThread i
         }
     };
 
-    MetaScanner.metaScan(conf, visitor, tableName, rowKey, 1);
+    MetaScanner.metaScan(conf, visitor, new StringBytes(tableName), rowKey, 1);
     return result.get();
   }
 
@@ -1866,7 +1867,7 @@ public class HMaster extends HasThread i
    */
   public HTableDescriptor getTableDescriptor(final byte [] tableName)
   throws IOException {
-    return this.connection.getHTableDescriptor(tableName);
+    return this.connection.getHTableDescriptor(new StringBytes(tableName));
   }
 
   @Override

Added: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/StringBytes.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/StringBytes.java?rev=1584189&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/StringBytes.java (added)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/StringBytes.java Wed Apr  2 21:10:53 2014
@@ -0,0 +1,149 @@
+/**
+ * Copyright 2014 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.util;
+
+
+/**
+ * An immutable data structure storing a byte array and corresponding string
+ * converted by {@code Bytes.toString(byte[])). The current encoding used {
+ * @code Bytes} is UTF-8.
+ *
+ * NOTE this class is supposed to store a byte array of table name or column
+ *      family name.
+ *
+ * {@ code StringBytes} is comparable and the hashCode and equals are based on
+ * contents.
+ */
+public class StringBytes implements Comparable<StringBytes> {
+  /**
+   * The byte array value.
+   */
+  protected final byte[] bytes;
+  /**
+   * A string cache. Could be null.
+   */
+  protected volatile String string = null;
+
+  /**
+   * The computed hashCode.
+   */
+  protected final int hashCode;
+
+  /**
+   * Constructs a StringBytes from a string.
+   *
+   * @param string the non-null string value.
+   */
+  public StringBytes(String string) {
+    if (string == null) {
+      throw new IllegalArgumentException("string cannot be null");
+    }
+
+    this.string = string;
+    this.bytes = Bytes.toBytes(string);
+    this.hashCode = Bytes.hashCode(this.bytes);
+  }
+
+  /**
+   * Constructs a StringBytes from a byte array.
+   *
+   * @param bytes the non-null byte array.
+   */
+  public StringBytes(byte[] bytes) {
+    if (bytes == null) {
+      throw new IllegalArgumentException("bytes cannot be null");
+    }
+
+    this.bytes = bytes;
+    this.hashCode = Bytes.hashCode(this.bytes);
+  }
+
+  private void checkString() {
+    if (this.string == null) {
+      this.string = Bytes.toString(bytes);
+    }
+  }
+
+  /**
+   * @return the string value.
+   */
+  public String getString() {
+    checkString();
+    return this.string;
+  }
+
+  /**
+   * @return the byte array value.
+   */
+  public byte[] getBytes() {
+    return this.bytes;
+  }
+
+  @Override
+  public int compareTo(StringBytes that) {
+    return Bytes.BYTES_COMPARATOR.compare(this.bytes, that.bytes);
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (this == that)
+      return true;
+    if (that == null)
+      return false;
+    if (!(that instanceof StringBytes))
+      return false;
+    StringBytes other = (StringBytes) that;
+    if (this.hashCode != other.hashCode) {
+      return false;
+    }
+    return Bytes.equals(bytes, other.bytes);
+  }
+
+  /**
+   * @return whether another byte array has the same contents as this.
+   */
+  public boolean equalBytes(byte[] that) {
+    return Bytes.equals(this.bytes, that);
+  }
+
+  /**
+   * @return whether another string has the same contents as this.
+   */
+  public boolean equalString(String that) {
+    return this.getString().equals(that);
+  }
+
+  @Override
+  public String toString() {
+    return this.getString();
+  }
+
+  /**
+   * @return whether this StringBytes contains zero-length bytes.
+   */
+  public boolean isEmpty() {
+    return bytes.length == 0;
+  }
+}

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Wed Apr  2 21:10:53 2014
@@ -90,6 +90,7 @@ import org.apache.hadoop.hbase.util.JVMC
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.RegionSplitter;
+import org.apache.hadoop.hbase.util.StringBytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
@@ -561,6 +562,21 @@ public class HBaseTestingUtility {
     return createTable(tableName, new byte[][]{family});
   }
 
+  /**
+   * Create a table.
+   *
+   * @param tableName
+   * @param family
+   * @return An HTableAsync instance for the created table, which is a sub-class
+   *         of the HTable class, and HTableAsyncInterface. So you can use both
+   *         the sync API of HTableInterface, and async API in
+   *         HTableAsyncInterface.
+   * @throws IOException
+   */
+  public HTableAsync createTable(StringBytes tableName, byte[] family)
+      throws IOException {
+    return createTable(tableName.getBytes(), new byte[][] { family });
+  }
 
   public HTable createTable(byte[] tableName, byte[][] families,
       int numVersions, byte[] startKey, byte[] endKey, int numRegions)

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Wed Apr  2 21:10:53 2014
@@ -117,7 +117,7 @@ public class TestZooKeeper {
         EmptyWatcher.instance, sessionID, password);
     zk.close();
     Thread.sleep(sessionTimeout * 3L);
-    connection.relocateRegion(HConstants.ROOT_TABLE_NAME,
+    connection.relocateRegion(HConstants.ROOT_TABLE_NAME_STRINGBYTES,
         HConstants.EMPTY_BYTE_ARRAY);
 
     // The zk session for connectionZK should NOT time out since the client will

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestBatchedUpload.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestBatchedUpload.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestBatchedUpload.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestBatchedUpload.java Wed Apr  2 21:10:53 2014
@@ -51,14 +51,17 @@ public class TestBatchedUpload {
   private static byte [] FAMILY = Bytes.toBytes("testFamily");
   private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static int SLAVES = 5;
-  
+
   private enum RegionServerAction {
     KILL_REGIONSERVER,
     MOVE_REGION
   }
-  
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().set(HBaseTestingUtility.FS_TYPE_KEY,
+        HBaseTestingUtility.FS_TYPE_LFS);
+
     TEST_UTIL.startMiniCluster(SLAVES);
   }
 
@@ -67,13 +70,13 @@ public class TestBatchedUpload {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  @Test(timeout=100000)
+  @Test(timeout = 200000)
   public void testBatchedUpload() throws Exception {
     byte [] TABLE = Bytes.toBytes("testBatchedUpload");
     int NUM_REGIONS = 10;
     HTable ht = TEST_UTIL.createTable(TABLE, new byte[][]{FAMILY},
         3, Bytes.toBytes("aaaaa"), Bytes.toBytes("zzzzz"), NUM_REGIONS);
-   
+
     int NUM_ROWS = 1000;
 
     // start batch processing
@@ -87,7 +90,7 @@ public class TestBatchedUpload {
     ht.close();
   }
 
-  @Test(timeout=100000)
+  @Test(timeout = 200000)
   /*
    * Test to make sure that if a region moves benignly, and both
    * the source and dest region servers are alive, then the batch
@@ -104,7 +107,7 @@ public class TestBatchedUpload {
     // do a bunch of puts
     // finish batch. Check for Exceptions.
     HMaster m = TEST_UTIL.getHBaseCluster().getMaster();
-    
+
     // Disable the load balancer as the movement of the region might cause the
     // load balancer to kick in and move the regions causing the end of batched
     // upload to fail.
@@ -125,7 +128,7 @@ public class TestBatchedUpload {
    * @param action -- enum RegionServerAction which defines the type of action.
    * @return number of attempts to complete the batch.
    * @throws IOException
-   * @throws InterruptedException 
+   * @throws InterruptedException
    */
   public int writeData(HTable table, long numRows, RegionServerAction action) throws IOException, InterruptedException {
     int attempts = 0;
@@ -151,24 +154,24 @@ public class TestBatchedUpload {
           byte[] value = rowKey; // value is the same as the row key
           put.add(FAMILY, QUALIFIER, value);
           put.setWriteToWAL(false);
-          
+
           prob = rand.nextDouble();
           if (kills < 2 && prob < killProb) { // kill up to 2 rs
             kills++;
             // Find the region server for the next put
             HRegionLocation regLoc = table.getRegionLocation(put.row);
             int srcRSIdx = cluster.getServerWith(regLoc.getRegionInfo().getRegionName());
- 
+
             LOG.debug("Try " + attempts + " written Puts : " + i);
             if (action == RegionServerAction.KILL_REGIONSERVER) {
               // abort the region server
               LOG.info("Killing region server " + srcRSIdx
-                  + " before the next put. Got probability " + 
+                  + " before the next put. Got probability " +
                   prob + " < " + killProb);
               cluster.abortRegionServer(srcRSIdx);
-              
+
             } else if (action == RegionServerAction.MOVE_REGION) {
-              
+
               // move the region to some other Region Server
               HRegionServer dstRS = cluster.getRegionServer(
                   (srcRSIdx + 1) % cluster.getLiveRegionServerThreads().size());
@@ -206,7 +209,7 @@ public class TestBatchedUpload {
       get.addColumn(FAMILY, QUALIFIER);
       get.setMaxVersions(1);
       Result result = table.get(get.create());
-      
+
       assertTrue(Arrays.equals(rowKey, result.getValue(FAMILY, QUALIFIER)));
     }
   }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Wed Apr  2 21:10:53 2014
@@ -76,6 +76,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.StringBytes;
 import org.apache.hadoop.hbase.util.TagRunner;
 import org.apache.hadoop.hbase.util.TestTag;
 import org.junit.After;
@@ -1117,7 +1118,7 @@ public class TestFromClientSide {
 
     // Null table name (should NOT work)
     try {
-      TEST_UTIL.createTable(null, FAMILY);
+      TEST_UTIL.createTable((StringBytes) null, FAMILY);
       fail("Creating a table with null name passed, should have failed");
     } catch(Exception e) {}
 
@@ -3984,7 +3985,7 @@ public class TestFromClientSide {
    public void testRegionCacheDeSerialization() throws Exception {
      // 1. test serialization.
      LOG.info("Starting testRegionCacheDeSerialization");
-     final byte[] TABLENAME = Bytes.toBytes("testCachePrewarm2");
+    final StringBytes TABLENAME = new StringBytes("testCachePrewarm2");
      final byte[] FAMILY = Bytes.toBytes("family");
      Configuration conf = TEST_UTIL.getConfiguration();
      TEST_UTIL.createTable(TABLENAME, FAMILY);
@@ -4023,8 +4024,8 @@ public class TestFromClientSide {
      // set the deserialized regions to the global cache.
      table.getConnectionAndResetOperationContext().clearRegionCache();
 
-     table.getConnectionAndResetOperationContext().prewarmRegionCache(table.getTableName(),
-         deserRegions);
+    table.getConnectionAndResetOperationContext().prewarmRegionCache(
+        table.getTableNameStringBytes(), deserRegions);
 
      // verify whether the 2 maps are identical or not.
      assertEquals("Number of cached region is incorrect",
@@ -4035,7 +4036,7 @@ public class TestFromClientSide {
      for (Map.Entry<HRegionInfo, HServerAddress> e: loadedRegions.entrySet()) {
        HRegionInfo hri = e.getKey();
        assertTrue(HConnectionManager.isRegionCached(conf,
-           hri.getTableDesc().getName(), hri.getStartKey()));
+           new StringBytes(hri.getTableDesc().getName()), hri.getStartKey()));
      }
 
      // delete the temp file
@@ -4050,7 +4051,7 @@ public class TestFromClientSide {
   @Test
   public void testRegionCachePreWarm() throws Exception {
     LOG.info("Starting testRegionCachePreWarm");
-    final byte [] TABLENAME = Bytes.toBytes("testCachePrewarm");
+    final StringBytes TABLENAME = new StringBytes("testCachePrewarm");
     Configuration conf = TEST_UTIL.getConfiguration();
 
     // Set up test table:
@@ -4058,9 +4059,9 @@ public class TestFromClientSide {
     TEST_UTIL.createTable(TABLENAME, FAMILY);
 
     // disable region cache for the table.
-    HTable.setRegionCachePrefetch(conf, TABLENAME, false);
+    HTable.setRegionCachePrefetch(conf, TABLENAME.getBytes(), false);
     assertFalse("The table is disabled for region cache prefetch",
-        HTable.getRegionCachePrefetch(conf, TABLENAME));
+        HTable.getRegionCachePrefetch(conf, TABLENAME.getBytes()));
 
     HTable table = new HTable(conf, TABLENAME);
 
@@ -4082,17 +4083,17 @@ public class TestFromClientSide {
         HConnectionManager.getCachedRegionCount(conf, TABLENAME));
 
     // now we enable cached prefetch.
-    HTable.setRegionCachePrefetch(conf, TABLENAME, true);
+    HTable.setRegionCachePrefetch(conf, TABLENAME.getBytes(), true);
     assertTrue("The table is enabled for region cache prefetch",
-        HTable.getRegionCachePrefetch(conf, TABLENAME));
+        HTable.getRegionCachePrefetch(conf, TABLENAME.getBytes()));
 
-    HTable.setRegionCachePrefetch(conf, TABLENAME, false);
+    HTable.setRegionCachePrefetch(conf, TABLENAME.getBytes(), false);
     assertFalse("The table is disabled for region cache prefetch",
-        HTable.getRegionCachePrefetch(conf, TABLENAME));
+        HTable.getRegionCachePrefetch(conf, TABLENAME.getBytes()));
 
-    HTable.setRegionCachePrefetch(conf, TABLENAME, true);
+    HTable.setRegionCachePrefetch(conf, TABLENAME.getBytes(), true);
     assertTrue("The table is enabled for region cache prefetch",
-        HTable.getRegionCachePrefetch(conf, TABLENAME));
+        HTable.getRegionCachePrefetch(conf, TABLENAME.getBytes()));
 
     table.getConnectionAndResetOperationContext().clearRegionCache();
 
@@ -4105,10 +4106,10 @@ public class TestFromClientSide {
 
     // Get the configured number of cache read-ahead regions.  For various
     // reasons, the meta may not yet have all regions in place (e.g. hbase-2757).
-    // That the prefetch gets at least half shows prefetch is bascially working.
+    // That the prefetch gets at least half shows prefetch is basically working.
     int prefetchRegionNumber = conf.getInt("hbase.client.prefetch.limit", 10) / 2;
 
-    // the total number of cached regions == region('aaa") + prefeched regions.
+    // the total number of cached regions == region("aaa") + prefetched regions.
     LOG.info("Testing how many regions cached");
     assertTrue(prefetchRegionNumber < HConnectionManager.getCachedRegionCount(conf, TABLENAME));
 

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Wed Apr  2 21:10:53 2014
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.StringBytes;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -54,7 +55,7 @@ public class TestHCM {
   private static final Log LOG = LogFactory.getLog(TestHCM.class);
 
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final byte[] TABLE_NAME = Bytes.toBytes("test");
+  private static final StringBytes TABLE_NAME = new StringBytes("test");
   private static final byte[] FAM_NAM = Bytes.toBytes("f");
   private static final byte[] ROW = Bytes.toBytes("bbd");
 

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTable.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTable.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestHTable.java Wed Apr  2 21:10:53 2014
@@ -19,6 +19,7 @@ import org.apache.hadoop.hbase.TableNotF
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.StringBytes;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -186,7 +187,7 @@ public class TestHTable {
     // Let's search for a non-existing table, and get a TableNotFoundException.
     HConnection connection =
       HConnectionManager.getConnection(TEST_UTIL.getConfiguration());
-    connection.getRegionLocation(Bytes.toBytes("foo"),
+    connection.getRegionLocation(new StringBytes("foo"),
                                  Bytes.toBytes("r1"), false);
   }
 }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java Wed Apr  2 21:10:53 2014
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HRegionLo
 import org.apache.hadoop.hbase.HServerAddress;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.StringBytes;
 import org.junit.Test;
 
 /**
@@ -36,15 +37,16 @@ public class TestMetaCache {
 
   @Test
   public void testBasic() {
-    final byte[] TABLE = Bytes.toBytes(this.getClass().getSimpleName());
-    final byte[] TABLE2 = Bytes.toBytes(this.getClass().getSimpleName() + "2");
+    final StringBytes TABLE = new StringBytes(this.getClass().getSimpleName());
+    final StringBytes TABLE2 =
+        new StringBytes(this.getClass().getSimpleName() + "2");
     final byte[] START_KEY = Bytes.toBytes("aaa");
     final byte[] END_KEY = Bytes.toBytes("ggg");
     final byte[] SMALL_ROW = Bytes.toBytes("a");
     final byte[] ROW = Bytes.toBytes("ddd");
 
     final HRegionLocation LOCATION = new HRegionLocation(new HRegionInfo(
-        new HTableDescriptor(TABLE), START_KEY, END_KEY),
+        new HTableDescriptor(TABLE.getBytes()), START_KEY, END_KEY),
         new HServerAddress("10.0.0.11:1234"));
 
     MetaCache metaCache = new MetaCache();
@@ -71,8 +73,8 @@ public class TestMetaCache {
 
     // Add another location of the same table at the same server
     metaCache.add(TABLE, new HRegionLocation(new HRegionInfo(
-        new HTableDescriptor(TABLE), HConstants.EMPTY_START_ROW, START_KEY),
-        new HServerAddress("10.0.0.11:1234")));
+        new HTableDescriptor(TABLE.getBytes()), HConstants.EMPTY_START_ROW,
+        START_KEY), new HServerAddress("10.0.0.11:1234")));
 
     Assert.assertEquals("should found", LOCATION, metaCache.getForRow(TABLE, ROW));
     Assert.assertNotNull("should found", metaCache.getForRow(TABLE, SMALL_ROW));
@@ -84,8 +86,8 @@ public class TestMetaCache {
 
     // Add another location of the different table at the different server
     metaCache.add(TABLE2, new HRegionLocation(new HRegionInfo(
-        new HTableDescriptor(TABLE2), END_KEY, HConstants.EMPTY_START_ROW),
-        new HServerAddress("10.0.0.12:1234")));
+        new HTableDescriptor(TABLE2.getBytes()), END_KEY,
+        HConstants.EMPTY_START_ROW), new HServerAddress("10.0.0.12:1234")));
 
     Assert.assertEquals("getNumber", 2, metaCache.getNumber(TABLE));
     Assert.assertEquals("get(table).size", 2, metaCache.getForTable(TABLE).size());

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/client/TestMetaScanner.java Wed Apr  2 21:10:53 2014
@@ -19,18 +19,24 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.StringBytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.mockito.Mockito.*;
-
 public class TestMetaScanner {
   final Log LOG = LogFactory.getLog(getClass());
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -51,7 +57,7 @@ public class TestMetaScanner {
   @Test
   public void testMetaScanner() throws Exception {
     LOG.info("Starting testMetaScanner");
-    final byte[] TABLENAME = Bytes.toBytes("testMetaScanner");
+    final StringBytes TABLENAME = new StringBytes("testMetaScanner");
     final byte[] FAMILY = Bytes.toBytes("family");
     TEST_UTIL.createTable(TABLENAME, FAMILY);
     Configuration conf = TEST_UTIL.getConfiguration();
@@ -63,34 +69,34 @@ public class TestMetaScanner {
           Bytes.toBytes("region_b")});
     // Make sure all the regions are deployed
     TEST_UTIL.countRows(table);
-    
-    MetaScanner.MetaScannerVisitor visitor = 
+
+    MetaScanner.MetaScannerVisitor visitor =
       mock(MetaScanner.MetaScannerVisitor.class);
     doReturn(true).when(visitor).processRow((Result)anyObject());
 
     // Scanning the entire table should give us three rows
     MetaScanner.metaScan(conf, visitor, TABLENAME);
     verify(visitor, times(3)).processRow((Result)anyObject());
-    
+
     // Scanning the table with a specified empty start row should also
     // give us three META rows
     reset(visitor);
     doReturn(true).when(visitor).processRow((Result)anyObject());
     MetaScanner.metaScan(conf, visitor, TABLENAME, HConstants.EMPTY_BYTE_ARRAY, 1000);
     verify(visitor, times(3)).processRow((Result)anyObject());
-    
+
     // Scanning the table starting in the middle should give us two rows:
     // region_a and region_b
     reset(visitor);
     doReturn(true).when(visitor).processRow((Result)anyObject());
     MetaScanner.metaScan(conf, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1000);
     verify(visitor, times(2)).processRow((Result)anyObject());
-    
+
     // Scanning with a limit of 1 should only give us one row
     reset(visitor);
     doReturn(true).when(visitor).processRow((Result)anyObject());
     MetaScanner.metaScan(conf, visitor, TABLENAME, Bytes.toBytes("region_ac"), 1);
     verify(visitor, times(1)).processRow((Result)anyObject());
-        
+
   }
 }

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestCachingOnCompaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestCachingOnCompaction.java?rev=1584189&r1=1584188&r2=1584189&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestCachingOnCompaction.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestCachingOnCompaction.java Wed Apr  2 21:10:53 2014
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.regionse
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.client.Pu
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.StringBytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -193,7 +195,8 @@ public class TestCachingOnCompaction {
     col.setBlockCacheEnabled(true);
     col.setDataBlockEncoding(type);
 
-    ht.getConnectionAndResetOperationContext().getHTableDescriptor(TABLE).addFamily(col);
+    ht.getConnectionAndResetOperationContext()
+        .getHTableDescriptor(new StringBytes(TABLE)).addFamily(col);
 
     region = TEST_UTIL.createTestRegion(tableName, col);
 

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestStringBytes.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestStringBytes.java?rev=1584189&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestStringBytes.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestStringBytes.java Wed Apr  2 21:10:53 2014
@@ -0,0 +1,39 @@
+/**
+ * Copyright 2014 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.util;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+/**
+ * Testcase for StringBytes
+ */
+public class TestStringBytes {
+  @Test
+  public void testBasic() throws Exception {
+    final String STR = "tableName";
+    final byte[] BYTES = Bytes.toBytes(STR);
+
+    Assert.assertTrue("BYTES should be equal",
+        Bytes.equals(BYTES, new StringBytes(STR).getBytes()));
+    Assert.assertEquals("STR", STR, new StringBytes(BYTES).getString());
+  }
+}