Kaynağa Gözat

Porting from 0.20-append - HDFS-1555. Disallow pipelien recovery if a file is already being
lease recovered. Contributed by Hairong Kuang.



git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security@1164772 13f79535-47bb-0310-9956-ffa450edef68

Suresh Srinivas 13 yıl önce
ebeveyn
işleme
80f53e81cd

+ 3 - 0
CHANGES.txt

@@ -89,6 +89,9 @@ Release 0.20.205.0 - unreleased
     HDFS-724.  Use a bidirectional heartbeat to detect stuck
     pipeline. (hairong)
 
+    HDFS-1555. Disallow pipelien recovery if a file is already being
+    lease recovered. (hairong)
+
   IMPROVEMENTS
 
     MAPREDUCE-2187. Reporter sends progress during sort/merge. (Anupam Seth via

+ 1 - 1
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1774,7 +1774,7 @@ public class DataNode extends Configured
 
     List<DatanodeID> successList = new ArrayList<DatanodeID>();
 
-    long generationstamp = namenode.nextGenerationStamp(block);
+    long generationstamp = namenode.nextGenerationStamp(block, closeFile);
     Block newblock = new Block(block.getBlockId(), block.getNumBytes(), generationstamp);
 
     for(BlockRecord r : syncList) {

+ 15 - 2
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -5194,8 +5194,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
   /**
    * Verifies that the block is associated with a file that has a lease.
    * Increments, logs and then returns the stamp
-   */
-  synchronized long nextGenerationStampForBlock(Block block) throws IOException {
+   *
+   * @param block block
+   * @param fromNN if it is for lease recovery initiated by NameNode
+   * @return a new generation stamp
+   */  
+  synchronized long nextGenerationStampForBlock(Block block, boolean fromNN) throws IOException {
     if (isInSafeMode()) {
       throw new SafeModeException("Cannot get nextGenStamp for " + block, safeMode);
     }
@@ -5211,6 +5215,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
       LOG.info(msg);
       throw new IOException(msg);
     }
+    // Disallow client-initiated recovery once
+    // NameNode initiated lease recovery starts
+    if (!fromNN && HdfsConstants.NN_RECOVERY_LEASEHOLDER.equals(
+        leaseManager.getLeaseByPath(FSDirectory.getFullPathName(fileINode)).getHolder())) {
+      String msg = block +
+        "is being recovered by NameNode, ignoring the request from a client";
+      LOG.info(msg);
+      throw new IOException(msg);
+    }
     if (!((INodeFileUnderConstruction)fileINode).setLastRecoveryTime(now())) {
       String msg = block + " is already being recovered, ignoring this request.";
       LOG.info(msg);

+ 6 - 4
src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -201,6 +201,12 @@ public class LeaseManager {
       this.holder = holder;
       renew();
     }
+
+    /** Get the holder of the lease */
+    public String getHolder() {
+      return holder;
+    }
+
     /** Only LeaseManager object can renew a lease */
     private void renew() {
       this.lastUpdate = FSNamesystem.now();
@@ -278,10 +284,6 @@ public class LeaseManager {
       return paths;
     }
 
-    String getHolder() {
-    	return holder;
-    }
-
     void replacePath(String oldpath, String newpath) {
       paths.remove(oldpath);
       paths.add(newpath);

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

@@ -673,8 +673,8 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   }
 
   /** {@inheritDoc} */
-  public long nextGenerationStamp(Block block) throws IOException{
-    return namesystem.nextGenerationStampForBlock(block);
+  public long nextGenerationStamp(Block block, boolean fromNN) throws IOException{
+    return namesystem.nextGenerationStampForBlock(block, fromNN);
   }
 
   /** {@inheritDoc} */

+ 11 - 9
src/hdfs/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java

@@ -41,13 +41,11 @@ import org.apache.hadoop.security.KerberosInfo;
     clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
 public interface DatanodeProtocol extends VersionedProtocol {
   /**
-   * 25: Serialized format of BlockTokenIdentifier changed to contain
-   *     multiple blocks within a single BlockTokenIdentifier
-   *     
-   *     (bumped to 25 to bring in line with trunk)
+   * 26: nextGenerationStamp has a new parameter indicating if it is for
+   * NameNode initiated lease recovery or not
    */
-  public static final long versionID = 25L;
-  
+  public static final long versionID = 26L;
+
   // error code
   final static int NOTIFY = 0;
   final static int DISK_ERROR = 1; // there are still valid volumes on DN
@@ -152,10 +150,14 @@ public interface DatanodeProtocol extends VersionedProtocol {
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
   
   /**
-   * @return the next GenerationStamp to be associated with the specified
-   * block. 
+   * Get the next GenerationStamp to be associated with the specified
+   * block.
+   * 
+   * @param block block
+   * @param fromNN if it is for lease recovery initiated by NameNode
+   * @return a new generation stamp
    */
-  public long nextGenerationStamp(Block block) throws IOException;
+  public long nextGenerationStamp(Block block, boolean fromNN) throws IOException;
 
   /**
    * Commit block synchronization in lease recovery