You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2020/03/07 10:11:17 UTC

[hbase] branch branch-2 updated: HBASE-23912 Resolve the TODO of FSTableDescriptor's construct method (#1225) (#1247)

This is an automated email from the ASF dual-hosted git repository.

zghao pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 8e26761  HBASE-23912 Resolve the TODO of FSTableDescriptor's construct method (#1225) (#1247)
8e26761 is described below

commit 8e26761fd01408471a25d481afed64faf34f7574
Author: Guanghao Zhang <zg...@apache.org>
AuthorDate: Sat Mar 7 18:11:01 2020 +0800

    HBASE-23912 Resolve the TODO of FSTableDescriptor's construct method (#1225) (#1247)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../org/apache/hadoop/hbase/master/HMaster.java    |  5 --
 .../hadoop/hbase/master/MasterFileSystem.java      |  3 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   | 20 ++----
 .../hadoop/hbase/util/FSTableDescriptors.java      | 83 ++++++++++------------
 .../hbase/TestFSTableDescriptorForceCreation.java  |  6 +-
 .../coprocessor/TestCoreMasterCoprocessor.java     |  1 -
 .../regionserver/TestGetClosestAtOrBefore.java     |  1 -
 .../hadoop/hbase/regionserver/TestHRegionInfo.java |  1 +
 .../hbase/regionserver/TestRegionInfoBuilder.java  |  1 +
 .../regionserver/wal/TestLogRollingNoCluster.java  |  4 +-
 .../TestRegionReplicaReplicationEndpoint.java      |  4 +-
 .../hadoop/hbase/util/TestFSTableDescriptors.java  | 46 ++++++------
 12 files changed, 75 insertions(+), 100 deletions(-)

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 11bfc4c..e1bb09f 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
@@ -49,7 +49,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import javax.servlet.ServletException;
@@ -686,10 +685,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     return connector.getLocalPort();
   }
 
-  @Override
-  protected Function<TableDescriptorBuilder, TableDescriptorBuilder> getMetaTableObserver() {
-    return builder -> builder.setRegionReplication(conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM));
-  }
   /**
    * For compatibility, if failed with regionserver credentials, try the master one
    */
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
index a20a402..6badf4e 100644
--- 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
@@ -300,7 +300,7 @@ public class MasterFileSystem {
     // 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);
+    FSTableDescriptors fsd = new FSTableDescriptors(fs, rd);
     fsd.createTableDescriptor(fsd.get(TableName.META_TABLE_NAME));
 
     return rd;
@@ -406,6 +406,7 @@ public class MasterFileSystem {
       // 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.
+      FSTableDescriptors.tryUpdateMetaTableDescriptor(c);
       TableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
       HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, rd,
           c, setInfoFamilyCachingForMeta(metaDescriptor, false), null);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 75122c4..fc0b693 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -53,7 +53,6 @@ import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.function.Function;
 import javax.management.MalformedObjectNameException;
 import javax.servlet.http.HttpServlet;
 import org.apache.commons.lang3.RandomUtils;
@@ -90,7 +89,6 @@ import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.client.locking.LockServiceClient;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
@@ -727,16 +725,13 @@ public class HRegionServer extends HasThread implements
     FSUtils.setFsDefault(this.conf, FSUtils.getRootDir(this.conf));
     this.dataFs = new HFileSystem(this.conf, useHBaseChecksum);
     this.dataRootDir = FSUtils.getRootDir(this.conf);
-    this.tableDescriptors = getFsTableDescriptors();
-  }
-
-  private TableDescriptors getFsTableDescriptors() throws IOException {
-    return new FSTableDescriptors(this.conf,
-      this.dataFs, this.dataRootDir, !canUpdateTableDescriptor(), false, getMetaTableObserver());
-  }
-
-  protected Function<TableDescriptorBuilder, TableDescriptorBuilder> getMetaTableObserver() {
-    return null;
+    this.tableDescriptors =
+        new FSTableDescriptors(this.dataFs, this.dataRootDir, !canUpdateTableDescriptor(), false);
+    if (this instanceof HMaster) {
+      FSTableDescriptors.tryUpdateMetaTableDescriptor(this.conf, this.dataFs, this.dataRootDir,
+        builder -> builder.setRegionReplication(
+          conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM)));
+    }
   }
 
   protected void login(UserProvider user, String host) throws IOException {
@@ -744,7 +739,6 @@ public class HRegionServer extends HasThread implements
       SecurityConstants.REGIONSERVER_KRB_PRINCIPAL, host);
   }
 
-
   /**
    * Wait for an active Master.
    * See override in Master superclass for how it is used.
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 585ebf9..467d66f 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
@@ -100,56 +100,49 @@ public class FSTableDescriptors implements TableDescriptors {
   private final Map<TableName, TableDescriptor> cache = new ConcurrentHashMap<>();
 
   /**
-   * Construct a FSTableDescriptors instance using the hbase root dir of the given
-   * conf and the filesystem where that root dir lives.
-   * This instance can do write operations (is not read only).
+   * Construct a FSTableDescriptors instance using the hbase root dir of the given conf and the
+   * filesystem where that root dir lives. This instance can do write operations (is not read only).
    */
   public FSTableDescriptors(final Configuration conf) throws IOException {
-    this(conf, FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf));
+    this(FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf));
   }
 
-  public FSTableDescriptors(final Configuration conf, final FileSystem fs, final Path rootdir)
-          throws IOException {
-    this(conf, fs, rootdir, false, true);
+  public FSTableDescriptors(final FileSystem fs, final Path rootdir) {
+    this(fs, rootdir, false, true);
   }
 
-  /**
-   * @param fsreadonly True if we are read-only when it comes to filesystem
-   *                   operations; i.e. on remove, we do not do delete in fs.
-   */
-  @VisibleForTesting
-  public FSTableDescriptors(final Configuration conf, final FileSystem fs,
-      final Path rootdir, final boolean fsreadonly, final boolean usecache) throws IOException {
-    this(conf, fs, rootdir, fsreadonly, usecache, null);
-  }
-
-  /**
-   * @param fsreadonly True if we are read-only when it comes to filesystem
-   *                   operations; i.e. on remove, we do not do delete in fs.
-   * @param metaObserver Used by HMaster. It need to modify the META_REPLICAS_NUM for meta table descriptor.
-   *                     see HMaster#finishActiveMasterInitialization
-   *                     TODO: This is a workaround. Should remove this ugly code...
-   */
-  public FSTableDescriptors(final Configuration conf, final FileSystem fs, final Path rootdir,
-    final boolean fsreadonly, final boolean usecache,
-    Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
+  public FSTableDescriptors(final FileSystem fs, final Path rootdir, final boolean fsreadonly,
+      final boolean usecache) {
     this.fs = fs;
     this.rootdir = rootdir;
     this.fsreadonly = fsreadonly;
     this.usecache = usecache;
-    if (!fsreadonly) {
-      // see if we already have meta descriptor on fs. Write one if not.
-      try {
-        getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
-      } catch (TableInfoMissingException e) {
-        TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf);
-        if (metaObserver != null) {
-          builder = metaObserver.apply(builder);
-        }
-        TableDescriptor td = builder.build();
-        LOG.info("Creating new hbase:meta table default descriptor/schema {}", td);
-        updateTableDescriptor(td);
+  }
+
+  public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOException {
+    tryUpdateMetaTableDescriptor(conf, FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf),
+      null);
+  }
+
+  public static void tryUpdateMetaTableDescriptor(Configuration conf, FileSystem fs, Path rootdir,
+      Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
+    // see if we already have meta descriptor on fs. Write one if not.
+    try {
+      getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
+    } catch (TableInfoMissingException e) {
+      TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf);
+      if (metaObserver != null) {
+        builder = metaObserver.apply(builder);
       }
+      TableDescriptor td = builder.build();
+      LOG.info("Creating new hbase:meta table descriptor {}", td);
+      TableName tableName = td.getTableName();
+      Path tableDir = FSUtils.getTableDir(rootdir, tableName);
+      Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(fs, tableDir, true));
+      if (p == null) {
+        throw new IOException("Failed update hbase:meta table descriptor");
+      }
+      LOG.info("Updated hbase:meta table descriptor to {}", p);
     }
   }
 
@@ -324,8 +317,7 @@ public class FSTableDescriptors implements TableDescriptors {
    * from the FileSystem.
    */
   @Override
-  public TableDescriptor remove(final TableName tablename)
-  throws IOException {
+  public TableDescriptor remove(final TableName tablename) throws IOException {
     if (fsreadonly) {
       throw new NotImplementedException("Cannot remove a table descriptor - in read only mode");
     }
@@ -339,8 +331,7 @@ public class FSTableDescriptors implements TableDescriptors {
     return descriptor;
   }
 
-  private FileStatus getTableInfoPath(Path tableDir)
-  throws IOException {
+  private FileStatus getTableInfoPath(Path tableDir) throws IOException {
     return getTableInfoPath(fs, tableDir, !fsreadonly);
   }
 
@@ -372,7 +363,7 @@ public class FSTableDescriptors implements TableDescriptors {
    * @return The file status of the current table info file or null if none exist
    */
   private static FileStatus getTableInfoPath(FileSystem fs, Path tableDir, boolean removeOldFiles)
-  throws IOException {
+      throws IOException {
     Path tableInfoDir = new Path(tableDir, TABLEINFO_DIR);
     return getCurrentTableInfoStatus(fs, tableInfoDir, removeOldFiles);
   }
@@ -711,8 +702,8 @@ public class FSTableDescriptors implements TableDescriptors {
    *         already exists and we weren't forcing the descriptor creation.
    * @throws IOException if a filesystem error occurs
    */
-  public static boolean createTableDescriptorForTableDirectory(FileSystem fs,
-      Path tableDir, TableDescriptor htd, boolean forceCreation) throws IOException {
+  public static boolean createTableDescriptorForTableDirectory(FileSystem fs, Path tableDir,
+      TableDescriptor htd, boolean forceCreation) throws IOException {
     FileStatus status = getTableInfoPath(fs, tableDir);
     if (status != null) {
       LOG.debug("Current path=" + status.getPath());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
index 57da486..d627244 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
@@ -52,7 +52,7 @@ public class TestFSTableDescriptorForceCreation {
     final String name = this.name.getMethodName();
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
+    FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir);
 
     assertTrue("Should create new table descriptor",
       fstd.createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build(), false));
@@ -65,7 +65,7 @@ public class TestFSTableDescriptorForceCreation {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     // Cleanup old tests if any detritus laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
+    FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir);
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
     fstd.update(htd);
     assertFalse("Should not create new table descriptor", fstd.createTableDescriptor(htd, false));
@@ -77,7 +77,7 @@ public class TestFSTableDescriptorForceCreation {
     final String name = this.name.getMethodName();
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
+    FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir);
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
     fstd.createTableDescriptor(htd, false);
     assertTrue("Should create new table descriptor",
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreMasterCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreMasterCoprocessor.java
index 5afc7b0..a83ff09 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreMasterCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoreMasterCoprocessor.java
@@ -43,7 +43,6 @@ import org.junit.rules.TestName;
  */
 @Category({CoprocessorTests.class, SmallTests.class})
 public class TestCoreMasterCoprocessor {
-
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestCoreMasterCoprocessor.class);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
index d7ef09a..b80b917 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
@@ -89,7 +89,6 @@ public class TestGetClosestAtOrBefore  {
     // Up flush size else we bind up when we use default catalog flush of 16k.
     TableDescriptorBuilder metaBuilder = UTIL.getMetaTableDescriptorBuilder()
             .setMemStoreFlushSize(64 * 1024 * 1024);
-
     HRegion mr = HBaseTestingUtility.createRegionAndWAL(HRegionInfo.FIRST_META_REGIONINFO,
         rootdir, this.conf, metaBuilder.build());
     try {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
index ff95f59..f9e2d5a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
@@ -146,6 +146,7 @@ public class TestHRegionInfo {
     Path basedir = htu.getDataTestDir();
     // Create a region.  That'll write the .regioninfo file.
     FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration());
+    FSTableDescriptors.tryUpdateMetaTableDescriptor(htu.getConfiguration());
     HRegion r = HBaseTestingUtility.createRegionAndWAL(hri, basedir, htu.getConfiguration(),
         fsTableDescriptors.get(TableName.META_TABLE_NAME));
     // Get modtime on the file.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
index 3f778c2..90bab5c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfoBuilder.java
@@ -103,6 +103,7 @@ public class TestRegionInfoBuilder {
     Path basedir = htu.getDataTestDir();
     // Create a region.  That'll write the .regioninfo file.
     FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration());
+    FSTableDescriptors.tryUpdateMetaTableDescriptor(htu.getConfiguration());
     HRegion r = HBaseTestingUtility.createRegionAndWAL(convert(ri), basedir, htu.getConfiguration(),
         fsTableDescriptors.get(TableName.META_TABLE_NAME));
     // Get modtime on the file.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index b852168..9ba10b9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -91,8 +91,8 @@ public class TestLogRollingNoCluster {
     final Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     conf.set(WALFactory.WAL_PROVIDER, "filesystem");
     FSUtils.setRootDir(conf, dir);
-    FSTableDescriptors fsTableDescriptors =
-      new FSTableDescriptors(TEST_UTIL.getConfiguration());
+    FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration());
+    FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration());
     TableDescriptor metaTableDescriptor = fsTableDescriptors.get(TableName.META_TABLE_NAME);
     conf.set("hbase.regionserver.hlog.writer.impl", HighLatencySyncWriter.class.getName());
     final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
index 04db81a..c86f2f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpoint.java
@@ -420,8 +420,8 @@ public class TestRegionReplicaReplicationEndpoint {
     RegionReplicaReplicationEndpoint.RegionReplicaOutputSink sink =
         mock(RegionReplicaReplicationEndpoint.RegionReplicaOutputSink.class);
     when(sink.getSkippedEditsCounter()).thenReturn(skippedEdits);
-    FSTableDescriptors fstd = new FSTableDescriptors(HTU.getConfiguration(),
-        FileSystem.get(HTU.getConfiguration()), HTU.getDefaultRootDirPath());
+    FSTableDescriptors fstd =
+        new FSTableDescriptors(FileSystem.get(HTU.getConfiguration()), HTU.getDefaultRootDirPath());
     RegionReplicaReplicationEndpoint.RegionReplicaSinkWriter sinkWriter =
         new RegionReplicaReplicationEndpoint.RegionReplicaSinkWriter(sink,
             (ClusterConnection) connection, Executors.newSingleThreadExecutor(), Integer.MAX_VALUE,
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 718cec3..3b451e3 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
@@ -29,7 +29,6 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.Map;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -87,7 +86,7 @@ public class TestFSTableDescriptors {
     Path testdir = UTIL.getDataTestDir(name.getMethodName());
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
-    FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
+    FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
     assertTrue(fstd.createTableDescriptor(htd));
     assertFalse(fstd.createTableDescriptor(htd));
     FileStatus [] statuses = fs.listStatus(testdir);
@@ -107,7 +106,7 @@ public class TestFSTableDescriptors {
     Path testdir = UTIL.getDataTestDir(name.getMethodName());
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
-    FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
+    FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
     Path p0 = fstd.updateTableDescriptor(htd);
     int i0 = FSTableDescriptors.getTableInfoSequenceId(p0);
     Path p1 = fstd.updateTableDescriptor(htd);
@@ -167,7 +166,7 @@ public class TestFSTableDescriptors {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     // Cleanup old tests if any detrius laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    TableDescriptors htds = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
+    TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
     htds.update(htd);
     assertNotNull(htds.remove(htd.getTableName()));
@@ -179,7 +178,7 @@ public class TestFSTableDescriptors {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
     Path rootdir = UTIL.getDataTestDir(name);
-    FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
+    FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir);
     fstd.createTableDescriptor(htd);
     TableDescriptor td2 =
       FSTableDescriptors.getTableDescriptorFromFs(fs, rootdir, htd.getTableName());
@@ -190,13 +189,13 @@ public class TestFSTableDescriptors {
     final String name = this.name.getMethodName();
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     Path rootdir = UTIL.getDataTestDir(name);
-    FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
+    FSTableDescriptors fstd = new FSTableDescriptors(fs, rootdir);
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
     Path descriptorFile = fstd.updateTableDescriptor(htd);
     try (FSDataOutputStream out = fs.create(descriptorFile, true)) {
       out.write(TableDescriptorBuilder.toByteArray(htd));
     }
-    FSTableDescriptors fstd2 = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
+    FSTableDescriptors fstd2 = new FSTableDescriptors(fs, rootdir);
     TableDescriptor td2 = fstd2.get(htd.getTableName());
     assertEquals(htd, td2);
     FileStatus descriptorFile2 =
@@ -215,7 +214,7 @@ public class TestFSTableDescriptors {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     // Cleanup old tests if any debris laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    FSTableDescriptors htds = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir) {
+    FSTableDescriptors htds = new FSTableDescriptors(fs, rootdir) {
       @Override
       public TableDescriptor get(TableName tablename)
           throws TableExistsException, FileNotFoundException, IOException {
@@ -261,8 +260,7 @@ public class TestFSTableDescriptors {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     // Cleanup old tests if any debris laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    FSTableDescriptors htds = new FSTableDescriptorsTest(UTIL.getConfiguration(), fs, rootdir,
-      false, false);
+    FSTableDescriptors htds = new FSTableDescriptorsTest(fs, rootdir, false);
     final int count = 10;
     // Write out table infos.
     for (int i = 0; i < count; i++) {
@@ -294,7 +292,7 @@ public class TestFSTableDescriptors {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     // Cleanup old tests if any debris laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    FSTableDescriptors htds = new FSTableDescriptorsTest(UTIL.getConfiguration(), fs, rootdir);
+    FSTableDescriptors htds = new FSTableDescriptorsTest(fs, rootdir);
     final int count = 4;
     // Write out table infos.
     for (int i = 0; i < count; i++) {
@@ -313,7 +311,7 @@ public class TestFSTableDescriptors {
     final String name = "testGetAllOrdering";
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     Path rootDir = new Path(UTIL.getDataTestDir(), name);
-    FSTableDescriptors tds = new FSTableDescriptorsTest(UTIL.getConfiguration(), fs, rootDir);
+    FSTableDescriptors tds = new FSTableDescriptorsTest(fs, rootDir);
 
     String[] tableNames = new String[] { "foo", "bar", "foo:bar", "bar:foo" };
     for (String tableName : tableNames) {
@@ -347,9 +345,8 @@ public class TestFSTableDescriptors {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     // Cleanup old tests if any debris laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    FSTableDescriptors chtds = new FSTableDescriptorsTest(UTIL.getConfiguration(), fs, rootdir);
-    FSTableDescriptors nonchtds = new FSTableDescriptorsTest(UTIL.getConfiguration(), fs,
-      rootdir, false, false);
+    FSTableDescriptors chtds = new FSTableDescriptorsTest(fs, rootdir);
+    FSTableDescriptors nonchtds = new FSTableDescriptorsTest(fs, rootdir, false);
 
     final int count = 10;
     // Write out table infos via non-cached FSTableDescriptors
@@ -387,7 +384,7 @@ public class TestFSTableDescriptors {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     // Cleanup old tests if any detrius laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    TableDescriptors htds = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
+    TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
     assertNull("There shouldn't be any HTD for this table",
       htds.get(TableName.valueOf("NoSuchTable")));
   }
@@ -398,7 +395,7 @@ public class TestFSTableDescriptors {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     // Cleanup old tests if any detrius laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
-    TableDescriptors htds = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
+    TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name)).build();
     htds.update(htd);
     htds.update(htd);
@@ -435,8 +432,7 @@ public class TestFSTableDescriptors {
   public void testReadingInvalidDirectoryFromFS() throws IOException {
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
     try {
-      new FSTableDescriptors(UTIL.getConfiguration(), fs,
-          FSUtils.getRootDir(UTIL.getConfiguration()))
+      new FSTableDescriptors(fs, FSUtils.getRootDir(UTIL.getConfiguration()))
           .get(TableName.valueOf(HConstants.HBASE_TEMP_DIRECTORY));
       fail("Shouldn't be able to read a table descriptor for the archive directory.");
     } catch (Exception e) {
@@ -450,7 +446,7 @@ public class TestFSTableDescriptors {
     Path testdir = UTIL.getDataTestDir(name.getMethodName());
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     FileSystem fs = FileSystem.get(UTIL.getConfiguration());
-    FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, testdir);
+    FSTableDescriptors fstd = new FSTableDescriptors(fs, testdir);
     assertTrue(fstd.createTableDescriptor(htd));
     assertFalse(fstd.createTableDescriptor(htd));
     htd = TableDescriptorBuilder.newBuilder(htd)
@@ -467,14 +463,12 @@ public class TestFSTableDescriptors {
 
   private static class FSTableDescriptorsTest extends FSTableDescriptors {
 
-    public FSTableDescriptorsTest(Configuration conf, FileSystem fs, Path rootdir)
-      throws IOException {
-      this(conf, fs, rootdir, false, true);
+    public FSTableDescriptorsTest(FileSystem fs, Path rootdir) {
+      this(fs, rootdir, true);
     }
 
-    public FSTableDescriptorsTest(Configuration conf, FileSystem fs, Path rootdir,
-      boolean fsreadonly, boolean usecache) throws IOException {
-      super(conf, fs, rootdir, fsreadonly, usecache);
+    public FSTableDescriptorsTest(FileSystem fs, Path rootdir, boolean usecache) {
+      super(fs, rootdir, false, usecache);
     }
 
     @Override