Просмотр исходного кода

HDFS-6119. FSNamesystem code cleanup. Contributed by Suresh Srinivas.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1582073 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 11 лет назад
Родитель
Сommit
78b9321539

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

@@ -264,6 +264,8 @@ Release 2.5.0 - UNRELEASED
     HDFS-6155. Fix Boxing/unboxing to parse a primitive findbugs warnings.
     (suresh)
 
+    HDFS-6119. FSNamesystem code cleanup. (suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES 

+ 36 - 39
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -616,12 +616,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
     if (namespaceDirs.size() == 1) {
       LOG.warn("Only one image storage directory ("
-          + DFS_NAMENODE_NAME_DIR_KEY + ") configured. Beware of dataloss"
+          + DFS_NAMENODE_NAME_DIR_KEY + ") configured. Beware of data loss"
           + " due to lack of redundant storage directories!");
     }
     if (editsDirs.size() == 1) {
       LOG.warn("Only one namespace edits storage directory ("
-          + DFS_NAMENODE_EDITS_DIR_KEY + ") configured. Beware of dataloss"
+          + DFS_NAMENODE_EDITS_DIR_KEY + ") configured. Beware of data loss"
           + " due to lack of redundant storage directories!");
     }
   }
@@ -951,8 +951,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   /** 
    * Start services common to both active and standby states
-   * @param haContext 
-   * @throws IOException
    */
   void startCommonServices(Configuration conf, HAContext haContext) throws IOException {
     this.registerMBean(); // register the MBean for the FSNamesystemState
@@ -979,7 +977,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   /** 
    * Stop services common to both active and standby states
-   * @throws IOException
    */
   void stopCommonServices() {
     writeLock();
@@ -1092,7 +1089,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /** 
    * Stop services required in active state
-   * @throws InterruptedException
    */
   void stopActiveServices() {
     LOG.info("Stopping services started for active state");
@@ -1340,7 +1336,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   /**
    * Returns edit directories that are shared between primary and secondary.
-   * @param conf
+   * @param conf configuration
    * @return Collection of edit directories.
    */
   public static List<URI> getSharedEditsDirs(Configuration conf) {
@@ -1772,13 +1768,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   /**
-   * Moves all the blocks from srcs and appends them to trg
-   * To avoid rollbacks we will verify validitity of ALL of the args
+   * Moves all the blocks from {@code srcs} and appends them to {@code target}
+   * To avoid rollbacks we will verify validity of ALL of the args
    * before we start actual move.
    * 
    * This does not support ".inodes" relative path
-   * @param target
-   * @param srcs
+   * @param target target file path to concatenate into
+   * @param srcs files that are concatenated
    * @throws IOException
    */
   void concat(String target, String [] srcs) 
@@ -1788,7 +1784,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       return; // Return previous response
     }
     
-    // Either there is no previous request in progres or it has failed
+    // Either there is no previous request in progress or it has failed
     if(FSNamesystem.LOG.isDebugEnabled()) {
       FSNamesystem.LOG.debug("concat " + Arrays.toString(srcs) +
           " to " + target);
@@ -1913,7 +1909,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
       // check replication and blocks size
       if(repl != srcInode.getBlockReplication()) {
-        throw new HadoopIllegalArgumentException("concat: the soruce file "
+        throw new HadoopIllegalArgumentException("concat: the source file "
             + src + " and the target file " + target
             + " should have the same replication: source replication is "
             + srcInode.getBlockReplication()
@@ -1928,7 +1924,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if(endSrc)
         idx = srcBlocks.length-2; // end block of endSrc is OK not to be full
       if(idx >= 0 && srcBlocks[idx].getNumBytes() != blockSize) {
-        throw new HadoopIllegalArgumentException("concat: the soruce file "
+        throw new HadoopIllegalArgumentException("concat: the source file "
             + src + " and the target file " + target
             + " should have the same blocks sizes: target block size is "
             + blockSize + " but the size of source block " + idx + " is "
@@ -1955,7 +1951,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   /**
    * stores the modification and access time for this inode. 
-   * The access time is precise upto an hour. The transaction, if needed, is
+   * The access time is precise up to an hour. The transaction, if needed, is
    * written to the edits log but is not flushed.
    */
   void setTimes(String src, long mtime, long atime) 
@@ -2166,12 +2162,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Create a new file entry in the namespace.
    * 
    * For description of parameters and exceptions thrown see
-   * {@link ClientProtocol#create()}, except it returns valid file status upon
+   * {@link ClientProtocol#create}, except it returns valid file status upon
    * success
-   * 
-   * For retryCache handling details see -
-   * {@link #getFileStatus(boolean, CacheEntryWithPayload)}
-   * 
    */
   HdfsFileStatus startFile(String src, PermissionStatus permissions,
       String holder, String clientMachine, EnumSet<CreateFlag> flag,
@@ -2257,10 +2249,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Create a new file or overwrite an existing file<br>
    * 
    * Once the file is create the client then allocates a new block with the next
-   * call using {@link NameNode#addBlock()}.
+   * call using {@link ClientProtocol#addBlock}.
    * <p>
    * For description of parameters and exceptions thrown see
-   * {@link ClientProtocol#create()}
+   * {@link ClientProtocol#create}
    */
   private void startFileInternal(FSPermissionChecker pc, String src,
       PermissionStatus permissions, String holder, String clientMachine,
@@ -2345,7 +2337,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * which can still be used for writing more data. The client uses the returned
    * block locations to form the data pipeline for this block.<br>
    * The method returns null if the last block is full. The client then
-   * allocates a new block with the next call using {@link NameNode#addBlock()}.
+   * allocates a new block with the next call using
+   * {@link ClientProtocol#addBlock}.
    * <p>
    * 
    * For description of parameters and exceptions thrown see
@@ -2848,7 +2841,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return lBlk;
   }
 
-  /** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */
+  /** @see ClientProtocol#getAdditionalDatanode */
   LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
       final DatanodeInfo[] existings, final String[] storageIDs,
       final Set<Node> excludes,
@@ -3053,8 +3046,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Save allocated block at the given pending filename
    * 
    * @param src path to the file
-   * @param inodesInPath representing each of the components of src. 
-   *                     The last INode is the INode for the file.
+   * @param inodesInPath representing each of the components of src.
+   *                     The last INode is the INode for {@code src} file.
+   * @param newBlock newly allocated block to be save
+   * @param targets target datanodes where replicas of the new block is placed
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
   BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
@@ -5483,7 +5478,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
    * Check whether current user have permissions to access the path. For more
    * details of the parameters, see
-   * {@link FSPermissionChecker#checkPermission()}.
+   * {@link FSPermissionChecker#checkPermission}.
    */
   private void checkPermission(FSPermissionChecker pc,
       String path, boolean doCheckOwner, FsAction ancestorAccess,
@@ -5496,7 +5491,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
    * Check whether current user have permissions to access the path. For more
    * details of the parameters, see
-   * {@link FSPermissionChecker#checkPermission()}.
+   * {@link FSPermissionChecker#checkPermission}.
    */
   private void checkPermission(FSPermissionChecker pc,
       String path, boolean doCheckOwner, FsAction ancestorAccess,
@@ -5993,7 +5988,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     LOG.info("updatePipeline(" + oldBlock + ") successfully to " + newBlock);
   }
 
-  /** @see #updatePipeline(String, ExtendedBlock, ExtendedBlock, DatanodeID[]) */
+  /**
+   * @see #updatePipeline(String,  ExtendedBlock, ExtendedBlock, DatanodeID[], String[])
+   */
   private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, 
       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs,
       boolean logRetryCache)
@@ -6114,10 +6111,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
   /**
    * @param path Restrict corrupt files to this portion of namespace.
-   * @param startBlockAfter Support for continuation; the set of files we return
-   *  back is ordered by blockid; startBlockAfter tells where to start from
+   * @param cookieTab Support for continuation; cookieTab  tells where
+   *                  to start from
    * @return a list in which each entry describes a corrupt file/block
-   * @throws AccessControlException
    * @throws IOException
    */
   Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
@@ -6212,7 +6208,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   /**
-   * @param renewer
+   * @param renewer Renewer information
    * @return Token<DelegationTokenIdentifier>
    * @throws IOException
    */
@@ -6255,7 +6251,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * 
-   * @param token
+   * @param token delegation token
    * @return New expiryTime of the token
    * @throws InvalidToken
    * @throws IOException
@@ -6289,7 +6285,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * 
-   * @param token
+   * @param token delegation token that needs to be canceled
    * @throws IOException
    */
   void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
@@ -6583,7 +6579,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * Returned information is a JSON representation of map with host name as the
-   * key and value is a map of decomisioning node attribute keys to its values
+   * key and value is a map of decommissioning node attribute keys to its
+   * values
    */
   @Override // NameNodeMXBean
   public String getDecomNodes() {
@@ -6867,7 +6864,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return snapshotManager;
   }
   
-  /** Allow snapshot on a directroy. */
+  /** Allow snapshot on a directory. */
   void allowSnapshot(String path) throws SafeModeException, IOException {
     checkOperation(OperationCategory.WRITE);
     writeLock();
@@ -7200,7 +7197,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * Update internal state to indicate that a rolling upgrade is in progress.
-   * @param startTime
+   * @param startTime start time of the rolling upgrade
    */
   void startRollingUpgradeInternal(long startTime)
       throws IOException {
@@ -7223,7 +7220,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
     checkRollingUpgrade("start rolling upgrade");
     getFSImage().checkUpgrade(this);
-    // in non-HA setup, we do an extra ckpt to generate a rollback image
+    // in non-HA setup, we do an extra checkpoint to generate a rollback image
     getFSImage().saveNamespace(this, NameNodeFile.IMAGE_ROLLBACK, null);
     LOG.info("Successfully saved namespace for preparing rolling upgrade.");