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 08:08:31 UTC

svn commit: r1511591 [11/23] - in /hbase/branches/0.95: 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/jav...

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/CreateTableHandler.java Thu Aug  8 06:08:23 2013
@@ -28,6 +28,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.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
@@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.master.Ma
 import org.apache.hadoop.hbase.master.TableLockManager;
 import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.apache.zookeeper.KeeperException;
 
@@ -77,7 +79,7 @@ public class CreateTableHandler extends 
     this.assignmentManager = masterServices.getAssignmentManager();
     this.tableLockManager = masterServices.getTableLockManager();
 
-    this.tableLock = this.tableLockManager.writeLock(this.hTableDescriptor.getName()
+    this.tableLock = this.tableLockManager.writeLock(this.hTableDescriptor.getTableName()
         , EventType.C_M_CREATE_TABLE.toString());
   }
 
@@ -100,7 +102,7 @@ public class CreateTableHandler extends 
     this.tableLock.acquire();
     boolean success = false;
     try {
-      String tableName = this.hTableDescriptor.getNameAsString();
+      TableName tableName = this.hTableDescriptor.getTableName();
       if (MetaReader.tableExists(catalogTracker, tableName)) {
         throw new TableExistsException(tableName);
       }
@@ -137,12 +139,12 @@ public class CreateTableHandler extends 
       name = server.getServerName().toString();
     }
     return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" +
-      this.hTableDescriptor.getNameAsString();
+      this.hTableDescriptor.getTableName();
   }
 
   @Override
   public void process() {
-    String tableName = this.hTableDescriptor.getNameAsString();
+    TableName tableName = this.hTableDescriptor.getTableName();
     LOG.info("Create table " + tableName);
 
     try {
@@ -174,11 +176,11 @@ public class CreateTableHandler extends 
       // It will block the creation saying TableAlreadyExists.
       try {
         this.assignmentManager.getZKTable().removeEnablingTable(
-            this.hTableDescriptor.getNameAsString(), false);
+            this.hTableDescriptor.getTableName(), false);
       } catch (KeeperException e) {
         // Keeper exception should not happen here
         LOG.error("Got a keeper exception while removing the ENABLING table znode "
-            + this.hTableDescriptor.getNameAsString(), e);
+            + this.hTableDescriptor.getTableName(), e);
       }
     }
   }
@@ -197,19 +199,19 @@ public class CreateTableHandler extends 
    *   [If something fails here: we still have the table in disabled state]
    * - Update ZooKeeper with the enabled state
    */
-  private void handleCreateTable(String tableName) throws IOException, KeeperException {
+  private void handleCreateTable(TableName tableName)
+      throws IOException, KeeperException {
     Path tempdir = fileSystemManager.getTempDir();
     FileSystem fs = fileSystemManager.getFileSystem();
 
     // 1. Create Table Descriptor
-    Path tempTableDir = new Path(tempdir, tableName);
+    Path tempTableDir = FSUtils.getTableDir(tempdir, tableName);
     new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory(
       tempTableDir, this.hTableDescriptor, false);
-    Path tableDir = new Path(fileSystemManager.getRootDir(), tableName);
+    Path tableDir = FSUtils.getTableDir(fileSystemManager.getRootDir(), tableName);
 
     // 2. Create Regions
     List<HRegionInfo> regionInfos = handleCreateHdfsRegions(tempdir, tableName);
-
     // 3. Move Table temp directory to the hbase root location
     if (!fs.rename(tempTableDir, tableDir)) {
       throw new IOException("Unable to move table from temp=" + tempTableDir +
@@ -258,7 +260,7 @@ public class CreateTableHandler extends 
    * @return the list of regions created
    */
   protected List<HRegionInfo> handleCreateHdfsRegions(final Path tableRootDir,
-    final String tableName)
+    final TableName tableName)
       throws IOException {
     return ModifyRegionUtils.createRegions(conf, tableRootDir,
         hTableDescriptor, newRegions, null);

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java Thu Aug  8 06:08:23 2013
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
@@ -46,7 +47,7 @@ import org.apache.zookeeper.KeeperExcept
 public class DeleteTableHandler extends TableEventHandler {
   private static final Log LOG = LogFactory.getLog(DeleteTableHandler.class);
 
-  public DeleteTableHandler(byte [] tableName, Server server,
+  public DeleteTableHandler(TableName tableName, Server server,
       final MasterServices masterServices) {
     super(EventType.C_M_DELETE_TABLE, tableName, server, masterServices);
   }
@@ -111,16 +112,15 @@ public class DeleteTableHandler extends 
         LOG.error("Couldn't delete " + tempTableDir);
       }
 
-      LOG.debug("Table '" + Bytes.toString(tableName) + "' archived!");
+      LOG.debug("Table '" + tableName + "' archived!");
     } finally {
-      String tableNameStr = Bytes.toString(tableName);
       // 6. Update table descriptor cache
-      LOG.debug("Removing '" + tableNameStr + "' descriptor.");
-      this.masterServices.getTableDescriptors().remove(Bytes.toString(tableName));
+      LOG.debug("Removing '" + tableName + "' descriptor.");
+      this.masterServices.getTableDescriptors().remove(tableName);
 
       // 7. If entry for this table in zk, and up in AssignmentManager, remove it.
-      LOG.debug("Marking '" + tableNameStr + "' as deleted.");
-      am.getZKTable().setDeletedTable(tableNameStr);
+      LOG.debug("Marking '" + tableName + "' as deleted.");
+      am.getZKTable().setDeletedTable(tableName);
     }
 
     if (cpHost != null) {
@@ -144,6 +144,6 @@ public class DeleteTableHandler extends 
     if(server != null && server.getServerName() != null) {
       name = server.getServerName().toString();
     }
-    return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" + tableNameStr;
+    return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" + tableName;
   }
 }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java Thu Aug  8 06:08:23 2013
@@ -25,12 +25,14 @@ import java.util.concurrent.ExecutorServ
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
 import org.apache.hadoop.hbase.catalog.MetaReader;
+import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.master.AssignmentManager;
@@ -41,7 +43,6 @@ import org.apache.hadoop.hbase.master.Re
 import org.apache.hadoop.hbase.master.TableLockManager;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.zookeeper.KeeperException;
 import org.cloudera.htrace.Trace;
 
@@ -51,20 +52,18 @@ import org.cloudera.htrace.Trace;
 @InterfaceAudience.Private
 public class DisableTableHandler extends EventHandler {
   private static final Log LOG = LogFactory.getLog(DisableTableHandler.class);
-  private final byte [] tableName;
-  private final String tableNameStr;
+  private final TableName tableName;
   private final AssignmentManager assignmentManager;
   private final TableLockManager tableLockManager;
   private final CatalogTracker catalogTracker;
   private final boolean skipTableStateCheck;
   private TableLock tableLock;
 
-  public DisableTableHandler(Server server, byte [] tableName,
+  public DisableTableHandler(Server server, TableName tableName,
       CatalogTracker catalogTracker, AssignmentManager assignmentManager,
       TableLockManager tableLockManager, boolean skipTableStateCheck) {
     super(server, EventType.C_M_DISABLE_TABLE);
     this.tableName = tableName;
-    this.tableNameStr = Bytes.toString(this.tableName);
     this.assignmentManager = assignmentManager;
     this.catalogTracker = catalogTracker;
     this.tableLockManager = tableLockManager;
@@ -73,6 +72,9 @@ public class DisableTableHandler extends
 
   public DisableTableHandler prepare()
       throws TableNotFoundException, TableNotEnabledException, IOException {
+    if(tableName.equals(TableName.META_TABLE_NAME)) {
+      throw new ConstraintException("Cannot disable catalog table");
+    }
     //acquire the table write lock, blocking
     this.tableLock = this.tableLockManager.writeLock(tableName,
         EventType.C_M_DISABLE_TABLE.toString());
@@ -81,8 +83,8 @@ public class DisableTableHandler extends
     boolean success = false;
     try {
       // Check if table exists
-      if (!MetaReader.tableExists(catalogTracker, this.tableNameStr)) {
-        throw new TableNotFoundException(this.tableNameStr);
+      if (!MetaReader.tableExists(catalogTracker, tableName)) {
+        throw new TableNotFoundException(tableName);
       }
 
       // There could be multiple client requests trying to disable or enable
@@ -93,9 +95,9 @@ public class DisableTableHandler extends
       if (!skipTableStateCheck) {
         try {
           if (!this.assignmentManager.getZKTable().checkEnabledAndSetDisablingTable
-            (this.tableNameStr)) {
-            LOG.info("Table " + tableNameStr + " isn't enabled; skipping disable");
-            throw new TableNotEnabledException(this.tableNameStr);
+            (this.tableName)) {
+            LOG.info("Table " + tableName + " isn't enabled; skipping disable");
+            throw new TableNotEnabledException(this.tableName);
           }
         } catch (KeeperException e) {
           throw new IOException("Unable to ensure that the table will be" +
@@ -119,13 +121,13 @@ public class DisableTableHandler extends
       name = server.getServerName().toString();
     }
     return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" +
-      tableNameStr;
+        tableName;
   }
 
   @Override
   public void process() {
     try {
-      LOG.info("Attempting to disable table " + this.tableNameStr);
+      LOG.info("Attempting to disable table " + this.tableName);
       MasterCoprocessorHost cpHost = ((HMaster) this.server)
           .getCoprocessorHost();
       if (cpHost != null) {
@@ -136,9 +138,9 @@ public class DisableTableHandler extends
         cpHost.postDisableTableHandler(this.tableName);
       }
     } catch (IOException e) {
-      LOG.error("Error trying to disable table " + this.tableNameStr, e);
+      LOG.error("Error trying to disable table " + this.tableName, e);
     } catch (KeeperException e) {
-      LOG.error("Error trying to disable table " + this.tableNameStr, e);
+      LOG.error("Error trying to disable table " + this.tableName, e);
     } finally {
       releaseTableLock();
     }
@@ -156,7 +158,7 @@ public class DisableTableHandler extends
 
   private void handleDisableTable() throws IOException, KeeperException {
     // Set table disabling flag up in zk.
-    this.assignmentManager.getZKTable().setDisablingTable(this.tableNameStr);
+    this.assignmentManager.getZKTable().setDisablingTable(this.tableName);
     boolean done = false;
     while (true) {
       // Get list of online regions that are of this table.  Regions that are
@@ -184,7 +186,7 @@ public class DisableTableHandler extends
       }
     }
     // Flip the table to disabled if success.
-    if (done) this.assignmentManager.getZKTable().setDisabledTable(this.tableNameStr);
+    if (done) this.assignmentManager.getZKTable().setDisabledTable(this.tableName);
     LOG.info("Disabled table is done=" + done);
   }
 

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java Thu Aug  8 06:08:23 2013
@@ -26,6 +26,7 @@ import java.util.concurrent.ExecutorServ
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
@@ -44,7 +45,6 @@ import org.apache.hadoop.hbase.master.Re
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.master.TableLockManager;
 import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.zookeeper.KeeperException;
 import org.cloudera.htrace.Trace;
@@ -55,20 +55,18 @@ import org.cloudera.htrace.Trace;
 @InterfaceAudience.Private
 public class EnableTableHandler extends EventHandler {
   private static final Log LOG = LogFactory.getLog(EnableTableHandler.class);
-  private final byte [] tableName;
-  private final String tableNameStr;
+  private final TableName tableName;
   private final AssignmentManager assignmentManager;
   private final TableLockManager tableLockManager;
   private final CatalogTracker catalogTracker;
   private boolean retainAssignment = false;
   private TableLock tableLock;
 
-  public EnableTableHandler(Server server, byte [] tableName,
+  public EnableTableHandler(Server server, TableName tableName,
       CatalogTracker catalogTracker, AssignmentManager assignmentManager,
       TableLockManager tableLockManager, boolean skipTableStateCheck) {
     super(server, EventType.C_M_ENABLE_TABLE);
     this.tableName = tableName;
-    this.tableNameStr = Bytes.toString(tableName);
     this.catalogTracker = catalogTracker;
     this.assignmentManager = assignmentManager;
     this.tableLockManager = tableLockManager;
@@ -85,16 +83,16 @@ public class EnableTableHandler extends 
     boolean success = false;
     try {
       // Check if table exists
-      if (!MetaReader.tableExists(catalogTracker, this.tableNameStr)) {
+      if (!MetaReader.tableExists(catalogTracker, tableName)) {
         // retainAssignment is true only during recovery.  In normal case it is false
         if (!this.retainAssignment) {
-          throw new TableNotFoundException(tableNameStr);
+          throw new TableNotFoundException(tableName);
         } 
         try {
-          this.assignmentManager.getZKTable().removeEnablingTable(tableNameStr, true);
+          this.assignmentManager.getZKTable().removeEnablingTable(tableName, true);
         } catch (KeeperException e) {
           // TODO : Use HBCK to clear such nodes
-          LOG.warn("Failed to delete the ENABLING node for the table " + tableNameStr
+          LOG.warn("Failed to delete the ENABLING node for the table " + tableName
               + ".  The table will remain unusable. Run HBCK to manually fix the problem.");
         }
       }
@@ -106,9 +104,9 @@ public class EnableTableHandler extends 
       if (!retainAssignment) {
         try {
           if (!this.assignmentManager.getZKTable().checkDisabledAndSetEnablingTable
-            (this.tableNameStr)) {
-            LOG.info("Table " + tableNameStr + " isn't disabled; skipping enable");
-            throw new TableNotDisabledException(this.tableNameStr);
+            (this.tableName)) {
+            LOG.info("Table " + tableName + " isn't disabled; skipping enable");
+            throw new TableNotDisabledException(this.tableName);
           }
         } catch (KeeperException e) {
           throw new IOException("Unable to ensure that the table will be" +
@@ -131,13 +129,13 @@ public class EnableTableHandler extends 
       name = server.getServerName().toString();
     }
     return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" +
-      tableNameStr;
+        tableName;
   }
 
   @Override
   public void process() {
     try {
-      LOG.info("Attempting to enable the table " + this.tableNameStr);
+      LOG.info("Attempting to enable the table " + this.tableName);
       MasterCoprocessorHost cpHost = ((HMaster) this.server)
           .getCoprocessorHost();
       if (cpHost != null) {
@@ -148,11 +146,11 @@ public class EnableTableHandler extends 
         cpHost.postEnableTableHandler(this.tableName);
       }
     } catch (IOException e) {
-      LOG.error("Error trying to enable the table " + this.tableNameStr, e);
+      LOG.error("Error trying to enable the table " + this.tableName, e);
     } catch (KeeperException e) {
-      LOG.error("Error trying to enable the table " + this.tableNameStr, e);
+      LOG.error("Error trying to enable the table " + this.tableName, e);
     } catch (InterruptedException e) {
-      LOG.error("Error trying to enable the table " + this.tableNameStr, e);
+      LOG.error("Error trying to enable the table " + this.tableName, e);
     } finally {
       releaseTableLock();
     }
@@ -173,7 +171,7 @@ public class EnableTableHandler extends 
     // that user first finish disabling but that might be obnoxious.
 
     // Set table enabling flag up in zk.
-    this.assignmentManager.getZKTable().setEnablingTable(this.tableNameStr);
+    this.assignmentManager.getZKTable().setEnablingTable(this.tableName);
     boolean done = false;
     // Get the regions of this table. We're done when all listed
     // tables are onlined.
@@ -185,7 +183,7 @@ public class EnableTableHandler extends 
     if (regionsCount == 0) {
       done = true;
     }
-    LOG.info("Table '" + this.tableNameStr + "' has " + countOfRegionsInTable
+    LOG.info("Table '" + this.tableName + "' has " + countOfRegionsInTable
       + " regions, of which " + regionsCount + " are offline.");
     BulkEnabler bd = new BulkEnabler(this.server, regions, countOfRegionsInTable,
         this.retainAssignment);
@@ -195,18 +193,18 @@ public class EnableTableHandler extends 
       }
     } catch (InterruptedException e) {
       LOG.warn("Enable operation was interrupted when enabling table '"
-        + this.tableNameStr + "'");
+        + this.tableName + "'");
       // Preserve the interrupt.
       Thread.currentThread().interrupt();
     }
     if (done) {
       // Flip the table to enabled.
       this.assignmentManager.getZKTable().setEnabledTable(
-        this.tableNameStr);
-      LOG.info("Table '" + this.tableNameStr
+        this.tableName);
+      LOG.info("Table '" + this.tableName
       + "' was successfully enabled. Status: done=" + done);
     } else {
-      LOG.warn("Table '" + this.tableNameStr
+      LOG.warn("Table '" + this.tableName
       + "' wasn't successfully enabled. Status: done=" + done);
     }
   }
@@ -232,7 +230,7 @@ public class EnableTableHandler extends 
       } else {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Skipping assign for the region " + hri + " during enable table "
-              + hri.getTableNameAsString() + " because its already in tranition or assigned.");
+              + hri.getTableName() + " because its already in tranition or assigned.");
         }
       }
     }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ModifyTableHandler.java Thu Aug  8 06:08:23 2013
@@ -25,6 +25,7 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
@@ -41,7 +42,7 @@ public class ModifyTableHandler extends 
 
   private final HTableDescriptor htd;
 
-  public ModifyTableHandler(final byte [] tableName,
+  public ModifyTableHandler(final TableName tableName,
       final HTableDescriptor htd, final Server server,
       final MasterServices masterServices) {
     super(EventType.C_M_MODIFY_TABLE, tableName, server, masterServices);
@@ -82,7 +83,7 @@ public class ModifyTableHandler extends 
       for (byte[] familyName: oldFamilies) {
         if (!newFamilies.contains(familyName)) {
           LOG.debug("Removing family=" + Bytes.toString(familyName) +
-                    " from table=" + this.tableName);
+                     " from table=" + this.tableName);
           for (HRegionInfo hri: hris) {
             // Delete the family directory in FS for all the regions one by one
             mfs.deleteFamilyFromFS(hri, familyName);
@@ -101,6 +102,6 @@ public class ModifyTableHandler extends 
       name = server.getServerName().toString();
     }
     return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" +
-      tableNameStr;
+      tableName;
   }
 }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java Thu Aug  8 06:08:23 2013
@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.executor.EventHandler;
@@ -46,7 +47,8 @@ public class OpenedRegionHandler extends
 
   private enum OpenedPriority {
     META (1),
-    USER (2);
+    SYSTEM (2),
+    USER (3);
 
     private final int value;
     OpenedPriority(int value) {
@@ -67,6 +69,9 @@ public class OpenedRegionHandler extends
     this.expectedVersion = expectedVersion;
     if(regionInfo.isMetaRegion()) {
       priority = OpenedPriority.META;
+    } else if(regionInfo.getTableName()
+        .getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
+      priority = OpenedPriority.SYSTEM;
     } else {
       priority = OpenedPriority.USER;
     }
@@ -109,7 +114,7 @@ public class OpenedRegionHandler extends
     }
     if (!openedNodeDeleted) {
       if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable(
-          regionInfo.getTableNameAsString())) {
+          regionInfo.getTableName())) {
         debugLog(regionInfo, "Opened region "
             + regionInfo.getShortNameToLog() + " but "
             + "this table is disabled, triggering close of region");

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java Thu Aug  8 06:08:23 2013
@@ -244,7 +244,7 @@ public class ServerShutdownHandler exten
             toAssignRegions.add(hri);
           } else if (rit != null) {
             if ((rit.isClosing() || rit.isPendingClose())
-                && am.getZKTable().isDisablingOrDisabledTable(hri.getTableNameAsString())) {
+                && am.getZKTable().isDisablingOrDisabledTable(hri.getTableName())) {
               // If the table was partially disabled and the RS went down, we should clear the RIT
               // and remove the node for the region.
               // The rit that we use may be stale in case the table was in DISABLING state
@@ -330,17 +330,17 @@ public class ServerShutdownHandler exten
       AssignmentManager assignmentManager, CatalogTracker catalogTracker)
   throws IOException {
     boolean tablePresent = assignmentManager.getZKTable().isTablePresent(
-        hri.getTableNameAsString());
+        hri.getTableName());
     if (!tablePresent) {
-      LOG.info("The table " + hri.getTableNameAsString()
+      LOG.info("The table " + hri.getTableName()
           + " was deleted.  Hence not proceeding.");
       return false;
     }
     // If table is not disabled but the region is offlined,
     boolean disabled = assignmentManager.getZKTable().isDisabledTable(
-        hri.getTableNameAsString());
+        hri.getTableName());
     if (disabled){
-      LOG.info("The table " + hri.getTableNameAsString()
+      LOG.info("The table " + hri.getTableName()
           + " was disabled.  Hence not proceeding.");
       return false;
     }
@@ -351,9 +351,9 @@ public class ServerShutdownHandler exten
       return false;
     }
     boolean disabling = assignmentManager.getZKTable().isDisablingTable(
-        hri.getTableNameAsString());
+        hri.getTableName());
     if (disabling) {
-      LOG.info("The table " + hri.getTableNameAsString()
+      LOG.info("The table " + hri.getTableName()
           + " is disabled.  Hence not assigning region" + hri.getEncodedName());
       return false;
     }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableAddFamilyHandler.java Thu Aug  8 06:08:23 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -40,7 +41,7 @@ public class TableAddFamilyHandler exten
 
   private final HColumnDescriptor familyDesc;
 
-  public TableAddFamilyHandler(byte[] tableName, HColumnDescriptor familyDesc,
+  public TableAddFamilyHandler(TableName tableName, HColumnDescriptor familyDesc,
       Server server, final MasterServices masterServices) {
     super(EventType.C_M_ADD_FAMILY, tableName, server, masterServices);
     this.familyDesc = familyDesc;
@@ -82,7 +83,7 @@ public class TableAddFamilyHandler exten
       family = familyDesc.getNameAsString();
     }
     return getClass().getSimpleName() + "-" + name + "-" +
-                              getSeqid() + "-" + tableNameStr + "-" + family;
+                              getSeqid() + "-" + tableName + "-" + family;
   }
 
 }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableDeleteFamilyHandler.java Thu Aug  8 06:08:23 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
@@ -40,7 +41,7 @@ public class TableDeleteFamilyHandler ex
 
   private byte [] familyName;
 
-  public TableDeleteFamilyHandler(byte[] tableName, byte [] familyName,
+  public TableDeleteFamilyHandler(TableName tableName, byte [] familyName,
       Server server, final MasterServices masterServices) throws IOException {
     super(EventType.C_M_DELETE_FAMILY, tableName, server, masterServices);
     this.familyName = familyName;
@@ -83,6 +84,7 @@ public class TableDeleteFamilyHandler ex
     if(familyName != null) {
       family = Bytes.toString(familyName);
     }
-    return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" + tableNameStr + "-" + family;
+    return getClass().getSimpleName() + "-" + name + "-" + getSeqid() +
+        "-" + tableName + "-" + family;
   }
 }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableEventHandler.java Thu Aug  8 06:08:23 2013
@@ -29,6 +29,7 @@ import java.util.TreeMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
@@ -60,17 +61,15 @@ import com.google.common.collect.Maps;
 public abstract class TableEventHandler extends EventHandler {
   private static final Log LOG = LogFactory.getLog(TableEventHandler.class);
   protected final MasterServices masterServices;
-  protected final byte [] tableName;
-  protected final String tableNameStr;
+  protected final TableName tableName;
   protected TableLock tableLock;
   private boolean isPrepareCalled = false;
 
-  public TableEventHandler(EventType eventType, byte [] tableName, Server server,
+  public TableEventHandler(EventType eventType, TableName tableName, Server server,
       MasterServices masterServices) {
     super(server, eventType);
     this.masterServices = masterServices;
     this.tableName = tableName;
-    this.tableNameStr = Bytes.toString(this.tableName);
   }
 
   public TableEventHandler prepare() throws IOException {
@@ -122,7 +121,7 @@ public abstract class TableEventHandler 
     }
     try {
       LOG.info("Handling table operation " + eventType + " on table " +
-          Bytes.toString(tableName));
+          tableName);
 
       List<HRegionInfo> hris =
         MetaReader.getTableRegions(this.server.getCatalogTracker(),
@@ -130,20 +129,20 @@ public abstract class TableEventHandler 
       handleTableOperation(hris);
       if (eventType.isOnlineSchemaChangeSupported() && this.masterServices.
           getAssignmentManager().getZKTable().
-          isEnabledTable(Bytes.toString(tableName))) {
+          isEnabledTable(tableName)) {
         if (reOpenAllRegions(hris)) {
           LOG.info("Completed table operation " + eventType + " on table " +
-              Bytes.toString(tableName));
+              tableName);
         } else {
           LOG.warn("Error on reopening the regions");
         }
       }
       completed(null);
     } catch (IOException e) {
-      LOG.error("Error manipulating table " + Bytes.toString(tableName), e);
+      LOG.error("Error manipulating table " + tableName, e);
       completed(e);
     } catch (KeeperException e) {
-      LOG.error("Error manipulating table " + Bytes.toString(tableName), e);
+      LOG.error("Error manipulating table " + tableName, e);
       completed(e);
     } finally {
       releaseTableLock();
@@ -226,11 +225,10 @@ public abstract class TableEventHandler 
    */
   public HTableDescriptor getTableDescriptor()
   throws FileNotFoundException, IOException {
-    final String name = Bytes.toString(tableName);
     HTableDescriptor htd =
-      this.masterServices.getTableDescriptors().get(name);
+      this.masterServices.getTableDescriptors().get(tableName);
     if (htd == null) {
-      throw new IOException("HTableDescriptor missing for " + name);
+      throw new IOException("HTableDescriptor missing for " + tableName);
     }
     return htd;
   }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TableModifyFamilyHandler.java Thu Aug  8 06:08:23 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -38,7 +39,7 @@ import org.apache.hadoop.hbase.master.Ma
 public class TableModifyFamilyHandler extends TableEventHandler {
   private final HColumnDescriptor familyDesc;
 
-  public TableModifyFamilyHandler(byte[] tableName,
+  public TableModifyFamilyHandler(TableName tableName,
       HColumnDescriptor familyDesc, Server server,
       final MasterServices masterServices) {
     super(EventType.C_M_MODIFY_FAMILY, tableName, server, masterServices);
@@ -76,7 +77,8 @@ public class TableModifyFamilyHandler ex
     if(familyDesc != null) {
       family = familyDesc.getNameAsString();
     }
-    return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" + tableNameStr + "-" + family;
+    return getClass().getSimpleName() + "-" + name + "-" + getSeqid() +
+        "-" + tableName + "-" + family;
   }
 
 }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/CloneSnapshotHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/CloneSnapshotHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/CloneSnapshotHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/CloneSnapshotHandler.java Thu Aug  8 06:08:23 2013
@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
@@ -47,6 +48,7 @@ import org.apache.hadoop.hbase.snapshot.
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
  * Handler to Clone a snapshot.
@@ -81,7 +83,7 @@ public class CloneSnapshotHandler extend
     // Monitor
     this.monitor = new ForeignExceptionDispatcher();
     this.status = TaskMonitor.get().createStatus("Cloning  snapshot '" + snapshot.getName() +
-      "' to table " + hTableDescriptor.getNameAsString());
+      "' to table " + hTableDescriptor.getTableName());
   }
 
   @Override
@@ -97,17 +99,16 @@ public class CloneSnapshotHandler extend
    */
   @Override
   protected List<HRegionInfo> handleCreateHdfsRegions(final Path tableRootDir,
-      final String tableName) throws IOException {
+      final TableName tableName) throws IOException {
     status.setStatus("Creating regions for table: " + tableName);
     FileSystem fs = fileSystemManager.getFileSystem();
     Path rootDir = fileSystemManager.getRootDir();
-    Path tableDir = new Path(tableRootDir, tableName);
 
     try {
       // 1. Execute the on-disk Clone
       Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
       RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(conf, fs,
-          snapshot, snapshotDir, hTableDescriptor, tableDir, monitor, status);
+          snapshot, snapshotDir, hTableDescriptor, tableRootDir, monitor, status);
       RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
 
       // Clone operation should not have stuff to restore or remove

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java Thu Aug  8 06:08:23 2013
@@ -115,13 +115,13 @@ public class DisabledTableSnapshotHandle
         new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call();
         monitor.rethrowException();
         status.setStatus("Completed copying recovered edits for offline snapshot of table: "
-            + snapshot.getTable());
+            + snapshotTable);
 
         // 2.3 reference all the files in the region
         new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call();
         monitor.rethrowException();
         status.setStatus("Completed referencing HFiles for offline snapshot of table: " +
-          snapshot.getTable());
+            snapshotTable);
       }
 
       // 3. write the table info to disk
@@ -131,14 +131,16 @@ public class DisabledTableSnapshotHandle
           FSUtils.getRootDir(conf));
       tableInfoCopyTask.call();
       monitor.rethrowException();
-      status.setStatus("Finished copying tableinfo for snapshot of table: " + snapshot.getTable());
+      status.setStatus("Finished copying tableinfo for snapshot of table: " +
+          snapshotTable);
     } catch (Exception e) {
       // make sure we capture the exception to propagate back to the client later
       String reason = "Failed snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
           + " due to exception:" + e.getMessage();
       ForeignException ee = new ForeignException(reason, e);
       monitor.receive(ee);
-      status.abort("Snapshot of table: "+ snapshot.getTable() +" failed because " + e.getMessage());
+      status.abort("Snapshot of table: "+ snapshotTable +
+          " failed because " + e.getMessage());
     } finally {
       LOG.debug("Marking snapshot" + ClientSnapshotDescriptionUtils.toString(snapshot)
           + " as finished.");

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java Thu Aug  8 06:08:23 2013
@@ -23,23 +23,22 @@ import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.catalog.MetaReader;
 import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
@@ -79,7 +78,7 @@ public final class MasterSnapshotVerifie
   private SnapshotDescription snapshot;
   private FileSystem fs;
   private Path rootDir;
-  private String tableName;
+  private TableName tableName;
   private MasterServices services;
 
   /**
@@ -92,7 +91,7 @@ public final class MasterSnapshotVerifie
     this.services = services;
     this.snapshot = snapshot;
     this.rootDir = rootDir;
-    this.tableName = snapshot.getTable();
+    this.tableName = TableName.valueOf(snapshot.getTable());
   }
 
   /**
@@ -141,7 +140,7 @@ public final class MasterSnapshotVerifie
    */
   private void verifyRegions(Path snapshotDir) throws IOException {
     List<HRegionInfo> regions = MetaReader.getTableRegions(this.services.getCatalogTracker(),
-      Bytes.toBytes(tableName));
+        tableName);
     for (HRegionInfo region : regions) {
       // if offline split parent, skip it
       if (region.isOffline() && (region.isSplit() || region.isSplitParent())) {
@@ -189,7 +188,7 @@ public final class MasterSnapshotVerifie
     if (columnFamilies == null) return;
 
     // setup the suffixes for the snapshot directories
-    Path tableNameSuffix = new Path(tableName);
+    Path tableNameSuffix = FSUtils.getTableDir(new Path("./"), tableName);
     Path regionNameSuffix = new Path(tableNameSuffix, region.getEncodedName());
 
     // get the potential real paths

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/RestoreSnapshotHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/RestoreSnapshotHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/RestoreSnapshotHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/RestoreSnapshotHandler.java Thu Aug  8 06:08:23 2013
@@ -28,6 +28,7 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
@@ -47,7 +48,7 @@ import org.apache.hadoop.hbase.snapshot.
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
-import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 
 /**
  * Handler to Restore a snapshot.
@@ -71,7 +72,7 @@ public class RestoreSnapshotHandler exte
   public RestoreSnapshotHandler(final MasterServices masterServices,
       final SnapshotDescription snapshot, final HTableDescriptor htd,
       final MetricsMaster metricsMaster) throws IOException {
-    super(EventType.C_M_RESTORE_SNAPSHOT, htd.getName(), masterServices, masterServices);
+    super(EventType.C_M_RESTORE_SNAPSHOT, htd.getTableName(), masterServices, masterServices);
     this.metricsMaster = metricsMaster;
 
     // Snapshot information
@@ -88,7 +89,7 @@ public class RestoreSnapshotHandler exte
 
     this.status = TaskMonitor.get().createStatus(
       "Restoring  snapshot '" + snapshot.getName() + "' to table "
-          + hTableDescriptor.getNameAsString());
+          + hTableDescriptor.getTableName());
   }
 
   public RestoreSnapshotHandler prepare() throws IOException {
@@ -109,8 +110,7 @@ public class RestoreSnapshotHandler exte
     CatalogTracker catalogTracker = masterServices.getCatalogTracker();
     FileSystem fs = fileSystemManager.getFileSystem();
     Path rootDir = fileSystemManager.getRootDir();
-    byte[] tableName = hTableDescriptor.getName();
-    Path tableDir = HTableDescriptor.getTableDir(rootDir, tableName);
+    TableName tableName = hTableDescriptor.getTableName();
 
     try {
       // 1. Update descriptor
@@ -121,7 +121,7 @@ public class RestoreSnapshotHandler exte
       Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
       RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(
           masterServices.getConfiguration(), fs,
-          snapshot, snapshotDir, hTableDescriptor, tableDir, monitor, status);
+          snapshot, snapshotDir, hTableDescriptor, rootDir, monitor, status);
       RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
 
       // 3. Applies changes to .META.
@@ -134,7 +134,7 @@ public class RestoreSnapshotHandler exte
 
       // At this point the restore is complete. Next step is enabling the table.
       LOG.info("Restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +
-        " on table=" + Bytes.toString(tableName) + " completed!");
+        " on table=" + tableName + " completed!");
     } catch (IOException e) {
       String msg = "restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot)
           + " failed. Try re-running the restore command.";

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java Thu Aug  8 06:08:23 2013
@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FSDataInputS
 import org.apache.hadoop.fs.FileStatus;
 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.HTableDescriptor;
 import org.apache.hadoop.hbase.Stoppable;
@@ -56,6 +57,7 @@ import org.apache.hadoop.hbase.procedure
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
 import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs;
 import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinatorRpcs;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
@@ -68,7 +70,6 @@ import org.apache.hadoop.hbase.snapshot.
 import org.apache.hadoop.hbase.snapshot.SnapshotExistsException;
 import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException;
 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -143,13 +144,15 @@ public class SnapshotManager implements 
   // The map is always accessed and modified under the object lock using synchronized.
   // snapshotTable() will insert an Handler in the table.
   // isSnapshotDone() will remove the handler requested if the operation is finished.
-  private Map<String, SnapshotSentinel> snapshotHandlers = new HashMap<String, SnapshotSentinel>();
+  private Map<TableName, SnapshotSentinel> snapshotHandlers =
+      new HashMap<TableName, SnapshotSentinel>();
 
   // Restore Sentinels map, with table name as key.
   // The map is always accessed and modified under the object lock using synchronized.
   // restoreSnapshot()/cloneSnapshot() will insert an Handler in the table.
   // isRestoreDone() will remove the handler requested if the operation is finished.
-  private Map<String, SnapshotSentinel> restoreHandlers = new HashMap<String, SnapshotSentinel>();
+  private Map<TableName, SnapshotSentinel> restoreHandlers =
+      new HashMap<TableName, SnapshotSentinel>();
 
   private final Path rootDir;
   private final ExecutorService executorService;
@@ -378,7 +381,7 @@ public class SnapshotManager implements 
    * @param tableName name of the table being snapshotted.
    * @return <tt>true</tt> if there is a snapshot in progress on the specified table.
    */
-  synchronized boolean isTakingSnapshot(final String tableName) {
+  synchronized boolean isTakingSnapshot(final TableName tableName) {
     SnapshotSentinel handler = this.snapshotHandlers.get(tableName);
     return handler != null && !handler.isFinished();
   }
@@ -393,10 +396,12 @@ public class SnapshotManager implements 
       throws HBaseSnapshotException {
     FileSystem fs = master.getMasterFileSystem().getFileSystem();
     Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
+    TableName snapshotTable =
+        TableName.valueOf(snapshot.getTable());
 
     // make sure we aren't already running a snapshot
-    if (isTakingSnapshot(snapshot.getTable())) {
-      SnapshotSentinel handler = this.snapshotHandlers.get(snapshot.getTable());
+    if (isTakingSnapshot(snapshotTable)) {
+      SnapshotSentinel handler = this.snapshotHandlers.get(snapshotTable);
       throw new SnapshotCreationException("Rejected taking "
           + ClientSnapshotDescriptionUtils.toString(snapshot)
           + " because we are already running another snapshot "
@@ -404,8 +409,8 @@ public class SnapshotManager implements 
     }
 
     // make sure we aren't running a restore on the same table
-    if (isRestoringTable(snapshot.getTable())) {
-      SnapshotSentinel handler = restoreHandlers.get(snapshot.getTable());
+    if (isRestoringTable(snapshotTable)) {
+      SnapshotSentinel handler = restoreHandlers.get(snapshotTable);
       throw new SnapshotCreationException("Rejected taking "
           + ClientSnapshotDescriptionUtils.toString(snapshot)
           + " because we are already have a restore in progress on the same snapshot "
@@ -478,7 +483,7 @@ public class SnapshotManager implements 
     try {
       handler.prepare();
       this.executorService.submit(handler);
-      this.snapshotHandlers.put(snapshot.getTable(), handler);
+      this.snapshotHandlers.put(TableName.valueOf(snapshot.getTable()), handler);
     } catch (Exception e) {
       // cleanup the working directory by trying to delete it from the fs.
       Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
@@ -518,7 +523,8 @@ public class SnapshotManager implements 
     // check to see if the table exists
     HTableDescriptor desc = null;
     try {
-      desc = master.getTableDescriptors().get(snapshot.getTable());
+      desc = master.getTableDescriptors().get(
+          TableName.valueOf(snapshot.getTable()));
     } catch (FileNotFoundException e) {
       String msg = "Table:" + snapshot.getTable() + " info doesn't exist!";
       LOG.error(msg);
@@ -543,14 +549,15 @@ public class SnapshotManager implements 
     }
 
     // if the table is enabled, then have the RS run actually the snapshot work
+    TableName snapshotTable = TableName.valueOf(snapshot.getTable());
     AssignmentManager assignmentMgr = master.getAssignmentManager();
-    if (assignmentMgr.getZKTable().isEnabledTable(snapshot.getTable())) {
+    if (assignmentMgr.getZKTable().isEnabledTable(snapshotTable)) {
       LOG.debug("Table enabled, starting distributed snapshot.");
       snapshotEnabledTable(snapshot);
       LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
     }
     // For disabled table, snapshot is created by the master
-    else if (assignmentMgr.getZKTable().isDisabledTable(snapshot.getTable())) {
+    else if (assignmentMgr.getZKTable().isDisabledTable(snapshotTable)) {
       LOG.debug("Table is disabled, running snapshot entirely on master.");
       snapshotDisabledTable(snapshot);
       LOG.debug("Started snapshot: " + ClientSnapshotDescriptionUtils.toString(snapshot));
@@ -577,7 +584,8 @@ public class SnapshotManager implements 
    *
    * TODO get rid of this if possible, repackaging, modify tests.
    */
-  public synchronized void setSnapshotHandlerForTesting(final String tableName,
+  public synchronized void setSnapshotHandlerForTesting(
+      final TableName tableName,
       final SnapshotSentinel handler) {
     if (handler != null) {
       this.snapshotHandlers.put(tableName, handler);
@@ -607,7 +615,6 @@ public class SnapshotManager implements 
     try {
       final Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
       FileSystem fs = master.getMasterFileSystem().getFileSystem();
-
       // check to see if the snapshot already exists
       return fs.exists(snapshotDir);
     } catch (IllegalArgumentException iae) {
@@ -624,7 +631,7 @@ public class SnapshotManager implements 
    */
   synchronized void cloneSnapshot(final SnapshotDescription snapshot,
       final HTableDescriptor hTableDescriptor) throws HBaseSnapshotException {
-    String tableName = hTableDescriptor.getNameAsString();
+    TableName tableName = hTableDescriptor.getTableName();
 
     // make sure we aren't running a snapshot on the same table
     if (isTakingSnapshot(tableName)) {
@@ -669,16 +676,19 @@ public class SnapshotManager implements 
     SnapshotDescription fsSnapshot = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
     HTableDescriptor snapshotTableDesc =
         FSTableDescriptors.getTableDescriptorFromFs(fs, snapshotDir);
-    String tableName = reqSnapshot.getTable();
+    TableName tableName = TableName.valueOf(reqSnapshot.getTable());
 
     // stop tracking "abandoned" handlers
     cleanupSentinels();
 
     // Execute the restore/clone operation
     if (MetaReader.tableExists(master.getCatalogTracker(), tableName)) {
-      if (master.getAssignmentManager().getZKTable().isEnabledTable(fsSnapshot.getTable())) {
+      if (master.getAssignmentManager().getZKTable().isEnabledTable(
+          TableName.valueOf(fsSnapshot.getTable()))) {
         throw new UnsupportedOperationException("Table '" +
-          fsSnapshot.getTable() + "' must be disabled in order to perform a restore operation.");
+            TableName.valueOf(fsSnapshot.getTable()) + "' must be disabled in order to " +
+            "perform a restore operation" +
+            ".");
       }
 
       // call coproc pre hook
@@ -692,8 +702,7 @@ public class SnapshotManager implements 
         cpHost.postRestoreSnapshot(reqSnapshot, snapshotTableDesc);
       }
     } else {
-      HTableDescriptor htd = RestoreSnapshotHelper.cloneTableSchema(snapshotTableDesc,
-                                                         Bytes.toBytes(tableName));
+      HTableDescriptor htd = RestoreSnapshotHelper.cloneTableSchema(snapshotTableDesc, tableName);
       if (cpHost != null) {
         cpHost.preCloneSnapshot(reqSnapshot, htd);
       }
@@ -715,7 +724,7 @@ public class SnapshotManager implements 
    */
   private synchronized void restoreSnapshot(final SnapshotDescription snapshot,
       final HTableDescriptor hTableDescriptor) throws HBaseSnapshotException {
-    String tableName = hTableDescriptor.getNameAsString();
+    TableName tableName = hTableDescriptor.getTableName();
 
     // make sure we aren't running a snapshot on the same table
     if (isTakingSnapshot(tableName)) {
@@ -731,7 +740,7 @@ public class SnapshotManager implements 
       RestoreSnapshotHandler handler =
         new RestoreSnapshotHandler(master, snapshot, hTableDescriptor, metricsMaster).prepare();
       this.executorService.submit(handler);
-      restoreHandlers.put(hTableDescriptor.getNameAsString(), handler);
+      restoreHandlers.put(tableName, handler);
     } catch (Exception e) {
       String msg = "Couldn't restore the snapshot=" + ClientSnapshotDescriptionUtils.toString(
           snapshot)  +
@@ -747,7 +756,7 @@ public class SnapshotManager implements 
    * @param tableName table under restore
    * @return <tt>true</tt> if there is a restore in progress of the specified table.
    */
-  private synchronized boolean isRestoringTable(final String tableName) {
+  private synchronized boolean isRestoringTable(final TableName tableName) {
     SnapshotSentinel sentinel = this.restoreHandlers.get(tableName);
     return(sentinel != null && !sentinel.isFinished());
   }
@@ -774,7 +783,8 @@ public class SnapshotManager implements 
     }
 
     LOG.debug("Verify snapshot=" + snapshot.getName() + " against="
-        + sentinel.getSnapshot().getName() + " table=" + snapshot.getTable());
+        + sentinel.getSnapshot().getName() + " table=" +
+        TableName.valueOf(snapshot.getTable()));
 
     // If the restore is failed, rethrow the exception
     sentinel.rethrowExceptionIfFailed();
@@ -801,8 +811,14 @@ public class SnapshotManager implements 
    * @return null if doesn't match, else a live handler.
    */
   private synchronized SnapshotSentinel removeSentinelIfFinished(
-      final Map<String, SnapshotSentinel> sentinels, final SnapshotDescription snapshot) {
-    SnapshotSentinel h = sentinels.get(snapshot.getTable());
+      final Map<TableName, SnapshotSentinel> sentinels,
+      final SnapshotDescription snapshot) {
+    if (!snapshot.hasTable()) {
+      return null;
+    }
+
+    TableName snapshotTable = TableName.valueOf(snapshot.getTable());
+    SnapshotSentinel h = sentinels.get(snapshotTable);
     if (h == null) {
       return null;
     }
@@ -814,7 +830,7 @@ public class SnapshotManager implements 
 
     // Remove from the "in-progress" list once completed
     if (h.isFinished()) {
-      sentinels.remove(snapshot.getTable());
+      sentinels.remove(snapshotTable);
     }
 
     return h;
@@ -837,11 +853,12 @@ public class SnapshotManager implements 
    * has exceeded the removal timeout.
    * @param sentinels map of sentinels to clean
    */
-  private synchronized void cleanupSentinels(final Map<String, SnapshotSentinel> sentinels) {
+  private synchronized void cleanupSentinels(final Map<TableName, SnapshotSentinel> sentinels) {
     long currentTime = EnvironmentEdgeManager.currentTimeMillis();
-    Iterator<Map.Entry<String, SnapshotSentinel>> it = sentinels.entrySet().iterator();
+    Iterator<Map.Entry<TableName, SnapshotSentinel>> it =
+        sentinels.entrySet().iterator();
     while (it.hasNext()) {
-      Map.Entry<String, SnapshotSentinel> entry = it.next();
+      Map.Entry<TableName, SnapshotSentinel> entry = it.next();
       SnapshotSentinel sentinel = entry.getValue();
       if (sentinel.isFinished() &&
           (currentTime - sentinel.getCompletionTimestamp()) > SNAPSHOT_SENTINELS_CLEANUP_TIMEOUT)

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java Thu Aug  8 06:08:23 2013
@@ -30,6 +30,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.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
@@ -46,12 +47,12 @@ import org.apache.hadoop.hbase.master.Ta
 import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.zookeeper.KeeperException;
 
@@ -83,6 +84,7 @@ public abstract class TakeSnapshotHandle
   protected final TableLockManager tableLockManager;
   protected final TableLock tableLock;
   protected final MonitoredTask status;
+  protected final TableName snapshotTable;
 
   /**
    * @param snapshot descriptor of the snapshot to take
@@ -97,6 +99,7 @@ public abstract class TakeSnapshotHandle
     this.master = masterServices;
     this.metricsMaster = metricsMaster;
     this.snapshot = snapshot;
+    this.snapshotTable = TableName.valueOf(snapshot.getTable());
     this.conf = this.master.getConfiguration();
     this.fs = this.master.getMasterFileSystem().getFileSystem();
     this.rootDir = this.master.getMasterFileSystem().getRootDir();
@@ -105,23 +108,23 @@ public abstract class TakeSnapshotHandle
     this.monitor = new ForeignExceptionDispatcher(snapshot.getName());
 
     this.tableLockManager = master.getTableLockManager();
-    this.tableLock = this.tableLockManager.writeLock(Bytes.toBytes(snapshot.getTable())
-      , EventType.C_M_SNAPSHOT_TABLE.toString());
+    this.tableLock = this.tableLockManager.writeLock(
+        snapshotTable,
+        EventType.C_M_SNAPSHOT_TABLE.toString());
 
     // prepare the verify
     this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, rootDir);
     // update the running tasks
     this.status = TaskMonitor.get().createStatus(
-      "Taking " + snapshot.getType() + " snapshot on table: " + snapshot.getTable());
+      "Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable);
   }
 
   private HTableDescriptor loadTableDescriptor()
       throws FileNotFoundException, IOException {
-    final String name = snapshot.getTable();
     HTableDescriptor htd =
-      this.master.getTableDescriptors().get(name);
+      this.master.getTableDescriptors().get(snapshotTable);
     if (htd == null) {
-      throw new IOException("HTableDescriptor missing for " + name);
+      throw new IOException("HTableDescriptor missing for " + snapshotTable);
     }
     return htd;
   }
@@ -150,7 +153,7 @@ public abstract class TakeSnapshotHandle
   @Override
   public void process() {
     String msg = "Running " + snapshot.getType() + " table snapshot " + snapshot.getName() + " "
-        + eventType + " on table " + snapshot.getTable();
+        + eventType + " on table " + snapshotTable;
     LOG.info(msg);
     status.setStatus(msg);
     try {
@@ -164,7 +167,7 @@ public abstract class TakeSnapshotHandle
 
       List<Pair<HRegionInfo, ServerName>> regionsAndLocations =
           MetaReader.getTableRegionsAndLocations(this.server.getCatalogTracker(),
-            Bytes.toBytes(snapshot.getTable()), true);
+              snapshotTable, true);
 
       // run the snapshot
       snapshotRegions(regionsAndLocations);
@@ -182,12 +185,12 @@ public abstract class TakeSnapshotHandle
 
       // complete the snapshot, atomically moving from tmp to .snapshot dir.
       completeSnapshot(this.snapshotDir, this.workingDir, this.fs);
-      status.markComplete("Snapshot " + snapshot.getName() + " of table " + snapshot.getTable()
+      status.markComplete("Snapshot " + snapshot.getName() + " of table " + snapshotTable
           + " completed");
       metricsMaster.addSnapshot(status.getCompletionTimestamp() - status.getStartTime());
     } catch (Exception e) {
       status.abort("Failed to complete snapshot " + snapshot.getName() + " on table " +
-          snapshot.getTable() + " because " + e.getMessage());
+          snapshotTable + " because " + e.getMessage());
       String reason = "Failed taking snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot)
           + " due to exception:" + e.getMessage();
       LOG.error(reason, e);

Added: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java?rev=1511591&view=auto
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java (added)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java Thu Aug  8 06:08:23 2013
@@ -0,0 +1,219 @@
+/**
+ * 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.migration;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.util.Tool;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Upgrades old 0.94 filesystem layout to namespace layout
+ * Does the following:
+ *
+ * - creates system namespace directory and move .META. table there
+ * renaming .META. table to hbase:meta,
+ * this in turn would require to re-encode the region directory name
+ */
+public class NamespaceUpgrade implements Tool {
+  private static final Log LOG = LogFactory.getLog(NamespaceUpgrade.class);
+
+  private Configuration conf;
+
+  private FileSystem fs;
+
+  private Path rootDir;
+  private Path sysNsDir;
+  private Path defNsDir;
+  private Path baseDirs[];
+
+  public NamespaceUpgrade() throws IOException {
+  }
+
+  public void init() throws IOException {
+    this.rootDir = FSUtils.getRootDir(conf);
+    this.fs = FileSystem.get(conf);
+    sysNsDir = FSUtils.getNamespaceDir(rootDir, NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR);
+    defNsDir = FSUtils.getNamespaceDir(rootDir, NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR);
+    baseDirs = new Path[]{rootDir,
+        new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY),
+        new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY)};
+  }
+
+
+  public void upgradeTableDirs()
+      throws IOException, DeserializationException {
+
+
+    //if new version is written then upgrade is done
+    if (verifyNSUpgrade(fs, rootDir)) {
+      return;
+    }
+
+    makeNamespaceDirs();
+
+    migrateTables();
+
+    migrateSnapshots();
+
+    migrateMeta();
+
+    FSUtils.setVersion(fs, rootDir);
+  }
+
+  public void makeNamespaceDirs() throws IOException {
+    if (!fs.exists(sysNsDir)) {
+      if (!fs.mkdirs(sysNsDir)) {
+        throw new IOException("Failed to create system namespace dir: " + sysNsDir);
+      }
+    }
+    if (!fs.exists(defNsDir)) {
+      if (!fs.mkdirs(defNsDir)) {
+        throw new IOException("Failed to create default namespace dir: " + defNsDir);
+      }
+    }
+  }
+
+  public void migrateTables() throws IOException {
+    List<String> sysTables = Lists.newArrayList("-ROOT-",".META.");
+
+    //migrate tables including archive and tmp
+    for(Path baseDir: baseDirs) {
+      List<Path> oldTableDirs = FSUtils.getLocalTableDirs(fs, baseDir);
+      for(Path oldTableDir: oldTableDirs) {
+        if (!sysTables.contains(oldTableDir.getName())) {
+          Path nsDir = FSUtils.getTableDir(baseDir,
+              TableName.valueOf(oldTableDir.getName()));
+          if(!fs.exists(nsDir.getParent())) {
+            if(!fs.mkdirs(nsDir.getParent())) {
+              throw new IOException("Failed to create namespace dir "+nsDir.getParent());
+            }
+          }
+          if (sysTables.indexOf(oldTableDir.getName()) < 0) {
+            LOG.info("Migrating table " + oldTableDir.getName() + " to " + nsDir);
+            if (!fs.rename(oldTableDir, nsDir)) {
+              throw new IOException("Failed to move "+oldTableDir+" to namespace dir "+nsDir);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  public void migrateSnapshots() throws IOException {
+    //migrate snapshot dir
+    Path oldSnapshotDir = new Path(rootDir, HConstants.OLD_SNAPSHOT_DIR_NAME);
+    Path newSnapshotDir = new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME);
+    if (fs.exists(oldSnapshotDir)) {
+      boolean foundOldSnapshotDir = false;
+      // Logic to verify old snapshot dir culled from SnapshotManager
+      // ignore all the snapshots in progress
+      FileStatus[] snapshots = fs.listStatus(oldSnapshotDir,
+        new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs));
+      // loop through all the completed snapshots
+      for (FileStatus snapshot : snapshots) {
+        Path info = new Path(snapshot.getPath(), SnapshotDescriptionUtils.SNAPSHOTINFO_FILE);
+        // if the snapshot is bad
+        if (fs.exists(info)) {
+          foundOldSnapshotDir = true;
+          break;
+        }
+      }
+      if(foundOldSnapshotDir) {
+        LOG.info("Migrating snapshot dir");
+        if (!fs.rename(oldSnapshotDir, newSnapshotDir)) {
+          throw new IOException("Failed to move old snapshot dir "+
+              oldSnapshotDir+" to new "+newSnapshotDir);
+        }
+      }
+    }
+  }
+
+  public void migrateMeta() throws IOException {
+    Path newMetaRegionDir = HRegion.getRegionDir(rootDir, HRegionInfo.FIRST_META_REGIONINFO);
+    Path newMetaDir = FSUtils.getTableDir(rootDir, TableName.META_TABLE_NAME);
+    Path oldMetaDir = new Path(rootDir, ".META.");
+    if (fs.exists(oldMetaDir)) {
+      LOG.info("Migrating meta table " + oldMetaDir.getName() + " to " + newMetaDir);
+      if (!fs.rename(oldMetaDir, newMetaDir)) {
+        throw new IOException("Failed to migrate meta table "
+            + oldMetaDir.getName() + " to " + newMetaDir);
+      }
+    }
+
+    //since meta table name has changed
+    //rename meta region dir from it's old encoding to new one
+    Path oldMetaRegionDir = HRegion.getRegionDir(rootDir,
+        new Path(newMetaDir, "1028785192").toString());
+    if (fs.exists(oldMetaRegionDir)) {
+      LOG.info("Migrating meta region " + oldMetaRegionDir + " to " + newMetaRegionDir);
+      if (!fs.rename(oldMetaRegionDir, newMetaRegionDir)) {
+        throw new IOException("Failed to migrate meta region "
+            + oldMetaRegionDir + " to " + newMetaRegionDir);
+      }
+    }
+  }
+
+  public static boolean verifyNSUpgrade(FileSystem fs, Path rootDir)
+      throws IOException {
+    try {
+      return FSUtils.getVersion(fs, rootDir).equals(HConstants.FILE_SYSTEM_VERSION);
+    } catch (DeserializationException e) {
+      throw new IOException("Failed to verify namespace upgrade", e);
+    }
+  }
+
+
+  @Override
+  public int run(String[] args) throws Exception {
+    if(args.length < 1 || !args[0].equals("--upgrade")) {
+      System.out.println("Usage: <CMD> --upgrade");
+      return 0;
+    }
+    init();
+    upgradeTableDirs();
+    return 0;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+}

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java Thu Aug  8 06:08:23 2013
@@ -119,7 +119,7 @@ public class ReplicationProtbufUtil {
       HLogKey key = entry.getKey();
       keyBuilder.setEncodedRegionName(
         ByteString.copyFrom(key.getEncodedRegionName()));
-      keyBuilder.setTableName(ByteString.copyFrom(key.getTablename()));
+      keyBuilder.setTableName(ByteString.copyFrom(key.getTablename().getName()));
       keyBuilder.setLogSequenceNumber(key.getLogSeqNum());
       keyBuilder.setWriteTime(key.getWriteTime());
       UUID clusterId = key.getClusterId();
@@ -190,4 +190,4 @@ public class ReplicationProtbufUtil {
       }
     };
   }
-}
\ No newline at end of file
+}

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java Thu Aug  8 06:08:23 2013
@@ -153,7 +153,7 @@ public class CompactionTool extends Conf
         final HRegionInfo hri, final String familyName, final boolean compactOnce,
         final boolean major) throws IOException {
       HStore store = getStore(conf, fs, tableDir, htd, hri, familyName, tmpDir);
-      LOG.info("Compact table=" + htd.getNameAsString() +
+      LOG.info("Compact table=" + htd.getTableName() +
         " region=" + hri.getRegionNameAsString() +
         " family=" + familyName);
       if (major) {

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java?rev=1511591&r1=1511590&r2=1511591&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java Thu Aug  8 06:08:23 2013
@@ -57,7 +57,7 @@ public class DelimitedKeyPrefixRegionSpl
           DELIMITER_KEY);
       if (delimiterString == null || delimiterString.length() == 0) {
         LOG.error(DELIMITER_KEY + " not specified for table "
-            + region.getTableDesc().getNameAsString()
+            + region.getTableDesc().getTableName()
             + ". Using default RegionSplitPolicy");
         return;
       }