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 so...@apache.org on 2020/07/30 14:56:46 UTC

[hadoop] branch branch-2.10 updated: HDFS-15313. Ensure inodes in active filesytem are not deleted during snapshot delete. Contributed by Shashikant Banerjee.

This is an automated email from the ASF dual-hosted git repository.

sodonnell pushed a commit to branch branch-2.10
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-2.10 by this push:
     new a1fe3ee  HDFS-15313. Ensure inodes in active filesytem are not deleted during snapshot delete. Contributed by Shashikant Banerjee.
a1fe3ee is described below

commit a1fe3eed9be102bff1cd6c3a291a87ea86fd2918
Author: S O'Donnell <so...@cloudera.com>
AuthorDate: Thu Jul 30 15:48:15 2020 +0100

    HDFS-15313. Ensure inodes in active filesytem are not deleted during snapshot delete. Contributed by Shashikant Banerjee.
---
 .../apache/hadoop/hdfs/server/namenode/INode.java  | 40 ++++++++++++++----
 .../snapshot/DirectoryWithSnapshotFeature.java     | 31 +++++++-------
 .../java/org/apache/hadoop/hdfs/util/Diff.java     | 48 +++++++++++++++++++++-
 .../server/namenode/TestFSImageWithSnapshot.java   | 40 ++++++++++++++++++
 .../namenode/snapshot/TestRenameWithSnapshots.java | 41 +++++++++++++++++-
 5 files changed, 174 insertions(+), 26 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index 1d2654e..d0f98e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -17,12 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.io.PrintStream;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.List;
-import java.util.Map;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
@@ -32,12 +28,12 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
@@ -48,8 +44,11 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.StringUtils;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.List;
+import java.util.Map;
 
 /**
  * We keep an in-memory representation of the file/block hierarchy.
@@ -226,6 +225,27 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     return this;
   }
 
+  /** Is this inode in the current state? */
+  public boolean isInCurrentState() {
+    if (isRoot()) {
+      return true;
+    }
+    final INodeDirectory parentDir = getParent();
+    if (parentDir == null) {
+      return false; // this inode is only referenced in snapshots
+    }
+    if (!parentDir.isInCurrentState()) {
+      return false;
+    }
+    final INode child = parentDir.getChild(getLocalNameBytes(),
+            Snapshot.CURRENT_STATE_ID);
+    if (this == child) {
+      return true;
+    }
+    return child != null && child.isReference() &&
+        this.equals(child.asReference().getReferredINode());
+  }
+
   /** Is this inode in the latest snapshot? */
   public final boolean isInLatestSnapshot(final int latestSnapshotId) {
     if (latestSnapshotId == Snapshot.CURRENT_STATE_ID ||
@@ -235,6 +255,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     // if parent is a reference node, parent must be a renamed node. We can 
     // stop the check at the reference node.
     if (parent != null && parent.isReference()) {
+      // TODO: Is it a bug to return true?
+      //       Some ancestor nodes may not be in the latest snapshot.
       return true;
     }
     final INodeDirectory parentDir = getParent();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
index d2dd3ed..031fd3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DirectoryWithSnapshotFeature.java
@@ -24,6 +24,7 @@ import java.util.Deque;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.ArrayList;
 import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -740,20 +741,22 @@ public class DirectoryWithSnapshotFeature implements INode.Feature {
           // were created before "prior" will be covered by the later 
           // cleanSubtreeRecursively call.
           if (priorCreated != null) {
-            if (currentINode.isLastReference() &&
-                    currentINode.getDiffs().getLastSnapshotId() == prior) {
-              // If this is the last reference of the directory inode and it
-              // can not be accessed in any of the subsequent snapshots i.e,
-              // this is the latest snapshot diff and if this is the last
-              // reference, the created list can be
-              // destroyed.
-              priorDiff.getChildrenDiff().destroyCreatedList(
-                  reclaimContext, currentINode);
-            } else {
-              // we only check the node originally in prior's created list
-              for (INode cNode : priorDiff.getChildrenDiff().getList(
-                  ListType.CREATED)) {
-                if (priorCreated.containsKey(cNode)) {
+            // The nodes in priorCreated must be destroyed if
+            //   (1) this is the last reference, and
+            //   (2) prior is the last snapshot, and
+            //   (3) currentINode is not in the current state.
+            final boolean destroy = currentINode.isLastReference()
+                && currentINode.getDiffs().getLastSnapshotId() == prior
+                && !currentINode.isInCurrentState();
+            // we only check the node originally in prior's created list
+            for (INode cNode : new ArrayList<>(priorDiff.
+                    diff.getCreatedUnmodifiable())) {
+              if (priorCreated.containsKey(cNode)) {
+                if (destroy) {
+                  cNode.destroyAndCollectBlocks(reclaimContext);
+                  currentINode.removeChild(cNode);
+                  priorDiff.diff.removeCreated(cNode);
+                } else {
                   cNode.cleanSubtree(reclaimContext, snapshot, NO_SNAPSHOT_ID);
                 }
               }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java
index 54e7103..db48437 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java
@@ -156,10 +156,54 @@ public class Diff<K, E extends Diff.Element<K>> {
     this.deleted = deleted;
   }
 
+
   /** @return the created list, which is never null. */
   public List<E> getList(final ListType type) {
-    final List<E> list = type == ListType.CREATED? created: deleted;
-    return list == null? Collections.<E>emptyList(): list;
+    final List<E> list = type == ListType.CREATED ? created : deleted;
+    return list == null ? Collections.<E>emptyList() : list;
+  }
+
+  public List<E> getCreatedUnmodifiable() {
+    return created != null? Collections.unmodifiableList(created)
+        : Collections.<E>emptyList();
+  }
+
+  public E setCreated(int index, E element) {
+    final E old = created.set(index, element);
+    if (old.compareTo(element.getKey()) != 0) {
+      throw new AssertionError("Element mismatched: element=" + element
+          + " but old=" + old);
+    }
+    return old;
+  }
+
+  public boolean removeCreated(final E element) {
+    if (created != null) {
+      final int i = search(created, element.getKey());
+      if (i >= 0 && created.get(i) == element) {
+        created.remove(i);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public void clearCreated() {
+    if (created != null) {
+      created.clear();
+    }
+  }
+
+  public List<E> getDeletedUnmodifiable() {
+    return deleted != null? Collections.unmodifiableList(deleted)
+        : Collections.<E>emptyList();
+  }
+
+  public boolean containsDeleted(final K key) {
+    if (deleted != null) {
+      return search(deleted, key) >= 0;
+    }
+    return false;
   }
 
   public int searchIndex(final ListType type, final K name) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
index 6059fcb..cad009c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
@@ -609,4 +609,44 @@ public class TestFSImageWithSnapshot {
     output.println(b);
     return b;
   }
+
+  @Test (timeout=60000)
+  public void testFSImageWithDoubleRename() throws Exception {
+    final Path dir1 = new Path("/dir1");
+    final Path dir2 = new Path("/dir2");
+    hdfs.mkdirs(dir1);
+    hdfs.mkdirs(dir2);
+    Path dira = new Path(dir1, "dira");
+    Path dirx = new Path(dir1, "dirx");
+    Path dirb = new Path(dira, "dirb");
+    hdfs.mkdirs(dira);
+    hdfs.mkdirs(dirb);
+    hdfs.mkdirs(dirx);
+    hdfs.allowSnapshot(dir1);
+    hdfs.createSnapshot(dir1, "s0");
+    Path file1 = new Path(dirb, "file1");
+    DFSTestUtil.createFile(hdfs, file1, BLOCKSIZE, (short) 1, seed);
+    Path rennamePath = new Path(dirx, "dirb");
+    // mv /dir1/dira/dirb to /dir1/dirx/dirb
+    hdfs.rename(dirb, rennamePath);
+    hdfs.createSnapshot(dir1, "s1");
+    DFSTestUtil.appendFile(hdfs, new Path("/dir1/dirx/dirb/file1"),
+            "more data");
+    Path renamePath1 = new Path(dir2, "dira");
+    hdfs.mkdirs(renamePath1);
+    //mv dirx/dirb to /dir2/dira/dirb
+    hdfs.rename(rennamePath, renamePath1);
+    hdfs.delete(renamePath1, true);
+    hdfs.deleteSnapshot(dir1, "s1");
+    // save namespace and restart cluster
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    cluster.shutdown();
+    cluster = new MiniDFSCluster.Builder(conf).format(false)
+            .numDataNodes(NUM_DATANODES).build();
+    cluster.waitActive();
+    fsn = cluster.getNamesystem();
+    hdfs = cluster.getFileSystem();
+  }
 }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
index 65b129a..0883a7d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
@@ -317,7 +317,46 @@ public class TestRenameWithSnapshots {
     assertTrue(existsInDiffReport(entries, DiffType.RENAME, sub2.getName(),
         sub3.getName()));
   }
-  
+
+  @Test (timeout=60000)
+  public void testRenameDirectoryAndFileInSnapshot() throws Exception {
+    final Path sub2 = new Path(sub1, "sub2");
+    final Path sub3 = new Path(sub1, "sub3");
+    final Path sub2file1 = new Path(sub2, "file1");
+    final Path sub2file2 = new Path(sub2, "file2");
+    final Path sub3file2 = new Path(sub3, "file2");
+    final Path sub3file3 = new Path(sub3, "file3");
+    final String sub1snap1 = "sub1snap1";
+    final String sub1snap2 = "sub1snap2";
+    final String sub1snap3 = "sub1snap3";
+    final String sub1snap4 = "sub1snap4";
+    hdfs.mkdirs(sub1);
+    hdfs.mkdirs(sub2);
+    DFSTestUtil.createFile(hdfs, sub2file1, BLOCKSIZE, REPL, SEED);
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, sub1snap1);
+    hdfs.rename(sub2file1, sub2file2);
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, sub1snap2);
+
+    // First rename the sub-directory.
+    hdfs.rename(sub2, sub3);
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, sub1snap3);
+    hdfs.rename(sub3file2, sub3file3);
+    SnapshotTestHelper.createSnapshot(hdfs, sub1, sub1snap4);
+    hdfs.deleteSnapshot(sub1, sub1snap1);
+    hdfs.deleteSnapshot(sub1, sub1snap2);
+    hdfs.deleteSnapshot(sub1, sub1snap3);
+    // check the internal details
+    INode sub3file3Inode = fsdir.getINode4Write(sub3file3.toString());
+    INodeReference ref = sub3file3Inode
+            .asReference();
+    INodeReference.WithCount withCount = (WithCount) ref
+            .getReferredINode();
+    Assert.assertEquals(withCount.getReferenceCount(), 1);
+    // Ensure name list is empty for the reference sub3file3Inode
+    Assert.assertNull(withCount.getLastWithName());
+    Assert.assertTrue(sub3file3Inode.isInCurrentState());
+  }
+
   /**
    * After the following steps:
    * <pre>


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