Ver código fonte

HDFS-6294. Use INode IDs to avoid conflicts when a file open for write is renamed (cmccabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1593634 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 11 anos atrás
pai
commit
f131dba8a3
26 arquivos alterados com 330 adições e 159 exclusões
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 20 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  4. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
  5. 17 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  6. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  7. 12 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  10. 97 43
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  11. 28 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
  12. 10 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  13. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  14. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSClientAdapter.java
  15. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java
  16. 2 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
  17. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  18. 46 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
  19. 11 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
  20. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
  21. 29 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.java
  22. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepDecreasing.java
  23. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java
  24. 6 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  25. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetaSave.java
  26. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java

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

@@ -343,6 +343,9 @@ Release 2.5.0 - UNRELEASED
     HDFS-6295. Add "decommissioning" state and node state filtering to
     dfsadmin. (wang)
 
+    HDFS-6294. Use INode IDs to avoid conflicts when a file open for write is
+    renamed. (cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)

+ 20 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -502,8 +502,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
    * that are currently being written by this client.
    * Note that a file can only be written by a single client.
    */
-  private final Map<String, DFSOutputStream> filesBeingWritten
-      = new HashMap<String, DFSOutputStream>();
+  private final Map<Long, DFSOutputStream> filesBeingWritten
+      = new HashMap<Long, DFSOutputStream>();
 
   /**
    * Same as this(NameNode.getAddress(conf), conf);
@@ -732,14 +732,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
   }
 
   /** Get a lease and start automatic renewal */
-  private void beginFileLease(final String src, final DFSOutputStream out) 
+  private void beginFileLease(final long inodeId, final DFSOutputStream out)
       throws IOException {
-    getLeaseRenewer().put(src, out, this);
+    getLeaseRenewer().put(inodeId, out, this);
   }
 
   /** Stop renewal of lease for the file. */
-  void endFileLease(final String src) throws IOException {
-    getLeaseRenewer().closeFile(src, this);
+  void endFileLease(final long inodeId) throws IOException {
+    getLeaseRenewer().closeFile(inodeId, this);
   }
     
 
@@ -747,9 +747,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
    *  enforced to consistently update its local dfsclients array and 
    *  client's filesBeingWritten map.
    */
-  void putFileBeingWritten(final String src, final DFSOutputStream out) {
+  void putFileBeingWritten(final long inodeId, final DFSOutputStream out) {
     synchronized(filesBeingWritten) {
-      filesBeingWritten.put(src, out);
+      filesBeingWritten.put(inodeId, out);
       // update the last lease renewal time only when there was no
       // writes. once there is one write stream open, the lease renewer
       // thread keeps it updated well with in anyone's expiration time.
@@ -760,9 +760,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
   }
 
   /** Remove a file. Only called from LeaseRenewer. */
-  void removeFileBeingWritten(final String src) {
+  void removeFileBeingWritten(final long inodeId) {
     synchronized(filesBeingWritten) {
-      filesBeingWritten.remove(src);
+      filesBeingWritten.remove(inodeId);
       if (filesBeingWritten.isEmpty()) {
         lastLeaseRenewal = 0;
       }
@@ -847,14 +847,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
   /** Close/abort all files being written. */
   private void closeAllFilesBeingWritten(final boolean abort) {
     for(;;) {
-      final String src;
+      final long inodeId;
       final DFSOutputStream out;
       synchronized(filesBeingWritten) {
         if (filesBeingWritten.isEmpty()) {
           return;
         }
-        src = filesBeingWritten.keySet().iterator().next();
-        out = filesBeingWritten.remove(src);
+        inodeId = filesBeingWritten.keySet().iterator().next();
+        out = filesBeingWritten.remove(inodeId);
       }
       if (out != null) {
         try {
@@ -864,8 +864,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
             out.close();
           }
         } catch(IOException ie) {
-          LOG.error("Failed to " + (abort? "abort": "close") + " file " + src,
-              ie);
+          LOG.error("Failed to " + (abort? "abort": "close") +
+                  " inode " + inodeId, ie);
         }
       }
     }
@@ -1463,7 +1463,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
     final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
         src, masked, flag, createParent, replication, blockSize, progress,
         buffersize, dfsClientConf.createChecksum(checksumOpt), favoredNodeStrs);
-    beginFileLease(src, result);
+    beginFileLease(result.getFileId(), result);
     return result;
   }
   
@@ -1511,7 +1511,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
           flag, createParent, replication, blockSize, progress, buffersize,
           checksum);
     }
-    beginFileLease(src, result);
+    beginFileLease(result.getFileId(), result);
     return result;
   }
   
@@ -1599,7 +1599,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
           + src + " on client " + clientName);
     }
     final DFSOutputStream result = callAppend(stat, src, buffersize, progress);
-    beginFileLease(src, result);
+    beginFileLease(result.getFileId(), result);
     return result;
   }
 
@@ -2436,8 +2436,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
   }
 
   @VisibleForTesting
-  ExtendedBlock getPreviousBlock(String file) {
-    return filesBeingWritten.get(file).getBlock();
+  ExtendedBlock getPreviousBlock(long fileId) {
+    return filesBeingWritten.get(fileId).getBlock();
   }
   
   /**

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -1012,7 +1012,7 @@ public class DFSOutputStream extends FSOutputSummer
       //get a new datanode
       final DatanodeInfo[] original = nodes;
       final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
-          src, block, nodes, storageIDs,
+          src, fileId, block, nodes, storageIDs,
           failed.toArray(new DatanodeInfo[failed.size()]),
           1, dfsClient.clientName);
       setPipeline(lb);
@@ -1268,7 +1268,8 @@ public class DFSOutputStream extends FSOutputSummer
 
         if (!success) {
           DFSClient.LOG.info("Abandoning " + block);
-          dfsClient.namenode.abandonBlock(block, src, dfsClient.clientName);
+          dfsClient.namenode.abandonBlock(block, fileId, src,
+              dfsClient.clientName);
           block = null;
           DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
           excludedNodes.put(nodes[errorIndex], nodes[errorIndex]);
@@ -1914,7 +1915,8 @@ public class DFSOutputStream extends FSOutputSummer
       // namenode.
       if (persistBlocks.getAndSet(false) || updateLength) {
         try {
-          dfsClient.namenode.fsync(src, dfsClient.clientName, lastBlockLength);
+          dfsClient.namenode.fsync(src, fileId,
+              dfsClient.clientName, lastBlockLength);
         } catch (IOException ioe) {
           DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
           // If we got an error here, it might be because some other thread called
@@ -2035,7 +2037,7 @@ public class DFSOutputStream extends FSOutputSummer
     streamer.setLastException(new IOException("Lease timeout of "
         + (dfsClient.getHdfsTimeout()/1000) + " seconds expired."));
     closeThreads(true);
-    dfsClient.endFileLease(src);
+    dfsClient.endFileLease(fileId);
   }
 
   // shutdown datastreamer and responseprocessor threads.
@@ -2090,7 +2092,7 @@ public class DFSOutputStream extends FSOutputSummer
       ExtendedBlock lastBlock = streamer.getBlock();
       closeThreads(false);
       completeFile(lastBlock);
-      dfsClient.endFileLease(src);
+      dfsClient.endFileLease(fileId);
     } catch (ClosedChannelException e) {
     } finally {
       closed = true;
@@ -2184,7 +2186,7 @@ public class DFSOutputStream extends FSOutputSummer
   }
 
   @VisibleForTesting
-  long getFileId() {
+  public long getFileId() {
     return fileId;
   }
 }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java

@@ -281,7 +281,7 @@ class LeaseRenewer {
         && Time.now() - emptyTime > gracePeriod;
   }
 
-  synchronized void put(final String src, final DFSOutputStream out,
+  synchronized void put(final long inodeId, final DFSOutputStream out,
       final DFSClient dfsc) {
     if (dfsc.isClientRunning()) {
       if (!isRunning() || isRenewerExpired()) {
@@ -319,7 +319,7 @@ class LeaseRenewer {
         });
         daemon.start();
       }
-      dfsc.putFileBeingWritten(src, out);
+      dfsc.putFileBeingWritten(inodeId, out);
       emptyTime = Long.MAX_VALUE;
     }
   }
@@ -330,8 +330,8 @@ class LeaseRenewer {
   }
 
   /** Close a file. */
-  void closeFile(final String src, final DFSClient dfsc) {
-    dfsc.removeFileBeingWritten(src);
+  void closeFile(final long inodeId, final DFSClient dfsc) {
+    dfsc.removeFileBeingWritten(inodeId);
 
     synchronized(this) {
       if (dfsc.isFilesBeingWrittenEmpty()) {

+ 17 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -290,13 +290,20 @@ public interface ClientProtocol {
    * file.
    * Any partial writes to the block will be discarded.
    * 
+   * @param b         Block to abandon
+   * @param fileId    The id of the file where the block resides.  Older clients
+   *                    will pass GRANDFATHER_INODE_ID here.
+   * @param src       The path of the file where the block resides.
+   * @param holder    Lease holder.
+   *
    * @throws AccessControlException If access is denied
    * @throws FileNotFoundException file <code>src</code> is not found
    * @throws UnresolvedLinkException If <code>src</code> contains a symlink
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void abandonBlock(ExtendedBlock b, String src, String holder)
+  public void abandonBlock(ExtendedBlock b, long fileId,
+      String src, String holder)
       throws AccessControlException, FileNotFoundException,
       UnresolvedLinkException, IOException;
 
@@ -344,6 +351,7 @@ public interface ClientProtocol {
    * Get a datanode for an existing pipeline.
    * 
    * @param src the file being written
+   * @param fileId the ID of the file being written
    * @param blk the block being written
    * @param existings the existing nodes in the pipeline
    * @param excludes the excluded nodes
@@ -359,8 +367,10 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
-      final DatanodeInfo[] existings, final String[] existingStorageIDs,
+  public LocatedBlock getAdditionalDatanode(final String src,
+      final long fileId, final ExtendedBlock blk,
+      final DatanodeInfo[] existings,
+      final String[] existingStorageIDs,
       final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
       ) throws AccessControlException, FileNotFoundException,
@@ -896,6 +906,8 @@ public interface ClientProtocol {
    * Write all metadata for this file into persistent storage.
    * The file must be currently open for writing.
    * @param src The string representation of the path
+   * @param inodeId The inode ID, or GRANDFATHER_INODE_ID if the client is
+   *                too old to support fsync with inode IDs.
    * @param client The string representation of the client
    * @param lastBlockLength The length of the last block (under construction) 
    *                        to be reported to NameNode 
@@ -905,7 +917,8 @@ public interface ClientProtocol {
    * @throws IOException If an I/O error occurred
    */
   @Idempotent
-  public void fsync(String src, String client, long lastBlockLength) 
+  public void fsync(String src, long inodeId, String client,
+                    long lastBlockLength)
       throws AccessControlException, FileNotFoundException, 
       UnresolvedLinkException, IOException;
 

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -422,8 +422,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public AbandonBlockResponseProto abandonBlock(RpcController controller,
       AbandonBlockRequestProto req) throws ServiceException {
     try {
-      server.abandonBlock(PBHelper.convert(req.getB()), req.getSrc(),
-          req.getHolder());
+      server.abandonBlock(PBHelper.convert(req.getB()), req.getFileId(),
+          req.getSrc(), req.getHolder());
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -461,7 +461,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       List<String> existingStorageIDsList = req.getExistingStorageUuidsList();
       List<DatanodeInfoProto> excludesList = req.getExcludesList();
       LocatedBlock result = server.getAdditionalDatanode(req.getSrc(),
-          PBHelper.convert(req.getBlk()),
+          req.getFileId(), PBHelper.convert(req.getBlk()),
           PBHelper.convert(existingList.toArray(
               new DatanodeInfoProto[existingList.size()])),
           existingStorageIDsList.toArray(
@@ -819,7 +819,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public FsyncResponseProto fsync(RpcController controller,
       FsyncRequestProto req) throws ServiceException {
     try {
-      server.fsync(req.getSrc(), req.getClient(), req.getLastBlockLength());
+      server.fsync(req.getSrc(), req.getFileId(),
+          req.getClient(), req.getLastBlockLength());
       return VOID_FSYNC_RESPONSE;
     } catch (IOException e) {
       throw new ServiceException(e);

+ 12 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -329,11 +329,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void abandonBlock(ExtendedBlock b, String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
+  public void abandonBlock(ExtendedBlock b, long fileId, String src,
+      String holder) throws AccessControlException, FileNotFoundException,
+        UnresolvedLinkException, IOException {
     AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
-        .setB(PBHelper.convert(b)).setSrc(src).setHolder(holder).build();
+        .setB(PBHelper.convert(b)).setSrc(src).setHolder(holder)
+            .setFileId(fileId).build();
     try {
       rpcProxy.abandonBlock(null, req);
     } catch (ServiceException e) {
@@ -365,8 +366,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
-      DatanodeInfo[] existings, String[] existingStorageIDs,
+  public LocatedBlock getAdditionalDatanode(String src, long fileId,
+      ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
       DatanodeInfo[] excludes,
       int numAdditionalNodes, String clientName) throws AccessControlException,
       FileNotFoundException, SafeModeException, UnresolvedLinkException,
@@ -374,6 +375,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
         .newBuilder()
         .setSrc(src)
+        .setFileId(fileId)
         .setBlk(PBHelper.convert(blk))
         .addAllExistings(PBHelper.convert(existings))
         .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
@@ -750,11 +752,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void fsync(String src, String client, long lastBlockLength)
+  public void fsync(String src, long fileId, String client,
+                    long lastBlockLength)
       throws AccessControlException, FileNotFoundException,
       UnresolvedLinkException, IOException {
     FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
-        .setClient(client).setLastBlockLength(lastBlockLength).build();
+        .setClient(client).setLastBlockLength(lastBlockLength)
+            .setFileId(fileId).build();
     try {
       rpcProxy.fsync(null, req);
     } catch (ServiceException e) {

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

@@ -1182,7 +1182,7 @@ public class FSEditLog implements LogsPurgeable {
    * Finalize the current log segment.
    * Transitions from IN_SEGMENT state to BETWEEN_LOG_SEGMENTS state.
    */
-  synchronized void endCurrentLogSegment(boolean writeEndTxn) {
+  public synchronized void endCurrentLogSegment(boolean writeEndTxn) {
     LOG.info("Ending log segment " + curSegmentTxId);
     Preconditions.checkState(isSegmentOpen(),
         "Bad state: %s", state);

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

@@ -542,7 +542,7 @@ public class FSImage implements Closeable {
   }
 
   @VisibleForTesting
-  void setEditLogForTesting(FSEditLog newLog) {
+  public void setEditLogForTesting(FSEditLog newLog) {
     editLog = newLog;
   }
 

+ 97 - 43
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2668,9 +2668,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       checkOperation(OperationCategory.READ);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
       LocatedBlock[] onRetryBlock = new LocatedBlock[1];
-      final INode[] inodes = analyzeFileState(
-          src, fileId, clientName, previous, onRetryBlock).getINodes();
-      final INodeFile pendingFile = inodes[inodes.length - 1].asFile();
+      final INodeFile pendingFile = analyzeFileState(
+          src, fileId, clientName, previous, onRetryBlock);
+      src = pendingFile.getFullPathName();
 
       if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
         // This is a retry. Just return the last block if having locations.
@@ -2704,10 +2704,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       // Run the full analysis again, since things could have changed
       // while chooseTarget() was executing.
       LocatedBlock[] onRetryBlock = new LocatedBlock[1];
-      INodesInPath inodesInPath =
+      final INodeFile pendingFile =
           analyzeFileState(src, fileId, clientName, previous, onRetryBlock);
-      INode[] inodes = inodesInPath.getINodes();
-      final INodeFile pendingFile = inodes[inodes.length - 1].asFile();
 
       if (onRetryBlock[0] != null) {
         if (onRetryBlock[0].getLocations().length > 0) {
@@ -2729,6 +2727,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
       // allocate new block, record block locations in INode.
       newBlock = createNewBlock();
+      INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
       saveAllocatedBlock(src, inodesInPath, newBlock, targets);
 
       dir.persistNewBlock(src, pendingFile);
@@ -2742,7 +2741,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return makeLocatedBlock(newBlock, targets, offset);
   }
 
-  INodesInPath analyzeFileState(String src,
+  INodeFile analyzeFileState(String src,
                                 long fileId,
                                 String clientName,
                                 ExtendedBlock previous,
@@ -2759,9 +2758,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     checkFsObjectLimit();
 
     Block previousBlock = ExtendedBlock.getLocalBlock(previous);
-    final INodesInPath iip = dir.getINodesInPath4Write(src);
-    final INodeFile pendingFile
-        = checkLease(src, fileId, clientName, iip.getLastINode());
+    INode inode;
+    if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+      // Older clients may not have given us an inode ID to work with.
+      // In this case, we have to try to resolve the path and hope it
+      // hasn't changed or been deleted since the file was opened for write.
+      final INodesInPath iip = dir.getINodesInPath4Write(src);
+      inode = iip.getLastINode();
+    } else {
+      // Newer clients pass the inode ID, so we can just get the inode
+      // directly.
+      inode = dir.getInode(fileId);
+      if (inode != null) src = inode.getFullPathName();
+    }
+    final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
     BlockInfo lastBlockInFile = pendingFile.getLastBlock();
     if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
       // The block that the client claims is the current last block
@@ -2819,7 +2829,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
             ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
             offset);
-        return iip;
+        return pendingFile;
       } else {
         // Case 3
         throw new IOException("Cannot allocate block in " + src + ": " +
@@ -2832,7 +2842,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if (!checkFileProgress(pendingFile, false)) {
       throw new NotReplicatedYetException("Not replicated yet: " + src);
     }
-    return iip;
+    return pendingFile;
   }
 
   LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
@@ -2845,8 +2855,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   /** @see ClientProtocol#getAdditionalDatanode */
-  LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
-      final DatanodeInfo[] existings, final String[] storageIDs,
+  LocatedBlock getAdditionalDatanode(String src, long fileId,
+      final ExtendedBlock blk, final DatanodeInfo[] existings,
+      final String[] storageIDs,
       final Set<Node> excludes,
       final int numAdditionalNodes, final String clientName
       ) throws IOException {
@@ -2866,7 +2877,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       src = FSDirectory.resolvePath(src, pathComponents, dir);
 
       //check lease
-      final INodeFile file = checkLease(src, clientName);
+      final INode inode;
+      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+        // Older clients may not have given us an inode ID to work with.
+        // In this case, we have to try to resolve the path and hope it
+        // hasn't changed or been deleted since the file was opened for write.
+        inode = dir.getINode(src);
+      } else {
+        inode = dir.getInode(fileId);
+        if (inode != null) src = inode.getFullPathName();
+      }
+      final INodeFile file = checkLease(src, clientName, inode, fileId);
       clientnode = file.getFileUnderConstructionFeature().getClientNode();
       preferredblocksize = file.getPreferredBlockSize();
 
@@ -2890,7 +2911,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
    * The client would like to let go of the given block
    */
-  boolean abandonBlock(ExtendedBlock b, String src, String holder)
+  boolean abandonBlock(ExtendedBlock b, long fileId, String src, String holder)
       throws LeaseExpiredException, FileNotFoundException,
       UnresolvedLinkException, IOException {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
@@ -2902,13 +2923,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot abandon block " + b + " for fle" + src);
+      checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
 
+      final INode inode;
+      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+        // Older clients may not have given us an inode ID to work with.
+        // In this case, we have to try to resolve the path and hope it
+        // hasn't changed or been deleted since the file was opened for write.
+        inode = dir.getINode(src);
+      } else {
+        inode = dir.getInode(fileId);
+        if (inode != null) src = inode.getFullPathName();
+      }
+      final INodeFile file = checkLease(src, holder, inode, fileId);
+
       //
       // Remove the block from the pending creates list
       //
-      INodeFile file = checkLease(src, holder);
       boolean removed = dir.removeBlock(src, file,
           ExtendedBlock.getLocalBlock(b));
       if (!removed) {
@@ -2926,39 +2958,39 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
     return true;
   }
-  
-  /** make sure that we still have the lease on this file. */
-  private INodeFile checkLease(String src, String holder)
-      throws LeaseExpiredException, UnresolvedLinkException,
-      FileNotFoundException {
-    return checkLease(src, INodeId.GRANDFATHER_INODE_ID, holder,
-        dir.getINode(src));
-  }
-  
-  private INodeFile checkLease(String src, long fileId, String holder,
-      INode inode) throws LeaseExpiredException, FileNotFoundException {
+
+  private INodeFile checkLease(String src, String holder, INode inode,
+                               long fileId)
+      throws LeaseExpiredException, FileNotFoundException {
     assert hasReadLock();
-    if (inode == null || !inode.isFile()) {
+    final String ident = src + " (inode " + fileId + ")";
+    if (inode == null) {
       Lease lease = leaseManager.getLease(holder);
       throw new LeaseExpiredException(
-          "No lease on " + src + ": File does not exist. "
+          "No lease on " + ident + ": File does not exist. "
           + (lease != null ? lease.toString()
               : "Holder " + holder + " does not have any open files."));
     }
+    if (!inode.isFile()) {
+      Lease lease = leaseManager.getLease(holder);
+      throw new LeaseExpiredException(
+          "No lease on " + ident + ": INode is not a regular file. "
+              + (lease != null ? lease.toString()
+              : "Holder " + holder + " does not have any open files."));
+    }
     final INodeFile file = inode.asFile();
     if (!file.isUnderConstruction()) {
       Lease lease = leaseManager.getLease(holder);
       throw new LeaseExpiredException(
-          "No lease on " + src + ": File is not open for writing. "
+          "No lease on " + ident + ": File is not open for writing. "
           + (lease != null ? lease.toString()
               : "Holder " + holder + " does not have any open files."));
     }
     String clientName = file.getFileUnderConstructionFeature().getClientName();
     if (holder != null && !clientName.equals(holder)) {
-      throw new LeaseExpiredException("Lease mismatch on " + src + " owned by "
-          + clientName + " but is accessed by " + holder);
+      throw new LeaseExpiredException("Lease mismatch on " + ident +
+          " owned by " + clientName + " but is accessed by " + holder);
     }
-    INodeId.checkId(fileId, file);
     return file;
   }
  
@@ -3001,10 +3033,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       String holder, Block last, long fileId) throws SafeModeException,
       UnresolvedLinkException, IOException {
     assert hasWriteLock();
-    final INodesInPath iip = dir.getLastINodeInPath(src);
     final INodeFile pendingFile;
     try {
-      pendingFile = checkLease(src, fileId, holder, iip.getINode(0));
+      final INode inode;
+      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+        // Older clients may not have given us an inode ID to work with.
+        // In this case, we have to try to resolve the path and hope it
+        // hasn't changed or been deleted since the file was opened for write.
+        final INodesInPath iip = dir.getLastINodeInPath(src);
+        inode = iip.getINode(0);
+      } else {
+        inode = dir.getInode(fileId);
+        if (inode != null) src = inode.getFullPathName();
+      }
+      pendingFile = checkLease(src, holder, inode, fileId);
     } catch (LeaseExpiredException lee) {
       final INode inode = dir.getINode(src);
       if (inode != null
@@ -3019,9 +3061,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         final Block realLastBlock = inode.asFile().getLastBlock();
         if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
           NameNode.stateChangeLog.info("DIR* completeFile: " +
-              "request from " + holder + " to complete " + src +
-              " which is already closed. But, it appears to be an RPC " +
-              "retry. Returning success");
+              "request from " + holder + " to complete inode " + fileId +
+              "(" + src + ") which is already closed. But, it appears to be " +
+              "an RPC retry. Returning success");
           return true;
         }
       }
@@ -3041,7 +3083,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     finalizeINodeFileUnderConstruction(src, pendingFile,
-        iip.getLatestSnapshotId());
+        Snapshot.CURRENT_STATE_ID);
     return true;
   }
 
@@ -3682,12 +3724,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /** Persist all metadata about this file.
    * @param src The string representation of the path
+   * @param fileId The inode ID that we're fsyncing.  Older clients will pass
+   *               INodeId.GRANDFATHER_INODE_ID here.
    * @param clientName The string representation of the client
    * @param lastBlockLength The length of the last block 
    *                        under construction reported from client.
    * @throws IOException if path does not exist
    */
-  void fsync(String src, String clientName, long lastBlockLength) 
+  void fsync(String src, long fileId, String clientName, long lastBlockLength)
       throws IOException, UnresolvedLinkException {
     NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
     checkOperation(OperationCategory.WRITE);
@@ -3697,7 +3741,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot fsync file " + src);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
-      INodeFile pendingFile  = checkLease(src, clientName);
+      final INode inode;
+      if (fileId == INodeId.GRANDFATHER_INODE_ID) {
+        // Older clients may not have given us an inode ID to work with.
+        // In this case, we have to try to resolve the path and hope it
+        // hasn't changed or been deleted since the file was opened for write.
+        inode = dir.getINode(src);
+      } else {
+        inode = dir.getInode(fileId);
+        if (inode != null) src = inode.getFullPathName();
+      }
+      final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
       if (lastBlockLength > 0) {
         pendingFile.getFileUnderConstructionFeature().updateLengthOfLastBlock(
             pendingFile, lastBlockLength);

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

@@ -46,6 +46,28 @@ public class INodesInPath {
         : Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, pathComponent);
   }
 
+  static INodesInPath fromINode(INode inode) {
+    int depth = 0, index;
+    INode tmp = inode;
+    while (tmp != null) {
+      depth++;
+      tmp = tmp.getParent();
+    }
+    final byte[][] path = new byte[depth][];
+    final INode[] inodes = new INode[depth];
+    final INodesInPath iip = new INodesInPath(path, depth);
+    tmp = inode;
+    index = depth;
+    while (tmp != null) {
+      index--;
+      path[index] = tmp.getKey();
+      inodes[index] = tmp;
+      tmp = tmp.getParent();
+    }
+    iip.setINodes(inodes);
+    return iip;
+  }
+
   /**
    * Given some components, create a path name.
    * @param components The path components
@@ -341,6 +363,11 @@ public class INodesInPath {
   private void addNode(INode node) {
     inodes[numNonNull++] = node;
   }
+
+  private void setINodes(INode inodes[]) {
+    this.inodes = inodes;
+    this.numNonNull = this.inodes.length;
+  }
   
   void setINode(int i, INode inode) {
     inodes[i >= 0? i: inodes.length + i] = inode;
@@ -417,4 +444,4 @@ public class INodesInPath {
           + ", this=" + toString(false));
     }
   }
-}
+}

+ 10 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -600,13 +600,15 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
+  public LocatedBlock getAdditionalDatanode(final String src,
+      final long fileId, final ExtendedBlock blk,
       final DatanodeInfo[] existings, final String[] existingStorageIDs,
       final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
       ) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("getAdditionalDatanode: src=" + src
+          + ", fileId=" + fileId
           + ", blk=" + blk
           + ", existings=" + Arrays.asList(existings)
           + ", excludes=" + Arrays.asList(excludes)
@@ -623,20 +625,20 @@ class NameNodeRpcServer implements NamenodeProtocols {
         excludeSet.add(node);
       }
     }
-    return namesystem.getAdditionalDatanode(src, blk, existings,
+    return namesystem.getAdditionalDatanode(src, fileId, blk, existings,
         existingStorageIDs, excludeSet, numAdditionalNodes, clientName);
   }
   /**
    * The client needs to give up on the block.
    */
   @Override // ClientProtocol
-  public void abandonBlock(ExtendedBlock b, String src, String holder)
-      throws IOException {
+  public void abandonBlock(ExtendedBlock b, long fileId, String src,
+        String holder) throws IOException {
     if(stateChangeLog.isDebugEnabled()) {
       stateChangeLog.debug("*BLOCK* NameNode.abandonBlock: "
           +b+" of file "+src);
     }
-    if (!namesystem.abandonBlock(b, src, holder)) {
+    if (!namesystem.abandonBlock(b, fileId, src, holder)) {
       throw new IOException("Cannot abandon block during write to " + src);
     }
   }
@@ -944,9 +946,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
   
   @Override // ClientProtocol
-  public void fsync(String src, String clientName, long lastBlockLength)
+  public void fsync(String src, long fileId, String clientName,
+                    long lastBlockLength)
       throws IOException {
-    namesystem.fsync(src, clientName, lastBlockLength);
+    namesystem.fsync(src, fileId, clientName, lastBlockLength);
   }
 
   @Override // ClientProtocol

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -117,6 +117,7 @@ message AbandonBlockRequestProto {
   required ExtendedBlockProto b = 1;
   required string src = 2;
   required string holder = 3;
+  optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
 }
 
 message AbandonBlockResponseProto { // void response
@@ -143,6 +144,7 @@ message GetAdditionalDatanodeRequestProto {
   required uint32 numAdditionalNodes = 5;
   required string clientName = 6;
   repeated string existingStorageUuids = 7;
+  optional uint64 fileId = 8 [default = 0];  // default to GRANDFATHER_INODE_ID
 }
 
 message GetAdditionalDatanodeResponseProto {
@@ -532,6 +534,7 @@ message FsyncRequestProto {
   required string src = 1;
   required string client = 2;
   optional sint64 lastBlockLength = 3 [default = -1];
+  optional uint64 fileId = 4 [default = 0];  // default to GRANDFATHER_INODE_ID
 }
 
 message FsyncResponseProto { // void response

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSClientAdapter.java

@@ -54,8 +54,8 @@ public class DFSClientAdapter {
     return dfs.dfs;
   }
 
-  public static ExtendedBlock getPreviousBlock(DFSClient client, String file) {
-    return client.getPreviousBlock(file);
+  public static ExtendedBlock getPreviousBlock(DFSClient client, long fileId) {
+    return client.getPreviousBlock(fileId);
   }
 
   public static long getFileId(DFSOutputStream out) {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java

@@ -71,6 +71,7 @@ public class TestAbandonBlock {
       fout.write(123);
     }
     fout.hflush();
+    long fileId = ((DFSOutputStream)fout.getWrappedStream()).getFileId();
 
     // Now abandon the last block
     DFSClient dfsclient = DFSClientAdapter.getDFSClient(fs);
@@ -78,11 +79,11 @@ public class TestAbandonBlock {
       dfsclient.getNamenode().getBlockLocations(src, 0, Integer.MAX_VALUE);
     int orginalNumBlocks = blocks.locatedBlockCount();
     LocatedBlock b = blocks.getLastLocatedBlock();
-    dfsclient.getNamenode().abandonBlock(b.getBlock(), src,
+    dfsclient.getNamenode().abandonBlock(b.getBlock(), fileId, src,
         dfsclient.clientName);
     
     // call abandonBlock again to make sure the operation is idempotent
-    dfsclient.getNamenode().abandonBlock(b.getBlock(), src,
+    dfsclient.getNamenode().abandonBlock(b.getBlock(), fileId, src,
         dfsclient.clientName);
 
     // And close the file

+ 2 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java

@@ -253,18 +253,9 @@ public class TestFileAppend3  {
     assertTrue(fs.rename(p, pnew));
 
     //d. Close file handle that was opened in (b). 
-    try {
-      out.close();
-      fail("close() should throw an exception");
-    } catch(Exception e) {
-      AppendTestUtil.LOG.info("GOOD!", e);
-    }
-
-    //wait for the lease recovery 
-    cluster.setLeasePeriod(1000, 1000);
-    AppendTestUtil.sleep(5000);
+    out.close();
 
-    //check block sizes 
+    //check block sizes
     final long len = fs.getFileStatus(pnew).getLen();
     final LocatedBlocks locatedblocks = fs.dfs.getNamenode().getBlockLocations(pnew.toString(), 0L, len);
     final int numblock = locatedblocks.locatedBlockCount();

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -74,6 +74,7 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.EnumSetWritable;
@@ -384,7 +385,6 @@ public class TestFileCreation {
       Path p = new Path("/testfile");
       FSDataOutputStream stm1 = fs.create(p);
       stm1.write(1);
-      stm1.hflush();
 
       // Create file again without overwrite
       try {
@@ -403,7 +403,8 @@ public class TestFileCreation {
         stm1.close();
         fail("Should have exception closing stm1 since it was deleted");
       } catch (IOException ioe) {
-        GenericTestUtils.assertExceptionContains("File is not open for writing", ioe);
+        GenericTestUtils.assertExceptionContains("No lease on /testfile", ioe);
+        GenericTestUtils.assertExceptionContains("File does not exist.", ioe);
       }
       
     } finally {
@@ -1189,8 +1190,8 @@ public class TestFileCreation {
         cluster.getNameNodeRpc()
             .complete(f.toString(), client.clientName, null, someOtherFileId);
         fail();
-      } catch(FileNotFoundException fnf) {
-        FileSystem.LOG.info("Caught Expected FileNotFoundException: ", fnf);
+      } catch(LeaseExpiredException e) {
+        FileSystem.LOG.info("Caught Expected LeaseExpiredException: ", e);
       }
     } finally {
       IOUtils.closeStream(dfs);

+ 46 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java

@@ -243,11 +243,56 @@ public class TestLease {
       Assert.assertTrue(pRenamedAgain+" not found", fs2.exists(pRenamedAgain));
       Assert.assertTrue("no lease for "+pRenamedAgain, hasLease(cluster, pRenamedAgain));
       Assert.assertEquals(1, leaseCount(cluster));
+      out.close();
     } finally {
       cluster.shutdown();
     }
   }
-  
+
+  /**
+   * Test that we can open up a file for write, move it to another location,
+   * and then create a new file in the previous location, without causing any
+   * lease conflicts.  This is possible because we now use unique inode IDs
+   * to identify files to the NameNode.
+   */
+  @Test
+  public void testLeaseAfterRenameAndRecreate() throws Exception {
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    try {
+      final Path path1 = new Path("/test-file");
+      final String contents1 = "contents1";
+      final Path path2 = new Path("/test-file-new-location");
+      final String contents2 = "contents2";
+
+      // open a file to get a lease
+      FileSystem fs = cluster.getFileSystem();
+      FSDataOutputStream out1 = fs.create(path1);
+      out1.writeBytes(contents1);
+      Assert.assertTrue(hasLease(cluster, path1));
+      Assert.assertEquals(1, leaseCount(cluster));
+
+      DistributedFileSystem fs2 = (DistributedFileSystem)
+          FileSystem.newInstance(fs.getUri(), fs.getConf());
+      fs2.rename(path1, path2);
+
+      FSDataOutputStream out2 = fs2.create(path1);
+      out2.writeBytes(contents2);
+      out2.close();
+
+      // The first file should still be open and valid
+      Assert.assertTrue(hasLease(cluster, path2));
+      out1.close();
+
+      // Contents should be as expected
+      DistributedFileSystem fs3 = (DistributedFileSystem)
+          FileSystem.newInstance(fs.getUri(), fs.getConf());
+      Assert.assertEquals(contents1, DFSTestUtil.readFile(fs3, path2));
+      Assert.assertEquals(contents2, DFSTestUtil.readFile(fs3, path1));
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   @Test
   public void testLease() throws Exception {
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java

@@ -107,8 +107,8 @@ public class TestLeaseRenewer {
     
     // Set up a file so that we start renewing our lease.
     DFSOutputStream mockStream = Mockito.mock(DFSOutputStream.class);
-    String filePath = "/foo";
-    renewer.put(filePath, mockStream, MOCK_DFSCLIENT);
+    long fileId = 123L;
+    renewer.put(fileId, mockStream, MOCK_DFSCLIENT);
 
     // Wait for lease to get renewed
     long failTime = Time.now() + 5000;
@@ -120,7 +120,7 @@ public class TestLeaseRenewer {
       Assert.fail("Did not renew lease at all!");
     }
 
-    renewer.closeFile(filePath, MOCK_DFSCLIENT);
+    renewer.closeFile(fileId, MOCK_DFSCLIENT);
   }
   
   /**
@@ -138,8 +138,8 @@ public class TestLeaseRenewer {
     
     // Set up a file so that we start renewing our lease.
     DFSOutputStream mockStream1 = Mockito.mock(DFSOutputStream.class);
-    String filePath = "/foo";
-    renewer.put(filePath, mockStream1, mockClient1);
+    long fileId = 456L;
+    renewer.put(fileId, mockStream1, mockClient1);
 
     // Second DFSClient does renew lease
     final DFSClient mockClient2 = createMockClient();
@@ -149,7 +149,7 @@ public class TestLeaseRenewer {
 
     // Set up a file so that we start renewing our lease.
     DFSOutputStream mockStream2 = Mockito.mock(DFSOutputStream.class);
-    renewer.put(filePath, mockStream2, mockClient2);
+    renewer.put(fileId, mockStream2, mockClient2);
 
     
     // Wait for lease to get renewed
@@ -170,19 +170,19 @@ public class TestLeaseRenewer {
       }
     }, 100, 10000);
 
-    renewer.closeFile(filePath, mockClient1);
-    renewer.closeFile(filePath, mockClient2);
+    renewer.closeFile(fileId, mockClient1);
+    renewer.closeFile(fileId, mockClient2);
   }
   
   @Test
   public void testThreadName() throws Exception {
     DFSOutputStream mockStream = Mockito.mock(DFSOutputStream.class);
-    String filePath = "/foo";
+    long fileId = 789L;
     Assert.assertFalse("Renewer not initially running",
         renewer.isRunning());
     
     // Pretend to open a file
-    renewer.put(filePath, mockStream, MOCK_DFSCLIENT);
+    renewer.put(fileId, mockStream, MOCK_DFSCLIENT);
     
     Assert.assertTrue("Renewer should have started running",
         renewer.isRunning());
@@ -192,7 +192,7 @@ public class TestLeaseRenewer {
     Assert.assertEquals("LeaseRenewer:myuser@hdfs://nn1/", threadName);
     
     // Pretend to close the file
-    renewer.closeFile(filePath, MOCK_DFSCLIENT);
+    renewer.closeFile(fileId, MOCK_DFSCLIENT);
     renewer.setEmptyTime(Time.now());
     
     // Should stop the renewer running within a few seconds

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -177,12 +178,13 @@ public class TestPersistBlocks {
       
       // Abandon the last block
       DFSClient dfsclient = DFSClientAdapter.getDFSClient((DistributedFileSystem)fs);
+      HdfsFileStatus fileStatus = dfsclient.getNamenode().getFileInfo(FILE_NAME);
       LocatedBlocks blocks = dfsclient.getNamenode().getBlockLocations(
           FILE_NAME, 0, BLOCK_SIZE * NUM_BLOCKS);
       assertEquals(NUM_BLOCKS, blocks.getLocatedBlocks().size());
       LocatedBlock b = blocks.getLastLocatedBlock();
-      dfsclient.getNamenode().abandonBlock(b.getBlock(), FILE_NAME,
-          dfsclient.clientName);
+      dfsclient.getNamenode().abandonBlock(b.getBlock(), fileStatus.getFileId(),
+          FILE_NAME, dfsclient.clientName);
       
       // explicitly do NOT close the file.
       cluster.restartNameNode();

+ 29 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRenameWhileOpen.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
 
@@ -26,11 +28,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.log4j.Level;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 public class TestRenameWhileOpen {
   {
@@ -57,6 +61,7 @@ public class TestRenameWhileOpen {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, TestFileCreation.blockSize);
 
     // create cluster
     System.out.println("Test 1*****************************");
@@ -65,6 +70,15 @@ public class TestRenameWhileOpen {
     try {
       cluster.waitActive();
       fs = cluster.getFileSystem();
+
+      // Normally, the in-progress edit log would be finalized by
+      // FSEditLog#endCurrentLogSegment.  For testing purposes, we
+      // disable that here.
+      FSEditLog spyLog =
+          spy(cluster.getNameNode().getFSImage().getEditLog());
+      doNothing().when(spyLog).endCurrentLogSegment(Mockito.anyBoolean());
+      cluster.getNameNode().getFSImage().setEditLogForTesting(spyLog);
+
       final int nnport = cluster.getNameNodePort();
 
       // create file1.
@@ -92,18 +106,21 @@ public class TestRenameWhileOpen {
 
       // create file3
       Path file3 = new Path(dir3, "file3");
-      FSDataOutputStream stm3 = TestFileCreation.createFile(fs, file3, 1);
-      TestFileCreation.writeFile(stm3);
-      // rename file3 to some bad name
-      try {
-        fs.rename(file3, new Path(dir3, "$ "));
-      } catch(Exception e) {
-        e.printStackTrace();
-      }
-      
-      // restart cluster with the same namenode port as before.
-      // This ensures that leases are persisted in fsimage.
+      FSDataOutputStream stm3 = fs.create(file3);
+      fs.rename(file3, new Path(dir3, "bozo"));
+      // Get a new block for the file.
+      TestFileCreation.writeFile(stm3, TestFileCreation.blockSize + 1);
+      stm3.hflush();
+
+      // Stop the NameNode before closing the files.
+      // This will ensure that the write leases are still active and present
+      // in the edit log.  Simiarly, there should be a pending ADD_BLOCK_OP
+      // for file3, since we just added a block to that file.
+      cluster.getNameNode().stop();
+
+      // Restart cluster with the same namenode port as before.
       cluster.shutdown();
+
       try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
       cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)
                                                 .format(false)
@@ -111,7 +128,7 @@ public class TestRenameWhileOpen {
       cluster.waitActive();
 
       // restart cluster yet again. This triggers the code to read in
-      // persistent leases from fsimage.
+      // persistent leases from the edit log.
       cluster.shutdown();
       try {Thread.sleep(5000);} catch (InterruptedException e) {}
       cluster = new MiniDFSCluster.Builder(conf).nameNodePort(nnport)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepDecreasing.java

@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.junit.Test;
 
 public class TestSetrepDecreasing {
-  @Test
+  @Test(timeout=120000)
   public void testSetrepDecreasing() throws IOException {
     TestSetrepIncreasing.setrep(5, 3, false);
   }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java

@@ -75,11 +75,11 @@ public class TestSetrepIncreasing {
     }
   }
 
-  @Test
+  @Test(timeout=120000)
   public void testSetrepIncreasing() throws IOException {
     setrep(3, 7, false);
   }
-  @Test
+  @Test(timeout=120000)
   public void testSetrepIncreasingSimulatedStorage() throws IOException {
     setrep(3, 7, true);
   }

+ 6 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -465,8 +465,8 @@ public class TestINodeFile {
     }
   }
 
-  @Test
-  public void testWriteToRenamedFile() throws IOException {
+  @Test(timeout=120000)
+  public void testWriteToDeletedFile() throws IOException {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
         .build();
@@ -483,18 +483,16 @@ public class TestINodeFile {
     Path filePath = new Path("/test1/file");
     FSDataOutputStream fos = fs.create(filePath);
 
-    // Rename /test1 to test2, and recreate /test1/file
-    Path renamedPath = new Path("/test2");
-    fs.rename(path, renamedPath);
-    fs.create(filePath, (short) 1);
+    // Delete the file
+    fs.delete(filePath, false);
 
-    // Add new block should fail since /test1/file has a different fileId
+    // Add new block should fail since /test1/file has been deleted.
     try {
       fos.write(data, 0, data.length);
       // make sure addBlock() request gets to NN immediately
       fos.hflush();
 
-      fail("Write should fail after rename");
+      fail("Write should fail after delete");
     } catch (Exception e) {
       /* Ignore */
     } finally {

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

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -92,8 +93,9 @@ public class TestMetaSave {
     try {
       reader = new BufferedReader(new InputStreamReader(in));
       String line = reader.readLine();
-      assertTrue(line.equals(
-          "3 files and directories, 2 blocks = 5 total filesystem objects"));
+      Assert.assertEquals(
+          "3 files and directories, 2 blocks = 5 total filesystem objects",
+          line);
       line = reader.readLine();
       assertTrue(line.equals("Live Datanodes: 1"));
       line = reader.readLine();

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java

@@ -36,6 +36,7 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -819,10 +820,13 @@ public class TestHASafeMode {
           null);
       create.write(testData.getBytes());
       create.hflush();
+      long fileId = ((DFSOutputStream)create.
+          getWrappedStream()).getFileId();
+      FileStatus fileStatus = dfs.getFileStatus(filePath);
       DFSClient client = DFSClientAdapter.getClient(dfs);
       // add one dummy block at NN, but not write to DataNode
-      ExtendedBlock previousBlock = DFSClientAdapter.getPreviousBlock(client,
-          pathString);
+      ExtendedBlock previousBlock =
+          DFSClientAdapter.getPreviousBlock(client, fileId);
       DFSClientAdapter.getNamenode(client).addBlock(
           pathString,
           client.getClientName(),