You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by in...@apache.org on 2016/10/03 22:58:36 UTC

[23/57] [abbrv] hadoop git commit: HDFS-10779. Rename does not need to re-solve destination. Contributed by Daryn Sharp.

HDFS-10779. Rename does not need to re-solve destination. Contributed by Daryn Sharp.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5f34402a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5f34402a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5f34402a

Branch: refs/heads/HDFS-10467
Commit: 5f34402adae191232fe78e62990396ca07f314bb
Parents: 0d6778d
Author: Kihwal Lee <ki...@apache.org>
Authored: Wed Sep 28 17:57:23 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Wed Sep 28 17:57:23 2016 -0500

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSDirRenameOp.java     | 184 ++++++++++---------
 .../hdfs/server/namenode/FSDirectory.java       |  14 --
 .../hdfs/server/namenode/FSNamesystem.java      |  11 +-
 3 files changed, 103 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f34402a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
index f98f8b1..0fdc545 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
@@ -38,23 +38,18 @@ import org.apache.hadoop.util.Time;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
-
 import static org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import static org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
 
 class FSDirRenameOp {
   @Deprecated
-  static RenameOldResult renameToInt(
-      FSDirectory fsd, final String srcArg, final String dstArg,
+  static RenameResult renameToInt(
+      FSDirectory fsd, final String src, final String dst,
       boolean logRetryCache)
       throws IOException {
-    String src = srcArg;
-    String dst = dstArg;
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src +
           " to " + dst);
@@ -64,18 +59,12 @@ class FSDirRenameOp {
     }
     FSPermissionChecker pc = fsd.getPermissionChecker();
 
-    HdfsFileStatus resultingStat = null;
     // Rename does not operate on link targets
     // Do not resolveLink when checking permissions of src and dst
     INodesInPath srcIIP = fsd.resolvePathForWrite(pc, src, false);
     INodesInPath dstIIP = fsd.resolvePathForWrite(pc, dst, false);
-    @SuppressWarnings("deprecation")
-    final boolean status = renameTo(fsd, pc, srcIIP, dstIIP, logRetryCache);
-    if (status) {
-      dstIIP = fsd.getINodesInPath(dstIIP.getPath(), false);
-      resultingStat = fsd.getAuditFileInfo(dstIIP);
-    }
-    return new RenameOldResult(status, resultingStat);
+    dstIIP = dstForRenameTo(srcIIP, dstIIP);
+    return renameTo(fsd, pc, srcIIP, dstIIP, logRetryCache);
   }
 
   /**
@@ -124,15 +113,30 @@ class FSDirRenameOp {
    * <br>
    */
   @Deprecated
-  @SuppressWarnings("deprecation")
-  static boolean renameForEditLog(FSDirectory fsd, String src, String dst,
+  static INodesInPath renameForEditLog(FSDirectory fsd, String src, String dst,
       long timestamp) throws IOException {
-    if (fsd.isDir(dst)) {
-      dst += Path.SEPARATOR + new Path(src).getName();
-    }
     final INodesInPath srcIIP = fsd.getINodesInPath4Write(src, false);
-    final INodesInPath dstIIP = fsd.getINodesInPath4Write(dst, false);
-    return unprotectedRenameTo(fsd, src, dst, srcIIP, dstIIP, timestamp);
+    INodesInPath dstIIP = fsd.getINodesInPath4Write(dst, false);
+    // this is wrong but accidentally works.  the edit contains the full path
+    // so the following will do nothing, but shouldn't change due to backward
+    // compatibility when maybe full path wasn't logged.
+    dstIIP = dstForRenameTo(srcIIP, dstIIP);
+    return unprotectedRenameTo(fsd, srcIIP, dstIIP, timestamp);
+  }
+
+  // if destination is a directory, append source child's name, else return
+  // iip as-is.
+  private static INodesInPath dstForRenameTo(
+      INodesInPath srcIIP, INodesInPath dstIIP) throws IOException {
+    INode dstINode = dstIIP.getLastINode();
+    if (dstINode != null && dstINode.isDirectory()) {
+      byte[] childName = srcIIP.getLastLocalName();
+      // new dest might exist so look it up.
+      INode childINode = dstINode.asDirectory().getChild(
+          childName, dstIIP.getPathSnapshotId());
+      dstIIP = INodesInPath.append(dstIIP, childINode, childName);
+    }
+    return dstIIP;
   }
 
   /**
@@ -141,12 +145,12 @@ class FSDirRenameOp {
    * @param fsd FSDirectory
    * @param src source path
    * @param dst destination path
-   * @return true if rename succeeds; false otherwise
+   * @return true INodesInPath if rename succeeds; null otherwise
    * @deprecated See {@link #renameToInt(FSDirectory, String, String,
    * boolean, Options.Rename...)}
    */
   @Deprecated
-  static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst,
+  static INodesInPath unprotectedRenameTo(FSDirectory fsd,
       final INodesInPath srcIIP, final INodesInPath dstIIP, long timestamp)
       throws IOException {
     assert fsd.hasWriteLock();
@@ -156,32 +160,34 @@ class FSDirRenameOp {
     } catch (SnapshotException e) {
       throw e;
     } catch (IOException ignored) {
-      return false;
+      return null;
     }
 
+    String src = srcIIP.getPath();
+    String dst = dstIIP.getPath();
     // validate the destination
     if (dst.equals(src)) {
-      return true;
+      return dstIIP;
     }
 
     try {
       validateDestination(src, dst, srcInode);
     } catch (IOException ignored) {
-      return false;
+      return null;
     }
 
     if (dstIIP.getLastINode() != null) {
       NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: " +
           "failed to rename " + src + " to " + dst + " because destination " +
           "exists");
-      return false;
+      return null;
     }
     INode dstParent = dstIIP.getINode(-2);
     if (dstParent == null) {
       NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: " +
           "failed to rename " + src + " to " + dst + " because destination's " +
           "parent does not exist");
-      return false;
+      return null;
     }
 
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
@@ -189,17 +195,19 @@ class FSDirRenameOp {
     verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
     verifyQuotaForRename(fsd, srcIIP, dstIIP);
 
-    RenameOperation tx = new RenameOperation(fsd, src, dst, srcIIP, dstIIP);
+    RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP);
 
     boolean added = false;
 
+    INodesInPath renamedIIP = null;
     try {
       // remove src
       if (!tx.removeSrc4OldRename()) {
-        return false;
+        return null;
       }
 
-      added = tx.addSourceToDestination();
+      renamedIIP = tx.addSourceToDestination();
+      added = (renamedIIP != null);
       if (added) {
         if (NameNode.stateChangeLog.isDebugEnabled()) {
           NameNode.stateChangeLog.debug("DIR* FSDirectory" +
@@ -209,7 +217,7 @@ class FSDirRenameOp {
         tx.updateMtimeAndLease(timestamp);
         tx.updateQuotasInSourceTree(fsd.getBlockStoragePolicySuite());
 
-        return true;
+        return renamedIIP;
       }
     } finally {
       if (!added) {
@@ -218,13 +226,13 @@ class FSDirRenameOp {
     }
     NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: " +
         "failed to rename " + src + " to " + dst);
-    return false;
+    return null;
   }
 
   /**
    * The new rename which has the POSIX semantic.
    */
-  static Map.Entry<BlocksMapUpdateInfo, HdfsFileStatus> renameToInt(
+  static RenameResult renameToInt(
       FSDirectory fsd, final String srcArg, final String dstArg,
       boolean logRetryCache, Options.Rename... options)
       throws IOException {
@@ -241,25 +249,19 @@ class FSDirRenameOp {
 
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     // returns resolved path
-    dst = renameTo(fsd, pc, src, dst, collectedBlocks, logRetryCache, options);
-    INodesInPath dstIIP = fsd.getINodesInPath(dst, false);
-    HdfsFileStatus resultingStat = fsd.getAuditFileInfo(dstIIP);
-
-    return new AbstractMap.SimpleImmutableEntry<>(
-        collectedBlocks, resultingStat);
+    return renameTo(fsd, pc, src, dst, collectedBlocks, logRetryCache, options);
   }
 
   /**
    * @see {@link #unprotectedRenameTo(FSDirectory, String, String, INodesInPath,
    * INodesInPath, long, BlocksMapUpdateInfo, Options.Rename...)}
    */
-  static String renameTo(FSDirectory fsd, FSPermissionChecker pc, String src,
-      String dst, BlocksMapUpdateInfo collectedBlocks, boolean logRetryCache,
-      Options.Rename... options) throws IOException {
+  static RenameResult renameTo(FSDirectory fsd, FSPermissionChecker pc,
+      String src, String dst, BlocksMapUpdateInfo collectedBlocks,
+      boolean logRetryCache,Options.Rename... options)
+          throws IOException {
     final INodesInPath srcIIP = fsd.resolvePathForWrite(pc, src, false);
     final INodesInPath dstIIP = fsd.resolvePathForWrite(pc, dst, false);
-    src = srcIIP.getPath();
-    dst = dstIIP.getPath();
     if (fsd.isPermissionEnabled()) {
       boolean renameToTrash = false;
       if (null != options &&
@@ -295,16 +297,19 @@ class FSDirRenameOp {
     }
     final long mtime = Time.now();
     fsd.writeLock();
+    final RenameResult result;
     try {
-      if (unprotectedRenameTo(fsd, src, dst, srcIIP, dstIIP, mtime,
-          collectedBlocks, options)) {
+      result = unprotectedRenameTo(fsd, srcIIP, dstIIP, mtime,
+          collectedBlocks, options);
+      if (result.filesDeleted) {
         FSDirDeleteOp.incrDeletedFileCount(1);
       }
     } finally {
       fsd.writeUnlock();
     }
-    fsd.getEditLog().logRename(src, dst, mtime, logRetryCache, options);
-    return dst;
+    fsd.getEditLog().logRename(
+        srcIIP.getPath(), dstIIP.getPath(), mtime, logRetryCache, options);
+    return result;
   }
 
   /**
@@ -327,7 +332,7 @@ class FSDirRenameOp {
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
     final INodesInPath srcIIP = fsd.getINodesInPath4Write(src, false);
     final INodesInPath dstIIP = fsd.getINodesInPath4Write(dst, false);
-    unprotectedRenameTo(fsd, src, dst, srcIIP, dstIIP, timestamp,
+    unprotectedRenameTo(fsd, srcIIP, dstIIP, timestamp,
         collectedBlocks, options);
     if (!collectedBlocks.getToDeleteList().isEmpty()) {
       fsd.getFSNamesystem().getBlockManager()
@@ -348,7 +353,7 @@ class FSDirRenameOp {
    * @param options         Rename options
    * @return whether a file/directory gets overwritten in the dst path
    */
-  static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst,
+  static RenameResult unprotectedRenameTo(FSDirectory fsd,
       final INodesInPath srcIIP, final INodesInPath dstIIP, long timestamp,
       BlocksMapUpdateInfo collectedBlocks, Options.Rename... options)
       throws IOException {
@@ -356,6 +361,8 @@ class FSDirRenameOp {
     boolean overwrite = options != null
         && Arrays.asList(options).contains(Options.Rename.OVERWRITE);
 
+    final String src = srcIIP.getPath();
+    final String dst = dstIIP.getPath();
     final String error;
     final INode srcInode = srcIIP.getLastINode();
     validateRenameSource(srcIIP);
@@ -401,7 +408,7 @@ class FSDirRenameOp {
     verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
     verifyQuotaForRename(fsd, srcIIP, dstIIP);
 
-    RenameOperation tx = new RenameOperation(fsd, src, dst, srcIIP, dstIIP);
+    RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP);
 
     boolean undoRemoveSrc = true;
     tx.removeSrc();
@@ -417,7 +424,8 @@ class FSDirRenameOp {
       }
 
       // add src as dst to complete rename
-      if (tx.addSourceToDestination()) {
+      INodesInPath renamedIIP = tx.addSourceToDestination();
+      if (renamedIIP != null) {
         undoRemoveSrc = false;
         if (NameNode.stateChangeLog.isDebugEnabled()) {
           NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: "
@@ -442,7 +450,8 @@ class FSDirRenameOp {
         }
 
         tx.updateQuotasInSourceTree(bsps);
-        return filesDeleted;
+        return createRenameResult(
+            fsd, renamedIIP, filesDeleted, collectedBlocks);
       }
     } finally {
       if (undoRemoveSrc) {
@@ -462,17 +471,9 @@ class FSDirRenameOp {
    * boolean, Options.Rename...)}
    */
   @Deprecated
-  @SuppressWarnings("deprecation")
-  private static boolean renameTo(FSDirectory fsd, FSPermissionChecker pc,
+  private static RenameResult renameTo(FSDirectory fsd, FSPermissionChecker pc,
       INodesInPath srcIIP, INodesInPath dstIIP, boolean logRetryCache)
           throws IOException {
-    String src = srcIIP.getPath();
-    String dst = dstIIP.getPath();
-    // Note: We should not be doing this.  This is move() not renameTo().
-    if (fsd.isDir(dst)) {
-      dstIIP = INodesInPath.append(dstIIP, null, srcIIP.getLastLocalName());
-    }
-    final String actualDst = dstIIP.getPath();
     if (fsd.isPermissionEnabled()) {
       // Check write access to parent of src
       fsd.checkPermission(pc, srcIIP, false, null, FsAction.WRITE, null, null,
@@ -483,22 +484,24 @@ class FSDirRenameOp {
     }
 
     if (NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: " + src + " to "
-          + dst);
+      NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: " +
+          srcIIP.getPath() + " to " + dstIIP.getPath());
     }
     final long mtime = Time.now();
-    boolean stat = false;
+    INodesInPath renameIIP;
     fsd.writeLock();
     try {
-      stat = unprotectedRenameTo(fsd, src, actualDst, srcIIP, dstIIP, mtime);
+      renameIIP = unprotectedRenameTo(fsd, srcIIP, dstIIP, mtime);
     } finally {
       fsd.writeUnlock();
     }
-    if (stat) {
-      fsd.getEditLog().logRename(src, actualDst, mtime, logRetryCache);
-      return true;
+    if (renameIIP != null) {
+      fsd.getEditLog().logRename(
+          srcIIP.getPath(), dstIIP.getPath(), mtime, logRetryCache);
     }
-    return false;
+    // this rename never overwrites the dest so files deleted and collected
+    // are irrelevant.
+    return createRenameResult(fsd, renameIIP, false, null);
   }
 
   private static void validateDestination(
@@ -584,8 +587,6 @@ class FSDirRenameOp {
     private final INodesInPath srcParentIIP;
     private INodesInPath dstIIP;
     private final INodesInPath dstParentIIP;
-    private final String src;
-    private final String dst;
     private final INodeReference.WithCount withCount;
     private final int srcRefDstSnapshot;
     private final INodeDirectory srcParent;
@@ -596,12 +597,9 @@ class FSDirRenameOp {
     private INode srcChild;
     private INode oldDstChild;
 
-    RenameOperation(FSDirectory fsd, String src, String dst,
-                    INodesInPath srcIIP, INodesInPath dstIIP)
+    RenameOperation(FSDirectory fsd, INodesInPath srcIIP, INodesInPath dstIIP)
         throws QuotaExceededException {
       this.fsd = fsd;
-      this.src = src;
-      this.dst = dst;
       this.srcIIP = srcIIP;
       this.dstIIP = dstIIP;
       this.srcParentIIP = srcIIP.getParentINodesInPath();
@@ -647,8 +645,8 @@ class FSDirRenameOp {
     long removeSrc() throws IOException {
       long removedNum = fsd.removeLastINode(srcIIP);
       if (removedNum == -1) {
-        String error = "Failed to rename " + src + " to " + dst +
-            " because the source can not be removed";
+        String error = "Failed to rename " + srcIIP.getPath() + " to " +
+            dstIIP.getPath() + " because the source can not be removed";
         NameNode.stateChangeLog.warn("DIR* FSDirRenameOp.unprotectedRenameTo:" +
             error);
         throw new IOException(error);
@@ -664,8 +662,8 @@ class FSDirRenameOp {
       final long removedSrc = fsd.removeLastINode(srcIIP);
       if (removedSrc == -1) {
         NameNode.stateChangeLog.warn("DIR* FSDirRenameOp.unprotectedRenameTo: "
-            + "failed to rename " + src + " to " + dst + " because the source" +
-            " can not be removed");
+            + "failed to rename " + srcIIP.getPath() + " to "
+            + dstIIP.getPath() + " because the source can not be removed");
         return false;
       } else {
         // update the quota count if necessary
@@ -686,7 +684,7 @@ class FSDirRenameOp {
       return removedNum;
     }
 
-    boolean addSourceToDestination() {
+    INodesInPath addSourceToDestination() {
       final INode dstParent = dstParentIIP.getLastINode();
       final byte[] dstChildName = dstIIP.getLastLocalName();
       final INode toDst;
@@ -698,7 +696,7 @@ class FSDirRenameOp {
         toDst = new INodeReference.DstReference(dstParent.asDirectory(),
             withCount, dstIIP.getLatestSnapshotId());
       }
-      return fsd.addLastINodeNoQuotaCheck(dstParentIIP, toDst) != null;
+      return fsd.addLastINodeNoQuotaCheck(dstParentIIP, toDst);
     }
 
     void updateMtimeAndLease(long timestamp) throws QuotaExceededException {
@@ -785,13 +783,27 @@ class FSDirRenameOp {
     }
   }
 
-  static class RenameOldResult {
+  private static RenameResult createRenameResult(FSDirectory fsd,
+      INodesInPath dst, boolean filesDeleted,
+      BlocksMapUpdateInfo collectedBlocks) throws IOException {
+    boolean success = (dst != null);
+    HdfsFileStatus auditStat = success ? fsd.getAuditFileInfo(dst) : null;
+    return new RenameResult(
+        success, auditStat, filesDeleted, collectedBlocks);
+  }
+
+  static class RenameResult {
     final boolean success;
     final HdfsFileStatus auditStat;
+    final boolean filesDeleted;
+    final BlocksMapUpdateInfo collectedBlocks;
 
-    RenameOldResult(boolean success, HdfsFileStatus auditStat) {
+    RenameResult(boolean success, HdfsFileStatus auditStat,
+        boolean filesDeleted, BlocksMapUpdateInfo collectedBlocks) {
       this.success = success;
       this.auditStat = auditStat;
+      this.filesDeleted = filesDeleted;
+      this.collectedBlocks = collectedBlocks;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f34402a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 2a3cabb..2c7a268 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -648,20 +648,6 @@ public class FSDirectory implements Closeable {
   }
 
   /**
-   * Check whether the path specifies a directory
-   */
-  boolean isDir(String src) throws UnresolvedLinkException {
-    src = normalizePath(src);
-    readLock();
-    try {
-      INode node = getINode(src, false);
-      return node != null && node.isDirectory();
-    } finally {
-      readUnlock();
-    }
-  }
-
-  /**
    * Tell the block manager to update the replication factors when delete
    * happens. Deleting a file or a snapshot might decrease the replication
    * factor of the blocks as the blocks are always replicated to the highest

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f34402a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 7f8981f..4700263 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -2845,7 +2845,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   @Deprecated
   boolean renameTo(String src, String dst, boolean logRetryCache)
       throws IOException {
-    FSDirRenameOp.RenameOldResult ret = null;
+    FSDirRenameOp.RenameResult ret = null;
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2857,7 +2857,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     } finally {
       writeUnlock();
     }
-    boolean success = ret != null && ret.success;
+    boolean success = ret.success;
     if (success) {
       getEditLog().logSync();
       logAuditEvent(success, "rename", src, dst, ret.auditStat);
@@ -2868,7 +2868,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void renameTo(final String src, final String dst,
                 boolean logRetryCache, Options.Rename... options)
       throws IOException {
-    Map.Entry<BlocksMapUpdateInfo, HdfsFileStatus> res = null;
+    FSDirRenameOp.RenameResult res = null;
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2884,15 +2884,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     getEditLog().logSync();
 
-    BlocksMapUpdateInfo collectedBlocks = res.getKey();
-    HdfsFileStatus auditStat = res.getValue();
+    BlocksMapUpdateInfo collectedBlocks = res.collectedBlocks;
     if (!collectedBlocks.getToDeleteList().isEmpty()) {
       removeBlocks(collectedBlocks);
       collectedBlocks.clear();
     }
 
     logAuditEvent(true, "rename (options=" + Arrays.toString(options) +
-        ")", src, dst, auditStat);
+        ")", src, dst, res.auditStat);
   }
 
   /**


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org