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/14 08:19:51 UTC

svn commit: r1513736 - in /hbase/trunk: hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/hbase/migration/ hbase-server/src/test/java/org/apache/hadoop/hbase/ hbase-server/src/test/java/org/apache/hadoop/h...

Author: stack
Date: Wed Aug 14 06:19:50 2013
New Revision: 1513736

URL: http://svn.apache.org/r1513736
Log:
HBASE-9128 Drop the '.' prefix from names under hbase.rootdir after namespaces goes in

Modified:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestNamespaceUpgrade.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java?rev=1513736&r1=1513735&r2=1513736&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java Wed Aug 14 06:19:50 2013
@@ -274,17 +274,16 @@ public final class HConstants {
   /** Default HBase client operation timeout, which is tantamount to a blocking call */
   public static final int DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT = Integer.MAX_VALUE;
 
-  /** Used to construct the name of the log directory for a region server
-   * Use '.' as a special character to seperate the log files from table data */
-  public static final String HREGION_LOGDIR_NAME = ".logs";
+  /** Used to construct the name of the log directory for a region server */
+  public static final String HREGION_LOGDIR_NAME = "WALs";
 
   /** Used to construct the name of the splitlog directory for a region server */
-  public static final String SPLIT_LOGDIR_NAME = "splitlog";
-
-  public static final String CORRUPT_DIR_NAME = ".corrupt";
+  public static final String SPLIT_LOGDIR_NAME = "splitWAL";
 
   /** Like the previous, but for old logs that are about to be deleted */
-  public static final String HREGION_OLDLOGDIR_NAME = ".oldlogs";
+  public static final String HREGION_OLDLOGDIR_NAME = "oldWALs";
+
+  public static final String CORRUPT_DIR_NAME = "corrupt";
 
   /** Used by HBCK to sideline backup data */
   public static final String HBCK_SIDELINEDIR_NAME = ".hbck";
@@ -355,7 +354,7 @@ public final class HConstants {
   // be the first to be reassigned if the server(s) they are being served by
   // should go down.
 
-  public static final String BASE_NAMESPACE_DIR = ".data";
+  public static final String BASE_NAMESPACE_DIR = "data";
 
   /** delimiter used between portions of a region name */
   public static final int META_ROW_DELIMITER = ',';
@@ -818,7 +817,7 @@ public final class HConstants {
   public static final int REPLAY_QOS = 6; // REPLICATION_QOS < REPLAY_QOS < high_QOS
 
   /** Directory under /hbase where archived hfiles are stored */
-  public static final String HFILE_ARCHIVE_DIRECTORY = ".archive";
+  public static final String HFILE_ARCHIVE_DIRECTORY = "archive";
 
   /**
    * Name of the directory to store all snapshots. See SnapshotDescriptionUtils for

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java?rev=1513736&r1=1513735&r2=1513736&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/migration/NamespaceUpgrade.java Wed Aug 14 06:19:50 2013
@@ -19,8 +19,11 @@
  */
 package org.apache.hadoop.hbase.migration;
 
-import com.google.common.collect.Lists;
-import com.google.common.primitives.Ints;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -37,7 +40,6 @@ import org.apache.hadoop.hbase.ServerNam
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.catalog.MetaEditor;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
@@ -50,9 +52,8 @@ import org.apache.hadoop.hbase.util.FSTa
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.util.Tool;
 
-import java.io.IOException;
-import java.util.Comparator;
-import java.util.List;
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Ints;
 
 /**
  * Upgrades old 0.94 filesystem layout to namespace layout
@@ -61,6 +62,8 @@ import java.util.List;
  * - 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
+ *
+ * <p>The pre-0.96 paths and dir names are hardcoded in here.
  */
 public class NamespaceUpgrade implements Tool {
   private static final Log LOG = LogFactory.getLog(NamespaceUpgrade.class);
@@ -74,15 +77,38 @@ public class NamespaceUpgrade implements
   private Path defNsDir;
   private Path baseDirs[];
   private Path backupDir;
+  // First move everything to this tmp .data dir in case there is a table named 'data'
+  private static final String TMP_DATA_DIR = ".data";
+  // Old dir names to migrate.
+  private static final String DOT_LOGS = ".logs";
+  private static final String DOT_OLD_LOGS = ".oldlogs";
+  private static final String DOT_CORRUPT = ".corrupt";
+  private static final String DOT_SPLITLOG = "splitlog";
+  private static final String DOT_ARCHIVE = ".archive";
+  private static final String OLD_ACL = "_acl_";
+  /** Directories that are not HBase table directories */
+  static final List<String> NON_USER_TABLE_DIRS = Arrays.asList(new String[] {
+      DOT_LOGS,
+      DOT_OLD_LOGS,
+      DOT_CORRUPT,
+      DOT_SPLITLOG,
+      HConstants.HBCK_SIDELINEDIR_NAME,
+      DOT_ARCHIVE,
+      HConstants.SNAPSHOT_DIR_NAME,
+      HConstants.HBASE_TEMP_DIRECTORY,
+      TMP_DATA_DIR,
+      OLD_ACL});
 
   public NamespaceUpgrade() throws IOException {
+    super();
   }
 
   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);
+    Path tmpDataDir = new Path(rootDir, TMP_DATA_DIR);
+    sysNsDir = new Path(tmpDataDir, NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR);
+    defNsDir = new Path(tmpDataDir, NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR);
     baseDirs = new Path[]{rootDir,
         new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY),
         new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY)};
@@ -90,29 +116,95 @@ public class NamespaceUpgrade implements
   }
 
 
-  public void upgradeTableDirs()
-      throws IOException, DeserializationException {
-
-
-    //if new version is written then upgrade is done
+  public void upgradeTableDirs() throws IOException, DeserializationException {
+    // if new version is written then upgrade is done
     if (verifyNSUpgrade(fs, rootDir)) {
       return;
     }
 
     makeNamespaceDirs();
 
+    migrateTables();
+
+    migrateSnapshots();
+
+    migrateDotDirs();
+
     migrateMeta();
 
     migrateACL();
 
-    migrateTables();
+    deleteRoot();
 
-    migrateSnapshots();
+    FSUtils.setVersion(fs, rootDir);
+  }
 
+  /**
+   * Remove the -ROOT- dir. No longer of use.
+   * @throws IOException
+   */
+  public void deleteRoot() throws IOException {
+    Path rootDir = new Path(this.rootDir, "-ROOT-");
+    if (this.fs.exists(rootDir)) {
+      if (!this.fs.delete(rootDir, true)) LOG.info("Failed remove of " + rootDir);
+      LOG.info("Deleted " + rootDir);
+    }
+  }
 
-    FSUtils.setVersion(fs, rootDir);
+  /**
+   * Rename all the dot dirs -- .data, .archive, etc. -- as data, archive, etc.; i.e. minus the dot.
+   * @throws IOException
+   */
+  public void migrateDotDirs() throws IOException {
+    // Dot dirs to rename.  Leave the tmp dir named '.tmp' and snapshots as .hbase-snapshot.
+    final Path archiveDir = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
+    Path [][] dirs = new Path[][] {
+      new Path [] {new Path(rootDir, DOT_CORRUPT), new Path(rootDir, HConstants.CORRUPT_DIR_NAME)},
+      new Path [] {new Path(rootDir, DOT_LOGS), new Path(rootDir, HConstants.HREGION_LOGDIR_NAME)},
+      new Path [] {new Path(rootDir, DOT_OLD_LOGS),
+        new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME)},
+      new Path [] {new Path(rootDir, TMP_DATA_DIR),
+        new Path(rootDir, HConstants.BASE_NAMESPACE_DIR)}};
+    for (Path [] dir: dirs) {
+      Path src = dir[0];
+      Path tgt = dir[1];
+      if (!this.fs.exists(src)) {
+        LOG.info("Does not exist: " + src);
+        continue;
+      }
+      rename(src, tgt);
+    }
+    // Do the .archive dir.  Need to move its subdirs to the default ns dir under data dir... so
+    // from '.archive/foo', to 'archive/data/default/foo'.
+    Path oldArchiveDir = new Path(rootDir, DOT_ARCHIVE);
+    if (this.fs.exists(oldArchiveDir)) {
+      // This is a pain doing two nn calls but portable over h1 and h2.
+      mkdirs(archiveDir);
+      Path archiveDataDir = new Path(archiveDir, HConstants.BASE_NAMESPACE_DIR);
+      mkdirs(archiveDataDir);
+      rename(oldArchiveDir, new Path(archiveDataDir,
+        NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR));
+    }
+    // Update the system and user namespace dirs removing the dot in front of .data.
+    Path dataDir = new Path(rootDir, HConstants.BASE_NAMESPACE_DIR);
+    sysNsDir = new Path(dataDir, NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR);
+    defNsDir = new Path(dataDir, NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR);
+  }
+
+  private void mkdirs(final Path p) throws IOException {
+    if (!this.fs.mkdirs(p)) throw new IOException("Failed make of " + p);
   }
 
+  private void rename(final Path src, final Path tgt) throws IOException {
+    if (!fs.rename(src, tgt)) {
+      throw new IOException("Failed move " + src + " to " + tgt);
+    }
+  }
+
+  /**
+   * Create the system and default namespaces dirs
+   * @throws IOException
+   */
   public void makeNamespaceDirs() throws IOException {
     if (!fs.exists(sysNsDir)) {
       if (!fs.mkdirs(sysNsDir)) {
@@ -126,27 +218,34 @@ public class NamespaceUpgrade implements
     }
   }
 
+  /**
+   * Migrate all tables into respective namespaces, either default or system.  We put them into
+   * a temporary location, '.data', in case a user table is name 'data'.  In a later method we will
+   * move stuff from .data to data.
+   * @throws IOException
+   */
   public void migrateTables() throws IOException {
     List<String> sysTables = Lists.newArrayList("-ROOT-",".META.");
 
-    //migrate tables including archive and tmp
-    for(Path baseDir: baseDirs) {
+    // Migrate tables including archive and tmp
+    for (Path baseDir: baseDirs) {
       if (!fs.exists(baseDir)) continue;
       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());
-            }
+      for (Path oldTableDir: oldTableDirs) {
+        if (NON_USER_TABLE_DIRS.contains(oldTableDir.getName())) continue;
+        if (sysTables.contains(oldTableDir.getName())) continue;
+        // Make the new directory under the ns to which we will move the table.
+        Path nsDir = new Path(this.defNsDir,
+          TableName.valueOf(oldTableDir.getName()).getQualifierAsString());
+        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);
-            }
+        }
+        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);
           }
         }
       }
@@ -183,8 +282,9 @@ public class NamespaceUpgrade implements
   }
 
   public void migrateMeta() throws IOException {
-    Path newMetaRegionDir = HRegion.getRegionDir(rootDir, HRegionInfo.FIRST_META_REGIONINFO);
-    Path newMetaDir = FSUtils.getTableDir(rootDir, TableName.META_TABLE_NAME);
+    Path newMetaDir = new Path(this.sysNsDir, TableName.META_TABLE_NAME.getQualifierAsString());
+    Path newMetaRegionDir =
+      new Path(newMetaDir, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
     Path oldMetaDir = new Path(rootDir, ".META.");
     if (fs.exists(oldMetaDir)) {
       LOG.info("Migrating meta table " + oldMetaDir.getName() + " to " + newMetaDir);
@@ -194,10 +294,9 @@ public class NamespaceUpgrade implements
       }
     }
 
-    //since meta table name has changed
-    //rename meta region dir from it's old encoding to new one
+    // 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());
+      new Path(newMetaDir, "1028785192").toString());
     if (fs.exists(oldMetaRegionDir)) {
       LOG.info("Migrating meta region " + oldMetaRegionDir + " to " + newMetaRegionDir);
       if (!fs.rename(oldMetaRegionDir, newMetaRegionDir)) {
@@ -214,7 +313,7 @@ public class NamespaceUpgrade implements
 
   public void migrateACL() throws IOException {
 
-    TableName oldTableName = TableName.valueOf("_acl_");
+    TableName oldTableName = TableName.valueOf(OLD_ACL);
     Path oldTablePath = new Path(rootDir, oldTableName.getNameAsString());
 
     if(!fs.exists(oldTablePath)) {
@@ -369,7 +468,7 @@ public class NamespaceUpgrade implements
 
   @Override
   public int run(String[] args) throws Exception {
-    if(args.length < 1 || !args[0].equals("--upgrade")) {
+    if (args.length < 1 || !args[0].equals("--upgrade")) {
       System.out.println("Usage: <CMD> --upgrade");
       return 0;
     }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java?rev=1513736&r1=1513735&r2=1513736&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java Wed Aug 14 06:19:50 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.client.HT
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
 import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.migration.NamespaceUpgrade;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestNamespaceUpgrade.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestNamespaceUpgrade.java?rev=1513736&r1=1513735&r2=1513736&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestNamespaceUpgrade.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/migration/TestNamespaceUpgrade.java Wed Aug 14 06:19:50 2013
@@ -19,14 +19,15 @@
  */
 package org.apache.hadoop.hbase.migration;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -51,6 +52,7 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -60,12 +62,12 @@ import org.junit.experimental.categories
  * Mainly tests that tables are migrated and consistent. Also verifies
  * that snapshots have been migrated correctly.
  *
- * Uses a tarball which is an image of an 0.94 hbase.rootdir.
+ * <p>Uses a tarball which is an image of an 0.94 hbase.rootdir.
  *
- * Contains tables with currentKeys as the stored keys:
+ * <p>Contains tables with currentKeys as the stored keys:
  * foo, ns1.foo, ns2.foo
  *
- * Contains snapshots with snapshot{num}Keys as the contents:
+ * <p>Contains snapshots with snapshot{num}Keys as the contents:
  * snapshot1Keys, snapshot2Keys
  *
  * Image also contains _acl_ table with one region and two storefiles.
@@ -118,9 +120,8 @@ public class TestNamespaceUpgrade {
     Configuration toolConf = TEST_UTIL.getConfiguration();
     conf.set(HConstants.HBASE_DIR, TEST_UTIL.getDefaultRootDirPath().toString());
     ToolRunner.run(toolConf, new NamespaceUpgrade(), new String[]{"--upgrade"});
-    doFsCommand(shell, new String [] {"-lsr", "/"});
-
     assertTrue(FSUtils.getVersion(fs, hbaseRootDir).equals(HConstants.FILE_SYSTEM_VERSION));
+    doFsCommand(shell, new String [] {"-lsr", "/"});
     TEST_UTIL.startMiniHBaseCluster(1, 1);
 
     for(String table: tables) {
@@ -186,7 +187,7 @@ public class TestNamespaceUpgrade {
   @Test
   public void testSnapshots() throws IOException, InterruptedException {
     String snapshots[][] = {snapshot1Keys, snapshot2Keys};
-    for(int i=1; i<=snapshots.length; i++) {
+    for(int i = 1; i <= snapshots.length; i++) {
       for(String table: tables) {
         TEST_UTIL.getHBaseAdmin().cloneSnapshot(table+"_snapshot"+i, table+"_clone"+i);
         FSUtils.logFileSystemState(FileSystem.get(TEST_UTIL.getConfiguration()),
@@ -203,7 +204,7 @@ public class TestNamespaceUpgrade {
   }
 
   @Test
-  public void testRenameUsingSnapshots() throws IOException, InterruptedException {
+  public void testRenameUsingSnapshots() throws Exception {
     String newNS = "newNS";
     TEST_UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(newNS).build());
     for(String table: tables) {
@@ -212,10 +213,9 @@ public class TestNamespaceUpgrade {
           Scan())) {
         assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
       }
-      TEST_UTIL.getHBaseAdmin().snapshot(table+"_snapshot3", table);
-      final String newTableName =
-          newNS+ TableName.NAMESPACE_DELIM+table+"_clone3";
-      TEST_UTIL.getHBaseAdmin().cloneSnapshot(table+"_snapshot3", newTableName);
+      TEST_UTIL.getHBaseAdmin().snapshot(table + "_snapshot3", table);
+      final String newTableName = newNS + TableName.NAMESPACE_DELIM + table + "_clone3";
+      TEST_UTIL.getHBaseAdmin().cloneSnapshot(table + "_snapshot3", newTableName);
       Thread.sleep(1000);
       count = 0;
       for(Result res: new HTable(TEST_UTIL.getConfiguration(), newTableName).getScanner(new
@@ -243,12 +243,12 @@ public class TestNamespaceUpgrade {
     String nextNS = "nextNS";
     TEST_UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(nextNS).build());
     for(String table: tables) {
-      String srcTable = newNS+TableName.NAMESPACE_DELIM+table+"_clone3";
-      TEST_UTIL.getHBaseAdmin().snapshot(table+"_snapshot4", srcTable);
-      String newTableName = nextNS+TableName.NAMESPACE_DELIM+table+"_clone4";
+      String srcTable = newNS + TableName.NAMESPACE_DELIM + table + "_clone3";
+      TEST_UTIL.getHBaseAdmin().snapshot(table + "_snapshot4", srcTable);
+      String newTableName = nextNS + TableName.NAMESPACE_DELIM + table + "_clone4";
       TEST_UTIL.getHBaseAdmin().cloneSnapshot(table+"_snapshot4", newTableName);
-      FSUtils.logFileSystemState(TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath()
-          , LOG);
+      FSUtils.logFileSystemState(TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath(),
+        LOG);
       int count = 0;
       for(Result res: new HTable(TEST_UTIL.getConfiguration(), newTableName).getScanner(new
           Scan())) {
@@ -256,7 +256,31 @@ public class TestNamespaceUpgrade {
       }
       Assert.assertEquals(newTableName, currentKeys.length, count);
     }
+  }
+
+  @Test
+  public void testOldDirsAreGonePostMigration() throws IOException {
+    FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
+    Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
+    List <String> dirs = new ArrayList<String>(NamespaceUpgrade.NON_USER_TABLE_DIRS);
+    // Remove those that are not renamed
+    dirs.remove(HConstants.HBCK_SIDELINEDIR_NAME);
+    dirs.remove(HConstants.SNAPSHOT_DIR_NAME);
+    dirs.remove(HConstants.HBASE_TEMP_DIRECTORY);
+    for (String dir: dirs) {
+      assertFalse(fs.exists(new Path(hbaseRootDir, dir)));
+    }
+  }
 
+  @Test
+  public void testNewDirsArePresentPostMigration() throws IOException {
+    FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
+    // Below list does not include 'corrupt' because there is no 'corrupt' in the tgz
+    String [] newdirs = new String [] {HConstants.BASE_NAMESPACE_DIR,
+      HConstants.HREGION_LOGDIR_NAME};
+    Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
+    for (String dir: newdirs) {
+      assertTrue(dir, fs.exists(new Path(hbaseRootDir, dir)));
+    }
   }
 }
-

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java?rev=1513736&r1=1513735&r2=1513736&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java Wed Aug 14 06:19:50 2013
@@ -726,17 +726,18 @@ public class TestHLog  {
     Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, hl + "qdf"));
     Assert.assertNull(HLogUtil.getServerNameFromHLogDirectoryName(conf, "sfqf" + hl + "qdf"));
 
+    final String wals = "/WALs/";
     ServerName parsed = HLogUtil.getServerNameFromHLogDirectoryName(conf,
-        FSUtils.getRootDir(conf).toUri().toString() +
-            "/.logs/" + sn + "/localhost%2C32984%2C1343316388997.1343316390417");
+      FSUtils.getRootDir(conf).toUri().toString() + wals + sn +
+      "/localhost%2C32984%2C1343316388997.1343316390417");
     Assert.assertEquals("standard",  sn, parsed);
 
     parsed = HLogUtil.getServerNameFromHLogDirectoryName(conf, hl + "/qdf");
     Assert.assertEquals("subdir", sn, parsed);
 
     parsed = HLogUtil.getServerNameFromHLogDirectoryName(conf,
-        FSUtils.getRootDir(conf).toUri().toString() +
-            "/.logs/" + sn + "-splitting/localhost%3A57020.1340474893931");
+      FSUtils.getRootDir(conf).toUri().toString() + wals + sn +
+      "-splitting/localhost%3A57020.1340474893931");
     Assert.assertEquals("split", sn, parsed);
   }