瀏覽代碼

HDFS-6557. Move the reference of fsimage to FSNamesystem. Contributed by Haohui Mai.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1604242 13f79535-47bb-0310-9956-ffa450edef68
Haohui Mai 11 年之前
父節點
當前提交
9ca79e8d32

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -456,6 +456,8 @@ Release 2.5.0 - UNRELEASED
     HDFS-6403. Add metrics for log warnings reported by JVM pauses. (Yongjun
     Zhang via atm)
 
+    HDFS-6557. Move the reference of fsimage to FSNamesystem. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

+ 2 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -117,7 +117,6 @@ public class FSDirectory implements Closeable {
   public final static byte[] DOT_INODES = 
       DFSUtil.string2Bytes(DOT_INODES_STRING);
   INodeDirectory rootDir;
-  FSImage fsImage;  
   private final FSNamesystem namesystem;
   private volatile boolean skipQuotaCheck = false; //skip while consuming edits
   private final int maxComponentLength;
@@ -170,11 +169,10 @@ public class FSDirectory implements Closeable {
    */
   private final NameCache<ByteArray> nameCache;
 
-  FSDirectory(FSImage fsImage, FSNamesystem ns, Configuration conf) {
+  FSDirectory(FSNamesystem ns, Configuration conf) {
     this.dirLock = new ReentrantReadWriteLock(true); // fair
     rootDir = createRoot(ns);
     inodeMap = INodeMap.newInstance(rootDir);
-    this.fsImage = fsImage;
     int configuredLimit = conf.getInt(
         DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
     this.lsLimit = configuredLimit>0 ?
@@ -231,9 +229,7 @@ public class FSDirectory implements Closeable {
    * Shutdown the filestore
    */
   @Override
-  public void close() throws IOException {
-    fsImage.close();
-  }
+  public void close() throws IOException {}
 
   void markNameCacheInitialized() {
     writeLock();

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -495,7 +495,6 @@ public class FSImage implements Closeable {
     FSImage realImage = target.getFSImage();
     FSImage ckptImage = new FSImage(conf, 
                                     checkpointDirs, checkpointEditsDirs);
-    target.dir.fsImage = ckptImage;
     // load from the checkpoint dirs
     try {
       ckptImage.recoverTransitionRead(StartupOption.REGULAR, target, null);
@@ -507,7 +506,6 @@ public class FSImage implements Closeable {
     realImage.getEditLog().setNextTxId(ckptImage.getEditLog().getLastWrittenTxId()+1);
     realImage.initEditLog(StartupOption.IMPORT);
 
-    target.dir.fsImage = realImage;
     realImage.getStorage().setBlockPoolID(ckptImage.getBlockPoolID());
 
     // and save it but keep the same checkpointTime

+ 23 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -517,6 +517,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   private volatile boolean imageLoaded = false;
   private final Condition cond;
+
+  private final FSImage fsImage;
+
   /**
    * Notify that loading of this FSDirectory is complete, and
    * it is imageLoaded for use
@@ -738,6 +741,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     LOG.info("fsLock is fair:" + fair);
     fsLock = new FSNamesystemLock(fair);
     cond = fsLock.writeLock().newCondition();
+    this.fsImage = fsImage;
     try {
       resourceRecheckInterval = conf.getLong(
           DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY,
@@ -827,7 +831,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_DEFAULT);
 
       this.dtSecretManager = createDelegationTokenSecretManager(conf);
-      this.dir = new FSDirectory(fsImage, this, conf);
+      this.dir = new FSDirectory(this, conf);
       this.snapshotManager = new SnapshotManager(dir);
       this.cacheManager = new CacheManager(this, conf, blockManager);
       this.safeMode = new SafeModeInfo(conf);
@@ -1055,7 +1059,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     LOG.info("Starting services required for active state");
     writeLock();
     try {
-      FSEditLog editLog = dir.fsImage.getEditLog();
+      FSEditLog editLog = getFSImage().getEditLog();
       
       if (!editLog.isOpenForWrite()) {
         // During startup, we're already open for write during initialization.
@@ -1084,12 +1088,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
               metaSaveAsString());
         }
         
-        long nextTxId = dir.fsImage.getLastAppliedTxId() + 1;
+        long nextTxId = getFSImage().getLastAppliedTxId() + 1;
         LOG.info("Will take over writing edit logs at txnid " + 
             nextTxId);
         editLog.setNextTxId(nextTxId);
 
-        dir.fsImage.editLog.openForWrite();
+        getFSImage().editLog.openForWrite();
       }
 
       // Enable quota checks.
@@ -1164,13 +1168,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         ((NameNodeEditLogRoller)nnEditLogRoller.getRunnable()).stop();
         nnEditLogRoller.interrupt();
       }
-      if (dir != null && dir.fsImage != null) {
-        if (dir.fsImage.editLog != null) {
-          dir.fsImage.editLog.close();
+      if (dir != null && getFSImage() != null) {
+        if (getFSImage().editLog != null) {
+          getFSImage().editLog.close();
         }
         // Update the fsimage with the last txid that we wrote
         // so that the tailer starts from the right spot.
-        dir.fsImage.updateLastAppliedTxIdFromWritten();
+        getFSImage().updateLastAppliedTxIdFromWritten();
       }
       if (cacheManager != null) {
         cacheManager.stopMonitorThread();
@@ -1193,9 +1197,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   void startStandbyServices(final Configuration conf) throws IOException {
     LOG.info("Starting services required for standby state");
-    if (!dir.fsImage.editLog.isOpenForRead()) {
+    if (!getFSImage().editLog.isOpenForRead()) {
       // During startup, we're already open for read.
-      dir.fsImage.editLog.initSharedJournalsForRead();
+      getFSImage().editLog.initSharedJournalsForRead();
     }
     
     blockManager.setPostponeBlocksFromFuture(true);
@@ -1242,8 +1246,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (editLogTailer != null) {
       editLogTailer.stop();
     }
-    if (dir != null && dir.fsImage != null && dir.fsImage.editLog != null) {
-      dir.fsImage.editLog.close();
+    if (dir != null && getFSImage() != null && getFSImage().editLog != null) {
+      getFSImage().editLog.close();
     }
   }
   
@@ -1486,9 +1490,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Version of @see #getNamespaceInfo() that is not protected by a lock.
    */
   NamespaceInfo unprotectedGetNamespaceInfo() {
-    return new NamespaceInfo(dir.fsImage.getStorage().getNamespaceID(),
+    return new NamespaceInfo(getFSImage().getStorage().getNamespaceID(),
         getClusterId(), getBlockPoolId(),
-        dir.fsImage.getStorage().getCTime());
+        getFSImage().getStorage().getCTime());
   }
 
   /**
@@ -1506,12 +1510,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       try {
         stopActiveServices();
         stopStandbyServices();
-        if (dir != null) {
-          dir.close();
-        }
       } catch (IOException ie) {
-        LOG.error("Error closing FSDirectory", ie);
+      } finally {
         IOUtils.cleanup(LOG, dir);
+        IOUtils.cleanup(LOG, fsImage);
       }
     }
   }
@@ -4503,7 +4505,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @return registration ID
    */
   String getRegistrationID() {
-    return Storage.getRegistrationID(dir.fsImage.getStorage());
+    return Storage.getRegistrationID(getFSImage().getStorage());
   }
 
   /**
@@ -4719,7 +4721,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   public FSImage getFSImage() {
-    return dir.fsImage;
+    return fsImage;
   }
 
   public FSEditLog getEditLog() {
@@ -7044,7 +7046,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   @Override  // NameNodeMXBean
   public String getClusterId() {
-    return dir.fsImage.getStorage().getClusterID();
+    return getFSImage().getStorage().getClusterID();
   }
   
   @Override  // NameNodeMXBean

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -831,7 +831,7 @@ public class NameNode implements NameNodeStatusMXBean {
   /** get FSImage */
   @VisibleForTesting
   public FSImage getFSImage() {
-    return namesystem.dir.fsImage;
+    return namesystem.getFSImage();
   }
 
   /**
@@ -1141,7 +1141,7 @@ public class NameNode implements NameNodeStatusMXBean {
         return true;
       }
     }
-    nsys.dir.fsImage.doRollback(nsys);
+    nsys.getFSImage().doRollback(nsys);
     return false;
   }
 

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.security.AccessControlException;
 import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
 
 /**
  * This is a utility class to expose NameNode functionality for unit tests.
@@ -177,8 +178,9 @@ public class NameNodeAdapter {
   }
 
   public static FSImage spyOnFsImage(NameNode nn1) {
-    FSImage spy = Mockito.spy(nn1.getNamesystem().dir.fsImage);
-    nn1.getNamesystem().dir.fsImage = spy;
+    FSNamesystem fsn = nn1.getNamesystem();
+    FSImage spy = Mockito.spy(fsn.getFSImage());
+    Whitebox.setInternalState(fsn, "fsImage", spy);
     return spy;
   }
   

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java

@@ -75,8 +75,7 @@ public class TestFSPermissionChecker {
         return new PermissionStatus(SUPERUSER, SUPERGROUP, perm);
       }
     }).when(fsn).createFsOwnerPermissions(any(FsPermission.class));
-    FSImage image = mock(FSImage.class);
-    dir = new FSDirectory(image, fsn, conf);
+    dir = new FSDirectory(fsn, conf);
     inodeRoot = dir.getRoot();
   }
 

+ 9 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java

@@ -61,6 +61,7 @@ import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.log4j.Level;
 import org.junit.Test;
 import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -124,14 +125,14 @@ public class TestSaveNamespace {
     FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
 
     // Replace the FSImage with a spy
-    FSImage originalImage = fsn.dir.fsImage;
+    FSImage originalImage = fsn.getFSImage();
     NNStorage storage = originalImage.getStorage();
 
     NNStorage spyStorage = spy(storage);
     originalImage.storage = spyStorage;
 
     FSImage spyImage = spy(originalImage);
-    fsn.dir.fsImage = spyImage;
+    Whitebox.setInternalState(fsn, "fsImage", spyImage);
 
     boolean shouldFail = false; // should we expect the save operation to fail
     // inject fault
@@ -233,11 +234,11 @@ public class TestSaveNamespace {
     FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
 
     // Replace the FSImage with a spy
-    FSImage originalImage = fsn.dir.fsImage;
+    FSImage originalImage = fsn.getFSImage();
     NNStorage storage = originalImage.getStorage();
 
     FSImage spyImage = spy(originalImage);
-    fsn.dir.fsImage = spyImage;
+    Whitebox.setInternalState(fsn, "fsImage", spyImage);
     
     FileSystem fs = FileSystem.getLocal(conf);
     File rootDir = storage.getStorageDir(0).getRoot();
@@ -367,14 +368,15 @@ public class TestSaveNamespace {
     FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
 
     // Replace the FSImage with a spy
-    final FSImage originalImage = fsn.dir.fsImage;
+    final FSImage originalImage = fsn.getFSImage();
     NNStorage storage = originalImage.getStorage();
     storage.close(); // unlock any directories that FSNamesystem's initialization may have locked
 
     NNStorage spyStorage = spy(storage);
     originalImage.storage = spyStorage;
     FSImage spyImage = spy(originalImage);
-    fsn.dir.fsImage = spyImage;
+    Whitebox.setInternalState(fsn, "fsImage", spyImage);
+
     spyImage.storage.setStorageDirectories(
         FSNamesystem.getNamespaceDirs(conf), 
         FSNamesystem.getNamespaceEditsDirs(conf));
@@ -504,7 +506,7 @@ public class TestSaveNamespace {
     FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);
 
     // Replace the FSImage with a spy
-    final FSImage image = fsn.dir.fsImage;
+    final FSImage image = fsn.getFSImage();
     NNStorage storage = image.getStorage();
     storage.close(); // unlock any directories that FSNamesystem's initialization may have locked
     storage.setStorageDirectories(