You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2016/09/29 06:57:31 UTC

[46/50] [abbrv] hbase git commit: HBASE-14439 Move fs stuff out pt1 - master fs + table desc

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/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..047f1a0 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,15 @@ 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 +298,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 +319,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/c3e022ad/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/c3e022ad/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 fdf0b82..4c87b57 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/c3e022ad/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/c3e022ad/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/c3e022ad/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;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
index e35c686..65b32a4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
@@ -60,7 +61,6 @@ import org.apache.hadoop.hbase.mapreduce.JobUtil;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 
 /*
@@ -111,13 +111,13 @@ public class CompactionTool extends Configured implements Tool {
       if (isFamilyDir(fs, path)) {
         Path regionDir = path.getParent();
         Path tableDir = regionDir.getParent();
-        HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
+        HTableDescriptor htd = LegacyTableDescriptor.getTableDescriptorFromFs(fs, tableDir);
         HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
         compactStoreFiles(tableDir, htd, hri,
             path.getName(), compactOnce, major);
       } else if (isRegionDir(fs, path)) {
         Path tableDir = path.getParent();
-        HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
+        HTableDescriptor htd = LegacyTableDescriptor.getTableDescriptorFromFs(fs, tableDir);
         compactRegion(tableDir, htd, path, compactOnce, major);
       } else if (isTableDir(fs, path)) {
         compactTable(path, compactOnce, major);
@@ -129,7 +129,7 @@ public class CompactionTool extends Configured implements Tool {
 
     private void compactTable(final Path tableDir, final boolean compactOnce, final boolean major)
         throws IOException {
-      HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir);
+      HTableDescriptor htd = LegacyTableDescriptor.getTableDescriptorFromFs(fs, tableDir);
       for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
         compactRegion(tableDir, htd, regionDir, compactOnce, major);
       }
@@ -198,7 +198,7 @@ public class CompactionTool extends Configured implements Tool {
   }
 
   private static boolean isTableDir(final FileSystem fs, final Path path) throws IOException {
-    return FSTableDescriptors.getTableInfoPath(fs, path) != null;
+    return LegacyTableDescriptor.getTableInfoPath(fs, path) != null;
   }
 
   private static boolean isFamilyDir(final FileSystem fs, final Path path) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index 1763b2f..0409441 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -55,7 +56,6 @@ import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 
@@ -347,7 +347,7 @@ public final class SnapshotManifest {
   private void load() throws IOException {
     switch (getSnapshotFormat(desc)) {
       case SnapshotManifestV1.DESCRIPTOR_VERSION: {
-        this.htd = FSTableDescriptors.getTableDescriptorFromFs(fs, workingDir);
+        this.htd = LegacyTableDescriptor.getTableDescriptorFromFs(fs, workingDir);
         ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
         try {
           this.regionManifests =
@@ -444,8 +444,7 @@ public final class SnapshotManifest {
       Path rootDir = FSUtils.getRootDir(conf);
       LOG.info("Using old Snapshot Format");
       // write a copy of descriptor to the snapshot directory
-      new FSTableDescriptors(conf, fs, rootDir)
-        .createTableDescriptorForTableDirectory(workingDir, htd, false);
+      LegacyTableDescriptor.createTableDescriptor(fs, workingDir, htd, false);
     } else {
       LOG.debug("Convert to Single Snapshot Manifest");
       convertToV2SingleManifest();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index 81dadd9..ebbe5df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -19,27 +19,18 @@ package org.apache.hadoop.hbase.util;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.Comparator;
-import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.primitives.Ints;
 import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-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.Coprocessor;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -48,8 +39,7 @@ 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.regionserver.BloomType;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 
 /**
  * Implementation of {@link TableDescriptors} that reads descriptors from the
@@ -72,20 +62,16 @@ import org.apache.hadoop.hbase.regionserver.BloomType;
 @InterfaceAudience.Private
 public class FSTableDescriptors implements TableDescriptors {
   private static final Log LOG = LogFactory.getLog(FSTableDescriptors.class);
-  private final FileSystem fs;
-  private final Path rootdir;
+
+  private final MasterFileSystem mfs;
   private final boolean fsreadonly;
+
   private volatile boolean usecache;
   private volatile boolean fsvisited;
 
   @VisibleForTesting long cachehits = 0;
   @VisibleForTesting long invocations = 0;
 
-  /** The file name prefix used to store HTD in HDFS  */
-  static final String TABLEINFO_FILE_PREFIX = ".tableinfo";
-  static final String TABLEINFO_DIR = ".tabledesc";
-  static final String TMP_DIR = ".tmp";
-
   // This cache does not age out the old stuff.  Thinking is that the amount
   // of data we keep up in here is so small, no need to do occasional purge.
   // TODO.
@@ -116,75 +102,18 @@ public class FSTableDescriptors implements TableDescriptors {
    * operations; i.e. on remove, we do not do delete in fs.
    */
   public FSTableDescriptors(final Configuration conf, final FileSystem fs,
-    final Path rootdir, final boolean fsreadonly, final boolean usecache) throws IOException {
+      final Path rootdir, final boolean fsreadonly, final boolean usecache) throws IOException {
+    this(MasterFileSystem.open(conf, fs, rootdir, false), fsreadonly, usecache);
+  }
+
+  private FSTableDescriptors(final MasterFileSystem mfs, boolean fsreadonly, boolean usecache)
+      throws IOException {
     super();
-    this.fs = fs;
-    this.rootdir = rootdir;
+    this.mfs = mfs;
     this.fsreadonly = fsreadonly;
     this.usecache = usecache;
 
-    this.metaTableDescriptor = createMetaTableDescriptor(conf);
-  }
-
-  @VisibleForTesting
-  public static HTableDescriptor createMetaTableDescriptor(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;
+    this.metaTableDescriptor = HTableDescriptor.metaTableDescriptor(mfs.getConfiguration());
   }
 
   @Override
@@ -235,15 +164,13 @@ public class FSTableDescriptors implements TableDescriptors {
     }
     HTableDescriptor tdmt = null;
     try {
-      tdmt = getTableDescriptorFromFs(fs, rootdir, tablename);
+      tdmt = mfs.getTableDescriptor(tablename);
     } catch (NullPointerException e) {
-      LOG.debug("Exception during readTableDecriptor. Current table name = "
-          + tablename, e);
+      LOG.debug("Exception during readTableDecriptor. Current table name = " + tablename, e);
     } catch (TableInfoMissingException e) {
       // ignore. This is regular operation
     } catch (IOException ioe) {
-      LOG.debug("Exception during readTableDecriptor. Current table name = "
-          + tablename, ioe);
+      LOG.debug("Exception during readTableDecriptor. Current table name = " + tablename, ioe);
     }
     // last HTD written wins
     if (usecache && tdmt != null) {
@@ -270,10 +197,11 @@ public class FSTableDescriptors implements TableDescriptors {
     } else {
       LOG.debug("Fetching table descriptors from the filesystem.");
       boolean allvisited = true;
-      for (Path d : FSUtils.getTableDirs(fs, rootdir)) {
+
+      for (TableName table: mfs.getTables()) {
         HTableDescriptor htd = null;
         try {
-          htd = get(FSUtils.getTableName(d));
+          htd = get(table);
         } catch (FileNotFoundException fnfe) {
           // inability of retrieving one HTD shouldn't stop getting the remaining
           LOG.warn("Trouble retrieving htd", fnfe);
@@ -309,21 +237,18 @@ public class FSTableDescriptors implements TableDescriptors {
     * @see #get(org.apache.hadoop.hbase.TableName)
     */
   @Override
-  public Map<String, HTableDescriptor> getByNamespace(String name)
-  throws IOException {
+  public Map<String, HTableDescriptor> getByNamespace(String name) throws IOException {
     Map<String, HTableDescriptor> htds = new TreeMap<String, HTableDescriptor>();
-    List<Path> tableDirs =
-        FSUtils.getLocalTableDirs(fs, FSUtils.getNamespaceDir(rootdir, name));
-    for (Path d: tableDirs) {
+    for (TableName table: mfs.getTables(name)) {
       HTableDescriptor htd = null;
       try {
-        htd = get(FSUtils.getTableName(d));
+        htd = get(table);
       } catch (FileNotFoundException fnfe) {
         // inability of retrieving one HTD shouldn't stop getting the remaining
         LOG.warn("Trouble retrieving htd", fnfe);
       }
       if (htd == null) continue;
-      htds.put(FSUtils.getTableName(d).getNameAsString(), htd);
+      htds.put(table.getNameAsString(), htd);
     }
     return htds;
   }
@@ -355,372 +280,29 @@ public class FSTableDescriptors implements TableDescriptors {
    * from the FileSystem.
    */
   @Override
-  public HTableDescriptor remove(final TableName tablename)
-  throws IOException {
+  public HTableDescriptor remove(final TableName tablename) throws IOException {
     if (fsreadonly) {
       throw new NotImplementedException("Cannot remove a table descriptor - in read only mode");
     }
-    Path tabledir = getTableDir(tablename);
-    if (this.fs.exists(tabledir)) {
-      if (!this.fs.delete(tabledir, true)) {
-        throw new IOException("Failed delete of " + tabledir.toString());
-      }
-    }
+    mfs.deleteTable(tablename); // for test only??
     HTableDescriptor descriptor = this.cache.remove(tablename);
     return descriptor;
   }
 
   /**
-   * Checks if a current table info file exists for the given table
-   *
-   * @param tableName name of table
-   * @return true if exists
-   * @throws IOException
-   */
-  public boolean isTableInfoExists(TableName tableName) throws IOException {
-    return getTableInfoPath(tableName) != null;
-  }
-
-  /**
-   * Find the most current table info file for the given table in the hbase root directory.
-   * @return The file status of the current table info file or null if it does not exist
-   */
-  private FileStatus getTableInfoPath(final TableName tableName) throws IOException {
-    Path tableDir = getTableDir(tableName);
-    return getTableInfoPath(tableDir);
-  }
-
-  private FileStatus getTableInfoPath(Path tableDir)
-  throws IOException {
-    return getTableInfoPath(fs, tableDir, !fsreadonly);
-  }
-
-  /**
-   * 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
-  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(file.getPath(), false)) {
-            LOG.warn("Failed cleanup of " + path);
-          } else {
-            LOG.debug("Cleaned up old tableinfo file " + path);
-          }
-        }
-      }
-    }
-    return mostCurrent;
-  }
-
-  /**
-   * Compare {@link FileStatus} instances by {@link Path#getName()}. Returns in
-   * reverse order.
-   */
-  @VisibleForTesting
-  static final Comparator<FileStatus> TABLEINFO_FILESTATUS_COMPARATOR =
-  new Comparator<FileStatus>() {
-    @Override
-    public int compare(FileStatus left, FileStatus right) {
-      return right.compareTo(left);
-    }};
-
-  /**
-   * Return the table directory in HDFS
-   */
-  @VisibleForTesting Path getTableDir(final TableName tableName) {
-    return FSUtils.getTableDir(rootdir, tableName);
-  }
-
-  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 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 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 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 getTableDescriptorFromFs(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 {
-    FileStatus status = getTableInfoPath(fs, tableDir, false);
-    if (status == null) {
-      throw new TableInfoMissingException("No table descriptor file under " + tableDir);
-    }
-    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];
-    FSDataInputStream fsDataInputStream = fs.open(status.getPath());
-    try {
-      fsDataInputStream.readFully(content);
-    } finally {
-      fsDataInputStream.close();
-    }
-    HTableDescriptor htd = null;
-    try {
-      htd = HTableDescriptor.parseFrom(content);
-    } catch (DeserializationException e) {
-      throw new IOException("content=" + Bytes.toShort(content), e);
-    }
-    return htd;
-  }
-
-  /**
    * Update table descriptor on the file system
    * @throws IOException Thrown if failed update.
    * @throws NotImplementedException if in read only mode
    */
-  @VisibleForTesting Path updateTableDescriptor(HTableDescriptor td)
-  throws IOException {
+  @VisibleForTesting
+  void updateTableDescriptor(HTableDescriptor td) throws IOException {
     if (fsreadonly) {
       throw new NotImplementedException("Cannot update a table descriptor - in read only mode");
     }
-    TableName tableName = td.getTableName();
-    Path tableDir = getTableDir(tableName);
-    Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(tableDir));
-    if (p == null) throw new IOException("Failed update");
-    LOG.info("Updated tableinfo=" + p);
+    mfs.updateTableDescriptor(td);
     if (usecache) {
       this.cache.put(td.getTableName(), td);
     }
-    return p;
-  }
-
-  /**
-   * Deletes all the table descriptor files from the file system.
-   * Used in unit tests only.
-   * @throws NotImplementedException if in read only mode
-   */
-  public void deleteTableDescriptorIfExists(TableName tableName) throws IOException {
-    if (fsreadonly) {
-      throw new NotImplementedException("Cannot delete a table descriptor - in read only mode");
-    }
-
-    Path tableDir = getTableDir(tableName);
-    Path tableInfoDir = new Path(tableDir, TABLEINFO_DIR);
-    deleteTableDescriptorFiles(fs, tableInfoDir, Integer.MAX_VALUE);
-  }
-
-  /**
-   * 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 {
-    FSDataOutputStream out = fs.create(p, false);
-    try {
-      // 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.
-      out.write(htd.toByteArray());
-    } finally {
-      out.close();
-    }
   }
 
   /**
@@ -740,42 +322,12 @@ public class FSTableDescriptors implements TableDescriptors {
    * @return True if we successfully created file.
    */
   public boolean createTableDescriptor(HTableDescriptor htd, boolean forceCreation)
-  throws IOException {
-    Path tableDir = getTableDir(htd.getTableName());
-    return createTableDescriptorForTableDirectory(tableDir, htd, forceCreation);
-  }
-
-  /**
-   * 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 boolean createTableDescriptorForTableDirectory(Path tableDir,
-      HTableDescriptor htd, boolean forceCreation) throws IOException {
+      throws IOException {
     if (fsreadonly) {
       throw new NotImplementedException("Cannot create a table descriptor - in read only mode");
     }
-    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;
-  }
 
+    // forceCreation???
+    return mfs.createTableDescriptor(htd, forceCreation);
+  }
 }
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index 15f079d..85a8080 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -266,8 +266,12 @@ public abstract class FSUtils {
    * @throws IOException e
    */
   public static boolean deleteDirectory(final FileSystem fs, final Path dir)
-  throws IOException {
-    return fs.exists(dir) && fs.delete(dir, true);
+      throws IOException {
+    try {
+      return !fs.delete(dir, true) ? !fs.exists(dir) : true;
+    } catch (FileNotFoundException e) {
+      return true;
+    }
   }
 
   /**
@@ -1180,7 +1184,7 @@ public abstract class FSUtils {
    * @param namespace namespace name
    * @return {@link org.apache.hadoop.fs.Path} for table
    */
-  public static Path getNamespaceDir(Path rootdir, final String namespace) {
+  private static Path getNamespaceDir(Path rootdir, final String namespace) {
     return new Path(rootdir, new Path(HConstants.BASE_NAMESPACE_DIR,
         new Path(namespace)));
   }
@@ -1322,7 +1326,7 @@ public abstract class FSUtils {
    * .logs, .oldlogs, .corrupt folders.
    * @throws IOException
    */
-  public static List<Path> getLocalTableDirs(final FileSystem fs, final Path rootdir)
+  private static List<Path> getLocalTableDirs(final FileSystem fs, final Path rootdir)
       throws IOException {
     // presumes any directory under hbase.rootdir is a table
     FileStatus[] dirs = fs.listStatus(rootdir, new UserTableDirFilter(fs));
@@ -1927,7 +1931,7 @@ public abstract class FSUtils {
    */
   public static void logFileSystemState(final FileSystem fs, final Path root, Log LOG)
       throws IOException {
-    LOG.debug("Current file system:");
+    LOG.debug("Current file system: " + root);
     logFSTree(LOG, fs, root, "|-");
   }
 
@@ -2208,4 +2212,23 @@ public abstract class FSUtils {
       return null;
     }
   }
+
+  public static void readFully(FileSystem fs, Path path, byte[] content) throws IOException {
+    FSDataInputStream in = fs.open(path);
+    try {
+      in.readFully(content);
+    } finally {
+      in.close();
+    }
+  }
+
+  public static void writeFully(FileSystem fs, Path path, byte[] content, boolean overwrite)
+      throws IOException {
+    FSDataOutputStream out = fs.create(path, overwrite);
+    try {
+      out.write(content);
+    } finally {
+      out.close();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index a1a6099..a12425d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -110,9 +110,10 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
@@ -123,6 +124,7 @@ import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.util.MetaUtils;
 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
@@ -1248,7 +1250,7 @@ public class HBaseFsck extends Configured implements Closeable {
         tablesInfo.put(tableName, modTInfo);
         try {
           HTableDescriptor htd =
-              FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, tableName);
+              LegacyTableDescriptor.getTableDescriptorFromFs(fs, hbaseRoot, tableName);
           modTInfo.htds.add(htd);
         } catch (IOException ioe) {
           if (!orphanTableDirs.containsKey(tableName)) {
@@ -1396,7 +1398,7 @@ public class HBaseFsck extends Configured implements Closeable {
     Configuration c = getConf();
     HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
     HTableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
-    MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, false);
+    MetaUtils.setInfoFamilyCachingForMeta(metaDescriptor, false);
     // The WAL subsystem will use the default rootDir rather than the passed in rootDir
     // unless I pass along via the conf.
     Configuration confForWAL = new Configuration(c);
@@ -1406,7 +1408,7 @@ public class HBaseFsck extends Configured implements Closeable {
         "hbck-meta-recovery-" + RandomStringUtils.randomNumeric(8))).
         getWAL(metaHRI.getEncodedNameAsBytes(), metaHRI.getTable().getNamespace());
     HRegion meta = HRegion.createHRegion(metaHRI, rootdir, c, metaDescriptor, wal);
-    MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, true);
+    MetaUtils.setInfoFamilyCachingForMeta(metaDescriptor, true);
     return meta;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
index a235696..2fbaa92 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java
@@ -133,7 +133,7 @@ public class HFileArchiveUtil {
 
   /**
    * Get the full path to the archive directory on the configured 
-   * {@link org.apache.hadoop.hbase.master.MasterFileSystem}
+   * {@link org.apache.hadoop.hbase.fs.MasterFileSystem}
    * @param conf to look for archive directory name and root directory. Cannot be null. Notes for
    *          testing: requires a FileSystem root directory to be specified.
    * @return the full {@link Path} to the archive directory, as defined by the configuration
@@ -145,7 +145,7 @@ public class HFileArchiveUtil {
 
   /**
    * Get the full path to the archive directory on the configured 
-   * {@link org.apache.hadoop.hbase.master.MasterFileSystem}
+   * {@link org.apache.hadoop.hbase.fs.MasterFileSystem}
    * @param rootdir {@link Path} to the root directory where hbase files are stored (for building
    *          the archive path)
    * @return the full {@link Path} to the archive directory, as defined by the configuration

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
index 307568c..1959c73 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HMerge.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.ipc.RemoteException;
@@ -153,7 +154,7 @@ class HMerge {
 
       this.rootDir = FSUtils.getRootDir(conf);
       Path tabledir = FSUtils.getTableDir(this.rootDir, tableName);
-      this.htd = FSTableDescriptors.getTableDescriptorFromFs(this.fs, tabledir);
+      this.htd = LegacyTableDescriptor.getTableDescriptorFromFs(this.fs, tabledir);
       String logname = "merge_" + System.currentTimeMillis() + HConstants.HREGION_LOGDIR_NAME;
 
       final Configuration walConf = new Configuration(conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
index 3c81cfe..a08a9f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/Merge.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -135,7 +136,7 @@ public class Merge extends Configured implements Tool {
     if (info2 == null) {
       throw new NullPointerException("info2 is null using key " + meta);
     }
-    HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(FileSystem.get(getConf()),
+    HTableDescriptor htd = LegacyTableDescriptor.getTableDescriptorFromFs(FileSystem.get(getConf()),
       this.rootdir, this.tableName);
     HRegion merged = merge(htd, meta, info1, info2);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java
index 7c89f11..1c940f7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/MetaUtils.java
@@ -30,7 +30,9 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -88,7 +90,7 @@ public class MetaUtils {
    */
   public synchronized WAL getLog(HRegionInfo info) throws IOException {
     if (this.walFactory == null) {
-      String logName = 
+      String logName =
           HConstants.HREGION_LOGDIR_NAME + "_" + System.currentTimeMillis();
       final Configuration walConf = new Configuration(this.conf);
       FSUtils.setRootDir(walConf, fs.getHomeDirectory());
@@ -152,4 +154,16 @@ public class MetaUtils {
     this.metaRegion.compactStores();
     return this.metaRegion;
   }
+
+  /**
+   * 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);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
index f129f9d..4abf350 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
@@ -24,9 +24,10 @@ import java.io.IOException;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -159,7 +160,7 @@ public class TestHColumnDescriptorDefaultVersions {
     // Verify descriptor from HDFS
     MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
     Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
-    HTableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
+    HTableDescriptor td = LegacyTableDescriptor.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
     hcds = td.getColumnFamilies();
     verifyHColumnDescriptor(expected, hcds, tableName, families);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
index a3fc640..53de8a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClient.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
index 8317376..fbd347a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
@@ -224,8 +224,8 @@ public class TestSnapshotFromClient {
     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
     Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
     LOG.debug("FS state after snapshot:");
-    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
-      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+    UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG);
+
     SnapshotTestingUtils.confirmSnapshotValid(
       ProtobufUtil.createHBaseProtosSnapshotDesc(snapshots.get(0)), TABLE_NAME, TEST_FAM,
       rootDir, admin, fs);
@@ -294,8 +294,7 @@ public class TestSnapshotFromClient {
     FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
     Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
     LOG.debug("FS state after snapshot:");
-    FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
-      FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
+    UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG);
 
     List<byte[]> emptyCfs = Lists.newArrayList(TEST_FAM); // no file in the region
     List<byte[]> nonEmptyCfs = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 140dcd9..4348d2b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
 import org.apache.hadoop.hbase.io.Reference;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
index bf13e7f..48e143d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.FSUtils;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
index 417987b..c143b9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
@@ -32,7 +32,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -275,7 +276,7 @@ public class TestTableDescriptorModificationFromClient {
     MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
     Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
     HTableDescriptor td =
-        FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
+        LegacyTableDescriptor.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
     verifyTableDescriptor(td, tableName, families);
   }
 
@@ -288,4 +289,4 @@ public class TestTableDescriptorModificationFromClient {
       assertTrue("Expected family " + Bytes.toString(familyName), htdFamilies.contains(familyName));
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
index 2a97119..035b17b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotManager.java
@@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.executor.ExecutorService;
-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.cleaner.HFileCleaner;

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index 8c9db88..49a290c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -35,8 +35,6 @@ import org.apache.commons.lang.StringUtils;
 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.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
@@ -65,6 +63,7 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
 import org.apache.hadoop.hbase.mapreduce.TableInputFormatBase;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionState;
@@ -190,8 +189,7 @@ public class TestNamespaceAuditor {
   @Test
   public void testValidQuotas() throws Exception {
     boolean exceptionCaught = false;
-    FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
-    Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+    MasterFileSystem mfs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem();
     NamespaceDescriptor nspDesc =
         NamespaceDescriptor.create(prefix + "vq1")
             .addConfiguration(TableNamespaceManager.KEY_MAX_REGIONS, "hihdufh")
@@ -203,7 +201,7 @@ public class TestNamespaceAuditor {
       exceptionCaught = true;
     } finally {
       assertTrue(exceptionCaught);
-      assertFalse(fs.exists(FSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
+      assertFalse(mfs.getNamespaces().contains(nspDesc.getName()));
     }
     nspDesc =
         NamespaceDescriptor.create(prefix + "vq2")
@@ -216,7 +214,7 @@ public class TestNamespaceAuditor {
       exceptionCaught = true;
     } finally {
       assertTrue(exceptionCaught);
-      assertFalse(fs.exists(FSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
+      assertFalse(mfs.getNamespaces().contains(nspDesc.getName()));
     }
     nspDesc =
         NamespaceDescriptor.create(prefix + "vq3")
@@ -229,7 +227,7 @@ public class TestNamespaceAuditor {
       exceptionCaught = true;
     } finally {
       assertTrue(exceptionCaught);
-      assertFalse(fs.exists(FSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
+      assertFalse(mfs.getNamespaces().contains(nspDesc.getName()));
     }
     nspDesc =
         NamespaceDescriptor.create(prefix + "vq4")
@@ -242,7 +240,7 @@ public class TestNamespaceAuditor {
       exceptionCaught = true;
     } finally {
       assertTrue(exceptionCaught);
-      assertFalse(fs.exists(FSUtils.getNamespaceDir(rootDir, nspDesc.getName())));
+      assertFalse(mfs.getNamespaces().contains(nspDesc.getName()));
     }
   }
 
@@ -716,8 +714,8 @@ public class TestNamespaceAuditor {
     ADMIN.createTable(tableDescOne);
     ADMIN.createTable(tableDescTwo, Bytes.toBytes("AAA"), Bytes.toBytes("ZZZ"), 4);
   }
-  
-  @Test(expected = QuotaExceededException.class)
+
+  @Test(expected = QuotaExceededException.class, timeout = 30000)
   public void testCloneSnapshotQuotaExceed() throws Exception {
     String nsp = prefix + "_testTableQuotaExceedWithCloneSnapshot";
     NamespaceDescriptor nspDesc =

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index fdc6c92..e4eff84 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -989,7 +989,7 @@ public class TestDefaultMemStore {
     edge.setCurrentTimeMillis(1234);
     WALFactory wFactory = new WALFactory(conf, null, "1234");
     HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, testDir,
-        conf, FSTableDescriptors.createMetaTableDescriptor(conf),
+        conf, HTableDescriptor.metaTableDescriptor(conf),
         wFactory.getMetaWAL(HRegionInfo.FIRST_META_REGIONINFO.
             getEncodedNameAsBytes()));
     HRegionInfo hri = new HRegionInfo(TableName.valueOf("testShouldFlushMeta"),

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
index 0a8dbc4..09532cf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
@@ -826,11 +826,10 @@ public class TestStoreFile extends HBaseTestCase {
 
     // Make up a directory hierarchy that has a regiondir ("7e0102") and familyname.
     Path storedir = new Path(new Path(testDir, "7e0102"), Bytes.toString(family));
-    Path dir = new Path(storedir, "1234567890");
     HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
     // Make a store file and write data to it.
     StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, this.fs)
-            .withOutputDir(dir)
+            .withOutputDir(storedir)
             .withFileContext(meta)
             .build();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index dfd00b3..958278d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -55,9 +55,10 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.fs.MasterFileSystem;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@@ -69,7 +70,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSVisitor;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.MD5Hash;
@@ -497,8 +497,8 @@ public final class SnapshotTestingUtils {
         this.desc = desc;
         this.tableRegions = tableRegions;
         this.snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir);
-        new FSTableDescriptors(conf)
-          .createTableDescriptorForTableDirectory(snapshotDir, htd, false);
+        LegacyTableDescriptor.createTableDescriptor(
+            fs, snapshotDir, htd, false);
       }
 
       public HTableDescriptor getTableDescriptor() {
@@ -690,7 +690,7 @@ public final class SnapshotTestingUtils {
     private RegionData[] createTable(final HTableDescriptor htd, final int nregions)
         throws IOException {
       Path tableDir = FSUtils.getTableDir(rootDir, htd.getTableName());
-      new FSTableDescriptors(conf).createTableDescriptorForTableDirectory(tableDir, htd, false);
+      LegacyTableDescriptor.createTableDescriptor(fs, tableDir, htd, false);
 
       assertTrue(nregions % 2 == 0);
       RegionData[] regions = new RegionData[nregions];

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
index bdc09fe..27d6dde 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.junit.Test;
@@ -62,12 +63,12 @@ public class TestFSTableDescriptors {
 
   @Test (expected=IllegalArgumentException.class)
   public void testRegexAgainstOldStyleTableInfo() {
-    Path p = new Path("/tmp", FSTableDescriptors.TABLEINFO_FILE_PREFIX);
-    int i = FSTableDescriptors.getTableInfoSequenceId(p);
+    Path p = new Path("/tmp", LegacyTableDescriptor.TABLEINFO_FILE_PREFIX);
+    int i = LegacyTableDescriptor.getTableInfoSequenceId(p);
     assertEquals(0, i);
     // Assert it won't eat garbage -- that it fails
     p = new Path("/tmp", "abc");
-    FSTableDescriptors.getTableInfoSequenceId(p);
+    LegacyTableDescriptor.getTableInfoSequenceId(p);
   }
 
   @Test
@@ -96,23 +97,28 @@ public class TestFSTableDescriptors {
     HTableDescriptor htd = new HTableDescriptor(
         TableName.valueOf("testSequenceidAdvancesOnTableInfo"));
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
+    Path tableDir = FSUtils.getTableDir(testdir, htd.getTableName());
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
-    Path p0 = fstd.updateTableDescriptor(htd);
-    int i0 = FSTableDescriptors.getTableInfoSequenceId(p0);
-    Path p1 = fstd.updateTableDescriptor(htd);
+    fstd.updateTableDescriptor(htd);
+    Path p0 = LegacyTableDescriptor.getTableInfoPath(fs, tableDir).getPath();
+    int i0 = LegacyTableDescriptor.getTableInfoSequenceId(p0);
+    fstd.updateTableDescriptor(htd);
+    Path p1 = LegacyTableDescriptor.getTableInfoPath(fs, tableDir).getPath();
     // Assert we cleaned up the old file.
     assertTrue(!fs.exists(p0));
-    int i1 = FSTableDescriptors.getTableInfoSequenceId(p1);
+    int i1 = LegacyTableDescriptor.getTableInfoSequenceId(p1);
     assertTrue(i1 == i0 + 1);
-    Path p2 = fstd.updateTableDescriptor(htd);
+    fstd.updateTableDescriptor(htd);
+    Path p2 = LegacyTableDescriptor.getTableInfoPath(fs, tableDir).getPath();
     // Assert we cleaned up the old file.
     assertTrue(!fs.exists(p1));
-    int i2 = FSTableDescriptors.getTableInfoSequenceId(p2);
+    int i2 = LegacyTableDescriptor.getTableInfoSequenceId(p2);
     assertTrue(i2 == i1 + 1);
-    Path p3 = fstd.updateTableDescriptor(htd);
+    fstd.updateTableDescriptor(htd);
+    Path p3 = LegacyTableDescriptor.getTableInfoPath(fs, tableDir).getPath();
     // Assert we cleaned up the old file.
     assertTrue(!fs.exists(p2));
-    int i3 = FSTableDescriptors.getTableInfoSequenceId(p3);
+    int i3 = LegacyTableDescriptor.getTableInfoSequenceId(p3);
     assertTrue(i3 == i2 + 1);
     HTableDescriptor descriptor = fstd.get(htd.getTableName());
     assertEquals(descriptor, htd);
@@ -123,29 +129,29 @@ public class TestFSTableDescriptors {
     Path p0 = assertWriteAndReadSequenceId(0);
     // Assert p0 has format we expect.
     StringBuilder sb = new StringBuilder();
-    for (int i = 0; i < FSTableDescriptors.WIDTH_OF_SEQUENCE_ID; i++) {
+    for (int i = 0; i < LegacyTableDescriptor.WIDTH_OF_SEQUENCE_ID; i++) {
       sb.append("0");
     }
-    assertEquals(FSTableDescriptors.TABLEINFO_FILE_PREFIX + "." + sb.toString(),
+    assertEquals(LegacyTableDescriptor.TABLEINFO_FILE_PREFIX + "." + sb.toString(),
       p0.getName());
     // Check a few more.
     Path p2 = assertWriteAndReadSequenceId(2);
     Path p10000 = assertWriteAndReadSequenceId(10000);
     // Get a .tablinfo that has no sequenceid suffix.
-    Path p = new Path(p0.getParent(), FSTableDescriptors.TABLEINFO_FILE_PREFIX);
+    Path p = new Path(p0.getParent(), LegacyTableDescriptor.TABLEINFO_FILE_PREFIX);
     FileStatus fs = new FileStatus(0, false, 0, 0, 0, p);
     FileStatus fs0 = new FileStatus(0, false, 0, 0, 0, p0);
     FileStatus fs2 = new FileStatus(0, false, 0, 0, 0, p2);
     FileStatus fs10000 = new FileStatus(0, false, 0, 0, 0, p10000);
-    Comparator<FileStatus> comparator = FSTableDescriptors.TABLEINFO_FILESTATUS_COMPARATOR;
+    Comparator<FileStatus> comparator = LegacyTableDescriptor.TABLEINFO_FILESTATUS_COMPARATOR;
     assertTrue(comparator.compare(fs, fs0) > 0);
     assertTrue(comparator.compare(fs0, fs2) > 0);
     assertTrue(comparator.compare(fs2, fs10000) > 0);
   }
 
   private Path assertWriteAndReadSequenceId(final int i) {
-    Path p = new Path("/tmp", FSTableDescriptors.getTableInfoFileName(i));
-    int ii = FSTableDescriptors.getTableInfoSequenceId(p);
+    Path p = new Path("/tmp", LegacyTableDescriptor.getTableInfoFileName(i));
+    int ii = LegacyTableDescriptor.getTableInfoSequenceId(p);
     assertEquals(i, ii);
     return p;
   }
@@ -171,7 +177,7 @@ public class TestFSTableDescriptors {
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
     fstd.createTableDescriptor(htd);
     HTableDescriptor td2 =
-      FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName());
+      LegacyTableDescriptor.getTableDescriptorFromFs(fs, rootdir, htd.getTableName());
     assertTrue(htd.equals(td2));
   }
 
@@ -181,20 +187,17 @@ public class TestFSTableDescriptors {
     Path rootdir = UTIL.getDataTestDir(name);
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
-    Path descriptorFile = fstd.updateTableDescriptor(htd);
-    try (FSDataOutputStream out = fs.create(descriptorFile, true)) {
-      out.write(htd.toByteArray());
-    }
+    Path tableDir = FSUtils.getTableDir(rootdir, htd.getTableName());
+    fstd.updateTableDescriptor(htd);
+    Path descriptorFile = LegacyTableDescriptor.getTableInfoPath(fs, tableDir).getPath();
+    FSUtils.writeFully(fs, descriptorFile, htd.toByteArray(), true);
     FSTableDescriptors fstd2 = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
-    HTableDescriptor td2 = fstd2.get(htd.getTableName());
+    HTableDescriptor td2 = fstd2.getDescriptor(htd.getTableName());
     assertEquals(htd, td2);
-    FileStatus descriptorFile2 =
-        FSTableDescriptors.getTableInfoPath(fs, fstd2.getTableDir(htd.getTableName()));
+    FileStatus descriptorFile2 = LegacyTableDescriptor.getTableInfoPath(fs, tableDir);
     byte[] buffer = htd.toByteArray();
-    try (FSDataInputStream in = fs.open(descriptorFile2.getPath())) {
-      in.readFully(buffer);
-    }
-    HTableDescriptor td3 = HTableDescriptor.parseFrom(buffer);
+    FSUtils.readFully(fs, descriptorFile2.getPath(), buffer);
+    TableDescriptor td3 = TableDescriptor.parseFrom(buffer);
     assertEquals(htd, td3);
   }
 
@@ -372,7 +375,7 @@ public class TestFSTableDescriptors {
   public void testTableInfoFileStatusComparator() {
     FileStatus bare =
       new FileStatus(0, false, 0, 0, -1,
-        new Path("/tmp", FSTableDescriptors.TABLEINFO_FILE_PREFIX));
+        new Path("/tmp", LegacyTableDescriptor.TABLEINFO_FILE_PREFIX));
     FileStatus future =
       new FileStatus(0, false, 0, 0, -1,
         new Path("/tmp/tablinfo." + System.currentTimeMillis()));
@@ -382,7 +385,7 @@ public class TestFSTableDescriptors {
     FileStatus [] alist = {bare, future, farFuture};
     FileStatus [] blist = {bare, farFuture, future};
     FileStatus [] clist = {farFuture, bare, future};
-    Comparator<FileStatus> c = FSTableDescriptors.TABLEINFO_FILESTATUS_COMPARATOR;
+    Comparator<FileStatus> c = LegacyTableDescriptor.TABLEINFO_FILESTATUS_COMPARATOR;
     Arrays.sort(alist, c);
     Arrays.sort(blist, c);
     Arrays.sort(clist, c);
@@ -419,12 +422,12 @@ public class TestFSTableDescriptors {
     assertFalse(fstd.createTableDescriptor(htd));
     htd.setValue(Bytes.toBytes("mykey"), Bytes.toBytes("myValue"));
     assertTrue(fstd.createTableDescriptor(htd)); //this will re-create
-    Path tableDir = fstd.getTableDir(htd.getTableName());
-    Path tmpTableDir = new Path(tableDir, FSTableDescriptors.TMP_DIR);
+    Path tableDir = FSUtils.getTableDir(testdir, htd.getTableName());
+    Path tmpTableDir = new Path(tableDir, LegacyTableDescriptor.TMP_DIR);
     FileStatus[] statuses = fs.listStatus(tmpTableDir);
     assertTrue(statuses.length == 0);
 
-    assertEquals(htd, FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir));
+    assertEquals(htd, LegacyTableDescriptor.getTableDescriptorFromFs(fs, tableDir));
   }
 
   private static class FSTableDescriptorsTest extends FSTableDescriptors {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e022ad/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index e03a0d5..e15609a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
@@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor;
 import org.apache.hadoop.hbase.io.hfile.TestHFile;
 import org.apache.hadoop.hbase.master.AssignmentManager;
 import org.apache.hadoop.hbase.master.RegionState;
@@ -230,7 +232,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
       Path hbaseTableDir = FSUtils.getTableDir(
           FSUtils.getRootDir(conf), table);
       fs = hbaseTableDir.getFileSystem(conf);
-      FileStatus status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
+      FileStatus status = LegacyTableDescriptor.getTableInfoPath(fs, hbaseTableDir);
       tableinfo = status.getPath();
       fs.rename(tableinfo, new Path("/.tableinfo"));
 
@@ -242,7 +244,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
       hbck = doFsck(conf, true);
       assertNoErrors(hbck);
       status = null;
-      status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
+      status = LegacyTableDescriptor.getTableInfoPath(fs, hbaseTableDir);
       assertNotNull(status);
 
       HTableDescriptor htd = admin.getTableDescriptor(table);
@@ -256,7 +258,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
       htd = admin.getTableDescriptor(table); // warms up cached htd on master
       hbck = doFsck(conf, true);
       assertNoErrors(hbck);
-      status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
+      status = LegacyTableDescriptor.getTableInfoPath(fs, hbaseTableDir);
       assertNotNull(status);
       htd = admin.getTableDescriptor(table);
       assertEquals(htd.getValue("NOT_DEFAULT"), "true");
@@ -420,8 +422,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
       htdDisabled.addFamily(new HColumnDescriptor(FAM));
 
       // Write the .tableinfo
-      FSTableDescriptors fstd = new FSTableDescriptors(conf);
-      fstd.createTableDescriptor(htdDisabled);
+      cluster.getMaster().getMasterFileSystem().createTableDescriptor(
+        new TableDescriptor(htdDisabled), true);
       List<HRegionInfo> disabledRegions =
           TEST_UTIL.createMultiRegionsInMeta(conf, htdDisabled, SPLIT_KEYS);