Browse Source

HDFS-16975. FileWithSnapshotFeature.isCurrentFileDeleted is not reloaded from FSImage. (#5546)

Tsz-Wo Nicholas Sze 2 years ago
parent
commit
dc78849f27

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

@@ -798,7 +798,7 @@ public class FSImageFormat {
       if (underConstruction) {
         file.toUnderConstruction(clientName, clientMachine);
       }
-      return fileDiffs == null ? file : new INodeFile(file, fileDiffs);
+      return fileDiffs == null ? file : file.loadSnapshotFeature(fileDiffs);
     } else if (numBlocks == -1) {
       //directory
       

+ 16 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -871,7 +871,14 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     long id = getId();
     return (int)(id^(id>>>32));  
   }
-  
+
+  @VisibleForTesting
+  public final StringBuilder dumpParentINodes() {
+    final StringBuilder b = parent == null? new StringBuilder()
+        : parent.dumpParentINodes().append("\n  ");
+    return b.append(toDetailString());
+  }
+
   /**
    * Dump the subtree starting from this inode.
    * @return a text representation of the tree.
@@ -896,10 +903,17 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   @VisibleForTesting
   public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
       int snapshotId) {
+    dumpINode(out, prefix, snapshotId);
+  }
+
+  public void dumpINode(PrintWriter out, StringBuilder prefix,
+      int snapshotId) {
     out.print(prefix);
     out.print(" ");
     final String name = getLocalName();
-    out.print(name.isEmpty()? "/": name);
+    out.print(name != null && name.isEmpty()? "/": name);
+    out.print(", isInCurrentState? ");
+    out.print(isInCurrentState());
     out.print("   (");
     out.print(getObjectString());
     out.print("), ");

+ 16 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -283,12 +283,6 @@ public class INodeFile extends INodeWithAdditionalFields
     setBlocks(that.blocks);
   }
   
-  public INodeFile(INodeFile that, FileDiffList diffs) {
-    this(that);
-    Preconditions.checkArgument(!that.isWithSnapshot());
-    this.addSnapshotFeature(diffs);
-  }
-
   /** @return true unconditionally. */
   @Override
   public final boolean isFile() {
@@ -458,7 +452,16 @@ public class INodeFile extends INodeWithAdditionalFields
     this.addFeature(sf);
     return sf;
   }
-  
+
+  /** Used by FSImage. */
+  public INodeFile loadSnapshotFeature(FileDiffList diffs) {
+    final FileWithSnapshotFeature sf = addSnapshotFeature(diffs);
+    if (!isInCurrentState()) {
+      sf.deleteCurrentFile();
+    }
+    return this;
+  }
+
   /**
    * If feature list contains a {@link FileWithSnapshotFeature}, return it;
    * otherwise, return null.
@@ -1092,7 +1095,12 @@ public class INodeFile extends INodeWithAdditionalFields
   @Override
   public void dumpTreeRecursively(PrintWriter out, StringBuilder prefix,
       final int snapshotId) {
-    super.dumpTreeRecursively(out, prefix, snapshotId);
+    dumpINodeFile(out, prefix, snapshotId);
+  }
+
+  public void dumpINodeFile(PrintWriter out, StringBuilder prefix,
+      final int snapshotId) {
+    dumpINode(out, prefix, snapshotId);
     out.print(", fileSize=" + computeFileSize(snapshotId));
     // only compare the first block
     out.print(", blocks=");

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

@@ -269,7 +269,7 @@ public class FSImageFormatPBSnapshot {
         }
         diffs.addFirst(diff);
       }
-      file.addSnapshotFeature(diffs);
+      file.loadSnapshotFeature(diffs);
       short repl = file.getPreferredBlockReplication();
       for (BlockInfo b : file.getBlocks()) {
         if (b.getReplication() < repl) {

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

@@ -243,6 +243,6 @@ public class FileWithSnapshotFeature implements INode.Feature {
 
   @Override
   public String toString() {
-    return "" + diffs;
+    return "isCurrentFileDeleted? " + isCurrentFileDeleted + ", " + diffs;
   }
 }

+ 3 - 32
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/visitor/NamespacePrintVisitor.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.namenode.visitor;
 
 import org.apache.hadoop.util.Preconditions;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -29,7 +28,6 @@ import org.apache.hadoop.hdfs.server.namenode.INodeSymlink;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature.DirectoryDiff;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 
 import java.io.PrintWriter;
@@ -63,7 +61,7 @@ public final class NamespacePrintVisitor implements NamespaceVisitor {
   }
 
   private final PrintWriter out;
-  private final StringBuffer prefix = new StringBuffer();
+  private final StringBuilder prefix = new StringBuilder();
 
   private NamespacePrintVisitor(PrintWriter out) {
     this.out = out;
@@ -74,39 +72,12 @@ public final class NamespacePrintVisitor implements NamespaceVisitor {
   }
 
   private void printINode(INode iNode, int snapshot) {
-    out.print(prefix);
-    out.print(" ");
-    final String name = iNode.getLocalName();
-    out.print(name != null && name.isEmpty()? "/": name);
-    out.print("   (");
-    out.print(iNode.getObjectString());
-    out.print("), ");
-    out.print(iNode.getParentString());
-    out.print(", " + iNode.getPermissionStatus(snapshot));
+    iNode.dumpINode(out, prefix, snapshot);
   }
 
   @Override
   public void visitFile(INodeFile file, int snapshot) {
-    printINode(file, snapshot);
-
-    out.print(", fileSize=" + file.computeFileSize(snapshot));
-    // print only the first block, if it exists
-    out.print(", blocks=");
-    final BlockInfo[] blocks = file.getBlocks();
-    out.print(blocks.length == 0 ? null: blocks[0]);
-    out.println();
-
-    final FileWithSnapshotFeature snapshotFeature
-        = file.getFileWithSnapshotFeature();
-    if (snapshotFeature != null) {
-      if (prefix.length() >= 2) {
-        prefix.setLength(prefix.length() - 2);
-        prefix.append("  ");
-      }
-      out.print(prefix);
-      out.print(snapshotFeature);
-    }
-    out.println();
+    file.dumpINodeFile(out, prefix, snapshot);
   }
 
   @Override

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

@@ -196,10 +196,8 @@ public class TestFSImageWithSnapshot {
     cluster.waitActive();
     fsn = cluster.getNamesystem();
     hdfs = cluster.getFileSystem();
-    
-    INodeDirectory rootNode = fsn.dir.getINode4Write(root.toString())
-        .asDirectory();
-    assertTrue("The children list of root should be empty", 
+    final INodeDirectory rootNode = fsn.dir.getRoot();
+    assertTrue("The children list of root should be empty",
         rootNode.getChildrenList(Snapshot.CURRENT_STATE_ID).isEmpty());
     // one snapshot on root: s1
     DiffList<DirectoryDiff> diffList = rootNode.getDiffs().asList();