Author: wheat9
Date: Thu Jun 19 04:15:56 2014
New Revision: 1603706

URL: http://svn.apache.org/r1603706
Log:
HDFS-6480. Merge r1603705 from trunk.

Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
    
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1603706&r1=1603705&r2=1603706&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt 
(original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt 
Thu Jun 19 04:15:56 2014
@@ -197,6 +197,8 @@ Release 2.5.0 - UNRELEASED
 
     HDFS-6530. Fix Balancer documentation.  (szetszwo)
 
+    HDFS-6480. Move waitForReady() from FSDirectory to FSNamesystem. (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java?rev=1603706&r1=1603705&r2=1603706&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
 Thu Jun 19 04:15:56 2014
@@ -252,7 +252,7 @@ class Checkpointer extends Daemon {
     
     backupNode.namesystem.writeLock();
     try {
-      backupNode.namesystem.dir.setReady();
+      backupNode.namesystem.setImageLoaded();
       if(backupNode.namesystem.getBlocksTotal() > 0) {
         backupNode.namesystem.setBlockTotal();
       }

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=1603706&r1=1603705&r2=1603706&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
 Thu Jun 19 04:15:56 2014
@@ -26,11 +26,10 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -84,15 +83,14 @@ import com.google.common.annotations.Vis
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
-/*************************************************
- * FSDirectory stores the filesystem directory state.
- * It handles writing/loading values to disk, and logging
- * changes as we go.
- *
- * It keeps the filename->blockset mapping always-current
- * and logged to disk.
- * 
- *************************************************/
+/**
+ * Both FSDirectory and FSNamesystem manage the state of the namespace.
+ * FSDirectory is a pure in-memory data structure, all of whose operations
+ * happen entirely in memory. In contrast, FSNamesystem persists the operations
+ * to the disk.
+ * @see org.apache.hadoop.hdfs.server.namenode.FSNamesystem
+ **/
+@InterfaceAudience.Private
 public class FSDirectory implements Closeable {
   private static INodeDirectorySnapshottable createRoot(FSNamesystem 
namesystem) {
     final INodeDirectory r = new INodeDirectory(
@@ -121,7 +119,6 @@ public class FSDirectory implements Clos
   INodeDirectory rootDir;
   FSImage fsImage;  
   private final FSNamesystem namesystem;
-  private volatile boolean ready = false;
   private volatile boolean skipQuotaCheck = false; //skip while consuming edits
   private final int maxComponentLength;
   private final int maxDirItems;
@@ -133,7 +130,6 @@ public class FSDirectory implements Clos
 
   // lock to protect the directory and BlockMap
   private final ReentrantReadWriteLock dirLock;
-  private final Condition cond;
 
   // utility methods to acquire and release read lock and write lock
   void readLock() {
@@ -176,7 +172,6 @@ public class FSDirectory implements Clos
 
   FSDirectory(FSImage fsImage, FSNamesystem ns, Configuration conf) {
     this.dirLock = new ReentrantReadWriteLock(true); // fair
-    this.cond = dirLock.writeLock().newCondition();
     rootDir = createRoot(ns);
     inodeMap = INodeMap.newInstance(rootDir);
     this.fsImage = fsImage;
@@ -233,38 +228,6 @@ public class FSDirectory implements Clos
   }
 
   /**
-   * Notify that loading of this FSDirectory is complete, and
-   * it is ready for use 
-   */
-  void imageLoadComplete() {
-    Preconditions.checkState(!ready, "FSDirectory already loaded");
-    setReady();
-  }
-
-  void setReady() {
-    if(ready) return;
-    writeLock();
-    try {
-      setReady(true);
-      this.nameCache.initialized();
-      cond.signalAll();
-    } finally {
-      writeUnlock();
-    }
-  }
-  
-  //This is for testing purposes only
-  @VisibleForTesting
-  boolean isReady() {
-    return ready;
-  }
-
-  // exposed for unit tests
-  protected void setReady(boolean flag) {
-    ready = flag;
-  }
-
-  /**
    * Shutdown the filestore
    */
   @Override
@@ -272,22 +235,12 @@ public class FSDirectory implements Clos
     fsImage.close();
   }
 
-  /**
-   * Block until the object is ready to be used.
-   */
-  void waitForReady() {
-    if (!ready) {
-      writeLock();
-      try {
-        while (!ready) {
-          try {
-            cond.await(5000, TimeUnit.MILLISECONDS);
-          } catch (InterruptedException ignored) {
-          }
-        }
-      } finally {
-        writeUnlock();
-      }
+  void markNameCacheInitialized() {
+    writeLock();
+    try {
+      nameCache.initialized();
+    } finally {
+      writeUnlock();
     }
   }
 
@@ -313,7 +266,6 @@ public class FSDirectory implements Clos
       String clientMachine, DatanodeDescriptor clientNode)
     throws FileAlreadyExistsException, QuotaExceededException,
       UnresolvedLinkException, SnapshotAccessControlException, AclException {
-    waitForReady();
 
     long modTime = now();
     INodeFile newNode = new INodeFile(namesystem.allocateNewInodeId(), null,
@@ -386,8 +338,6 @@ public class FSDirectory implements Clos
    */
   BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block,
       DatanodeStorageInfo[] targets) throws IOException {
-    waitForReady();
-
     writeLock();
     try {
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();
@@ -425,8 +375,6 @@ public class FSDirectory implements Clos
   boolean removeBlock(String path, INodeFile fileNode, Block block)
       throws IOException {
     Preconditions.checkArgument(fileNode.isUnderConstruction());
-    waitForReady();
-
     writeLock();
     try {
       return unprotectedRemoveBlock(path, fileNode, block);
@@ -470,7 +418,6 @@ public class FSDirectory implements Clos
       NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: "
           +src+" to "+dst);
     }
-    waitForReady();
     writeLock();
     try {
       if (!unprotectedRenameTo(src, dst, mtime))
@@ -493,7 +440,6 @@ public class FSDirectory implements Clos
       NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: " + src
           + " to " + dst);
     }
-    waitForReady();
     writeLock();
     try {
       if (unprotectedRenameTo(src, dst, mtime, options)) {
@@ -1025,7 +971,6 @@ public class FSDirectory implements Clos
   Block[] setReplication(String src, short replication, short[] blockRepls)
       throws QuotaExceededException, UnresolvedLinkException,
       SnapshotAccessControlException {
-    waitForReady();
     writeLock();
     try {
       return unprotectedSetReplication(src, replication, blockRepls);
@@ -1148,7 +1093,6 @@ public class FSDirectory implements Clos
     writeLock();
     try {
       // actual move
-      waitForReady();
       unprotectedConcat(target, srcs, timestamp);
     } finally {
       writeUnlock();
@@ -1231,7 +1175,6 @@ public class FSDirectory implements Clos
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: " + src);
     }
-    waitForReady();
     final long filesRemoved;
     writeLock();
     try {
@@ -1704,7 +1647,7 @@ public class FSDirectory implements Clos
                            long nsDelta, long dsDelta, boolean checkQuota)
                            throws QuotaExceededException {
     assert hasWriteLock();
-    if (!ready) {
+    if (!namesystem.isImageLoaded()) {
       //still initializing. do not check or update quotas.
       return;
     }
@@ -1897,7 +1840,7 @@ public class FSDirectory implements Clos
    */
   private void verifyQuotaForRename(INode[] src, INode[] dst)
       throws QuotaExceededException {
-    if (!ready || skipQuotaCheck) {
+    if (!namesystem.isImageLoaded() || skipQuotaCheck) {
       // Do not check quota if edits log is still being processed
       return;
     }
@@ -1953,7 +1896,7 @@ public class FSDirectory implements Clos
   void verifyINodeName(byte[] childName) throws HadoopIllegalArgumentException 
{
     if (Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, childName)) {
       String s = "\"" + HdfsConstants.DOT_SNAPSHOT_DIR + "\" is a reserved 
name.";
-      if (!ready) {
+      if (!namesystem.isImageLoaded()) {
         s += "  Please rename it before upgrade.";
       }
       throw new HadoopIllegalArgumentException(s);
@@ -1980,7 +1923,7 @@ public class FSDirectory implements Clos
           getFullPathName((INode[])parentPath, pos - 1): (String)parentPath;
       final PathComponentTooLongException e = new 
PathComponentTooLongException(
           maxComponentLength, length, p, DFSUtil.bytes2String(childName));
-      if (ready) {
+      if (namesystem.isImageLoaded()) {
         throw e;
       } else {
         // Do not throw if edits log is still being processed
@@ -2004,7 +1947,7 @@ public class FSDirectory implements Clos
     if (count >= maxDirItems) {
       final MaxDirectoryItemsExceededException e
           = new MaxDirectoryItemsExceededException(maxDirItems, count);
-      if (ready) {
+      if (namesystem.isImageLoaded()) {
         e.setPathName(getFullPathName(pathComponents, pos - 1));
         throw e;
       } else {
@@ -2341,7 +2284,6 @@ public class FSDirectory implements Clos
   void reset() {
     writeLock();
     try {
-      setReady(false);
       rootDir = createRoot(getFSNamesystem());
       inodeMap.clear();
       addToInodeMap(rootDir);

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1603706&r1=1603705&r2=1603706&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
 Thu Jun 19 04:15:56 2014
@@ -105,6 +105,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -503,6 +504,59 @@ public class FSNamesystem implements Nam
 
   private final NNConf nnConf;
 
+  private volatile boolean imageLoaded = false;
+  private final Condition cond;
+  /**
+   * Notify that loading of this FSDirectory is complete, and
+   * it is imageLoaded for use
+   */
+  void imageLoadComplete() {
+    Preconditions.checkState(!imageLoaded, "FSDirectory already loaded");
+    setImageLoaded();
+  }
+
+  void setImageLoaded() {
+    if(imageLoaded) return;
+    writeLock();
+    try {
+      setImageLoaded(true);
+      dir.markNameCacheInitialized();
+      cond.signalAll();
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  //This is for testing purposes only
+  @VisibleForTesting
+  boolean isImageLoaded() {
+    return imageLoaded;
+  }
+
+  // exposed for unit tests
+  protected void setImageLoaded(boolean flag) {
+    imageLoaded = flag;
+  }
+
+  /**
+   * Block until the object is imageLoaded to be used.
+   */
+  void waitForLoadingFSImage() {
+    if (!imageLoaded) {
+      writeLock();
+      try {
+        while (!imageLoaded) {
+          try {
+            cond.await(5000, TimeUnit.MILLISECONDS);
+          } catch (InterruptedException ignored) {
+          }
+        }
+      } finally {
+        writeUnlock();
+      }
+    }
+  }
+
   /**
    * Set the last allocated inode id when fsimage or editlog is loaded. 
    */
@@ -544,6 +598,7 @@ public class FSNamesystem implements Nam
     inodeId.setCurrentValue(INodeId.LAST_RESERVED_ID);
     snapshotManager.clearSnapshottableDirs();
     cacheManager.clear();
+    setImageLoaded(false);
   }
 
   @VisibleForTesting
@@ -671,6 +726,7 @@ public class FSNamesystem implements Nam
     boolean fair = conf.getBoolean("dfs.namenode.fslock.fair", true);
     LOG.info("fsLock is fair:" + fair);
     fsLock = new FSNamesystemLock(fair);
+    cond = fsLock.writeLock().newCondition();
     try {
       resourceRecheckInterval = conf.getLong(
           DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY,
@@ -912,7 +968,7 @@ public class FSNamesystem implements Nam
       }
       writeUnlock();
     }
-    dir.imageLoadComplete();
+    imageLoadComplete();
   }
 
   private void startSecretManager() {
@@ -1830,6 +1886,7 @@ public class FSNamesystem implements Nam
     HdfsFileStatus resultingStat = null;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
+    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2109,6 +2166,7 @@ public class FSNamesystem implements Nam
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = 
FSDirectory.getPathComponentsForReservedPath(src);
+    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2236,6 +2294,8 @@ public class FSNamesystem implements Nam
     byte[][] pathComponents = 
FSDirectory.getPathComponentsForReservedPath(src);
     boolean create = flag.contains(CreateFlag.CREATE);
     boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
+
+    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2729,6 +2789,7 @@ public class FSNamesystem implements Nam
     Block newBlock = null;
     long offset;
     checkOperation(OperationCategory.WRITE);
+    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2951,6 +3012,7 @@ public class FSNamesystem implements Nam
     }
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = 
FSDirectory.getPathComponentsForReservedPath(src);
+    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -3049,6 +3111,7 @@ public class FSNamesystem implements Nam
     boolean success = false;
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = 
FSDirectory.getPathComponentsForReservedPath(src);
+    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -3248,6 +3311,7 @@ public class FSNamesystem implements Nam
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot rename " + src);
+      waitForLoadingFSImage();
       src = FSDirectory.resolvePath(src, srcComponents, dir);
       dst = FSDirectory.resolvePath(dst, dstComponents, dir);
       checkOperation(OperationCategory.WRITE);
@@ -3355,6 +3419,7 @@ public class FSNamesystem implements Nam
           false);
     }
 
+    waitForLoadingFSImage();
     long mtime = now();
     dir.renameTo(src, dst, mtime, options);
     getEditLog().logRename(src, dst, mtime, logRetryCache, options);
@@ -3428,6 +3493,8 @@ public class FSNamesystem implements Nam
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = 
FSDirectory.getPathComponentsForReservedPath(src);
     boolean ret = false;
+
+    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -3901,6 +3968,8 @@ public class FSNamesystem implements Nam
     NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + 
clientName);
     checkOperation(OperationCategory.WRITE);
     byte[][] pathComponents = 
FSDirectory.getPathComponentsForReservedPath(src);
+
+    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -4102,6 +4171,7 @@ public class FSNamesystem implements Nam
       INodeFile pendingFile, int latestSnapshot) throws IOException,
       UnresolvedLinkException {
     assert hasWriteLock();
+
     FileUnderConstructionFeature uc = 
pendingFile.getFileUnderConstructionFeature();
     Preconditions.checkArgument(uc != null);
     leaseManager.removeLease(uc.getClientName(), src);
@@ -4113,6 +4183,7 @@ public class FSNamesystem implements Nam
     // since we just remove the uc feature from pendingFile
     final INodeFile newFile = pendingFile.toCompleteFile(now());
 
+    waitForLoadingFSImage();
     // close file and persist block allocations for this file
     closeFile(src, newFile);
 
@@ -4171,6 +4242,7 @@ public class FSNamesystem implements Nam
              + ")");
     checkOperation(OperationCategory.WRITE);
     String src = "";
+    waitForLoadingFSImage();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -4516,7 +4588,7 @@ public class FSNamesystem implements Nam
    */
   private void closeFile(String path, INodeFile file) {
     assert hasWriteLock();
-    dir.waitForReady();
+    waitForLoadingFSImage();
     // file is closed
     getEditLog().logCloseFile(path, file);
     if (NameNode.stateChangeLog.isDebugEnabled()) {
@@ -4540,7 +4612,7 @@ public class FSNamesystem implements Nam
                                   boolean createParent, boolean logRetryCache)
       throws UnresolvedLinkException, FileAlreadyExistsException,
       QuotaExceededException, SnapshotAccessControlException, AclException {
-    dir.waitForReady();
+    waitForLoadingFSImage();
 
     final long modTime = now();
     if (createParent) {
@@ -5803,7 +5875,7 @@ public class FSNamesystem implements Nam
       boolean ignoreEmptyDir, boolean resolveLink)
       throws AccessControlException, UnresolvedLinkException {
     if (!pc.isSuperUser()) {
-      dir.waitForReady();
+      waitForLoadingFSImage();
       readLock();
       try {
         pc.checkPermission(path, dir, doCheckOwner, ancestorAccess,
@@ -6270,6 +6342,7 @@ public class FSNamesystem implements Nam
              + ", newNodes=" + Arrays.asList(newNodes)
              + ", clientName=" + clientName
              + ")");
+    waitForLoadingFSImage();
     writeLock();
     boolean success = false;
     try {

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java?rev=1603706&r1=1603705&r2=1603706&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
 Thu Jun 19 04:15:56 2014
@@ -1064,7 +1064,7 @@ public class SecondaryNameNode implement
       } finally {
         dstNamesystem.writeUnlock();
       }
-      dstNamesystem.dir.imageLoadComplete();
+      dstNamesystem.imageLoadComplete();
     }
     // error simulation code for junit test
     CheckpointFaultInjector.getInstance().duringMerge();   

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java?rev=1603706&r1=1603705&r2=1603706&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
 Thu Jun 19 04:15:56 2014
@@ -32,7 +32,6 @@ import java.io.IOException;
 
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Mockito.*;
 
 /**
@@ -50,6 +49,7 @@ public class TestCommitBlockSynchronizat
     final DatanodeStorageInfo[] targets = {};
 
     FSNamesystem namesystem = new FSNamesystem(conf, image);
+    namesystem.setImageLoaded(true);
     FSNamesystem namesystemSpy = spy(namesystem);
     BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
         block, 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, 
targets);

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java?rev=1603706&r1=1603705&r2=1603706&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirectory.java
 Thu Jun 19 04:15:56 2014
@@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.DFSConfigK
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -129,16 +128,6 @@ public class TestFSDirectory {
   }
   
   @Test
-  public void testReset() throws Exception {
-    fsdir.reset();
-    Assert.assertFalse(fsdir.isReady());
-    final INodeDirectory root = (INodeDirectory) fsdir.getINode("/");
-    
Assert.assertTrue(root.getChildrenList(Snapshot.CURRENT_STATE_ID).isEmpty());
-    fsdir.imageLoadComplete();
-    Assert.assertTrue(fsdir.isReady());
-  }
-
-  @Test
   public void testSkipQuotaCheck() throws Exception {
     try {
       // set quota. nsQuota of 1 means no files can be created

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java?rev=1603706&r1=1603705&r2=1603706&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
 Thu Jun 19 04:15:56 2014
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.MiniDFSClu
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.junit.After;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -194,4 +195,22 @@ public class TestFSNamesystem {
     assertFalse(rwLock.isWriteLockedByCurrentThread());
     assertEquals(0, rwLock.getWriteHoldCount());
   }
+
+  @Test
+  public void testReset() throws Exception {
+    Configuration conf = new Configuration();
+    FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);
+    FSImage fsImage = Mockito.mock(FSImage.class);
+    Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog);
+    FSNamesystem fsn = new FSNamesystem(conf, fsImage);
+    fsn.imageLoadComplete();
+    assertTrue(fsn.isImageLoaded());
+    fsn.clear();
+    assertFalse(fsn.isImageLoaded());
+    final INodeDirectory root = (INodeDirectory) fsn.getFSDirectory()
+            .getINode("/");
+    assertTrue(root.getChildrenList(Snapshot.CURRENT_STATE_ID).isEmpty());
+    fsn.imageLoadComplete();
+    assertTrue(fsn.isImageLoaded());
+  }
 }

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java?rev=1603706&r1=1603705&r2=1603706&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
 Thu Jun 19 04:15:56 2014
@@ -19,12 +19,9 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
-import static org.apache.hadoop.util.Time.now;
 import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyObject;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import java.io.File;
 import java.io.IOException;
@@ -57,7 +54,7 @@ public class TestFsLimits {
     FSEditLog editLog = mock(FSEditLog.class);
     doReturn(editLog).when(fsImage).getEditLog();
     FSNamesystem fsn = new FSNamesystem(conf, fsImage);
-    fsn.getFSDirectory().setReady(fsIsReady);
+    fsn.setImageLoaded(fsIsReady);
     return fsn;
   }
 


Reply via email to