HBASE-14439 Move fs stuff out pt1 - master fs + table desc

Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6796171f
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6796171f
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6796171f

Branch: refs/heads/hbase-14439
Commit: 6796171fc43c551ab4aed46b95840cc4c72208a4
Parents: 1384c9a
Author: Matteo Bertozzi <matteo.berto...@cloudera.com>
Authored: Tue Oct 6 16:02:20 2015 -0700
Committer: Sean Busbey <bus...@apache.org>
Committed: Thu Sep 22 11:19:52 2016 -0500

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  63 +++
 .../SplitLogManagerCoordination.java            |   4 +-
 .../org/apache/hadoop/hbase/fs/FsContext.java   |  28 +
 .../hadoop/hbase/fs/MasterFileSystem.java       | 320 ++++++++++++
 .../hadoop/hbase/fs/legacy/LegacyLayout.java    |  69 +++
 .../hbase/fs/legacy/LegacyMasterFileSystem.java | 262 ++++++++++
 .../hbase/fs/legacy/LegacyTableDescriptor.java  | 399 +++++++++++++++
 .../org/apache/hadoop/hbase/master/HMaster.java |   4 +-
 .../hadoop/hbase/master/MasterFileSystem.java   | 324 ------------
 .../hadoop/hbase/master/MasterServices.java     |   1 +
 .../hadoop/hbase/master/MasterWalManager.java   |   2 +-
 .../procedure/CloneSnapshotProcedure.java       |   6 +-
 .../procedure/CreateNamespaceProcedure.java     |   5 +-
 .../master/procedure/CreateTableProcedure.java  |  10 +-
 .../procedure/DeleteNamespaceProcedure.java     |  20 +-
 .../master/procedure/DeleteTableProcedure.java  |   2 +-
 .../procedure/MasterDDLOperationHelper.java     |   2 +-
 .../procedure/RestoreSnapshotProcedure.java     |   2 +-
 .../hbase/master/snapshot/SnapshotManager.java  |   2 +-
 .../hbase/regionserver/CompactionTool.java      |  10 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |   7 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   | 510 ++-----------------
 .../org/apache/hadoop/hbase/util/FSUtils.java   |  33 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  10 +-
 .../hadoop/hbase/util/HFileArchiveUtil.java     |   4 +-
 .../org/apache/hadoop/hbase/util/HMerge.java    |   3 +-
 .../org/apache/hadoop/hbase/util/Merge.java     |   3 +-
 .../org/apache/hadoop/hbase/util/MetaUtils.java |  16 +-
 .../TestHColumnDescriptorDefaultVersions.java   |   5 +-
 .../client/TestRestoreSnapshotFromClient.java   |   2 +-
 .../hbase/client/TestSnapshotFromClient.java    |   7 +-
 .../hadoop/hbase/master/TestCatalogJanitor.java |   1 +
 .../hbase/master/TestMasterFileSystem.java      |   1 +
 ...stTableDescriptorModificationFromClient.java |   7 +-
 .../master/snapshot/TestSnapshotManager.java    |   2 +-
 .../hbase/namespace/TestNamespaceAuditor.java   |  18 +-
 .../hbase/regionserver/TestDefaultMemStore.java |   2 +-
 .../hbase/regionserver/TestStoreFile.java       |   3 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |  10 +-
 .../hbase/util/TestFSTableDescriptors.java      |  71 +--
 .../hadoop/hbase/util/TestHBaseFsckOneRS.java   |  12 +-
 41 files changed, 1330 insertions(+), 932 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index 9abdf42..867dc0d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -34,6 +34,7 @@ import java.util.regex.Matcher;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
+import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -1567,4 +1569,65 @@ public class HTableDescriptor implements 
Comparable<HTableDescriptor> {
   public void removeConfiguration(final String key) {
     configuration.remove(key);
   }
+
+  @InterfaceAudience.Private
+  public static HTableDescriptor metaTableDescriptor(final Configuration conf)
+      throws IOException {
+    HTableDescriptor metaDescriptor = new HTableDescriptor(
+        TableName.META_TABLE_NAME,
+        new HColumnDescriptor[] {
+            new HColumnDescriptor(HConstants.CATALOG_FAMILY)
+                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+                    HConstants.DEFAULT_HBASE_META_VERSIONS))
+                .setInMemory(true)
+                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+                    HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
+                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                    // Disable blooms for meta.  Needs work.  Seems to mess w/ 
getClosestOrBefore.
+                .setBloomFilterType(BloomType.NONE)
+                    // Enable cache of data blocks in L1 if more than one 
caching tier deployed:
+                    // e.g. if using CombinedBlockCache (BucketCache).
+                .setCacheDataInL1(true),
+            new HColumnDescriptor(HConstants.REPLICATION_BARRIER_FAMILY)
+                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+                    HConstants.DEFAULT_HBASE_META_VERSIONS))
+                .setInMemory(true)
+                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+                    HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
+                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                // Disable blooms for meta.  Needs work.  Seems to mess w/ 
getClosestOrBefore.
+                .setBloomFilterType(BloomType.NONE)
+                // Enable cache of data blocks in L1 if more than one caching 
tier deployed:
+                // e.g. if using CombinedBlockCache (BucketCache).
+                .setCacheDataInL1(true),
+            new HColumnDescriptor(HConstants.REPLICATION_POSITION_FAMILY)
+                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+                    HConstants.DEFAULT_HBASE_META_VERSIONS))
+                .setInMemory(true)
+                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+                    HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
+                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                // Disable blooms for meta.  Needs work.  Seems to mess w/ 
getClosestOrBefore.
+                .setBloomFilterType(BloomType.NONE)
+                // Enable cache of data blocks in L1 if more than one caching 
tier deployed:
+                // e.g. if using CombinedBlockCache (BucketCache).
+                .setCacheDataInL1(true),
+            new HColumnDescriptor(HConstants.TABLE_FAMILY)
+                // Ten is arbitrary number.  Keep versions to help debugging.
+                .setMaxVersions(10)
+                .setInMemory(true)
+                .setBlocksize(8 * 1024)
+                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                    // Disable blooms for meta.  Needs work.  Seems to mess w/ 
getClosestOrBefore.
+                .setBloomFilterType(BloomType.NONE)
+                    // Enable cache of data blocks in L1 if more than one 
caching tier deployed:
+                    // e.g. if using CombinedBlockCache (BucketCache).
+                .setCacheDataInL1(true)
+        }) {
+    };
+    metaDescriptor.addCoprocessor(
+        "org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
+        null, Coprocessor.PRIORITY_SYSTEM, null);
+    return metaDescriptor;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
index 3b1bbb7..3afce96 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/SplitLogManagerCoordination.java
@@ -42,11 +42,11 @@ import com.google.common.annotations.VisibleForTesting;
  * <P>
  * Methods required for task life circle: <BR>
  * {@link #markRegionsRecovering(ServerName, Set)} mark regions for log 
replaying. Used by
- * {@link org.apache.hadoop.hbase.master.MasterFileSystem} <BR>
+ * {@link org.apache.hadoop.hbase.fs.MasterFileSystem} <BR>
  * {@link #removeRecoveringRegions(Set, Boolean)} make regions cleanup that 
previous were marked as
  * recovering. Called after all tasks processed <BR>
  * {@link #removeStaleRecoveringRegions(Set)} remove stale recovering. called 
by
- * {@link org.apache.hadoop.hbase.master.MasterFileSystem} after Active Master 
is initialized <BR>
+ * {@link org.apache.hadoop.hbase.fs.MasterFileSystem} after Active Master is 
initialized <BR>
  * {@link #getLastRecoveryTime()} required for garbage collector and should 
indicate when the last
  * recovery has been made<BR>
  * {@link #checkTaskStillAvailable(String)} Check that task is still there <BR>

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/FsContext.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/FsContext.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/FsContext.java
new file mode 100644
index 0000000..ced095d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/FsContext.java
@@ -0,0 +1,28 @@
+/**
+ *
+ * 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.fs;
+
+public enum FsContext {
+  TEMP,
+  DATA,
+  ARCHIVE,
+  SNAPSHOT,
+  SIDELINE,
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterFileSystem.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterFileSystem.java
new file mode 100644
index 0000000..2a58b9e
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterFileSystem.java
@@ -0,0 +1,320 @@
+/**
+ *
+ * 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.fs;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.ClusterId;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.fs.legacy.LegacyMasterFileSystem;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.mob.MobUtils;
+
+@InterfaceAudience.Private
+public abstract class MasterFileSystem {
+  private static Log LOG = LogFactory.getLog(MasterFileSystem.class);
+
+  // Persisted unique cluster ID
+  private ClusterId clusterId;
+
+  private Configuration conf;
+  private FileSystem fs;
+  private Path rootDir;
+
+  protected MasterFileSystem(Configuration conf, FileSystem fs, Path rootDir) {
+    this.rootDir = rootDir;
+    this.conf = conf;
+    this.fs = fs;
+  }
+
+  public Configuration getConfiguration() { return conf; }
+  public FileSystem getFileSystem() { return fs; }
+  public Path getRootDir() { return rootDir; }
+
+  // ==========================================================================
+  //  PUBLIC Interfaces - Visitors
+  // ==========================================================================
+  public interface NamespaceVisitor {
+    void visitNamespace(String namespace) throws IOException;
+  }
+
+  public interface TableVisitor {
+    void visitTable(TableName tableName) throws IOException;
+  }
+
+  public interface RegionVisitor {
+    void visitRegion(HRegionInfo regionInfo) throws IOException;
+  }
+
+  // ==========================================================================
+  //  PUBLIC Methods - Namespace related
+  // ==========================================================================
+  public abstract void createNamespace(NamespaceDescriptor nsDescriptor) 
throws IOException;
+  public abstract void deleteNamespace(String namespaceName) throws 
IOException;
+  public abstract Collection<String> getNamespaces(FsContext ctx) throws 
IOException;
+
+  public Collection<String> getNamespaces() throws IOException {
+    return getNamespaces(FsContext.DATA);
+  }
+  // should return or get a NamespaceDescriptor? how is that different from 
HTD?
+
+  // ==========================================================================
+  //  PUBLIC Methods - Table Descriptor related
+  // ==========================================================================
+  public HTableDescriptor getTableDescriptor(TableName tableName)
+      throws IOException {
+    return getTableDescriptor(FsContext.DATA, tableName);
+  }
+
+  public boolean createTableDescriptor(HTableDescriptor tableDesc, boolean 
force)
+      throws IOException {
+    return createTableDescriptor(FsContext.DATA, tableDesc, force);
+  }
+
+  public void updateTableDescriptor(HTableDescriptor tableDesc) throws 
IOException {
+    updateTableDescriptor(FsContext.DATA, tableDesc);
+  }
+
+  public abstract HTableDescriptor getTableDescriptor(FsContext ctx, TableName 
tableName)
+      throws IOException;
+  public abstract boolean createTableDescriptor(FsContext ctx, 
HTableDescriptor tableDesc,
+      boolean force) throws IOException;
+  public abstract void updateTableDescriptor(FsContext ctx, HTableDescriptor 
tableDesc)
+      throws IOException;
+
+  // ==========================================================================
+  //  PUBLIC Methods - Table related
+  // ==========================================================================
+  public void deleteTable(TableName tableName) throws IOException {
+    deleteTable(FsContext.DATA, tableName);
+  }
+
+  public Collection<TableName> getTables(String namespace) throws IOException {
+    return getTables(FsContext.DATA, namespace);
+  }
+
+  public abstract void deleteTable(FsContext ctx, TableName tableName) throws 
IOException;
+
+  public abstract Collection<TableName> getTables(FsContext ctx, String 
namespace)
+    throws IOException;
+
+  public Collection<TableName> getTables() throws IOException {
+    ArrayList<TableName> tables = new ArrayList<TableName>();
+    for (String ns: getNamespaces()) {
+      tables.addAll(getTables(ns));
+    }
+    return tables;
+  }
+
+  // ==========================================================================
+  //  PUBLIC Methods - bootstrap
+  // ==========================================================================
+  public abstract Path getTempDir();
+
+  public void logFileSystemState(Log log) throws IOException {
+    FSUtils.logFileSystemState(getFileSystem(), getRootDir(), LOG);
+  }
+
+  /**
+   * @return The unique identifier generated for this cluster
+   */
+  public ClusterId getClusterId() {
+    return clusterId;
+  }
+
+  protected void bootstrap() throws IOException {
+    // check if the root directory exists
+    createInitialLayout(getRootDir(), conf, this.fs);
+
+    // check if temp directory exists and clean it
+    startupCleanup();
+  }
+
+  protected abstract void bootstrapMeta() throws IOException;
+  protected abstract void startupCleanup() throws IOException;
+
+  /**
+   * Create initial layout in filesystem.
+   * <ol>
+   * <li>Check if the meta region exists and is readable, if not create it.
+   * Create hbase.version and the hbase:meta directory if not one.
+   * </li>
+   * <li>Create a log archive directory for RS to put archived logs</li>
+   * </ol>
+   * Idempotent.
+   */
+  private void createInitialLayout(final Path rd, final Configuration c, final 
FileSystem fs)
+      throws IOException {
+    // If FS is in safe mode wait till out of it.
+    FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 
1000));
+
+    boolean isSecurityEnabled = 
"kerberos".equalsIgnoreCase(c.get("hbase.security.authentication"));
+    FsPermission rootDirPerms = new FsPermission(c.get("hbase.rootdir.perms", 
"700"));
+
+    // Filesystem is good. Go ahead and check for hbase.rootdir.
+    try {
+      if (!fs.exists(rd)) {
+        if (isSecurityEnabled) {
+          fs.mkdirs(rd, rootDirPerms);
+        } else {
+          fs.mkdirs(rd);
+        }
+        // DFS leaves safe mode with 0 DNs when there are 0 blocks.
+        // We used to handle this by checking the current DN count and waiting 
until
+        // it is nonzero. With security, the check for datanode count doesn't 
work --
+        // it is a privileged op. So instead we adopt the strategy of the 
jobtracker
+        // and simply retry file creation during bootstrap indefinitely. As 
soon as
+        // there is one datanode it will succeed. Permission problems should 
have
+        // already been caught by mkdirs above.
+        FSUtils.setVersion(fs, rd, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
+          10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
+            HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
+      } else {
+        if (!fs.isDirectory(rd)) {
+          throw new IllegalArgumentException(rd.toString() + " is not a 
directory");
+        }
+        if (isSecurityEnabled && 
!rootDirPerms.equals(fs.getFileStatus(rd).getPermission())) {
+          // check whether the permission match
+          LOG.warn("Found rootdir permissions NOT matching expected 
\"hbase.rootdir.perms\" for "
+              + "rootdir=" + rd.toString() + " permissions=" + 
fs.getFileStatus(rd).getPermission()
+              + " and  \"hbase.rootdir.perms\" configured as "
+              + c.get("hbase.rootdir.perms", "700") + ". Automatically setting 
the permissions. You"
+              + " can change the permissions by setting 
\"hbase.rootdir.perms\" in hbase-site.xml "
+              + "and restarting the master");
+          fs.setPermission(rd, rootDirPerms);
+        }
+        // as above
+        FSUtils.checkVersion(fs, rd, true, 
c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
+          10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
+            HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
+      }
+    } catch (DeserializationException de) {
+      LOG.fatal("Please fix invalid configuration for " + 
HConstants.HBASE_DIR, de);
+      IOException ioe = new IOException();
+      ioe.initCause(de);
+      throw ioe;
+    } catch (IllegalArgumentException iae) {
+      LOG.fatal("Please fix invalid configuration for "
+        + HConstants.HBASE_DIR + " " + rd.toString(), iae);
+      throw iae;
+    }
+    // Make sure cluster ID exists
+    if (!FSUtils.checkClusterIdExists(fs, rd, c.getInt(
+        HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000))) {
+      FSUtils.setClusterId(fs, rd, new ClusterId(), 
c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
+    }
+    clusterId = FSUtils.getClusterId(fs, rd);
+
+    // Make sure the meta region exists!
+    bootstrapMeta();
+  }
+
+  // TODO: Move in HRegionFileSystem
+  public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName, 
boolean hasMob)
+      throws IOException {
+    // archive family store files
+    Path tableDir = FSUtils.getTableDir(getRootDir(), region.getTable());
+    HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
+
+    // delete the family folder
+    Path familyDir = new Path(tableDir,
+      new Path(region.getEncodedName(), Bytes.toString(familyName)));
+    if (fs.delete(familyDir, true) == false) {
+      if (fs.exists(familyDir)) {
+        throw new IOException("Could not delete family "
+            + Bytes.toString(familyName) + " from FileSystem for region "
+            + region.getRegionNameAsString() + "(" + region.getEncodedName()
+            + ")");
+      }
+    }
+
+    // archive and delete mob files
+    if (hasMob) {
+      Path mobTableDir =
+          FSUtils.getTableDir(new Path(getRootDir(), 
MobConstants.MOB_DIR_NAME), region.getTable());
+      HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(region.getTable());
+      Path mobFamilyDir =
+          new Path(mobTableDir,
+              new Path(mobRegionInfo.getEncodedName(), 
Bytes.toString(familyName)));
+      // archive mob family store files
+      MobUtils.archiveMobStoreFiles(conf, fs, mobRegionInfo, mobFamilyDir, 
familyName);
+
+      if (!fs.delete(mobFamilyDir, true)) {
+        throw new IOException("Could not delete mob store files for family "
+            + Bytes.toString(familyName) + " from FileSystem region "
+            + mobRegionInfo.getRegionNameAsString() + "(" + 
mobRegionInfo.getEncodedName() + ")");
+      }
+    }
+  }
+
+  // ==========================================================================
+  //  PUBLIC
+  // ==========================================================================
+  public static MasterFileSystem open(Configuration conf, boolean bootstrap)
+      throws IOException {
+    return open(conf, FSUtils.getCurrentFileSystem(conf), 
FSUtils.getRootDir(conf), bootstrap);
+  }
+
+  public static MasterFileSystem open(Configuration conf, FileSystem fs,
+      Path rootDir, boolean bootstrap) throws IOException {
+    // Cover both bases, the old way of setting default fs and the new.
+    // We're supposed to run on 0.20 and 0.21 anyways.
+    fs = rootDir.getFileSystem(conf);
+    FSUtils.setFsDefault(conf, new Path(fs.getUri()));
+    // make sure the fs has the same conf
+    fs.setConf(conf);
+
+    MasterFileSystem mfs = getInstance(conf, fs, rootDir);
+    if (bootstrap) {
+      mfs.bootstrap();
+    }
+    HFileSystem.addLocationsOrderInterceptor(conf);
+    return mfs;
+  }
+
+  private static MasterFileSystem getInstance(Configuration conf, final 
FileSystem fs,
+      Path rootDir) throws IOException {
+    String fsType = conf.get("hbase.fs.layout.type", "legacy").toLowerCase();
+    switch (fsType) {
+      case "legacy":
+        return new LegacyMasterFileSystem(conf, fs, rootDir);
+      default:
+        throw new IOException("Invalid filesystem type " + fsType);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java
new file mode 100644
index 0000000..3af40fa
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java
@@ -0,0 +1,69 @@
+/**
+ *
+ * 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.fs.legacy;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+
+public final class LegacyLayout {
+  /** Name of the region info file that resides just under the region 
directory. */
+  public final static String REGION_INFO_FILE = ".regioninfo";
+
+  private LegacyLayout() {}
+
+  public static Path getDataDir(final Path rootDir) {
+    return new Path(rootDir, HConstants.BASE_NAMESPACE_DIR);
+  }
+
+  public static Path getSidelineDir(Path rootDir) {
+    return new Path(rootDir, HConstants.HBCK_SIDELINEDIR_NAME);
+  }
+
+  public static Path getSnapshotDir(Path rootDir) {
+    return new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME);
+  }
+
+  public static Path getSnapshotDir(Path baseSnapshotDir, String snapshotName) 
{
+    return new Path(baseSnapshotDir, snapshotName);
+  }
+
+  public static Path getArchiveDir(Path rootDir) {
+    return new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
+  }
+
+  public static Path getTempDir(Path rootDir) {
+    return new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY);
+  }
+
+  public static Path getNamespaceDir(Path baseDataDir, String namespace) {
+    return new Path(baseDataDir, namespace);
+  }
+
+  public static Path getTableDir(Path baseDataDir, TableName table) {
+    Path nsDir = getNamespaceDir(baseDataDir, table.getNamespaceAsString());
+    return new Path(nsDir, table.getQualifierAsString());
+  }
+
+  public static Path getRegionDir(Path baseDataDir, TableName table, 
HRegionInfo hri) {
+    return new Path(getTableDir(baseDataDir, table), hri.getEncodedName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterFileSystem.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterFileSystem.java
new file mode 100644
index 0000000..f330b6e
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterFileSystem.java
@@ -0,0 +1,262 @@
+/**
+ *
+ * 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.fs.legacy;
+
+import java.io.IOException;
+import java.io.FileNotFoundException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.fs.FsContext;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.MetaUtils;
+
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+
+@InterfaceAudience.Private
+public class LegacyMasterFileSystem extends MasterFileSystem {
+  private static final Log LOG = 
LogFactory.getLog(LegacyMasterFileSystem.class);
+
+  private final Path sidelineDir;
+  private final Path snapshotDir;
+  private final Path archiveDataDir;
+  private final Path archiveDir;
+  private final Path tmpDataDir;
+  private final Path dataDir;
+  private final Path tmpDir;
+
+  public LegacyMasterFileSystem(Configuration conf, FileSystem fs, Path 
rootDir) {
+    super(conf, fs, rootDir);
+
+    // base directories
+    this.sidelineDir = LegacyLayout.getSidelineDir(rootDir);
+    this.snapshotDir = LegacyLayout.getSnapshotDir(rootDir);
+    this.archiveDir = LegacyLayout.getArchiveDir(rootDir);
+    this.archiveDataDir = LegacyLayout.getDataDir(this.archiveDir);
+    this.dataDir = LegacyLayout.getDataDir(rootDir);
+    this.tmpDir = LegacyLayout.getTempDir(rootDir);
+    this.tmpDataDir = LegacyLayout.getDataDir(this.tmpDir);
+  }
+
+  // ==========================================================================
+  //  PUBLIC Methods - Namespace related
+  // ==========================================================================
+  public void createNamespace(NamespaceDescriptor nsDescriptor) throws 
IOException {
+    getFileSystem().mkdirs(getNamespaceDir(FsContext.DATA, 
nsDescriptor.getName()));
+  }
+
+  public void deleteNamespace(String namespaceName) throws IOException {
+    FileSystem fs = getFileSystem();
+    Path nsDir = getNamespaceDir(FsContext.DATA, namespaceName);
+
+    try {
+      for (FileStatus status : fs.listStatus(nsDir)) {
+        if 
(!HConstants.HBASE_NON_TABLE_DIRS.contains(status.getPath().getName())) {
+          throw new IOException("Namespace directory contains table dir: " + 
status.getPath());
+        }
+      }
+      if (!fs.delete(nsDir, true)) {
+        throw new IOException("Failed to remove namespace: " + namespaceName);
+      }
+    } catch (FileNotFoundException e) {
+      // File already deleted, continue
+      LOG.debug("deleteDirectory throws exception: " + e);
+    }
+  }
+
+  public Collection<String> getNamespaces(FsContext ctx) throws IOException {
+    FileStatus[] stats = FSUtils.listStatus(getFileSystem(), 
getNamespaceDir(ctx));
+    if (stats == null) return Collections.emptyList();
+
+    ArrayList<String> namespaces = new ArrayList<String>(stats.length);
+    for (int i = 0; i < stats.length; ++i) {
+      namespaces.add(stats[i].getPath().getName());
+    }
+    return namespaces;
+  }
+
+  // should return or get a NamespaceDescriptor? how is that different from 
HTD?
+
+  // ==========================================================================
+  //  PUBLIC Methods - Table Descriptor related
+  // 
==========================================================================s
+  @Override
+  public boolean createTableDescriptor(FsContext ctx, HTableDescriptor 
tableDesc, boolean force)
+      throws IOException {
+    return LegacyTableDescriptor.createTableDescriptor(getFileSystem(),
+      getTableDir(ctx, tableDesc.getTableName()), tableDesc, force);
+  }
+
+  @Override
+  public void updateTableDescriptor(FsContext ctx, HTableDescriptor tableDesc) 
throws IOException {
+    LegacyTableDescriptor.updateTableDescriptor(getFileSystem(),
+        getTableDir(ctx, tableDesc.getTableName()), tableDesc);
+  }
+
+  @Override
+  public HTableDescriptor getTableDescriptor(FsContext ctx, TableName 
tableName)
+      throws IOException {
+    return LegacyTableDescriptor.getTableDescriptorFromFs(
+        getFileSystem(), getTableDir(ctx, tableName));
+  }
+
+  // ==========================================================================
+  //  PUBLIC Methods - Table related
+  // ==========================================================================
+  @Override
+  public void deleteTable(FsContext ctx, TableName tableName) throws 
IOException {
+    Path tableDir = getTableDir(ctx, tableName);
+    if (!FSUtils.deleteDirectory(getFileSystem(), tableDir)) {
+      throw new IOException("Failed delete of " + tableName);
+    }
+  }
+
+  @Override
+  public Collection<TableName> getTables(FsContext ctx, String namespace)
+      throws IOException {
+    FileStatus[] stats = FSUtils.listStatus(getFileSystem(),
+        getNamespaceDir(ctx, namespace), new 
FSUtils.UserTableDirFilter(getFileSystem()));
+    if (stats == null) return Collections.emptyList();
+
+    ArrayList<TableName> tables = new ArrayList<TableName>(stats.length);
+    for (int i = 0; i < stats.length; ++i) {
+      tables.add(TableName.valueOf(namespace, stats[i].getPath().getName()));
+    }
+    return tables;
+  }
+
+  // ==========================================================================
+  //  PROTECTED Methods - Bootstrap
+  // ==========================================================================
+  @Override
+  protected void bootstrapMeta() throws IOException {
+    if (!FSUtils.metaRegionExists(getFileSystem(), getRootDir())) {
+      bootstrapMeta(getRootDir(), getConfiguration());
+    }
+
+    // Create tableinfo-s for hbase:meta if not already there.
+    // assume, created table descriptor is for enabling table
+    // meta table is a system table, so descriptors are predefined,
+    // we should get them from registry.
+    
createTableDescriptor(HTableDescriptor.metaTableDescriptor(getConfiguration()), 
false);
+  }
+
+  private static void bootstrapMeta(final Path rd, final Configuration c) 
throws IOException {
+    LOG.info("BOOTSTRAP: creating hbase:meta region");
+    try {
+      // Bootstrapping, make sure blockcache is off.  Else, one will be
+      // created here in bootstrap and it'll need to be cleaned up.  Better to
+      // not make it in first place.  Turn off block caching for bootstrap.
+      // Enable after.
+      HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
+      HTableDescriptor metaDescriptor = 
HTableDescriptor.metaTableDescriptor(c);
+      MetaUtils.setInfoFamilyCachingForMeta(metaDescriptor, false);
+      HRegion meta = HRegion.createHRegion(metaHRI, rd, c, metaDescriptor, 
null);
+      MetaUtils.setInfoFamilyCachingForMeta(metaDescriptor, true);
+      meta.close();
+    } catch (IOException e) {
+        e = e instanceof RemoteException ?
+                ((RemoteException)e).unwrapRemoteException() : e;
+      LOG.error("bootstrap", e);
+      throw e;
+    }
+  }
+
+  @Override
+  protected void startupCleanup() throws IOException {
+    checkTempDir(getTempDir(), getConfiguration(), getFileSystem());
+  }
+
+  /**
+   * Make sure the hbase temp directory exists and is empty.
+   * NOTE that this method is only executed once just after the master becomes 
the active one.
+   */
+  private void checkTempDir(final Path tmpdir, final Configuration c, final 
FileSystem fs)
+      throws IOException {
+    // If the temp directory exists, clear the content (left over, from the 
previous run)
+    if (fs.exists(tmpdir)) {
+      // Archive table in temp, maybe left over from failed deletion,
+      // if not the cleaner will take care of them.
+      for (Path tabledir: FSUtils.getTableDirs(fs, tmpdir)) {
+        for (Path regiondir: FSUtils.getRegionDirs(fs, tabledir)) {
+          HFileArchiver.archiveRegion(fs, getRootDir(), tabledir, regiondir);
+        }
+      }
+      if (!fs.delete(tmpdir, true)) {
+        throw new IOException("Unable to clean the temp directory: " + tmpdir);
+      }
+    }
+
+    // Create the temp directory
+    if (!fs.mkdirs(tmpdir)) {
+      throw new IOException("HBase temp directory '" + tmpdir + "' creation 
failure.");
+    }
+  }
+
+  // ==========================================================================
+  //  PROTECTED Methods - Path
+  // ==========================================================================
+  protected Path getNamespaceDir(FsContext ctx) {
+    return getBaseDirFromContext(ctx);
+  }
+
+  protected Path getNamespaceDir(FsContext ctx, String namespace) {
+    return LegacyLayout.getNamespaceDir(getBaseDirFromContext(ctx), namespace);
+  }
+
+  protected Path getTableDir(FsContext ctx, TableName table) {
+    return LegacyLayout.getTableDir(getBaseDirFromContext(ctx), table);
+  }
+
+  protected Path getRegionDir(FsContext ctx, TableName table, HRegionInfo hri) 
{
+    return LegacyLayout.getRegionDir(getBaseDirFromContext(ctx), table, hri);
+  }
+
+  public Path getTempDir() {
+    return tmpDir;
+  }
+
+  protected Path getBaseDirFromContext(FsContext ctx) {
+    switch (ctx) {
+      case TEMP: return tmpDataDir;
+      case DATA: return dataDir;
+      case ARCHIVE: return archiveDataDir;
+      case SNAPSHOT: return snapshotDir;
+      case SIDELINE: return sidelineDir;
+      default: throw new RuntimeException("Invalid context: " + ctx);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyTableDescriptor.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyTableDescriptor.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyTableDescriptor.java
new file mode 100644
index 0000000..e4d5bc1
--- /dev/null
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyTableDescriptor.java
@@ -0,0 +1,399 @@
+/**
+ * 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.fs.legacy;
+
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.primitives.Ints;
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.HTableDescriptor;
+import org.apache.hadoop.hbase.TableDescriptors;
+import org.apache.hadoop.hbase.TableInfoMissingException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.util.*;
+
+/**
+ * Implementation of {@link TableDescriptors} that reads descriptors from the
+ * passed filesystem.  It expects descriptors to be in a file in the
+ * {@link #TABLEINFO_DIR} subdir of the table's directory in FS.  Can be 
read-only
+ *  -- i.e. does not modify the filesystem or can be read and write.
+ *
+ * <p>Also has utility for keeping up the table descriptors tableinfo file.
+ * The table schema file is kept in the {@link #TABLEINFO_DIR} subdir
+ * of the table directory in the filesystem.
+ * It has a {@link #TABLEINFO_FILE_PREFIX} and then a suffix that is the
+ * edit sequenceid: e.g. <code>.tableinfo.0000000003</code>.  This sequenceid
+ * is always increasing.  It starts at zero.  The table schema file with the
+ * highest sequenceid has the most recent schema edit. Usually there is one 
file
+ * only, the most recent but there may be short periods where there are more
+ * than one file. Old files are eventually cleaned.  Presumption is that there
+ * will not be lots of concurrent clients making table schema edits.  If so,
+ * the below needs a bit of a reworking and perhaps some supporting api in 
hdfs.
+ */
+@InterfaceAudience.Private
+public final class LegacyTableDescriptor {
+  private static final Log LOG = 
LogFactory.getLog(LegacyTableDescriptor.class);
+
+  /** The file name prefix used to store HTD in HDFS  */
+  public static final String TABLEINFO_FILE_PREFIX = ".tableinfo";
+  public static final String TABLEINFO_DIR = ".tabledesc";
+  public static final String TMP_DIR = ".tmp";
+
+  private LegacyTableDescriptor() {}
+
+  /**
+   * Find the most current table info file for the table located in the given 
table directory.
+   *
+   * Looks within the {@link #TABLEINFO_DIR} subdirectory of the given 
directory for any table info
+   * files and takes the 'current' one - meaning the one with the highest 
sequence number if present
+   * or no sequence number at all if none exist (for backward compatibility 
from before there
+   * were sequence numbers).
+   *
+   * @return The file status of the current table info file or null if it does 
not exist
+   * @throws IOException
+   */
+  public static FileStatus getTableInfoPath(FileSystem fs, Path tableDir) 
throws IOException {
+    return getTableInfoPath(fs, tableDir, false);
+  }
+
+  /**
+   * Find the most current table info file for the table in the given table 
directory.
+   *
+   * Looks within the {@link #TABLEINFO_DIR} subdirectory of the given 
directory for any table info
+   * files and takes the 'current' one - meaning the one with the highest 
sequence number if
+   * present or no sequence number at all if none exist (for backward 
compatibility from before
+   * there were sequence numbers).
+   * If there are multiple table info files found and removeOldFiles is true 
it also deletes the
+   * older files.
+   *
+   * @return The file status of the current table info file or null if none 
exist
+   * @throws IOException
+   */
+  private static FileStatus getTableInfoPath(FileSystem fs, Path tableDir, 
boolean removeOldFiles)
+      throws IOException {
+    Path tableInfoDir = new Path(tableDir, TABLEINFO_DIR);
+    return getCurrentTableInfoStatus(fs, tableInfoDir, removeOldFiles);
+  }
+
+  /**
+   * Find the most current table info file in the given directory
+   *
+   * Looks within the given directory for any table info files
+   * and takes the 'current' one - meaning the one with the highest sequence 
number if present
+   * or no sequence number at all if none exist (for backward compatibility 
from before there
+   * were sequence numbers).
+   * If there are multiple possible files found
+   * and the we're not in read only mode it also deletes the older files.
+   *
+   * @return The file status of the current table info file or null if it does 
not exist
+   * @throws IOException
+   */
+  // only visible for FSTableDescriptorMigrationToSubdir, can be removed with 
that
+  public static FileStatus getCurrentTableInfoStatus(FileSystem fs, Path dir, 
boolean removeOldFiles)
+      throws IOException {
+    FileStatus [] status = FSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
+    if (status == null || status.length < 1) return null;
+    FileStatus mostCurrent = null;
+    for (FileStatus file : status) {
+      if (mostCurrent == null || TABLEINFO_FILESTATUS_COMPARATOR.compare(file, 
mostCurrent) < 0) {
+        mostCurrent = file;
+      }
+    }
+    if (removeOldFiles && status.length > 1) {
+      // Clean away old versions
+      for (FileStatus file : status) {
+        Path path = file.getPath();
+        if (file != mostCurrent) {
+          if (!fs.delete(path, false)) {
+            LOG.warn("Failed cleanup of " + path);
+          } else {
+            LOG.debug("Cleaned up old tableinfo file " + path);
+          }
+        }
+      }
+    }
+    return mostCurrent;
+  }
+
+  /**
+   * Update table descriptor on the file system
+   * @throws IOException Thrown if failed update.
+   * @throws NotImplementedException if in read only mode
+   */
+  public static void updateTableDescriptor(FileSystem fs, Path tableDir, 
HTableDescriptor td)
+      throws IOException {
+    TableName tableName = td.getTableName();
+    Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(fs, 
tableDir));
+    if (p == null) throw new IOException("Failed update");
+    LOG.info("Updated tableinfo=" + p);
+  }
+
+  /**
+   * Compare {@link FileStatus} instances by {@link Path#getName()}. Returns in
+   * reverse order.
+   */
+  @VisibleForTesting
+  public static final Comparator<FileStatus> TABLEINFO_FILESTATUS_COMPARATOR =
+  new Comparator<FileStatus>() {
+    @Override
+    public int compare(FileStatus left, FileStatus right) {
+      return right.compareTo(left);
+    }};
+
+  private static final PathFilter TABLEINFO_PATHFILTER = new PathFilter() {
+    @Override
+    public boolean accept(Path p) {
+      // Accept any file that starts with TABLEINFO_NAME
+      return p.getName().startsWith(TABLEINFO_FILE_PREFIX);
+    }};
+
+  /**
+   * Width of the sequenceid that is a suffix on a tableinfo file.
+   */
+  @VisibleForTesting public static final int WIDTH_OF_SEQUENCE_ID = 10;
+
+  /*
+   * @param number Number to use as suffix.
+   * @return Returns zero-prefixed decimal version of passed
+   * number (Does absolute in case number is negative).
+   */
+  private static String formatTableInfoSequenceId(final int number) {
+    byte [] b = new byte[WIDTH_OF_SEQUENCE_ID];
+    int d = Math.abs(number);
+    for (int i = b.length - 1; i >= 0; i--) {
+      b[i] = (byte)((d % 10) + '0');
+      d /= 10;
+    }
+    return Bytes.toString(b);
+  }
+
+  /**
+   * Regex to eat up sequenceid suffix on a .tableinfo file.
+   * Use regex because may encounter oldstyle .tableinfos where there is no
+   * sequenceid on the end.
+   */
+  private static final Pattern TABLEINFO_FILE_REGEX =
+    Pattern.compile(TABLEINFO_FILE_PREFIX + "(\\.([0-9]{" + 
WIDTH_OF_SEQUENCE_ID + "}))?$");
+
+  /**
+   * @param p Path to a <code>.tableinfo</code> file.
+   * @return The current editid or 0 if none found.
+   */
+  @VisibleForTesting
+  public static int getTableInfoSequenceId(final Path p) {
+    if (p == null) return 0;
+    Matcher m = TABLEINFO_FILE_REGEX.matcher(p.getName());
+    if (!m.matches()) throw new IllegalArgumentException(p.toString());
+    String suffix = m.group(2);
+    if (suffix == null || suffix.length() <= 0) return 0;
+    return Integer.parseInt(m.group(2));
+  }
+
+  /**
+   * @param sequenceid
+   * @return Name of tableinfo file.
+   */
+  @VisibleForTesting
+  public static String getTableInfoFileName(final int sequenceid) {
+    return TABLEINFO_FILE_PREFIX + "." + formatTableInfoSequenceId(sequenceid);
+  }
+
+  /**
+   * Returns the latest table descriptor for the given table directly from the 
file system
+   * if it exists, bypassing the local cache.
+   * Returns null if it's not found.
+   */
+  public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs,
+      Path hbaseRootDir, TableName tableName) throws IOException {
+    Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
+    return readTableDescriptor(fs, tableDir);
+  }
+
+  /**
+   * Returns the latest table descriptor for the table located at the given 
directory
+   * directly from the file system if it exists.
+   * @throws TableInfoMissingException if there is no descriptor
+   */
+  public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs, Path 
tableDir)
+      throws IOException {
+    final HTableDescriptor htd = readTableDescriptor(fs, tableDir);
+    if (htd == null) {
+      throw new TableInfoMissingException("No table descriptor file under " + 
tableDir);
+    }
+    return htd;
+  }
+
+  /**
+   * @return null if not found
+   */
+  private static HTableDescriptor readTableDescriptor(FileSystem fs, Path 
tableDir)
+      throws IOException {
+    FileStatus status = getTableInfoPath(fs, tableDir, false);
+    if (status == null) {
+      return null;
+    }
+    return readTableDescriptor(fs, status);
+  }
+
+  private static HTableDescriptor readTableDescriptor(FileSystem fs, 
FileStatus status)
+      throws IOException {
+    int len = Ints.checkedCast(status.getLen());
+    byte[] content = new byte[len];
+    FSUtils.readFully(fs, status.getPath(), content);
+    try {
+      HTableDescriptor td = HTableDescriptor.parseFrom(content);
+      return td;
+    } catch (DeserializationException exception) {
+      throw new IOException("failed to deserialize descriptor.", exception);
+    }
+  }
+
+  /**
+   * Deletes files matching the table info file pattern within the given 
directory
+   * whose sequenceId is at most the given max sequenceId.
+   */
+  private static void deleteTableDescriptorFiles(FileSystem fs, Path dir, int 
maxSequenceId)
+      throws IOException {
+    FileStatus [] status = FSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
+    for (FileStatus file : status) {
+      Path path = file.getPath();
+      int sequenceId = getTableInfoSequenceId(path);
+      if (sequenceId <= maxSequenceId) {
+        boolean success = FSUtils.delete(fs, path, false);
+        if (success) {
+          LOG.debug("Deleted table descriptor at " + path);
+        } else {
+          LOG.error("Failed to delete descriptor at " + path);
+        }
+      }
+    }
+  }
+
+  /**
+   * Attempts to write a new table descriptor to the given table's directory.
+   * It first writes it to the .tmp dir then uses an atomic rename to move it 
into place.
+   * It begins at the currentSequenceId + 1 and tries 10 times to find a new 
sequence number
+   * not already in use.
+   * Removes the current descriptor file if passed in.
+   *
+   * @return Descriptor file or null if we failed write.
+   */
+  private static Path writeTableDescriptor(final FileSystem fs,
+    final HTableDescriptor htd, final Path tableDir,
+    final FileStatus currentDescriptorFile) throws IOException {
+    // Get temporary dir into which we'll first write a file to avoid 
half-written file phenomenon.
+    // This directory is never removed to avoid removing it out from under a 
concurrent writer.
+    Path tmpTableDir = new Path(tableDir, TMP_DIR);
+    Path tableInfoDir = new Path(tableDir, TABLEINFO_DIR);
+
+    // What is current sequenceid?  We read the current sequenceid from
+    // the current file.  After we read it, another thread could come in and
+    // compete with us writing out next version of file.  The below retries
+    // should help in this case some but its hard to do guarantees in face of
+    // concurrent schema edits.
+    int currentSequenceId = currentDescriptorFile == null ? 0 :
+      getTableInfoSequenceId(currentDescriptorFile.getPath());
+    int newSequenceId = currentSequenceId;
+
+    // Put arbitrary upperbound on how often we retry
+    int retries = 10;
+    int retrymax = currentSequenceId + retries;
+    Path tableInfoDirPath = null;
+    do {
+      newSequenceId += 1;
+      String filename = getTableInfoFileName(newSequenceId);
+      Path tempPath = new Path(tmpTableDir, filename);
+      if (fs.exists(tempPath)) {
+        LOG.debug(tempPath + " exists; retrying up to " + retries + " times");
+        continue;
+      }
+      tableInfoDirPath = new Path(tableInfoDir, filename);
+      try {
+        writeTD(fs, tempPath, htd);
+        fs.mkdirs(tableInfoDirPath.getParent());
+        if (!fs.rename(tempPath, tableInfoDirPath)) {
+          throw new IOException("Failed rename of " + tempPath + " to " + 
tableInfoDirPath);
+        }
+        LOG.debug("Wrote descriptor into: " + tableInfoDirPath);
+      } catch (IOException ioe) {
+        // Presume clash of names or something; go around again.
+        LOG.debug("Failed write and/or rename; retrying", ioe);
+        if (!FSUtils.deleteDirectory(fs, tempPath)) {
+          LOG.warn("Failed cleanup of " + tempPath);
+        }
+        tableInfoDirPath = null;
+        continue;
+      }
+      break;
+    } while (newSequenceId < retrymax);
+    if (tableInfoDirPath != null) {
+      // if we succeeded, remove old table info files.
+      deleteTableDescriptorFiles(fs, tableInfoDir, newSequenceId - 1);
+    }
+    return tableInfoDirPath;
+  }
+
+  private static void writeTD(final FileSystem fs, final Path p, final 
HTableDescriptor htd)
+      throws IOException {
+    // We used to write this file out as a serialized HTD Writable followed by 
two '\n's and then
+    // the toString version of HTD.  Now we just write out the pb 
serialization.
+    FSUtils.writeFully(fs, p, htd.toByteArray(), false);
+  }
+
+  /**
+   * Create a new HTableDescriptor in HDFS in the specified table directory. 
Happens when we create
+   * a new table or snapshot a table.
+   * @param tableDir table directory under which we should write the file
+   * @param htd description of the table to write
+   * @param forceCreation if <tt>true</tt>,then even if previous table 
descriptor is present it will
+   *          be overwritten
+   * @return <tt>true</tt> if the we successfully created the file, 
<tt>false</tt> if the file
+   *         already exists and we weren't forcing the descriptor creation.
+   * @throws IOException if a filesystem error occurs
+   */
+  public static boolean createTableDescriptor(FileSystem fs, Path tableDir,
+      HTableDescriptor htd, boolean forceCreation) throws IOException {
+    FileStatus status = getTableInfoPath(fs, tableDir);
+    if (status != null) {
+      LOG.debug("Current tableInfoPath = " + status.getPath());
+      if (!forceCreation) {
+        if (fs.exists(status.getPath()) && status.getLen() > 0) {
+          if (readTableDescriptor(fs, status).equals(htd)) {
+            LOG.debug("TableInfo already exists.. Skipping creation");
+            return false;
+          }
+        }
+      }
+    }
+    Path p = writeTableDescriptor(fs, htd, tableDir, status);
+    return p != null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 810f95b..c9056e3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -90,6 +90,7 @@ import 
org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
 import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
@@ -681,7 +682,7 @@ public class HMaster extends HRegionServer implements 
MasterServices {
 
     this.masterActiveTime = System.currentTimeMillis();
     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or 
Spring.
-    this.fileSystemManager = new MasterFileSystem(this);
+    this.fileSystemManager = MasterFileSystem.open(conf, true);
     this.walManager = new MasterWalManager(this);
 
     // enable table descriptors cache
@@ -1024,7 +1025,6 @@ public class HMaster extends HRegionServer implements 
MasterServices {
     if (this.serverManager != null) this.serverManager.stop();
     if (this.assignmentManager != null) this.assignmentManager.stop();
     if (this.walManager != null) this.walManager.stop();
-    if (this.fileSystemManager != null) this.fileSystemManager.stop();
     if (this.mpmHost != null) this.mpmHost.stop("server shutting down.");
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
deleted file mode 100644
index 0ce7411..0000000
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ /dev/null
@@ -1,324 +0,0 @@
-/**
- *
- * 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.master;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.ClusterId;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.backup.HFileArchiver;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-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.ipc.RemoteException;
-
-/**
- * This class abstracts a bunch of operations the HMaster needs to interact 
with
- * the underlying file system like creating the initial layout, checking file
- * system status, etc.
- */
-@InterfaceAudience.Private
-public class MasterFileSystem {
-  private static final Log LOG = LogFactory.getLog(MasterFileSystem.class);
-
-  // HBase configuration
-  private final Configuration conf;
-  // Persisted unique cluster ID
-  private ClusterId clusterId;
-  // Keep around for convenience.
-  private final FileSystem fs;
-  // root hbase directory on the FS
-  private final Path rootdir;
-  // hbase temp directory used for table construction and deletion
-  private final Path tempdir;
-
-  private final MasterServices services;
-
-  public MasterFileSystem(MasterServices services) throws IOException {
-    this.conf = services.getConfiguration();
-    this.services = services;
-    // Set filesystem to be that of this.rootdir else we get complaints about
-    // mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
-    // default localfs.  Presumption is that rootdir is fully-qualified before
-    // we get to here with appropriate fs scheme.
-    this.rootdir = FSUtils.getRootDir(conf);
-    this.tempdir = new Path(this.rootdir, HConstants.HBASE_TEMP_DIRECTORY);
-    // Cover both bases, the old way of setting default fs and the new.
-    // We're supposed to run on 0.20 and 0.21 anyways.
-    this.fs = this.rootdir.getFileSystem(conf);
-    FSUtils.setFsDefault(conf, new Path(this.fs.getUri()));
-    // make sure the fs has the same conf
-    fs.setConf(conf);
-    // setup the filesystem variable
-    createInitialFileSystemLayout();
-    HFileSystem.addLocationsOrderInterceptor(conf);
-  }
-
-  /**
-   * Create initial layout in filesystem.
-   * <ol>
-   * <li>Check if the meta region exists and is readable, if not create it.
-   * Create hbase.version and the hbase:meta directory if not one.
-   * </li>
-   * </ol>
-   * Idempotent.
-   */
-  private void createInitialFileSystemLayout() throws IOException {
-    // check if the root directory exists
-    checkRootDir(this.rootdir, conf, this.fs);
-
-    // check if temp directory exists and clean it
-    checkTempDir(this.tempdir, conf, this.fs);
-  }
-
-  public FileSystem getFileSystem() {
-    return this.fs;
-  }
-
-  public Configuration getConfiguration() {
-    return this.conf;
-  }
-
-  /**
-   * @return HBase root dir.
-   */
-  public Path getRootDir() {
-    return this.rootdir;
-  }
-
-  /**
-   * @return HBase temp dir.
-   */
-  public Path getTempDir() {
-    return this.tempdir;
-  }
-
-  /**
-   * @return The unique identifier generated for this cluster
-   */
-  public ClusterId getClusterId() {
-    return clusterId;
-  }
-
-  /**
-   * Get the rootdir.  Make sure its wholesome and exists before returning.
-   * @param rd
-   * @param c
-   * @param fs
-   * @return hbase.rootdir (after checks for existence and bootstrapping if
-   * needed populating the directory with necessary bootup files).
-   * @throws IOException
-   */
-  private Path checkRootDir(final Path rd, final Configuration c, final 
FileSystem fs)
-      throws IOException {
-    // If FS is in safe mode wait till out of it.
-    FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 
1000));
-
-    boolean isSecurityEnabled = 
"kerberos".equalsIgnoreCase(c.get("hbase.security.authentication"));
-    FsPermission rootDirPerms = new FsPermission(c.get("hbase.rootdir.perms", 
"700"));
-
-    // Filesystem is good. Go ahead and check for hbase.rootdir.
-    try {
-      if (!fs.exists(rd)) {
-        if (isSecurityEnabled) {
-          fs.mkdirs(rd, rootDirPerms);
-        } else {
-          fs.mkdirs(rd);
-        }
-        // DFS leaves safe mode with 0 DNs when there are 0 blocks.
-        // We used to handle this by checking the current DN count and waiting 
until
-        // it is nonzero. With security, the check for datanode count doesn't 
work --
-        // it is a privileged op. So instead we adopt the strategy of the 
jobtracker
-        // and simply retry file creation during bootstrap indefinitely. As 
soon as
-        // there is one datanode it will succeed. Permission problems should 
have
-        // already been caught by mkdirs above.
-        FSUtils.setVersion(fs, rd, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
-          10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
-            HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
-      } else {
-        if (!fs.isDirectory(rd)) {
-          throw new IllegalArgumentException(rd.toString() + " is not a 
directory");
-        }
-        if (isSecurityEnabled && 
!rootDirPerms.equals(fs.getFileStatus(rd).getPermission())) {
-          // check whether the permission match
-          LOG.warn("Found rootdir permissions NOT matching expected 
\"hbase.rootdir.perms\" for "
-              + "rootdir=" + rd.toString() + " permissions=" + 
fs.getFileStatus(rd).getPermission()
-              + " and  \"hbase.rootdir.perms\" configured as "
-              + c.get("hbase.rootdir.perms", "700") + ". Automatically setting 
the permissions. You"
-              + " can change the permissions by setting 
\"hbase.rootdir.perms\" in hbase-site.xml "
-              + "and restarting the master");
-          fs.setPermission(rd, rootDirPerms);
-        }
-        // as above
-        FSUtils.checkVersion(fs, rd, true, 
c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
-          10 * 1000), c.getInt(HConstants.VERSION_FILE_WRITE_ATTEMPTS,
-            HConstants.DEFAULT_VERSION_FILE_WRITE_ATTEMPTS));
-      }
-    } catch (DeserializationException de) {
-      LOG.fatal("Please fix invalid configuration for " + 
HConstants.HBASE_DIR, de);
-      IOException ioe = new IOException();
-      ioe.initCause(de);
-      throw ioe;
-    } catch (IllegalArgumentException iae) {
-      LOG.fatal("Please fix invalid configuration for "
-        + HConstants.HBASE_DIR + " " + rd.toString(), iae);
-      throw iae;
-    }
-    // Make sure cluster ID exists
-    if (!FSUtils.checkClusterIdExists(fs, rd, c.getInt(
-        HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000))) {
-      FSUtils.setClusterId(fs, rd, new ClusterId(), 
c.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000));
-    }
-    clusterId = FSUtils.getClusterId(fs, rd);
-
-    // Make sure the meta region directory exists!
-    if (!FSUtils.metaRegionExists(fs, rd)) {
-      bootstrap(rd, c);
-    }
-
-    // Create tableinfo-s for hbase:meta if not already there.
-    // assume, created table descriptor is for enabling table
-    // meta table is a system table, so descriptors are predefined,
-    // we should get them from registry.
-    FSTableDescriptors fsd = new FSTableDescriptors(c, fs, rd);
-    fsd.createTableDescriptor(fsd.get(TableName.META_TABLE_NAME));
-
-    return rd;
-  }
-
-  /**
-   * Make sure the hbase temp directory exists and is empty.
-   * NOTE that this method is only executed once just after the master becomes 
the active one.
-   */
-  private void checkTempDir(final Path tmpdir, final Configuration c, final 
FileSystem fs)
-      throws IOException {
-    // If the temp directory exists, clear the content (left over, from the 
previous run)
-    if (fs.exists(tmpdir)) {
-      // Archive table in temp, maybe left over from failed deletion,
-      // if not the cleaner will take care of them.
-      for (Path tabledir: FSUtils.getTableDirs(fs, tmpdir)) {
-        for (Path regiondir: FSUtils.getRegionDirs(fs, tabledir)) {
-          HFileArchiver.archiveRegion(fs, this.rootdir, tabledir, regiondir);
-        }
-      }
-      if (!fs.delete(tmpdir, true)) {
-        throw new IOException("Unable to clean the temp directory: " + tmpdir);
-      }
-    }
-
-    // Create the temp directory
-    if (!fs.mkdirs(tmpdir)) {
-      throw new IOException("HBase temp directory '" + tmpdir + "' creation 
failure.");
-    }
-  }
-
-  private static void bootstrap(final Path rd, final Configuration c)
-  throws IOException {
-    LOG.info("BOOTSTRAP: creating hbase:meta region");
-    try {
-      // Bootstrapping, make sure blockcache is off.  Else, one will be
-      // created here in bootstrap and it'll need to be cleaned up.  Better to
-      // not make it in first place.  Turn off block caching for bootstrap.
-      // Enable after.
-      HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
-      HTableDescriptor metaDescriptor = new 
FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
-      setInfoFamilyCachingForMeta(metaDescriptor, false);
-      HRegion meta = HRegion.createHRegion(metaHRI, rd, c, metaDescriptor, 
null);
-      setInfoFamilyCachingForMeta(metaDescriptor, true);
-      meta.close();
-    } catch (IOException e) {
-        e = e instanceof RemoteException ?
-                ((RemoteException)e).unwrapRemoteException() : e;
-      LOG.error("bootstrap", e);
-      throw e;
-    }
-  }
-
-  /**
-   * Enable in memory caching for hbase:meta
-   */
-  public static void setInfoFamilyCachingForMeta(HTableDescriptor 
metaDescriptor, final boolean b) {
-    for (HColumnDescriptor hcd: metaDescriptor.getColumnFamilies()) {
-      if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
-        hcd.setBlockCacheEnabled(b);
-        hcd.setInMemory(b);
-      }
-    }
-  }
-
-  public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName, 
boolean hasMob)
-      throws IOException {
-    // archive family store files
-    Path tableDir = FSUtils.getTableDir(rootdir, region.getTable());
-    HFileArchiver.archiveFamily(fs, conf, region, tableDir, familyName);
-
-    // delete the family folder
-    Path familyDir = new Path(tableDir,
-      new Path(region.getEncodedName(), Bytes.toString(familyName)));
-    if (fs.delete(familyDir, true) == false) {
-      if (fs.exists(familyDir)) {
-        throw new IOException("Could not delete family "
-            + Bytes.toString(familyName) + " from FileSystem for region "
-            + region.getRegionNameAsString() + "(" + region.getEncodedName()
-            + ")");
-      }
-    }
-
-    // archive and delete mob files
-    if (hasMob) {
-      Path mobTableDir =
-          FSUtils.getTableDir(new Path(getRootDir(), 
MobConstants.MOB_DIR_NAME), region.getTable());
-      HRegionInfo mobRegionInfo = MobUtils.getMobRegionInfo(region.getTable());
-      Path mobFamilyDir =
-          new Path(mobTableDir,
-              new Path(mobRegionInfo.getEncodedName(), 
Bytes.toString(familyName)));
-      // archive mob family store files
-      MobUtils.archiveMobStoreFiles(conf, fs, mobRegionInfo, mobFamilyDir, 
familyName);
-
-      if (!fs.delete(mobFamilyDir, true)) {
-        throw new IOException("Could not delete mob store files for family "
-            + Bytes.toString(familyName) + " from FileSystem region "
-            + mobRegionInfo.getRegionNameAsString() + "(" + 
mobRegionInfo.getEncodedName() + ")");
-      }
-    }
-  }
-
-  public void stop() {
-  }
-
-  public void logFileSystemState(Log log) throws IOException {
-    FSUtils.logFileSystemState(fs, rootdir, log);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 670642f..3dfe4a6 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
index 4d19e9e..58b646b 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
@@ -347,4 +347,4 @@ public class MasterWalManager {
   public RecoveryMode getLogRecoveryMode() {
     return this.splitLogManager.getRecoveryMode();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
index 9c21040..f52e321 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java
@@ -41,8 +41,9 @@ import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.fs.FsContext;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MetricsSnapshot;
 import 
org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateHdfsRegions;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -419,8 +420,7 @@ public class CloneSnapshotProcedure
     // using a copy of descriptor, table will be created enabling first
     HTableDescriptor underConstruction = new 
HTableDescriptor(hTableDescriptor);
     final Path tempTableDir = FSUtils.getTableDir(tempdir, 
hTableDescriptor.getTableName());
-    ((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
-      .createTableDescriptorForTableDirectory(tempTableDir, underConstruction, 
false);
+    mfs.createTableDescriptor(FsContext.TEMP, underConstruction, false);
 
     // 2. Create Regions
     newRegions = hdfsRegionHandler.createHdfsRegions(

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
index c350185..7562a04 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java
@@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceExistException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.TableNamespaceManager;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
@@ -203,9 +202,7 @@ public class CreateNamespaceProcedure
   protected static void createDirectory(
       final MasterProcedureEnv env,
       final NamespaceDescriptor nsDescriptor) throws IOException {
-    MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    mfs.getFileSystem().mkdirs(
-      FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
+    
env.getMasterServices().getMasterFileSystem().createNamespace(nsDescriptor);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
index 0dde420..f0a71b0 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java
@@ -38,14 +38,16 @@ import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.fs.FsContext;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@@ -297,10 +299,9 @@ public class CreateTableProcedure
 
     // 1. Create Table Descriptor
     // using a copy of descriptor, table will be created enabling first
+    HTableDescriptor underConstruction = new 
HTableDescriptor(hTableDescriptor);
     final Path tempTableDir = FSUtils.getTableDir(tempdir, 
hTableDescriptor.getTableName());
-    ((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
-        .createTableDescriptorForTableDirectory(
-          tempTableDir, hTableDescriptor, false);
+    mfs.createTableDescriptor(FsContext.TEMP, underConstruction, false);
 
     // 2. Create Regions
     newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir,
@@ -319,6 +320,7 @@ public class CreateTableProcedure
     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
     final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), 
hTableDescriptor.getTableName());
     FileSystem fs = mfs.getFileSystem();
+
     if (!fs.delete(tableDir, true) && fs.exists(tableDir)) {
       throw new IOException("Couldn't delete " + tableDir);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
index 2b042dd..c6ca374 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.master.TableNamespaceManager;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
@@ -275,23 +275,7 @@ public class DeleteNamespaceProcedure
   protected static void deleteDirectory(
       final MasterProcedureEnv env,
       final String namespaceName) throws IOException {
-    MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    FileSystem fs = mfs.getFileSystem();
-    Path p = FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName);
-
-    try {
-      for(FileStatus status : fs.listStatus(p)) {
-        if 
(!HConstants.HBASE_NON_TABLE_DIRS.contains(status.getPath().getName())) {
-          throw new IOException("Namespace directory contains table dir: " + 
status.getPath());
-        }
-      }
-      if (!fs.delete(FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName), 
true)) {
-        throw new IOException("Failed to remove namespace: " + namespaceName);
-      }
-    } catch (FileNotFoundException e) {
-      // File already deleted, continue
-      LOG.debug("deleteDirectory throws exception: " + e);
-    }
+    
env.getMasterServices().getMasterFileSystem().deleteNamespace(namespaceName);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
index f8dd0b6..63a4f0f 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java
@@ -43,10 +43,10 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.exceptions.HBaseException;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
index 8fa4183..d37159b 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterDDLOperationHelper.java
@@ -36,9 +36,9 @@ import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.BulkReOpen;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
index b27a3b8..feb9228 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MetricsSnapshot;
 import org.apache.hadoop.hbase.master.RegionStates;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6796171f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
index abd69ad..3a5b974 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java
@@ -51,8 +51,8 @@ import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.MetricsMaster;
 import org.apache.hadoop.hbase.master.SnapshotSentinel;

Reply via email to