Browse Source

HDFS-6583. Merge r1604541 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1604542 13f79535-47bb-0310-9956-ffa450edef68
Haohui Mai 11 years ago
parent
commit
dbfde21372

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

@@ -214,6 +214,7 @@ Release 2.5.0 - UNRELEASED
     HDFS-6460. Ignore stale and decommissioned nodes in
     NetworkTopology#sortByDistance. (Yongjun Zhang via wang)
 
+    HDFS-6583. Remove clientNode in FileUnderConstructionFeature. (wheat9)
   BUG FIXES 
 
     HDFS-6112. NFS Gateway docs are incorrect for allowed hosts configuration.

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

@@ -259,8 +259,8 @@ public class FSDirectory implements Closeable {
    * @throws SnapshotAccessControlException 
    */
   INodeFile addFile(String path, PermissionStatus permissions,
-      short replication, long preferredBlockSize, String clientName,
-      String clientMachine, DatanodeDescriptor clientNode)
+                    short replication, long preferredBlockSize,
+                    String clientName, String clientMachine)
     throws FileAlreadyExistsException, QuotaExceededException,
       UnresolvedLinkException, SnapshotAccessControlException, AclException {
 
@@ -268,7 +268,7 @@ public class FSDirectory implements Closeable {
     INodeFile newNode = new INodeFile(namesystem.allocateNewInodeId(), null,
         permissions, modTime, modTime, BlockInfo.EMPTY_ARRAY, replication,
         preferredBlockSize);
-    newNode.toUnderConstruction(clientName, clientMachine, clientNode);
+    newNode.toUnderConstruction(clientName, clientMachine);
 
     boolean added = false;
     writeLock();
@@ -306,7 +306,7 @@ public class FSDirectory implements Closeable {
       newNode = new INodeFile(id, null, permissions, modificationTime,
           modificationTime, BlockInfo.EMPTY_ARRAY, replication,
           preferredBlockSize);
-      newNode.toUnderConstruction(clientName, clientMachine, null);
+      newNode.toUnderConstruction(clientName, clientMachine);
 
     } else {
       newNode = new INodeFile(id, null, permissions, modificationTime, atime,

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

@@ -377,8 +377,7 @@ public class FSEditLogLoader {
                 "for append");
           }
           LocatedBlock lb = fsNamesys.prepareFileForWrite(path,
-              oldFile, addCloseOp.clientName, addCloseOp.clientMachine, null,
-              false, iip.getLatestSnapshotId(), false);
+              oldFile, addCloseOp.clientName, addCloseOp.clientMachine, false, iip.getLatestSnapshotId(), false);
           newFile = INodeFile.valueOf(fsDir.getINode(path),
               path, true);
           

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

@@ -783,7 +783,7 @@ public class FSImageFormat {
       final INodeFile file = new INodeFile(inodeId, localName, permissions,
           modificationTime, atime, blocks, replication, blockSize);
       if (underConstruction) {
-        file.toUnderConstruction(clientName, clientMachine, null);
+        file.toUnderConstruction(clientName, clientMachine);
       }
         return fileDiffs == null ? file : new INodeFile(file, fileDiffs);
       } else if (numBlocks == -1) {
@@ -938,8 +938,7 @@ public class FSImageFormat {
         }
 
         FileUnderConstructionFeature uc = cons.getFileUnderConstructionFeature();
-        oldnode.toUnderConstruction(uc.getClientName(), uc.getClientMachine(),
-            uc.getClientNode());
+        oldnode.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         if (oldnode.numBlocks() > 0) {
           BlockInfo ucBlock = cons.getLastBlock();
           // we do not replace the inode, just replace the last block of oldnode

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -53,7 +53,6 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatu
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrCompactProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
-import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 
 import com.google.common.base.Preconditions;
@@ -298,8 +297,7 @@ public final class FSImageFormatPBINode {
       // under-construction information
       if (f.hasFileUC()) {
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
-        file.toUnderConstruction(uc.getClientName(), uc.getClientMachine(),
-            null);
+        file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         if (blocks.length > 0) {
           BlockInfo lastBlk = file.getLastBlock();
           // replace the last block of file

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -149,7 +149,7 @@ public class FSImageSerialization {
 
     INodeFile file = new INodeFile(inodeId, name, perm, modificationTime,
         modificationTime, blocks, blockReplication, preferredBlockSize);
-    file.toUnderConstruction(clientName, clientMachine, null);
+    file.toUnderConstruction(clientName, clientMachine);
     return file;
   }
 

+ 12 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2381,9 +2381,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       }
 
       checkFsObjectLimit();
-      final DatanodeDescriptor clientNode = 
-          blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
-
       INodeFile newNode = null;
 
       // Always do an implicit mkdirs for parent directory tree.
@@ -2391,7 +2388,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (parent != null && mkdirsRecursively(parent.toString(),
               permissions, true, now())) {
         newNode = dir.addFile(src, permissions, replication, blockSize,
-                holder, clientMachine, clientNode);
+                holder, clientMachine);
       }
 
       if (newNode == null) {
@@ -2466,10 +2463,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new IOException("append: lastBlock=" + lastBlock +
             " of src=" + src + " is not sufficiently replicated yet.");
       }
-      final DatanodeDescriptor clientNode = 
-          blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
-      return prepareFileForWrite(src, myFile, holder, clientMachine, clientNode,
-          true, iip.getLatestSnapshotId(), logRetryCache);
+      return prepareFileForWrite(src, myFile, holder, clientMachine, true,
+              iip.getLatestSnapshotId(), logRetryCache);
     } catch (IOException ie) {
       NameNode.stateChangeLog.warn("DIR* NameSystem.append: " +ie.getMessage());
       throw ie;
@@ -2484,7 +2479,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @param file existing file object
    * @param leaseHolder identifier of the lease holder on this file
    * @param clientMachine identifier of the client machine
-   * @param clientNode if the client is collocated with a DN, that DN's descriptor
    * @param writeToEditLog whether to persist this change to the edit log
    * @param logRetryCache whether to record RPC ids in editlog for retry cache
    *                      rebuilding
@@ -2493,12 +2487,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException
    */
   LocatedBlock prepareFileForWrite(String src, INodeFile file,
-      String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
-      boolean writeToEditLog, int latestSnapshot, boolean logRetryCache)
+                                   String leaseHolder, String clientMachine,
+                                   boolean writeToEditLog,
+                                   int latestSnapshot, boolean logRetryCache)
       throws IOException {
     file = file.recordModification(latestSnapshot);
-    final INodeFile cons = file.toUnderConstruction(leaseHolder, clientMachine,
-        clientNode);
+    final INodeFile cons = file.toUnderConstruction(leaseHolder, clientMachine);
 
     leaseManager.addLease(cons.getFileUnderConstructionFeature()
         .getClientName(), src);
@@ -2776,7 +2770,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
             + maxBlocksPerFile);
       }
       blockSize = pendingFile.getPreferredBlockSize();
-      clientNode = pendingFile.getFileUnderConstructionFeature().getClientNode();
+      clientNode = blockManager.getDatanodeManager().getDatanodeByHost(
+              pendingFile.getFileUnderConstructionFeature().getClientMachine());
       replication = pendingFile.getFileReplication();
     } finally {
       readUnlock();
@@ -2982,7 +2977,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         if (inode != null) src = inode.getFullPathName();
       }
       final INodeFile file = checkLease(src, clientName, inode, fileId);
-      clientnode = file.getFileUnderConstructionFeature().getClientNode();
+      String clientMachine = file.getFileUnderConstructionFeature()
+              .getClientMachine();
+      clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
       preferredblocksize = file.getPreferredBlockSize();
 
       //find datanode storages

+ 1 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java

@@ -32,15 +32,10 @@ import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 public class FileUnderConstructionFeature implements INode.Feature {
   private String clientName; // lease holder
   private final String clientMachine;
-  // if client is a cluster node too.
-  private final DatanodeDescriptor clientNode;
 
-  public FileUnderConstructionFeature(final String clientName,
-      final String clientMachine,
-      final DatanodeDescriptor clientNode) {
+  public FileUnderConstructionFeature(final String clientName, final String clientMachine) {
     this.clientName = clientName;
     this.clientMachine = clientMachine;
-    this.clientNode = clientNode;
   }
 
   public String getClientName() {
@@ -55,10 +50,6 @@ public class FileUnderConstructionFeature implements INode.Feature {
     return clientMachine;
   }
 
-  public DatanodeDescriptor getClientNode() {
-    return clientNode;
-  }
-
   /**
    * Update the length for the last block
    *

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

@@ -33,7 +33,6 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
@@ -170,12 +169,11 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Convert this file to an {@link INodeFileUnderConstruction}. */
-  INodeFile toUnderConstruction(String clientName, String clientMachine,
-      DatanodeDescriptor clientNode) {
+  INodeFile toUnderConstruction(String clientName, String clientMachine) {
     Preconditions.checkState(!isUnderConstruction(),
         "file is already under construction");
     FileUnderConstructionFeature uc = new FileUnderConstructionFeature(
-        clientName, clientMachine, clientNode);
+        clientName, clientMachine);
     addFeature(uc);
     return this;
   }

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

@@ -83,7 +83,7 @@ public class CreateEditsLog {
 
       final INodeFile inode = new INodeFile(inodeId.nextValue(), null,
           p, 0L, 0L, blocks, replication, blockSize);
-      inode.toUnderConstruction("", "", null);
+      inode.toUnderConstruction("", "");
 
      // Append path to filename with information about blockIDs 
       String path = "_" + iF + "_B" + blocks[0].getBlockId() + 
@@ -98,7 +98,7 @@ public class CreateEditsLog {
       }
       INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
           p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
-      fileUc.toUnderConstruction("", "", null);
+      fileUc.toUnderConstruction("", "");
       editLog.logOpenFile(filePath, fileUc, false);
       editLog.logCloseFile(filePath, inode);
 

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

@@ -195,7 +195,7 @@ public class TestEditLog {
       for (int i = 0; i < numTransactions; i++) {
         INodeFile inode = new INodeFile(namesystem.allocateNewInodeId(), null,
             p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
-        inode.toUnderConstruction("", "", null);
+        inode.toUnderConstruction("", "");
 
         editLog.logOpenFile("/filename" + (startIndex + i), inode, false);
         editLog.logCloseFile("/filename" + (startIndex + i), inode);

+ 2 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -28,8 +28,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -321,7 +319,7 @@ public class TestINodeFile {
     {//cast from INodeFileUnderConstruction
       final INode from = new INodeFile(
           INodeId.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, null, replication, 1024L);
-      from.asFile().toUnderConstruction("client", "machine", null);
+      from.asFile().toUnderConstruction("client", "machine");
     
       //cast to INodeFile, should success
       final INodeFile f = INodeFile.valueOf(from, path);
@@ -1034,12 +1032,11 @@ public class TestINodeFile {
 
     final String clientName = "client";
     final String clientMachine = "machine";
-    file.toUnderConstruction(clientName, clientMachine, null);
+    file.toUnderConstruction(clientName, clientMachine);
     assertTrue(file.isUnderConstruction());
     FileUnderConstructionFeature uc = file.getFileUnderConstructionFeature();
     assertEquals(clientName, uc.getClientName());
     assertEquals(clientMachine, uc.getClientMachine());
-    Assert.assertNull(uc.getClientNode());
 
     file.toCompleteFile(Time.now());
     assertFalse(file.isUnderConstruction());