瀏覽代碼

HDFS-10506. OIV's ReverseXML processor cannot reconstruct some snapshot details. Contributed by Akira Ajisaka.

Wei-Chiu Chuang 8 年之前
父節點
當前提交
05391c1845

+ 60 - 25
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java

@@ -567,6 +567,13 @@ class OfflineImageReconstructor {
   private void processFileXml(Node node, INodeSection.INode.Builder inodeBld)
       throws IOException {
     inodeBld.setType(INodeSection.INode.Type.FILE);
+    INodeSection.INodeFile.Builder bld = createINodeFileBuilder(node);
+    inodeBld.setFile(bld);
+    // Will check remaining keys and serialize in processINodeXml
+  }
+
+  private INodeSection.INodeFile.Builder createINodeFileBuilder(Node node)
+      throws IOException {
     INodeSection.INodeFile.Builder bld = INodeSection.INodeFile.newBuilder();
     Integer ival = node.removeChildInt(SECTION_REPLICATION);
     if (ival != null) {
@@ -595,24 +602,7 @@ class OfflineImageReconstructor {
         if (block == null) {
           break;
         }
-        HdfsProtos.BlockProto.Builder blockBld =
-            HdfsProtos.BlockProto.newBuilder();
-        Long id = block.removeChildLong(SECTION_ID);
-        if (id == null) {
-          throw new IOException("<block> found without <id>");
-        }
-        blockBld.setBlockId(id);
-        Long genstamp = block.removeChildLong(INODE_SECTION_GEMSTAMP);
-        if (genstamp == null) {
-          throw new IOException("<block> found without <genstamp>");
-        }
-        blockBld.setGenStamp(genstamp);
-        Long numBytes = block.removeChildLong(INODE_SECTION_NUM_BYTES);
-        if (numBytes == null) {
-          throw new IOException("<block> found without <numBytes>");
-        }
-        blockBld.setNumBytes(numBytes);
-        bld.addBlocks(blockBld);
+        bld.addBlocks(createBlockBuilder(block));
       }
     }
     Node fileUnderConstruction =
@@ -663,13 +653,42 @@ class OfflineImageReconstructor {
             blockType);
       }
     }
-    inodeBld.setFile(bld);
-    // Will check remaining keys and serialize in processINodeXml
+    return bld;
+  }
+
+  private HdfsProtos.BlockProto.Builder createBlockBuilder(Node block)
+      throws IOException {
+    HdfsProtos.BlockProto.Builder blockBld =
+        HdfsProtos.BlockProto.newBuilder();
+    Long id = block.removeChildLong(SECTION_ID);
+    if (id == null) {
+      throw new IOException("<block> found without <id>");
+    }
+    blockBld.setBlockId(id);
+    Long genstamp = block.removeChildLong(INODE_SECTION_GEMSTAMP);
+    if (genstamp == null) {
+      throw new IOException("<block> found without <genstamp>");
+    }
+    blockBld.setGenStamp(genstamp);
+    Long numBytes = block.removeChildLong(INODE_SECTION_NUM_BYTES);
+    if (numBytes == null) {
+      throw new IOException("<block> found without <numBytes>");
+    }
+    blockBld.setNumBytes(numBytes);
+    return blockBld;
   }
 
   private void processDirectoryXml(Node node,
           INodeSection.INode.Builder inodeBld) throws IOException {
     inodeBld.setType(INodeSection.INode.Type.DIRECTORY);
+    INodeSection.INodeDirectory.Builder bld =
+        createINodeDirectoryBuilder(node);
+    inodeBld.setDirectory(bld);
+    // Will check remaining keys and serialize in processINodeXml
+  }
+
+  private INodeSection.INodeDirectory.Builder
+      createINodeDirectoryBuilder(Node node) throws IOException {
     INodeSection.INodeDirectory.Builder bld =
         INodeSection.INodeDirectory.newBuilder();
     Long lval = node.removeChildLong(INODE_SECTION_MTIME);
@@ -723,8 +742,7 @@ class OfflineImageReconstructor {
       qf.addQuotas(qbld);
     }
     bld.setTypeQuotas(qf);
-    inodeBld.setDirectory(bld);
-    // Will check remaining keys and serialize in processINodeXml
+    return bld;
   }
 
   private void processSymlinkXml(Node node,
@@ -1368,7 +1386,11 @@ class OfflineImageReconstructor {
         if (name != null) {
           bld.setName(ByteString.copyFrom(name, "UTF8"));
         }
-        // TODO: add missing snapshotCopy field to XML
+        Node snapshotCopy = dirDiff.removeChild(
+            SNAPSHOT_DIFF_SECTION_SNAPSHOT_COPY);
+        if (snapshotCopy != null) {
+          bld.setSnapshotCopy(createINodeDirectoryBuilder(snapshotCopy));
+        }
         Integer expectedCreatedListSize = dirDiff.removeChildInt(
             SNAPSHOT_DIFF_SECTION_CREATED_LIST_SIZE);
         if (expectedCreatedListSize == null) {
@@ -1467,8 +1489,21 @@ class OfflineImageReconstructor {
         if (name != null) {
           bld.setName(ByteString.copyFrom(name, "UTF8"));
         }
-        // TODO: missing snapshotCopy
-        // TODO: missing blocks
+        Node snapshotCopy = fileDiff.removeChild(
+            SNAPSHOT_DIFF_SECTION_SNAPSHOT_COPY);
+        if (snapshotCopy != null) {
+          bld.setSnapshotCopy(createINodeFileBuilder(snapshotCopy));
+        }
+        Node blocks = fileDiff.removeChild(INODE_SECTION_BLOCKS);
+        if (blocks != null) {
+          while (true) {
+            Node block = blocks.removeChild(INODE_SECTION_BLOCK);
+            if (block == null) {
+              break;
+            }
+            bld.addBlocks(createBlockBuilder(block));
+          }
+        }
         fileDiff.verifyNoRemainingKeys("fileDiff");
         bld.build().writeDelimitedTo(out);
       }

+ 27 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java

@@ -190,6 +190,8 @@ public final class PBImageXmlWriter {
       "childrenSize";
   public static final String SNAPSHOT_DIFF_SECTION_IS_SNAPSHOT_ROOT =
       "isSnapshotRoot";
+  public static final String SNAPSHOT_DIFF_SECTION_SNAPSHOT_COPY =
+      "snapshotCopy";
   public static final String SNAPSHOT_DIFF_SECTION_CREATED_LIST_SIZE =
       "createdListSize";
   public static final String SNAPSHOT_DIFF_SECTION_DELETED_INODE =
@@ -667,6 +669,23 @@ public final class PBImageXmlWriter {
           o(SNAPSHOT_DIFF_SECTION_SNAPSHOT_ID, f.getSnapshotId())
               .o(SNAPSHOT_DIFF_SECTION_SIZE, f.getFileSize())
               .o(SECTION_NAME, f.getName().toStringUtf8());
+          INodeSection.INodeFile snapshotCopy = f.getSnapshotCopy();
+          if (snapshotCopy != null) {
+            out.print("<" + SNAPSHOT_DIFF_SECTION_SNAPSHOT_COPY + ">");
+            dumpINodeFile(snapshotCopy);
+            out.print("</" + SNAPSHOT_DIFF_SECTION_SNAPSHOT_COPY + ">\n");
+          }
+          if (f.getBlocksCount() > 0) {
+            out.print("<" + INODE_SECTION_BLOCKS + ">");
+            for (BlockProto b : f.getBlocksList()) {
+              out.print("<" + INODE_SECTION_BLOCK + ">");
+              o(SECTION_ID, b.getBlockId())
+                  .o(INODE_SECTION_GEMSTAMP, b.getGenStamp())
+                  .o(INODE_SECTION_NUM_BYTES, b.getNumBytes());
+              out.print("</" + INODE_SECTION_BLOCK + ">\n");
+            }
+            out.print("</" + INODE_SECTION_BLOCKS + ">\n");
+          }
           out.print("</" + SNAPSHOT_DIFF_SECTION_FILE_DIFF + ">\n");
         }
       }
@@ -679,9 +698,14 @@ public final class PBImageXmlWriter {
           o(SNAPSHOT_DIFF_SECTION_SNAPSHOT_ID, d.getSnapshotId())
               .o(SNAPSHOT_DIFF_SECTION_CHILDREN_SIZE, d.getChildrenSize())
               .o(SNAPSHOT_DIFF_SECTION_IS_SNAPSHOT_ROOT, d.getIsSnapshotRoot())
-              .o(SECTION_NAME, d.getName().toStringUtf8())
-              .o(SNAPSHOT_DIFF_SECTION_CREATED_LIST_SIZE,
-                  d.getCreatedListSize());
+              .o(SECTION_NAME, d.getName().toStringUtf8());
+          INodeDirectory snapshotCopy = d.getSnapshotCopy();
+          if (snapshotCopy != null) {
+            out.print("<" + SNAPSHOT_DIFF_SECTION_SNAPSHOT_COPY + ">");
+            dumpINodeDirectory(snapshotCopy);
+            out.print("</" + SNAPSHOT_DIFF_SECTION_SNAPSHOT_COPY + ">\n");
+          }
+          o(SNAPSHOT_DIFF_SECTION_CREATED_LIST_SIZE, d.getCreatedListSize());
           for (long did : d.getDeletedINodeList()) {
             o(SNAPSHOT_DIFF_SECTION_DELETED_INODE, did);
           }

+ 14 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -181,14 +181,27 @@ public class TestOfflineImageViewer {
       hdfs.mkdirs(src);
       dirCount++;
       writtenFiles.put(src.toString(), hdfs.getFileStatus(src));
+
+      // Create snapshot and snapshotDiff.
       final Path orig = new Path("/src/orig");
       hdfs.mkdirs(orig);
+      final Path file1 = new Path("/src/file");
+      FSDataOutputStream o = hdfs.create(file1);
+      o.write(23);
+      o.write(45);
+      o.close();
       hdfs.allowSnapshot(src);
       hdfs.createSnapshot(src, "snapshot");
       final Path dst = new Path("/dst");
+      // Rename a directory in the snapshot directory to add snapshotCopy
+      // field to the dirDiff entry.
       hdfs.rename(orig, dst);
       dirCount++;
       writtenFiles.put(dst.toString(), hdfs.getFileStatus(dst));
+      // Truncate a file in the snapshot directory to add snapshotCopy and
+      // blocks fields to the fileDiff entry.
+      hdfs.truncate(file1, 1);
+      writtenFiles.put(file1.toString(), hdfs.getFileStatus(file1));
 
       // Set XAttrs so the fsimage contains XAttr ops
       final Path xattr = new Path("/xattr");
@@ -279,7 +292,7 @@ public class TestOfflineImageViewer {
     Matcher matcher = p.matcher(outputString);
     assertTrue(matcher.find() && matcher.groupCount() == 1);
     int totalFiles = Integer.parseInt(matcher.group(1));
-    assertEquals(NUM_DIRS * FILES_PER_DIR, totalFiles);
+    assertEquals(NUM_DIRS * FILES_PER_DIR + 1, totalFiles);
 
     p = Pattern.compile("totalDirectories = (\\d+)\n");
     matcher = p.matcher(outputString);