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 2013/08/08 06:19:56 UTC

svn commit: r1511577 [9/23] - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/ hbase-client/src/main/java/org/apache/hadoop/hbase/catalog/ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apa...

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java Thu Aug  8 04:19:49 2013
@@ -21,10 +21,12 @@ package org.apache.hadoop.hbase.coproces
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@@ -59,156 +61,180 @@ public class BaseMasterObserver implemen
 
   @Override
   public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
+      TableName tableName) throws IOException {
   }
 
   @Override
   public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
+      TableName tableName) throws IOException {
   }
 
   @Override
   public void preDeleteTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName)
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException{
   }
 
   @Override
   public void postDeleteTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName)
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException {
   }
 
   @Override
   public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HTableDescriptor htd) throws IOException {
+      TableName tableName, HTableDescriptor htd) throws IOException {
   }
 
   @Override
   public void postModifyTableHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName,
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
       HTableDescriptor htd) throws IOException {
   }
 
   @Override
   public void preModifyTableHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName,
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
       HTableDescriptor htd) throws IOException {
   }
 
   @Override
   public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HTableDescriptor htd) throws IOException {
+      TableName tableName, HTableDescriptor htd) throws IOException {
+  }
+
+  @Override
+  public void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException {
+  }
+
+  @Override
+  public void postCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException {
+  }
+
+  @Override
+  public void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException {
+  }
+
+  @Override
+  public void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException {
+  }
+
+  @Override
+  public void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException {
+  }
+
+  @Override
+  public void postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException {
   }
 
   @Override
   public void preAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor column) throws IOException {
+      TableName tableName, HColumnDescriptor column) throws IOException {
   }
 
   @Override
   public void postAddColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor column) throws IOException {
+      TableName tableName, HColumnDescriptor column) throws IOException {
   }
 
   @Override
   public void preAddColumnHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName,
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
       HColumnDescriptor column) throws IOException {
   }
 
   @Override
   public void postAddColumnHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName,
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
       HColumnDescriptor column) throws IOException {
   }
 
   @Override
   public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor descriptor) throws IOException {
+      TableName tableName, HColumnDescriptor descriptor) throws IOException {
   }
 
   @Override
   public void postModifyColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor descriptor) throws IOException {
+      TableName tableName, HColumnDescriptor descriptor) throws IOException {
   }
 
   @Override
   public void preModifyColumnHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName,
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
       HColumnDescriptor descriptor) throws IOException {
   }
 
   @Override
   public void postModifyColumnHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName,
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
       HColumnDescriptor descriptor) throws IOException {
   }
 
   @Override
   public void preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, byte[] c) throws IOException {
+      TableName tableName, byte[] c) throws IOException {
   }
 
   @Override
   public void postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, byte[] c) throws IOException {
+      TableName tableName, byte[] c) throws IOException {
   }
 
   @Override
   public void preDeleteColumnHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName,
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
       byte[] c) throws IOException {
   }
 
   @Override
   public void postDeleteColumnHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName,
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
       byte[] c) throws IOException {
   }
 
 
   @Override
   public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
+      TableName tableName) throws IOException {
   }
 
   @Override
   public void postEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
+      TableName tableName) throws IOException {
   }
 
   @Override
   public void preEnableTableHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName)
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException {
   }
 
   @Override
   public void postEnableTableHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName)
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException {
   }
 
   @Override
   public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
+      TableName tableName) throws IOException {
   }
 
   @Override
   public void postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException {
+      TableName tableName) throws IOException {
   }
 
   @Override
   public void preDisableTableHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName)
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException {
   }
 
   @Override
   public void postDisableTableHandler(
-      ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName)
+      ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException {
   }
 
@@ -346,7 +372,8 @@ public class BaseMasterObserver implemen
 
   @Override
   public void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      List<String> tableNamesList, List<HTableDescriptor> descriptors) throws IOException {
+      List<TableName> tableNamesList, List<HTableDescriptor> descriptors)
+      throws IOException {
   }
 
   @Override

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java Thu Aug  8 04:19:49 2013
@@ -39,6 +39,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
@@ -366,10 +367,10 @@ public abstract class CoprocessorHost<E 
      */
     class HTableWrapper implements HTableInterface {
 
-      private byte[] tableName;
+      private TableName tableName;
       private HTable table;
 
-      public HTableWrapper(byte[] tableName) throws IOException {
+      public HTableWrapper(TableName tableName) throws IOException {
         this.tableName = tableName;
         this.table = new HTable(conf, tableName);
         openTables.add(this);
@@ -481,8 +482,14 @@ public abstract class CoprocessorHost<E 
         return table.getTableDescriptor();
       }
 
+      @Override
       public byte[] getTableName() {
-        return tableName;
+        return tableName.getName();
+      }
+
+      @Override
+      public TableName getName() {
+        return table.getName();
       }
 
       @Override
@@ -667,7 +674,7 @@ public abstract class CoprocessorHost<E 
      * @exception java.io.IOException Exception
      */
     @Override
-    public HTableInterface getTable(byte[] tableName) throws IOException {
+    public HTableInterface getTable(TableName tableName) throws IOException {
       return new HTableWrapper(tableName);
     }
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java Thu Aug  8 04:19:49 2013
@@ -25,9 +25,11 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@@ -96,7 +98,7 @@ public interface MasterObserver extends 
    * @param tableName the name of the table
    */
   void preDeleteTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException;
+      TableName tableName) throws IOException;
 
   /**
    * Called after the deleteTable operation has been requested.  Called as part
@@ -105,7 +107,7 @@ public interface MasterObserver extends 
    * @param tableName the name of the table
    */
   void postDeleteTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName) throws IOException;
+      TableName tableName) throws IOException;
 
   /**
    * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
@@ -116,7 +118,7 @@ public interface MasterObserver extends 
    * @param tableName the name of the table
    */
   void preDeleteTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName)
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException;
 
   /**
@@ -128,7 +130,7 @@ public interface MasterObserver extends 
    * @param tableName the name of the table
    */
   void postDeleteTableHandler(
-      final ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] tableName)
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
       throws IOException;
 
   /**
@@ -140,7 +142,7 @@ public interface MasterObserver extends 
    * @param htd the HTableDescriptor
    */
   void preModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName, HTableDescriptor htd) throws IOException;
+      final TableName tableName, HTableDescriptor htd) throws IOException;
 
   /**
    * Called after the modifyTable operation has been requested.  Called as part
@@ -150,7 +152,7 @@ public interface MasterObserver extends 
    * @param htd the HTableDescriptor
    */
   void postModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName, HTableDescriptor htd) throws IOException;
+      final TableName tableName, HTableDescriptor htd) throws IOException;
 
   /**
    * Called prior to modifying a table's properties.  Called as part of modify
@@ -162,7 +164,7 @@ public interface MasterObserver extends 
    */
   void preModifyTableHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName, HTableDescriptor htd) throws IOException;
+      final TableName tableName, HTableDescriptor htd) throws IOException;
 
   /**
    * Called after to modifying a table's properties.  Called as part of modify
@@ -174,7 +176,7 @@ public interface MasterObserver extends 
    */
   void postModifyTableHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName, HTableDescriptor htd) throws IOException;
+      final TableName tableName, HTableDescriptor htd) throws IOException;
 
   /**
    * Called prior to adding a new column family to the table.  Called as part of
@@ -184,7 +186,7 @@ public interface MasterObserver extends 
    * @param column the HColumnDescriptor
    */
   void preAddColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor column) throws IOException;
+      TableName tableName, HColumnDescriptor column) throws IOException;
 
   /**
    * Called after the new column family has been created.  Called as part of
@@ -194,7 +196,7 @@ public interface MasterObserver extends 
    * @param column the HColumnDescriptor
    */
   void postAddColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor column) throws IOException;
+      TableName tableName, HColumnDescriptor column) throws IOException;
 
   /**
    * Called prior to adding a new column family to the table.  Called as part of
@@ -205,7 +207,7 @@ public interface MasterObserver extends 
    */
   void preAddColumnHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor column) throws IOException;
+      TableName tableName, HColumnDescriptor column) throws IOException;
 
   /**
    * Called after the new column family has been created.  Called as part of
@@ -216,7 +218,7 @@ public interface MasterObserver extends 
    */
   void postAddColumnHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor column) throws IOException;
+      TableName tableName, HColumnDescriptor column) throws IOException;
 
   /**
    * Called prior to modifying a column family's attributes.  Called as part of
@@ -226,7 +228,7 @@ public interface MasterObserver extends 
    * @param descriptor the HColumnDescriptor
    */
   void preModifyColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte [] tableName, HColumnDescriptor descriptor) throws IOException;
+      TableName tableName, HColumnDescriptor descriptor) throws IOException;
 
   /**
    * Called after the column family has been updated.  Called as part of modify
@@ -236,7 +238,7 @@ public interface MasterObserver extends 
    * @param descriptor the HColumnDescriptor
    */
   void postModifyColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor descriptor) throws IOException;
+      TableName tableName, HColumnDescriptor descriptor) throws IOException;
 
   /**
    * Called prior to modifying a column family's attributes.  Called as part of
@@ -247,7 +249,7 @@ public interface MasterObserver extends 
    */
   void preModifyColumnHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor descriptor) throws IOException;
+      TableName tableName, HColumnDescriptor descriptor) throws IOException;
 
   /**
    * Called after the column family has been updated.  Called as part of modify
@@ -258,7 +260,7 @@ public interface MasterObserver extends 
    */
   void postModifyColumnHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      byte[] tableName, HColumnDescriptor descriptor) throws IOException;
+      TableName tableName, HColumnDescriptor descriptor) throws IOException;
 
 
   /**
@@ -269,7 +271,7 @@ public interface MasterObserver extends 
    * @param c the column
    */
   void preDeleteColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte [] tableName, final byte[] c) throws IOException;
+      final TableName tableName, final byte[] c) throws IOException;
 
   /**
    * Called after the column family has been deleted.  Called as part of delete
@@ -279,7 +281,7 @@ public interface MasterObserver extends 
    * @param c the column
    */
   void postDeleteColumn(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte [] tableName, final byte[] c) throws IOException;
+      final TableName tableName, final byte[] c) throws IOException;
 
   /**
    * Called prior to deleting the entire column family.  Called as part of
@@ -290,7 +292,7 @@ public interface MasterObserver extends 
    */
   void preDeleteColumnHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName, final byte[] c) throws IOException;
+      final TableName tableName, final byte[] c) throws IOException;
 
   /**
    * Called after the column family has been deleted.  Called as part of
@@ -301,7 +303,7 @@ public interface MasterObserver extends 
    */
   void postDeleteColumnHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName, final byte[] c) throws IOException;
+      final TableName tableName, final byte[] c) throws IOException;
 
   /**
    * Called prior to enabling a table.  Called as part of enable table RPC call.
@@ -310,7 +312,7 @@ public interface MasterObserver extends 
    * @param tableName the name of the table
    */
   void preEnableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName) throws IOException;
+      final TableName tableName) throws IOException;
 
   /**
    * Called after the enableTable operation has been requested.  Called as part
@@ -319,7 +321,7 @@ public interface MasterObserver extends 
    * @param tableName the name of the table
    */
   void postEnableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName) throws IOException;
+      final TableName tableName) throws IOException;
 
   /**
    * Called prior to enabling a table.  Called as part of enable table handler
@@ -330,7 +332,7 @@ public interface MasterObserver extends 
    */
   void preEnableTableHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName) throws IOException;
+      final TableName tableName) throws IOException;
 
   /**
    * Called after the enableTable operation has been requested.  Called as part
@@ -340,7 +342,7 @@ public interface MasterObserver extends 
    */
   void postEnableTableHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName) throws IOException;
+      final TableName tableName) throws IOException;
 
   /**
    * Called prior to disabling a table.  Called as part of disable table RPC
@@ -350,7 +352,7 @@ public interface MasterObserver extends 
    * @param tableName the name of the table
    */
   void preDisableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName) throws IOException;
+      final TableName tableName) throws IOException;
 
   /**
    * Called after the disableTable operation has been requested.  Called as part
@@ -359,7 +361,7 @@ public interface MasterObserver extends 
    * @param tableName the name of the table
    */
   void postDisableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName) throws IOException;
+      final TableName tableName) throws IOException;
 
   /**
    * Called prior to disabling a table.  Called as part of disable table handler
@@ -370,7 +372,7 @@ public interface MasterObserver extends 
    */
   void preDisableTableHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName) throws IOException;
+      final TableName tableName) throws IOException;
 
   /**
    * Called after the disableTable operation has been requested.  Called as part
@@ -380,7 +382,7 @@ public interface MasterObserver extends 
    */
   void postDisableTableHandler(
       final ObserverContext<MasterCoprocessorEnvironment> ctx,
-      final byte[] tableName) throws IOException;
+      final TableName tableName) throws IOException;
 
   /**
    * Called prior to moving a given region from one region server to another.
@@ -619,7 +621,8 @@ public interface MasterObserver extends 
    * @throws IOException
    */
   void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
-      List<String> tableNamesList, List<HTableDescriptor> descriptors) throws IOException;
+      List<TableName> tableNamesList,
+      List<HTableDescriptor> descriptors) throws IOException;
 
   /**
    * Called after a getTableDescriptors request has been processed.
@@ -629,4 +632,58 @@ public interface MasterObserver extends 
    */
   void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
       List<HTableDescriptor> descriptors) throws IOException;
+
+  /**
+   * Called before a new namespace is created by
+   * {@link org.apache.hadoop.hbase.master.HMaster}.
+   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+   * @param ctx the environment to interact with the framework and master
+   * @param ns the NamespaceDescriptor for the table
+   * @throws IOException
+   */
+  void preCreateNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      NamespaceDescriptor ns) throws IOException;
+  /**
+   * Called after the createNamespace operation has been requested.
+   * @param ctx the environment to interact with the framework and master
+   * @param ns the NamespaceDescriptor for the table
+   * @throws IOException
+   */
+  void postCreateNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+       NamespaceDescriptor ns) throws IOException;
+
+  /**
+   * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
+   * namespace
+   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+   * @param ctx the environment to interact with the framework and master
+   * @param namespace the name of the namespace
+   */
+  void preDeleteNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String namespace) throws IOException;
+
+  /**
+   * Called after the deleteNamespace operation has been requested.
+   * @param ctx the environment to interact with the framework and master
+   * @param namespace the name of the namespace
+   */
+  void postDeleteNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      String namespace) throws IOException;
+
+  /**
+   * Called prior to modifying a namespace's properties.
+   * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+   * @param ctx the environment to interact with the framework and master
+   * @param ns the NamespaceDescriptor
+   */
+  void preModifyNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      NamespaceDescriptor ns) throws IOException;
+
+  /**
+   * Called after the modifyNamespace operation has been requested.
+   * @param ctx the environment to interact with the framework and master
+   * @param ns the NamespaceDescriptor
+   */
+  void postModifyNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+      NamespaceDescriptor ns) throws IOException;
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java Thu Aug  8 04:19:49 2013
@@ -262,7 +262,7 @@ policy implementations, perhaps) ahead o
     "TestClassloading.jar");
 
   // create a table that references the jar
-  HTableDescriptor htd = new HTableDescriptor(getClass().getName());
+  HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(getClass().getTableName()));
   htd.addFamily(new HColumnDescriptor("test"));
   htd.setValue("Coprocessor$1",
     path.toString() +

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HFileLink.java Thu Aug  8 04:19:49 2013
@@ -28,13 +28,14 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.hadoop.hbase.util.Pair;
 
 /**
  * HFileLink describes a link to an hfile.
@@ -66,12 +67,15 @@ public class HFileLink extends FileLink 
    * and the bulk loaded (_SeqId_[0-9]+_) hfiles.
    */
   public static final String LINK_NAME_REGEX =
-    String.format("%s=%s-%s", HTableDescriptor.VALID_USER_TABLE_REGEX,
+    String.format("(?:(?:%s=)?)%s=%s-%s",
+      TableName.VALID_NAMESPACE_REGEX, TableName.VALID_TABLE_QUALIFIER_REGEX,
       HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX);
 
   /** Define the HFile Link name parser in the form of: table=region-hfile */
-  private static final Pattern LINK_NAME_PATTERN =
-    Pattern.compile(String.format("^(%s)=(%s)-(%s)$", HTableDescriptor.VALID_USER_TABLE_REGEX,
+  //made package private for testing
+  static final Pattern LINK_NAME_PATTERN =
+    Pattern.compile(String.format("^(?:(%s)(?:\\=))?(%s)=(%s)-(%s)$",
+      TableName.VALID_NAMESPACE_REGEX, TableName.VALID_TABLE_QUALIFIER_REGEX,
       HRegionInfo.ENCODED_REGION_NAME_REGEX, StoreFileInfo.HFILE_NAME_REGEX));
 
   /**
@@ -79,7 +83,8 @@ public class HFileLink extends FileLink 
    * that can be found in /hbase/table/region/family/
    */
   private static final Pattern REF_OR_HFILE_LINK_PATTERN =
-    Pattern.compile(String.format("^(%s)=(%s)-(.+)$", HTableDescriptor.VALID_USER_TABLE_REGEX,
+    Pattern.compile(String.format("^(?:(%s)(?:=))?(%s)=(%s)-(.+)$",
+      TableName.VALID_NAMESPACE_REGEX, TableName.VALID_TABLE_QUALIFIER_REGEX,
       HRegionInfo.ENCODED_REGION_NAME_REGEX));
 
   private final Path archivePath;
@@ -138,8 +143,7 @@ public class HFileLink extends FileLink 
   public static boolean isHFileLink(String fileName) {
     Matcher m = LINK_NAME_PATTERN.matcher(fileName);
     if (!m.matches()) return false;
-
-    return m.groupCount() > 2 && m.group(3) != null && m.group(2) != null && m.group(1) != null;
+    return m.groupCount() > 2 && m.group(4) != null && m.group(3) != null && m.group(2) != null;
   }
 
   /**
@@ -159,11 +163,13 @@ public class HFileLink extends FileLink 
     }
 
     // Convert the HFileLink name into a real table/region/cf/hfile path.
-    String tableName = m.group(1);
-    String regionName = m.group(2);
-    String hfileName = m.group(3);
+    TableName tableName = TableName.valueOf(m.group(1), m.group(2));
+    String regionName = m.group(3);
+    String hfileName = m.group(4);
     String familyName = path.getParent().getName();
-    return new Path(new Path(tableName, regionName), new Path(familyName, hfileName));
+    Path tableDir = FSUtils.getTableDir(new Path("./"), tableName);
+    return new Path(tableDir, new Path(regionName, new Path(familyName,
+        hfileName)));
   }
 
   /**
@@ -177,7 +183,7 @@ public class HFileLink extends FileLink 
     if (!m.matches()) {
       throw new IllegalArgumentException(fileName + " is not a valid HFileLink name!");
     }
-    return(m.group(3));
+    return(m.group(4));
   }
 
   /**
@@ -191,7 +197,7 @@ public class HFileLink extends FileLink 
     if (!m.matches()) {
       throw new IllegalArgumentException(fileName + " is not a valid HFileLink name!");
     }
-    return(m.group(2));
+    return(m.group(3));
   }
 
   /**
@@ -200,12 +206,12 @@ public class HFileLink extends FileLink 
    * @param fileName HFileLink file name
    * @return the name of the referenced Table
    */
-  public static String getReferencedTableName(final String fileName) {
+  public static TableName getReferencedTableName(final String fileName) {
     Matcher m = REF_OR_HFILE_LINK_PATTERN.matcher(fileName);
     if (!m.matches()) {
       throw new IllegalArgumentException(fileName + " is not a valid HFileLink name!");
     }
-    return(m.group(1));
+    return(TableName.valueOf(m.group(1), m.group(2)));
   }
 
   /**
@@ -217,7 +223,7 @@ public class HFileLink extends FileLink 
    */
   public static String createHFileLinkName(final HRegionInfo hfileRegionInfo,
       final String hfileName) {
-    return createHFileLinkName(hfileRegionInfo.getTableNameAsString(),
+    return createHFileLinkName(hfileRegionInfo.getTableName(),
                       hfileRegionInfo.getEncodedName(), hfileName);
   }
 
@@ -229,9 +235,12 @@ public class HFileLink extends FileLink 
    * @param hfileName - Linked HFile name
    * @return file name of the HFile Link
    */
-  public static String createHFileLinkName(final String tableName,
+  public static String createHFileLinkName(final TableName tableName,
       final String regionName, final String hfileName) {
-    return String.format("%s=%s-%s", tableName, regionName, hfileName);
+    String s = String.format("%s=%s-%s",
+        tableName.getNameAsString().replace(TableName.NAMESPACE_DELIM, '='),
+        regionName, hfileName);
+    return s;
   }
 
   /**
@@ -251,7 +260,7 @@ public class HFileLink extends FileLink 
   public static boolean create(final Configuration conf, final FileSystem fs,
       final Path dstFamilyPath, final HRegionInfo hfileRegionInfo,
       final String hfileName) throws IOException {
-    String linkedTable = hfileRegionInfo.getTableNameAsString();
+    TableName linkedTable = hfileRegionInfo.getTableName();
     String linkedRegion = hfileRegionInfo.getEncodedName();
     return create(conf, fs, dstFamilyPath, linkedTable, linkedRegion, hfileName);
   }
@@ -272,11 +281,12 @@ public class HFileLink extends FileLink 
    * @throws IOException on file or parent directory creation failure
    */
   public static boolean create(final Configuration conf, final FileSystem fs,
-      final Path dstFamilyPath, final String linkedTable, final String linkedRegion,
+      final Path dstFamilyPath, final TableName linkedTable, final String linkedRegion,
       final String hfileName) throws IOException {
     String familyName = dstFamilyPath.getName();
     String regionName = dstFamilyPath.getParent().getName();
-    String tableName = dstFamilyPath.getParent().getParent().getName();
+    String tableName = FSUtils.getTableName(dstFamilyPath.getParent().getParent())
+        .getNameAsString();
 
     String name = createHFileLinkName(linkedTable, linkedRegion, hfileName);
     String refName = createBackReferenceName(tableName, regionName);
@@ -323,14 +333,18 @@ public class HFileLink extends FileLink 
     if (!m.matches()) {
       throw new IllegalArgumentException(hfileLinkName + " is not a valid HFileLink name!");
     }
-    return create(conf, fs, dstFamilyPath, m.group(1), m.group(2), m.group(3));
+    return create(conf, fs, dstFamilyPath, TableName.valueOf(m.group(1), m.group(2)),
+        m.group(3), m.group(4));
   }
 
   /**
    * Create the back reference name
    */
-  private static String createBackReferenceName(final String tableName, final String regionName) {
-    return regionName + "." + tableName;
+  //package-private for testing
+  static String createBackReferenceName(final String tableNameStr,
+                                        final String regionName) {
+
+    return regionName + "." + tableNameStr.replace(TableName.NAMESPACE_DELIM, '=');
   }
 
   /**
@@ -342,20 +356,31 @@ public class HFileLink extends FileLink 
    * @throws IOException on unexpected error.
    */
   public static Path getHFileFromBackReference(final Path rootDir, final Path linkRefPath) {
-    int separatorIndex = linkRefPath.getName().indexOf('.');
-    String linkRegionName = linkRefPath.getName().substring(0, separatorIndex);
-    String linkTableName = linkRefPath.getName().substring(separatorIndex + 1);
+    Pair<TableName, String> p = parseBackReferenceName(linkRefPath.getName());
+    TableName linkTableName = p.getFirst();
+    String linkRegionName = p.getSecond();
+
     String hfileName = getBackReferenceFileName(linkRefPath.getParent());
     Path familyPath = linkRefPath.getParent().getParent();
     Path regionPath = familyPath.getParent();
     Path tablePath = regionPath.getParent();
 
-    String linkName = createHFileLinkName(tablePath.getName(), regionPath.getName(), hfileName);
-    Path linkTableDir = FSUtils.getTablePath(rootDir, linkTableName);
+    String linkName = createHFileLinkName(FSUtils.getTableName(tablePath),
+        regionPath.getName(), hfileName);
+    Path linkTableDir = FSUtils.getTableDir(rootDir, linkTableName);
     Path regionDir = HRegion.getRegionDir(linkTableDir, linkRegionName);
     return new Path(new Path(regionDir, familyPath.getName()), linkName);
   }
 
+  static Pair<TableName, String> parseBackReferenceName(String name) {
+    int separatorIndex = name.indexOf('.');
+    String linkRegionName = name.substring(0, separatorIndex);
+    String tableSubstr = name.substring(separatorIndex + 1)
+        .replace('=', TableName.NAMESPACE_DELIM);
+    TableName linkTableName = TableName.valueOf(tableSubstr);
+    return new Pair<TableName, String>(linkTableName, linkRegionName);
+  }
+
   /**
    * Get the full path of the HFile referenced by the back reference
    *
@@ -368,4 +393,5 @@ public class HFileLink extends FileLink 
       throws IOException {
     return getHFileFromBackReference(FSUtils.getRootDir(conf), linkRefPath);
   }
+
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java Thu Aug  8 04:19:49 2013
@@ -38,6 +38,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
@@ -140,7 +141,7 @@ public class HFilePrettyPrinter {
       byte[] rn = Bytes.toBytes(regionName);
       byte[][] hri = HRegionInfo.parseRegionName(rn);
       Path rootDir = FSUtils.getRootDir(conf);
-      Path tableDir = new Path(rootDir, Bytes.toString(hri[0]));
+      Path tableDir = FSUtils.getTableDir(rootDir, TableName.valueOf(hri[0]));
       String enc = HRegionInfo.encodeRegionName(rn);
       Path regionDir = new Path(tableDir, enc);
       if (verbose)

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java Thu Aug  8 04:19:49 2013
@@ -66,6 +66,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -2138,9 +2139,9 @@ public class RpcServer implements RpcSer
         params[1] instanceof Operation) {
       // if the slow process is a query, we want to log its table as well
       // as its own fingerprint
-      byte [] tableName =
-          HRegionInfo.parseRegionName((byte[]) params[0])[0];
-      responseInfo.put("table", Bytes.toStringBinary(tableName));
+      TableName tableName = TableName.valueOf(
+          HRegionInfo.parseRegionName((byte[]) params[0])[0]);
+      responseInfo.put("table", tableName.getNameAsString());
       // annotate the response map with operation details
       responseInfo.putAll(((Operation) params[1]).toMap());
       // report to the log file

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java Thu Aug  8 04:19:49 2013
@@ -135,7 +135,7 @@ implements InputFormat<ImmutableBytesWri
       lastPos = startKeys.length % realNumSplits > i ? lastPos + 1 : lastPos;
       String regionLocation = table.getRegionLocation(startKeys[startPos]).
         getHostname();
-      splits[i] = new TableSplit(this.table.getTableName(),
+      splits[i] = new TableSplit(this.table.getName(),
         startKeys[startPos], ((i + 1) < realNumSplits) ? startKeys[lastPos]:
           HConstants.EMPTY_START_ROW, regionLocation);
       LOG.info("split: " + i + "->" + splits[i]);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSplit.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSplit.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSplit.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/TableSplit.java Thu Aug  8 04:19:49 2013
@@ -22,6 +22,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapred.InputSplit;
@@ -31,14 +32,14 @@ import org.apache.hadoop.mapred.InputSpl
  */
 @Deprecated
 public class TableSplit implements InputSplit, Comparable<TableSplit> {
-  private byte [] m_tableName;
+  private TableName m_tableName;
   private byte [] m_startRow;
   private byte [] m_endRow;
   private String m_regionLocation;
 
   /** default constructor */
   public TableSplit() {
-    this(HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
+    this((TableName)null, HConstants.EMPTY_BYTE_ARRAY,
       HConstants.EMPTY_BYTE_ARRAY, "");
   }
 
@@ -49,7 +50,7 @@ public class TableSplit implements Input
    * @param endRow
    * @param location
    */
-  public TableSplit(byte [] tableName, byte [] startRow, byte [] endRow,
+  public TableSplit(TableName tableName, byte [] startRow, byte [] endRow,
       final String location) {
     this.m_tableName = tableName;
     this.m_startRow = startRow;
@@ -57,11 +58,21 @@ public class TableSplit implements Input
     this.m_regionLocation = location;
   }
 
+  public TableSplit(byte [] tableName, byte [] startRow, byte [] endRow,
+      final String location) {
+    this(TableName.valueOf(tableName), startRow, endRow, location);
+  }
+
   /** @return table name */
-  public byte [] getTableName() {
+  public TableName getTable() {
     return this.m_tableName;
   }
 
+  /** @return table name */
+   public byte [] getTableName() {
+     return this.m_tableName.getName();
+   }
+
   /** @return starting row key */
   public byte [] getStartRow() {
     return this.m_startRow;
@@ -87,14 +98,14 @@ public class TableSplit implements Input
   }
 
   public void readFields(DataInput in) throws IOException {
-    this.m_tableName = Bytes.readByteArray(in);
+    this.m_tableName = TableName.valueOf(Bytes.readByteArray(in));
     this.m_startRow = Bytes.readByteArray(in);
     this.m_endRow = Bytes.readByteArray(in);
     this.m_regionLocation = Bytes.toString(Bytes.readByteArray(in));
   }
 
   public void write(DataOutput out) throws IOException {
-    Bytes.writeByteArray(out, this.m_tableName);
+    Bytes.writeByteArray(out, this.m_tableName.getName());
     Bytes.writeByteArray(out, this.m_startRow);
     Bytes.writeByteArray(out, this.m_endRow);
     Bytes.writeByteArray(out, Bytes.toBytes(this.m_regionLocation));
@@ -117,7 +128,7 @@ public class TableSplit implements Input
       return false;
     }
     TableSplit other = (TableSplit)o;
-    return Bytes.equals(m_tableName, other.m_tableName) &&
+    return m_tableName.equals(other.m_tableName) &&
       Bytes.equals(m_startRow, other.m_startRow) &&
       Bytes.equals(m_endRow, other.m_endRow) &&
       m_regionLocation.equals(other.m_regionLocation);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java Thu Aug  8 04:19:49 2013
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseConf
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
@@ -358,7 +359,7 @@ public class ImportTsv extends Configure
 
   private static void createTable(HBaseAdmin admin, String tableName, String[] columns)
       throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(tableName.getBytes());
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
     Set<String> cfSet = new HashSet<String>();
     for (String aColumn : columns) {
       if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)) continue;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java Thu Aug  8 04:19:49 2013
@@ -51,6 +51,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -61,7 +62,6 @@ import org.apache.hadoop.hbase.client.HB
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.RegionServerCallable;
-import org.apache.hadoop.hbase.client.RpcRetryingCaller;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.coprocessor.SecureBulkLoadClient;
 import org.apache.hadoop.hbase.io.HalfStoreFileReader;
@@ -200,7 +200,7 @@ public class LoadIncrementalHFiles exten
   {
     final HConnection conn = table.getConnection();
 
-    if (!conn.isTableAvailable(table.getTableName())) {
+    if (!conn.isTableAvailable(table.getName())) {
       throw new TableNotFoundException("Table " +
           Bytes.toStringBinary(table.getTableName()) +
           "is not currently available.");
@@ -261,7 +261,7 @@ public class LoadIncrementalHFiles exten
         if(User.isSecurityEnabled()) {
          userToken = fs.getDelegationToken("renewer");
         }
-        bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(table.getTableName());
+        bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(table.getName());
       }
 
       // Assumes that region splits can happen while this occurs.
@@ -339,7 +339,8 @@ public class LoadIncrementalHFiles exten
 
       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
         public List<LoadQueueItem> call() throws Exception {
-          List<LoadQueueItem> toRetry = tryAtomicRegionLoad(conn, table.getTableName(), first, lqis);
+          List<LoadQueueItem> toRetry =
+              tryAtomicRegionLoad(conn, table.getName(), first, lqis);
           return toRetry;
         }
       };
@@ -420,8 +421,8 @@ public class LoadIncrementalHFiles exten
   }
 
   // unique file name for the table
-  String getUniqueName(byte[] tableName) {
-    String name = Bytes.toStringBinary(tableName) + "," + regionCount.incrementAndGet();
+  String getUniqueName(TableName tableName) {
+    String name = tableName + "," + regionCount.incrementAndGet();
     return name;
   }
 
@@ -437,7 +438,7 @@ public class LoadIncrementalHFiles exten
     LOG.info("HFile at " + hfilePath + " no longer fits inside a single " +
     "region. Splitting...");
 
-    String uniqueName = getUniqueName(table.getTableName());
+    String uniqueName = getUniqueName(table.getName());
     HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
     Path botOut = new Path(tmpDir, uniqueName + ".bottom");
     Path topOut = new Path(tmpDir, uniqueName + ".top");
@@ -530,7 +531,8 @@ public class LoadIncrementalHFiles exten
    * failure
    */
   protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
-      byte[] tableName, final byte[] first, Collection<LoadQueueItem> lqis) throws IOException {
+      final TableName tableName,
+      final byte[] first, Collection<LoadQueueItem> lqis) throws IOException {
 
     final List<Pair<byte[], String>> famPaths =
       new ArrayList<Pair<byte[], String>>(lqis.size());
@@ -595,7 +597,7 @@ public class LoadIncrementalHFiles exten
       if (!success) {
         LOG.warn("Attempt to bulk load region containing "
             + Bytes.toStringBinary(first) + " into table "
-            + Bytes.toStringBinary(tableName)  + " with files " + lqis
+            + tableName  + " with files " + lqis
             + " failed.  This is recoverable and they will be retried.");
         toRetry.addAll(lqis); // return lqi's to retry
       }
@@ -678,7 +680,7 @@ public class LoadIncrementalHFiles exten
     return !HFile.isReservedFileInfoKey(key);
   }
 
-  private boolean doesTableExist(String tableName) throws Exception {
+  private boolean doesTableExist(TableName tableName) throws Exception {
     return hbAdmin.tableExists(tableName);
   }
 
@@ -716,7 +718,7 @@ public class LoadIncrementalHFiles exten
    * If the table is created for the first time, then "completebulkload" reads the files twice.
    * More modifications necessary if we want to avoid doing it.
    */
-  private void createTable(String tableName, String dirPath) throws Exception {
+  private void createTable(TableName tableName, String dirPath) throws Exception {
     Path hfofDir = new Path(dirPath);
     FileSystem fs = hfofDir.getFileSystem(getConf());
 
@@ -797,7 +799,7 @@ public class LoadIncrementalHFiles exten
     }
 
     String dirPath   = args[0];
-    String tableName = args[1];
+    TableName tableName = TableName.valueOf(args[1]);
 
     boolean tableExists   = this.doesTableExist(tableName);
     if (!tableExists) this.createTable(tableName,dirPath);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableInputFormatBase.java Thu Aug  8 04:19:49 2013
@@ -152,7 +152,8 @@ public abstract class MultiTableInputFor
                     stopRow) <= 0) && keys.getSecond()[i].length > 0 ? keys
                     .getSecond()[i] : stopRow;
             InputSplit split =
-                new TableSplit(tableName, scan, splitStart, splitStop, regionLocation);
+                new TableSplit(table.getName(),
+                    scan, splitStart, splitStop, regionLocation);
             splits.add(split);
             if (LOG.isDebugEnabled())
               LOG.debug("getSplits: split -> " + (count++) + " -> " + split);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java Thu Aug  8 04:19:49 2013
@@ -166,7 +166,7 @@ extends InputFormat<ImmutableBytesWritab
         throw new IOException("Expecting at least one region.");
       }
       List<InputSplit> splits = new ArrayList<InputSplit>(1);
-      InputSplit split = new TableSplit(table.getTableName(),
+      InputSplit split = new TableSplit(table.getName(),
           HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, regLoc
               .getHostnamePort().split(Addressing.HOSTNAME_PORT_SEPARATOR)[0]);
       splits.add(split);
@@ -206,7 +206,7 @@ extends InputFormat<ImmutableBytesWritab
           Bytes.compareTo(keys.getSecond()[i], stopRow) <= 0) &&
           keys.getSecond()[i].length > 0 ?
             keys.getSecond()[i] : stopRow;
-        InputSplit split = new TableSplit(table.getTableName(),
+        InputSplit split = new TableSplit(table.getName(),
           splitStart, splitStop, regionLocation);
         splits.add(split);
         if (LOG.isDebugEnabled()) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSplit.java Thu Aug  8 04:19:49 2013
@@ -27,6 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -76,7 +77,7 @@ implements Writable, Comparable<TableSpl
   }
   
   private static final Version VERSION = Version.INITIAL;
-  private byte [] tableName;
+  private TableName tableName;
   private byte [] startRow;
   private byte [] endRow;
   private String regionLocation;
@@ -84,7 +85,7 @@ implements Writable, Comparable<TableSpl
 
   /** Default constructor. */
   public TableSplit() {
-    this(HConstants.EMPTY_BYTE_ARRAY, null, HConstants.EMPTY_BYTE_ARRAY,
+    this(null, null, HConstants.EMPTY_BYTE_ARRAY,
       HConstants.EMPTY_BYTE_ARRAY, "");
   }
 
@@ -97,7 +98,7 @@ implements Writable, Comparable<TableSpl
    * @param endRow  The end row of the split.
    * @param location  The location of the region.
    */
-  public TableSplit(byte [] tableName, Scan scan, byte [] startRow, byte [] endRow,
+  public TableSplit(TableName tableName, Scan scan, byte [] startRow, byte [] endRow,
       final String location) {
     this.tableName = tableName;
     try {
@@ -119,7 +120,7 @@ implements Writable, Comparable<TableSpl
    * @param endRow The end row of the split.
    * @param location The location of the region.
    */
-  public TableSplit(byte[] tableName, byte[] startRow, byte[] endRow,
+  public TableSplit(TableName tableName, byte[] startRow, byte[] endRow,
       final String location) {
     this(tableName, null, startRow, endRow, location);
   }
@@ -139,7 +140,7 @@ implements Writable, Comparable<TableSpl
    *
    * @return The table name.
    */
-  public byte [] getTableName() {
+  public TableName getTableName() {
     return tableName;
   }
 
@@ -216,8 +217,9 @@ implements Writable, Comparable<TableSpl
       version = Version.fromCode(len);
       len = WritableUtils.readVInt(in);
     }
-    tableName = new byte[len];
-    in.readFully(tableName);
+    byte[] tableNameBytes = new byte[len];
+    in.readFully(tableNameBytes);
+    tableName = TableName.valueOf(tableNameBytes);
     startRow = Bytes.readByteArray(in);
     endRow = Bytes.readByteArray(in);
     regionLocation = Bytes.toString(Bytes.readByteArray(in));
@@ -235,7 +237,7 @@ implements Writable, Comparable<TableSpl
   @Override
   public void write(DataOutput out) throws IOException {
     WritableUtils.writeVInt(out, VERSION.code);
-    Bytes.writeByteArray(out, tableName);
+    Bytes.writeByteArray(out, tableName.getName());
     Bytes.writeByteArray(out, startRow);
     Bytes.writeByteArray(out, endRow);
     Bytes.writeByteArray(out, Bytes.toBytes(regionLocation));
@@ -266,7 +268,7 @@ implements Writable, Comparable<TableSpl
     // If The table name of the two splits is the same then compare start row
     // otherwise compare based on table names
     int tableNameComparison =
-        Bytes.compareTo(getTableName(), split.getTableName());
+        getTableName().compareTo(split.getTableName());
     return tableNameComparison != 0 ? tableNameComparison : Bytes.compareTo(
         getStartRow(), split.getStartRow());
   }
@@ -276,7 +278,7 @@ implements Writable, Comparable<TableSpl
     if (o == null || !(o instanceof TableSplit)) {
       return false;
     }
-    return Bytes.equals(tableName, ((TableSplit)o).tableName) &&
+    return tableName.equals(((TableSplit)o).tableName) &&
       Bytes.equals(startRow, ((TableSplit)o).startRow) &&
       Bytes.equals(endRow, ((TableSplit)o).endRow) &&
       regionLocation.equals(((TableSplit)o).regionLocation);
@@ -284,7 +286,7 @@ implements Writable, Comparable<TableSpl
 
     @Override
     public int hashCode() {
-        int result = tableName != null ? Arrays.hashCode(tableName) : 0;
+        int result = tableName != null ? tableName.hashCode() : 0;
         result = 31 * result + (scan != null ? scan.hashCode() : 0);
         result = 31 * result + (startRow != null ? Arrays.hashCode(startRow) : 0);
         result = 31 * result + (endRow != null ? Arrays.hashCode(endRow) : 0);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java Thu Aug  8 04:19:49 2013
@@ -28,6 +28,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Delete;
@@ -79,7 +80,7 @@ public class WALPlayer extends Configure
     throws IOException {
       try {
         // skip all other tables
-        if (Bytes.equals(table, key.getTablename())) {
+        if (Bytes.equals(table, key.getTablename().getName())) {
           for (KeyValue kv : value.getKeyValues()) {
             if (WALEdit.isMetaEditFamily(kv.getFamily())) continue;
             context.write(new ImmutableBytesWritable(kv.getRow()), kv);
@@ -108,7 +109,8 @@ public class WALPlayer extends Configure
    */
   static class HLogMapper
   extends Mapper<HLogKey, WALEdit, ImmutableBytesWritable, Mutation> {
-    private Map<byte[], byte[]> tables = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
+    private Map<TableName, TableName> tables =
+        new TreeMap<TableName, TableName>();
 
     @Override
     public void map(HLogKey key, WALEdit value,
@@ -116,10 +118,10 @@ public class WALPlayer extends Configure
     throws IOException {
       try {
         if (tables.isEmpty() || tables.containsKey(key.getTablename())) {
-          byte[] targetTable = tables.isEmpty() ?
+          TableName targetTable = tables.isEmpty() ?
                 key.getTablename() :
                 tables.get(key.getTablename());
-          ImmutableBytesWritable tableOut = new ImmutableBytesWritable(targetTable);
+          ImmutableBytesWritable tableOut = new ImmutableBytesWritable(targetTable.getName());
           Put put = null;
           Delete del = null;
           KeyValue lastKV = null;
@@ -168,7 +170,8 @@ public class WALPlayer extends Configure
       }
       int i = 0;
       for (String table : tablesToUse) {
-        tables.put(Bytes.toBytes(table), Bytes.toBytes(tableMap[i++]));
+        tables.put(TableName.valueOf(table),
+            TableName.valueOf(tableMap[i++]));
       }
     }
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Thu Aug  8 04:19:49 2013
@@ -39,11 +39,15 @@ import java.util.concurrent.atomic.Atomi
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Chore;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.RegionTransition;
@@ -53,7 +57,6 @@ import org.apache.hadoop.hbase.Stoppable
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
@@ -89,7 +92,6 @@ import org.apache.zookeeper.KeeperExcept
 import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.apache.zookeeper.data.Stat;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.LinkedHashMultimap;
 
 /**
@@ -355,7 +357,7 @@ public class AssignmentManager extends Z
    * @return Pair indicating the status of the alter command
    * @throws IOException
    */
-  public Pair<Integer, Integer> getReopenStatus(byte[] tableName)
+  public Pair<Integer, Integer> getReopenStatus(TableName tableName)
       throws IOException {
     List <HRegionInfo> hris =
       MetaReader.getTableRegions(this.server.getCatalogTracker(), tableName, true);
@@ -450,7 +452,8 @@ public class AssignmentManager extends Z
       // its a clean cluster startup, else its a failover.
       Map<HRegionInfo, ServerName> regions = regionStates.getRegionAssignments();
       for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
-        if (!e.getKey().isMetaTable() && e.getValue() != null) {
+        if (!HTableDescriptor.isSystemTable(e.getKey().getTableName())
+            && e.getValue() != null) {
           LOG.debug("Found " + e + " out on cluster");
           failover = true;
           break;
@@ -1259,7 +1262,7 @@ public class AssignmentManager extends Z
               LOG.info("The master has opened "
                 + regionNameStr + " that was online on " + serverName);
               boolean disabled = getZKTable().isDisablingOrDisabledTable(
-                regionInfo.getTableNameAsString());
+                regionInfo.getTableName());
               if (!serverManager.isServerOnline(serverName) && !disabled) {
                 LOG.info("Opened " + regionNameStr
                   + "but the region server is offline, reassign the region");
@@ -1863,7 +1866,7 @@ public class AssignmentManager extends Z
           // When we have a case such as all the regions are added directly into .META. and we call
           // assignRegion then we need to make the table ENABLED. Hence in such case the table
           // will not be in ENABLING or ENABLED state.
-          String tableName = region.getTableNameAsString();
+          TableName tableName = region.getTableName();
           if (!zkTable.isEnablingTable(tableName) && !zkTable.isEnabledTable(tableName)) {
             LOG.debug("Setting table " + tableName + " to ENABLED state.");
             setEnabledTable(tableName);
@@ -2043,7 +2046,7 @@ public class AssignmentManager extends Z
   }
 
   private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
-    String tableName = region.getTableNameAsString();
+    TableName tableName = region.getTableName();
     boolean disabled = this.zkTable.isDisabledTable(tableName);
     if (disabled || this.zkTable.isDisablingTable(tableName)) {
       LOG.info("Table " + tableName + (disabled ? " disabled;" : " disabling;") +
@@ -2520,10 +2523,10 @@ public class AssignmentManager extends Z
     // Skip assignment for regions of tables in DISABLING state because during clean cluster startup
     // no RS is alive and regions map also doesn't have any information about the regions.
     // See HBASE-6281.
-    Set<String> disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(watcher);
+    Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(watcher);
     disabledOrDisablingOrEnabling.addAll(ZKTable.getEnablingTables(watcher));
     // Scan META for all user regions, skipping any disabled tables
-    Map<HRegionInfo, ServerName> allRegions = null;
+    Map<HRegionInfo, ServerName> allRegions;
     if (this.shouldAssignRegionsWithFavoredNodes) {
       allRegions = FavoredNodeAssignmentHelper.fullScan(
         catalogTracker, disabledOrDisablingOrEnabling, true, (FavoredNodeLoadBalancer)balancer);
@@ -2531,7 +2534,18 @@ public class AssignmentManager extends Z
       allRegions = MetaReader.fullScan(
         catalogTracker, disabledOrDisablingOrEnabling, true);
     }
-    if (allRegions == null || allRegions.isEmpty()) return;
+
+    if (allRegions == null) return;
+
+    //remove system tables because they would have been assigned earlier
+    for(Iterator<HRegionInfo> iter = allRegions.keySet().iterator();
+        iter.hasNext();) {
+      if (HTableDescriptor.isSystemTable(iter.next().getTableName())) {
+        iter.remove();
+      }
+    }
+
+    if (allRegions.isEmpty()) return;
 
     // Determine what type of assignment to do on startup
     boolean retainAssignment = server.getConfiguration().
@@ -2545,7 +2559,7 @@ public class AssignmentManager extends Z
     }
 
     for (HRegionInfo hri : allRegions.keySet()) {
-      String tableName = hri.getTableNameAsString();
+      TableName tableName = hri.getTableName();
       if (!zkTable.isEnabledTable(tableName)) {
         setEnabledTable(tableName);
       }
@@ -2586,10 +2600,10 @@ public class AssignmentManager extends Z
    * @throws IOException
    */
   Map<ServerName, List<HRegionInfo>> rebuildUserRegions() throws IOException, KeeperException {
-    Set<String> enablingTables = ZKTable.getEnablingTables(watcher);
-    Set<String> disabledOrEnablingTables = ZKTable.getDisabledTables(watcher);
+    Set<TableName> enablingTables = ZKTable.getEnablingTables(watcher);
+    Set<TableName> disabledOrEnablingTables = ZKTable.getDisabledTables(watcher);
     disabledOrEnablingTables.addAll(enablingTables);
-    Set<String> disabledOrDisablingOrEnabling = ZKTable.getDisablingTables(watcher);
+    Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisablingTables(watcher);
     disabledOrDisablingOrEnabling.addAll(disabledOrEnablingTables);
 
     // Region assignment from META
@@ -2607,7 +2621,7 @@ public class AssignmentManager extends Z
       ServerName regionLocation = region.getSecond();
       if (regionInfo == null) continue;
       regionStates.createRegionState(regionInfo);
-      String tableName = regionInfo.getTableNameAsString();
+      TableName tableName = regionInfo.getTableName();
       if (regionLocation == null) {
         // regionLocation could be null if createTable didn't finish properly.
         // When createTable is in progress, HMaster restarts.
@@ -2678,14 +2692,14 @@ public class AssignmentManager extends Z
    */
   private void recoverTableInDisablingState()
       throws KeeperException, TableNotFoundException, IOException {
-    Set<String> disablingTables = ZKTable.getDisablingTables(watcher);
+    Set<TableName> disablingTables = ZKTable.getDisablingTables(watcher);
     if (disablingTables.size() != 0) {
-      for (String tableName : disablingTables) {
+      for (TableName tableName : disablingTables) {
         // Recover by calling DisableTableHandler
         LOG.info("The table " + tableName
             + " is in DISABLING state.  Hence recovering by moving the table"
             + " to DISABLED state.");
-        new DisableTableHandler(this.server, tableName.getBytes(), catalogTracker,
+        new DisableTableHandler(this.server, tableName, catalogTracker,
             this, tableLockManager, true).prepare().process();
       }
     }
@@ -2701,16 +2715,16 @@ public class AssignmentManager extends Z
    */
   private void recoverTableInEnablingState()
       throws KeeperException, TableNotFoundException, IOException {
-    Set<String> enablingTables = ZKTable.getEnablingTables(watcher);
+    Set<TableName> enablingTables = ZKTable.getEnablingTables(watcher);
     if (enablingTables.size() != 0) {
-      for (String tableName : enablingTables) {
+      for (TableName tableName : enablingTables) {
         // Recover by calling EnableTableHandler
         LOG.info("The table " + tableName
             + " is in ENABLING state.  Hence recovering by moving the table"
             + " to ENABLED state.");
         // enableTable in sync way during master startup,
         // no need to invoke coprocessor
-        new EnableTableHandler(this.server, tableName.getBytes(),
+        new EnableTableHandler(this.server, tableName,
             catalogTracker, this, tableLockManager, true).prepare().process();
       }
     }
@@ -3109,8 +3123,7 @@ public class AssignmentManager extends Z
           } catch (KeeperException ke) {
             server.abort("Unexpected ZK exception deleting node " + hri, ke);
           }
-
-          if (zkTable.isDisablingOrDisabledTable(hri.getTableNameAsString())) {
+          if (zkTable.isDisablingOrDisabledTable(hri.getTableName())) {
             it.remove();
             regionStates.regionOffline(hri);
             continue;
@@ -3143,7 +3156,7 @@ public class AssignmentManager extends Z
     // that case. This is not racing with the region server itself since RS
     // report is done after the split transaction completed.
     if (this.zkTable.isDisablingOrDisabledTable(
-        parent.getTableNameAsString())) {
+        parent.getTableName())) {
       unassign(a);
       unassign(b);
     }
@@ -3166,7 +3179,7 @@ public class AssignmentManager extends Z
     // the master to disable, we need to make sure we close those regions in
     // that case. This is not racing with the region server itself since RS
     // report is done after the regions merge transaction completed.
-    if (this.zkTable.isDisablingOrDisabledTable(merged.getTableNameAsString())) {
+    if (this.zkTable.isDisablingOrDisabledTable(merged.getTableName())) {
       unassign(merged);
     }
   }
@@ -3200,7 +3213,7 @@ public class AssignmentManager extends Z
     zkEventWorkers.shutdownNow();
   }
 
-  protected void setEnabledTable(String tableName) {
+  protected void setEnabledTable(TableName tableName) {
     try {
       this.zkTable.setEnabledTable(tableName);
     } catch (KeeperException e) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java?rev=1511577&r1=1511576&r2=1511577&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java Thu Aug  8 04:19:49 2013
@@ -33,6 +33,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Chore;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.client.Me
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.PairOfSameType;
 import org.apache.hadoop.hbase.util.Triple;
@@ -128,8 +130,8 @@ public class CatalogJanitor extends Chor
    * @throws IOException
    */
   Triple<Integer, Map<HRegionInfo, Result>, Map<HRegionInfo, Result>> getMergedRegionsAndSplitParents(
-      final byte[] tableName) throws IOException {
-    final boolean isTableSpecified = (tableName != null && tableName.length != 0);
+      final TableName tableName) throws IOException {
+    final boolean isTableSpecified = (tableName != null);
     // TODO: Only works with single .META. region currently.  Fix.
     final AtomicInteger count = new AtomicInteger(0);
     // Keep Map of found split parents.  There are candidates for cleanup.
@@ -147,7 +149,7 @@ public class CatalogJanitor extends Chor
         HRegionInfo info = HRegionInfo.getHRegionInfo(r);
         if (info == null) return true; // Keep scanning
         if (isTableSpecified
-            && Bytes.compareTo(info.getTableName(), tableName) > 0) {
+            && info.getTableName().compareTo(tableName) > 0) {
           // Another table, stop scanning
           return false;
         }
@@ -182,10 +184,9 @@ public class CatalogJanitor extends Chor
       final HRegionInfo regionA, final HRegionInfo regionB) throws IOException {
     FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
     Path rootdir = this.services.getMasterFileSystem().getRootDir();
-    Path tabledir = HTableDescriptor.getTableDir(rootdir,
+    Path tabledir = FSUtils.getTableDir(rootdir,
         mergedRegion.getTableName());
-    HTableDescriptor htd = getTableDescriptor(mergedRegion
-        .getTableNameAsString());
+    HTableDescriptor htd = getTableDescriptor(mergedRegion.getTableName());
     HRegionFileSystem regionFs = null;
     try {
       regionFs = HRegionFileSystem.openRegionFromFileSystem(
@@ -289,7 +290,7 @@ public class CatalogJanitor extends Chor
       if (left == null) return -1;
       if (right == null) return 1;
       // Same table name.
-      int result = Bytes.compareTo(left.getTableName(),
+      int result = left.getTableName().compareTo(
           right.getTableName());
       if (result != 0) return result;
       // Compare start keys.
@@ -374,7 +375,7 @@ public class CatalogJanitor extends Chor
 
     FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
     Path rootdir = this.services.getMasterFileSystem().getRootDir();
-    Path tabledir = HTableDescriptor.getTableDir(rootdir, daughter.getTableName());
+    Path tabledir = FSUtils.getTableDir(rootdir, daughter.getTableName());
 
     HRegionFileSystem regionFs = null;
     try {
@@ -386,7 +387,7 @@ public class CatalogJanitor extends Chor
     }
 
     boolean references = false;
-    HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTableNameAsString());
+    HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTableName());
     for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
       if ((references = regionFs.hasReferences(family.getNameAsString()))) {
         break;
@@ -395,7 +396,7 @@ public class CatalogJanitor extends Chor
     return new Pair<Boolean, Boolean>(Boolean.TRUE, Boolean.valueOf(references));
   }
 
-  private HTableDescriptor getTableDescriptor(final String tableName)
+  private HTableDescriptor getTableDescriptor(final TableName tableName)
       throws FileNotFoundException, IOException {
     return this.services.getTableDescriptors().get(tableName);
   }