You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by ha...@apache.org on 2010/11/23 08:04:27 UTC

svn commit: r1038001 - in /hadoop/hdfs/trunk: ./ src/java/org/apache/hadoop/hdfs/server/namenode/

Author: hairong
Date: Tue Nov 23 07:04:26 2010
New Revision: 1038001

URL: http://svn.apache.org/viewvc?rev=1038001&view=rev
Log:
HDFS-1481. NameNode should validate fsimage before rolling. Contributed by Hairong Kuang.

Modified:
    hadoop/hdfs/trunk/CHANGES.txt
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=1038001&r1=1038000&r2=1038001&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Tue Nov 23 07:04:26 2010
@@ -230,6 +230,8 @@ Release 0.22.0 - Unreleased
 
     HDFS-1513. Fix a number of warnings. (eli)
 
+    HDFS-1481. NameNode should validate fsimage before rolling. (hairong)
+
   OPTIMIZATIONS
 
     HDFS-1140. Speedup INode.getPathComponents. (Dmytro Molkov via shv)

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java?rev=1038001&r1=1038000&r2=1038001&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java Tue Nov 23 07:04:26 2010
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.na
 
 import java.io.IOException;
 import java.io.File;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.util.Collection;
 
@@ -188,7 +187,7 @@ class Checkpointer extends Daemon {
     File[] files = list.toArray(new File[list.size()]);
     assert files.length > 0 : "No checkpoint targets.";
     String nnHttpAddr = backupNode.nnHttpAddress;
-    TransferFsImage.getFileClient(nnHttpAddr, fileid, files);
+    TransferFsImage.getFileClient(nnHttpAddr, fileid, files, false);
     LOG.info("Downloaded file " + files[0].getName() + " size " +
              files[0].length() + " bytes.");
 
@@ -197,7 +196,7 @@ class Checkpointer extends Daemon {
     list = getFSImage().getFiles(NameNodeFile.EDITS, NameNodeDirType.EDITS);
     files = list.toArray(new File[list.size()]);
     assert files.length > 0 : "No checkpoint targets.";
-    TransferFsImage.getFileClient(nnHttpAddr, fileid, files);
+    TransferFsImage.getFileClient(nnHttpAddr, fileid, files, false);
     LOG.info("Downloaded file " + files[0].getName() + " size " +
         files[0].length() + " bytes.");
   }
@@ -214,7 +213,8 @@ class Checkpointer extends Daemon {
       "&token=" + sig.toString() +
       "&newChecksum=" + getFSImage().imageDigest.toString();
     LOG.info("Posted URL " + backupNode.nnHttpAddress + fileid);
-    TransferFsImage.getFileClient(backupNode.nnHttpAddress, fileid, (File[])null);
+    TransferFsImage.getFileClient(backupNode.nnHttpAddress, 
+        fileid, (File[])null, false);
   }
 
   /**

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java?rev=1038001&r1=1038000&r2=1038001&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java Tue Nov 23 07:04:26 2010
@@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
+import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 
@@ -91,14 +92,20 @@ public class GetImageServlet extends Htt
             // issue a HTTP get request to download the new fsimage 
             nnImage.validateCheckpointUpload(ff.getToken());
             nnImage.newImageDigest = ff.getNewChecksum();
-            reloginIfNecessary().doAs(new PrivilegedExceptionAction<Void>() {
+            MD5Hash downloadImageDigest = reloginIfNecessary().doAs(
+                new PrivilegedExceptionAction<MD5Hash>() {
                 @Override
-                public Void run() throws Exception {
-                  TransferFsImage.getFileClient(ff.getInfoServer(), "getimage=1", 
-                      nnImage.getFsImageNameCheckpoint());
-                  return null;
+                public MD5Hash run() throws Exception {
+                  return TransferFsImage.getFileClient(
+                      ff.getInfoServer(), "getimage=1", 
+                      nnImage.getFsImageNameCheckpoint(), true);
                 }
             });
+            if (!nnImage.newImageDigest.equals(downloadImageDigest)) {
+              throw new IOException("The downloaded image is corrupt," +
+                  " expecting a checksum " + nnImage.newImageDigest +
+                  " but received a checksum " + downloadImageDigest);
+            }
            nnImage.checkpointUploadDone();
           }
           return null;

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java?rev=1038001&r1=1038000&r2=1038001&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java Tue Nov 23 07:04:26 2010
@@ -343,7 +343,7 @@ public class SecondaryNameNode implement
                 NameNodeDirType.IMAGE);
             File[] srcNames = list.toArray(new File[list.size()]);
             assert srcNames.length > 0 : "No checkpoint targets.";
-            TransferFsImage.getFileClient(fsName, fileid, srcNames);
+            TransferFsImage.getFileClient(fsName, fileid, srcNames, false);
             LOG.info("Downloaded file " + srcNames[0].getName() + " size " +
                      srcNames[0].length() + " bytes.");
         
@@ -352,7 +352,7 @@ public class SecondaryNameNode implement
             list = getFSImage().getFiles(NameNodeFile.EDITS, NameNodeDirType.EDITS);
             srcNames = list.toArray(new File[list.size()]);;
             assert srcNames.length > 0 : "No checkpoint targets.";
-            TransferFsImage.getFileClient(fsName, fileid, srcNames);
+            TransferFsImage.getFileClient(fsName, fileid, srcNames, false);
             LOG.info("Downloaded file " + srcNames[0].getName() + " size " +
                 srcNames[0].length() + " bytes.");
         
@@ -374,7 +374,7 @@ public class SecondaryNameNode implement
       "&token=" + sig.toString() +
       "&newChecksum=" + checkpointImage.imageDigest;
     LOG.info("Posted URL " + fsName + fileid);
-    TransferFsImage.getFileClient(fsName, fileid, (File[])null);
+    TransferFsImage.getFileClient(fsName, fileid, (File[])null, false);
   }
 
   /**

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java?rev=1038001&r1=1038000&r2=1038001&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java Tue Nov 23 07:04:26 2010
@@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.na
 
 import java.io.*;
 import java.net.*;
+import java.security.DigestInputStream;
+import java.security.MessageDigest;
 import java.util.Iterator;
 import java.util.Map;
 import java.lang.Math;
@@ -169,8 +171,11 @@ class TransferFsImage implements FSConst
   /**
    * Client-side Method to fetch file from a server
    * Copies the response from the URL to a list of local files.
+   * 
+   * @Return a digest of the received file if getChecksum is true
    */
-  static void getFileClient(String fsName, String id, File[] localPath)
+  static MD5Hash getFileClient(String fsName, String id, File[] localPath,
+      boolean getChecksum)
     throws IOException {
     byte[] buf = new byte[BUFFER_SIZE];
     String proto = UserGroupInformation.isSecurityEnabled() ? "https://" : "http://";
@@ -195,6 +200,11 @@ class TransferFsImage implements FSConst
     }
     long received = 0;
     InputStream stream = connection.getInputStream();
+    MessageDigest digester = null;
+    if (getChecksum) {
+      digester = MD5Hash.getDigester();
+      stream = new DigestInputStream(stream, digester);
+    }
     FileOutputStream[] output = null;
 
     try {
@@ -230,5 +240,6 @@ class TransferFsImage implements FSConst
                               advertisedSize);
       }
     }
+    return digester==null ? null : new MD5Hash(digester.digest());
   }
 }