Browse Source

HDFS-14924. RenameSnapshot not updating new modification time. Contributed by hemanthboyina

Takanobu Asanuma 5 years ago
parent
commit
b25e94ce29

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

@@ -129,15 +129,17 @@ class FSDirSnapshotOp {
       fsd.checkOwner(pc, iip);
       fsd.checkOwner(pc, iip);
     }
     }
     verifySnapshotName(fsd, snapshotNewName, path);
     verifySnapshotName(fsd, snapshotNewName, path);
+    // time of snapshot modification
+    final long now = Time.now();
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
       snapshotManager.renameSnapshot(iip, path, snapshotOldName,
       snapshotManager.renameSnapshot(iip, path, snapshotOldName,
-          snapshotNewName);
+          snapshotNewName, now);
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
     fsd.getEditLog().logRenameSnapshot(path, snapshotOldName,
     fsd.getEditLog().logRenameSnapshot(path, snapshotOldName,
-        snapshotNewName, logRetryCache);
+        snapshotNewName, logRetryCache, now);
   }
   }
 
 
   static SnapshottableDirectoryStatus[] getSnapshottableDirListing(
   static SnapshottableDirectoryStatus[] getSnapshottableDirListing(

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

@@ -1140,11 +1140,19 @@ public class FSEditLog implements LogsPurgeable {
     logEdit(op);
     logEdit(op);
   }
   }
   
   
+  /**
+   * Log that a snapshot is renamed.
+   * @param path Root of the snapshot.
+   * @param snapOldName Old name of the snapshot.
+   * @param snapNewName New name the snapshot will be renamed to.
+   * @param toLogRpcIds If it is logging RPC ids.
+   * @param mtime The snapshot modification time set by Time.now().
+   */
   void logRenameSnapshot(String path, String snapOldName, String snapNewName,
   void logRenameSnapshot(String path, String snapOldName, String snapNewName,
-      boolean toLogRpcIds) {
+      boolean toLogRpcIds, long mtime) {
     RenameSnapshotOp op = RenameSnapshotOp.getInstance(cache.get())
     RenameSnapshotOp op = RenameSnapshotOp.getInstance(cache.get())
         .setSnapshotRoot(path).setSnapshotOldName(snapOldName)
         .setSnapshotRoot(path).setSnapshotOldName(snapOldName)
-        .setSnapshotNewName(snapNewName);
+        .setSnapshotNewName(snapNewName).setSnapshotMTime(mtime);
     logRpcIds(op, toLogRpcIds);
     logRpcIds(op, toLogRpcIds);
     logEdit(op);
     logEdit(op);
   }
   }

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

@@ -841,7 +841,7 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(snapshotRoot, DirOp.WRITE);
       INodesInPath iip = fsDir.getINodesInPath(snapshotRoot, DirOp.WRITE);
       fsNamesys.getSnapshotManager().renameSnapshot(iip,
       fsNamesys.getSnapshotManager().renameSnapshot(iip,
           snapshotRoot, renameSnapshotOp.snapshotOldName,
           snapshotRoot, renameSnapshotOp.snapshotOldName,
-          renameSnapshotOp.snapshotNewName);
+          renameSnapshotOp.snapshotNewName, renameSnapshotOp.mtime);
       
       
       if (toAddRetryCache) {
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(renameSnapshotOp.rpcClientId,
         fsNamesys.addCacheEntry(renameSnapshotOp.rpcClientId,

+ 20 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -3606,6 +3606,9 @@ public abstract class FSEditLogOp {
     String snapshotRoot;
     String snapshotRoot;
     String snapshotOldName;
     String snapshotOldName;
     String snapshotNewName;
     String snapshotNewName;
+    /** Modification time of the edit set by Time.now(). */
+    long mtime;
+
     
     
     RenameSnapshotOp() {
     RenameSnapshotOp() {
       super(OP_RENAME_SNAPSHOT);
       super(OP_RENAME_SNAPSHOT);
@@ -3620,13 +3623,16 @@ public abstract class FSEditLogOp {
       snapshotRoot = null;
       snapshotRoot = null;
       snapshotOldName = null;
       snapshotOldName = null;
       snapshotNewName = null;
       snapshotNewName = null;
+      mtime = 0L;
     }
     }
-    
+
+    /* set the old name of the snapshot. */
     RenameSnapshotOp setSnapshotOldName(String snapshotOldName) {
     RenameSnapshotOp setSnapshotOldName(String snapshotOldName) {
       this.snapshotOldName = snapshotOldName;
       this.snapshotOldName = snapshotOldName;
       return this;
       return this;
     }
     }
 
 
+    /* set the new name of the snapshot. */
     RenameSnapshotOp setSnapshotNewName(String snapshotNewName) {
     RenameSnapshotOp setSnapshotNewName(String snapshotNewName) {
       this.snapshotNewName = snapshotNewName;
       this.snapshotNewName = snapshotNewName;
       return this;
       return this;
@@ -3637,11 +3643,18 @@ public abstract class FSEditLogOp {
       return this;
       return this;
     }
     }
     
     
+    /* The snapshot rename time set by Time.now(). */
+    RenameSnapshotOp setSnapshotMTime(long mTime) {
+      this.mtime = mTime;
+      return this;
+    }
+
     @Override
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
     void readFields(DataInputStream in, int logVersion) throws IOException {
       snapshotRoot = FSImageSerialization.readString(in);
       snapshotRoot = FSImageSerialization.readString(in);
       snapshotOldName = FSImageSerialization.readString(in);
       snapshotOldName = FSImageSerialization.readString(in);
       snapshotNewName = FSImageSerialization.readString(in);
       snapshotNewName = FSImageSerialization.readString(in);
+      mtime = FSImageSerialization.readLong(in);
       
       
       // read RPC ids if necessary
       // read RPC ids if necessary
       readRpcIds(in, logVersion);
       readRpcIds(in, logVersion);
@@ -3652,6 +3665,7 @@ public abstract class FSEditLogOp {
       FSImageSerialization.writeString(snapshotRoot, out);
       FSImageSerialization.writeString(snapshotRoot, out);
       FSImageSerialization.writeString(snapshotOldName, out);
       FSImageSerialization.writeString(snapshotOldName, out);
       FSImageSerialization.writeString(snapshotNewName, out);
       FSImageSerialization.writeString(snapshotNewName, out);
+      FSImageSerialization.writeLong(mtime, out);
       
       
       writeRpcIds(rpcClientId, rpcCallId, out);
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
     }
@@ -3661,6 +3675,7 @@ public abstract class FSEditLogOp {
       XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
       XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
       XMLUtils.addSaxString(contentHandler, "SNAPSHOTOLDNAME", snapshotOldName);
       XMLUtils.addSaxString(contentHandler, "SNAPSHOTOLDNAME", snapshotOldName);
       XMLUtils.addSaxString(contentHandler, "SNAPSHOTNEWNAME", snapshotNewName);
       XMLUtils.addSaxString(contentHandler, "SNAPSHOTNEWNAME", snapshotNewName);
+      XMLUtils.addSaxString(contentHandler, "MTIME", Long.toString(mtime));
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
       appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
     }
     }
 
 
@@ -3669,6 +3684,7 @@ public abstract class FSEditLogOp {
       snapshotRoot = st.getValue("SNAPSHOTROOT");
       snapshotRoot = st.getValue("SNAPSHOTROOT");
       snapshotOldName = st.getValue("SNAPSHOTOLDNAME");
       snapshotOldName = st.getValue("SNAPSHOTOLDNAME");
       snapshotNewName = st.getValue("SNAPSHOTNEWNAME");
       snapshotNewName = st.getValue("SNAPSHOTNEWNAME");
+      this.mtime = Long.parseLong(st.getValue("MTIME"));
       
       
       readRpcIdsFromXml(st);
       readRpcIdsFromXml(st);
     }
     }
@@ -3681,7 +3697,9 @@ public abstract class FSEditLogOp {
           .append(", snapshotOldName=")
           .append(", snapshotOldName=")
           .append(snapshotOldName)
           .append(snapshotOldName)
           .append(", snapshotNewName=")
           .append(", snapshotNewName=")
-          .append(snapshotNewName);
+          .append(snapshotNewName)
+          .append(", mtime=")
+          .append(mtime);
       appendRpcIdsToString(builder, rpcClientId, rpcCallId);
       appendRpcIdsToString(builder, rpcClientId, rpcCallId);
       builder.append("]");
       builder.append("]");
       return builder.toString();
       return builder.toString();

+ 11 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -296,9 +296,17 @@ public class INodeDirectory extends INodeWithAdditionalFields
         reclaimContext, this, snapshotName);
         reclaimContext, this, snapshotName);
   }
   }
 
 
-  public void renameSnapshot(String path, String oldName, String newName)
-      throws SnapshotException {
-    getDirectorySnapshottableFeature().renameSnapshot(path, oldName, newName);
+  /**
+   * Rename a snapshot.
+   * @param path The directory path where the snapshot was taken.
+   * @param oldName Old name of the snapshot
+   * @param newName New name the snapshot will be renamed to
+   * @param mtime The snapshot modification time set by Time.now().
+   */
+  public void renameSnapshot(String path, String oldName, String newName,
+      long mtime) throws SnapshotException {
+    getDirectorySnapshottableFeature().renameSnapshot(path, oldName, newName,
+        mtime);
   }
   }
 
 
   /** add DirectorySnapshottableFeature */
   /** add DirectorySnapshottableFeature */

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectorySnapshottableFeature.java

@@ -112,12 +112,14 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
    *          Old name of the snapshot
    *          Old name of the snapshot
    * @param newName
    * @param newName
    *          New name the snapshot will be renamed to
    *          New name the snapshot will be renamed to
+   * @param mtime The snapshot modification time set by Time.now().
    * @throws SnapshotException
    * @throws SnapshotException
    *           Throw SnapshotException when either the snapshot with the old
    *           Throw SnapshotException when either the snapshot with the old
    *           name does not exist or a snapshot with the new name already
    *           name does not exist or a snapshot with the new name already
    *           exists
    *           exists
    */
    */
-  public void renameSnapshot(String path, String oldName, String newName)
+  public void renameSnapshot(String path, String oldName, String newName,
+      long mtime)
       throws SnapshotException {
       throws SnapshotException {
     final int indexOfOld = searchSnapshot(DFSUtil.string2Bytes(oldName));
     final int indexOfOld = searchSnapshot(DFSUtil.string2Bytes(oldName));
     if (indexOfOld < 0) {
     if (indexOfOld < 0) {
@@ -137,6 +139,7 @@ public class DirectorySnapshottableFeature extends DirectoryWithSnapshotFeature
       Snapshot snapshot = snapshotsByNames.remove(indexOfOld);
       Snapshot snapshot = snapshotsByNames.remove(indexOfOld);
       final INodeDirectory ssRoot = snapshot.getRoot();
       final INodeDirectory ssRoot = snapshot.getRoot();
       ssRoot.setLocalName(newNameBytes);
       ssRoot.setLocalName(newNameBytes);
+      ssRoot.setModificationTime(mtime, Snapshot.CURRENT_STATE_ID);
       indexOfNew = -indexOfNew - 1;
       indexOfNew = -indexOfNew - 1;
       if (indexOfNew <= indexOfOld) {
       if (indexOfNew <= indexOfOld) {
         snapshotsByNames.add(indexOfNew, snapshot);
         snapshotsByNames.add(indexOfNew, snapshot);

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

@@ -362,6 +362,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    *          Old name of the snapshot
    *          Old name of the snapshot
    * @param newSnapshotName
    * @param newSnapshotName
    *          New name of the snapshot
    *          New name of the snapshot
+   * @param now is the snapshot modification time set by Time.now().
    * @throws IOException
    * @throws IOException
    *           Throw IOException when 1) the given path does not lead to an
    *           Throw IOException when 1) the given path does not lead to an
    *           existing snapshottable directory, and/or 2) the snapshot with the
    *           existing snapshottable directory, and/or 2) the snapshot with the
@@ -369,10 +370,10 @@ public class SnapshotManager implements SnapshotStatsMXBean {
    *           a snapshot with the new name for the directory
    *           a snapshot with the new name for the directory
    */
    */
   public void renameSnapshot(final INodesInPath iip, final String snapshotRoot,
   public void renameSnapshot(final INodesInPath iip, final String snapshotRoot,
-      final String oldSnapshotName, final String newSnapshotName)
+      final String oldSnapshotName, final String newSnapshotName, long now)
       throws IOException {
       throws IOException {
     final INodeDirectory srcRoot = getSnapshottableRoot(iip);
     final INodeDirectory srcRoot = getSnapshottableRoot(iip);
-    srcRoot.renameSnapshot(snapshotRoot, oldSnapshotName, newSnapshotName);
+    srcRoot.renameSnapshot(snapshotRoot, oldSnapshotName, newSnapshotName, now);
   }
   }
   
   
   public int getNumSnapshottableDirs() {
   public int getNumSnapshottableDirs() {

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
@@ -472,6 +473,23 @@ public class TestSnapshot {
         newSnapshotStatus.getModificationTime());
         newSnapshotStatus.getModificationTime());
   }
   }
 
 
+  @Test(timeout = 60000)
+  public void testRenameSnapshotMtime() throws Exception {
+    Path dir = new Path("/dir");
+    Path sub = new Path(dir, "sub");
+    Path subFile = new Path(sub, "file");
+    DFSTestUtil.createFile(hdfs, subFile, BLOCKSIZE, REPLICATION, seed);
+
+    hdfs.allowSnapshot(dir);
+    Path snapshotPath = hdfs.createSnapshot(dir, "s1");
+    FileStatus oldSnapshotStatus = hdfs.getFileStatus(snapshotPath);
+    hdfs.renameSnapshot(dir, "s1", "s2");
+    Path snapshotRenamePath = new Path("/dir/.snapshot/s2");
+    FileStatus newSnapshotStatus = hdfs.getFileStatus(snapshotRenamePath);
+    assertNotEquals(oldSnapshotStatus.getModificationTime(),
+        newSnapshotStatus.getModificationTime());
+  }
+
   /**
   /**
    * Prepare a list of modifications. A modification may be a file creation,
    * Prepare a list of modifications. A modification may be a file creation,
    * file deletion, or a modification operation such as appending to an existing
    * file deletion, or a modification operation such as appending to an existing

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -288,6 +288,7 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
+      <MTIME>1512607197715</MTIME>
       <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
       <RPC_CLIENTID>cab1aa2d-e08a-4d2f-8216-76e167eccd94</RPC_CLIENTID>
       <RPC_CALLID>55</RPC_CALLID>
       <RPC_CALLID>55</RPC_CALLID>
     </DATA>
     </DATA>