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 dh...@apache.org on 2010/03/31 08:35:19 UTC

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

Author: dhruba
Date: Wed Mar 31 06:35:18 2010
New Revision: 929406

URL: http://svn.apache.org/viewvc?rev=929406&view=rev
Log:
HDFS-1024. SecondaryNameNode verifies size of fsimage and edits file.
(Dmytro Molkov via dhruba)


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

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=929406&r1=929405&r2=929406&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Wed Mar 31 06:35:18 2010
@@ -114,7 +114,10 @@ Trunk (unreleased changes)
     block report. (Dmytro Molkov via jhoman)
 
     HDFS-1032. fsck has an option to list corrupt files.
-    (André Orianivia dhruba)
+    (André Oriai via dhruba)
+
+    HDFS-1024. SecondaryNameNode verifies size of fsimage and edits file.
+    (Dmytro Molkov via dhruba)
 
   OPTIMIZATIONS
 

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=929406&r1=929405&r2=929406&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 Wed Mar 31 06:35:18 2010
@@ -45,10 +45,14 @@ public class GetImageServlet extends Htt
       FSImage nnImage = (FSImage)context.getAttribute("name.system.image");
       TransferFsImage ff = new TransferFsImage(pmap, request, response);
       if (ff.getImage()) {
+        response.setHeader(TransferFsImage.CONTENT_LENGTH,
+          String.valueOf(nnImage.getFsImageName().length()));
         // send fsImage
         TransferFsImage.getFileServer(response.getOutputStream(),
                                       nnImage.getFsImageName()); 
       } else if (ff.getEdit()) {
+        response.setHeader(TransferFsImage.CONTENT_LENGTH,
+          String.valueOf(nnImage.getFsEditName().length()));
         // send edits
         TransferFsImage.getFileServer(response.getOutputStream(),
                                       nnImage.getFsEditName());

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=929406&r1=929405&r2=929406&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 Wed Mar 31 06:35:18 2010
@@ -21,6 +21,7 @@ import java.io.*;
 import java.net.*;
 import java.util.Iterator;
 import java.util.Map;
+import java.lang.Math;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletRequest;
 
@@ -32,6 +33,8 @@ import org.apache.hadoop.hdfs.DFSUtil.Er
  */
 class TransferFsImage implements FSConstants {
   
+  public final static String CONTENT_LENGTH = "Content-Length";
+  
   private boolean isGetImage;
   private boolean isGetEdit;
   private boolean isPutImage;
@@ -118,6 +121,16 @@ class TransferFsImage implements FSConst
         throw new IOException("If this exception is not caught by the " +
             "name-node fs image will be truncated.");
       }
+      
+      if (ErrorSimulator.getErrorSimulation(3)
+          && localfile.getAbsolutePath().contains("fsimage")) {
+          // Test sending image shorter than localfile
+          long len = localfile.length();
+          buf = new byte[(int)Math.min(len/2, BUFFER_SIZE)];
+          // This will read at most half of the image
+          // and the rest of the image will be sent over the wire
+          infile.read(buf);
+      }
       int num = 1;
       while (num > 0) {
         num = infile.read(buf);
@@ -148,6 +161,15 @@ class TransferFsImage implements FSConst
     //
     URL url = new URL(str.toString());
     URLConnection connection = url.openConnection();
+    long advertisedSize;
+    String contentLength = connection.getHeaderField(CONTENT_LENGTH);
+    if (contentLength != null) {
+      advertisedSize = Long.parseLong(contentLength);
+    } else {
+      throw new IOException(CONTENT_LENGTH + " header is not provided " +
+                            "by the namenode when trying to fetch " + str);
+    }
+    long received = 0;
     InputStream stream = connection.getInputStream();
     FileOutputStream[] output = null;
 
@@ -162,6 +184,7 @@ class TransferFsImage implements FSConst
       while (num > 0) {
         num = stream.read(buf);
         if (num > 0 && localPath != null) {
+          received += num;
           for (int i = 0; i < output.length; i++) {
             output[i].write(buf, 0, num);
           }
@@ -176,6 +199,11 @@ class TransferFsImage implements FSConst
           }
         }
       }
+      if (received != advertisedSize) {
+        throw new IOException("File " + str + " received length " + received +
+                              " is not of the advertised size " +
+                              advertisedSize);
+      }
     }
   }
 }

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java?rev=929406&r1=929405&r2=929406&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java Wed Mar 31 06:35:18 2010
@@ -390,6 +390,55 @@ public class TestCheckpoint extends Test
   }
 
   /**
+   * Simulate namenode failing to send the whole file
+   * secondary namenode sometimes assumed it received all of it
+   */
+  @SuppressWarnings("deprecation")
+  void testNameNodeImageSendFail(Configuration conf)
+    throws IOException {
+    System.out.println("Starting testNameNodeImageSendFail");
+    Path file1 = new Path("checkpointww.dat");
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, 
+                                                false, null);
+    cluster.waitActive();
+    FileSystem fileSys = cluster.getFileSystem();
+    try {
+      assertTrue(!fileSys.exists(file1));
+      //
+      // Make the checkpoint fail after rolling the edit log.
+      //
+      SecondaryNameNode secondary = startSecondaryNameNode(conf);
+      ErrorSimulator.setErrorSimulation(3);
+
+      try {
+        secondary.doCheckpoint();  // this should fail
+        fail("Did not get expected exception");
+      } catch (IOException e) {
+        // We only sent part of the image. Have to trigger this exception
+        assertTrue(e.getMessage().contains("is not of the advertised size"));
+      }
+      ErrorSimulator.clearErrorSimulation(3);
+      secondary.shutdown(); // secondary namenode crash!
+
+      // start new instance of secondary and verify that 
+      // a new rollEditLog suceedes inspite of the fact that 
+      // edits.new already exists.
+      //
+      secondary = startSecondaryNameNode(conf);
+      secondary.doCheckpoint();  // this should work correctly
+      secondary.shutdown();
+
+      //
+      // Create a new file
+      //
+      writeFile(fileSys, file1, replication);
+      checkFile(fileSys, file1, replication);
+    } finally {
+      fileSys.close();
+      cluster.shutdown();
+    }
+  }
+  /**
    * Test different startup scenarios.
    * <p><ol>
    * <li> Start of primary name-node in secondary directory must succeed. 
@@ -610,7 +659,7 @@ public class TestCheckpoint extends Test
       // Take a checkpoint
       //
       SecondaryNameNode secondary = startSecondaryNameNode(conf);
-      ErrorSimulator.initializeErrorSimulationEvent(3);
+      ErrorSimulator.initializeErrorSimulationEvent(4);
       secondary.doCheckpoint();
       secondary.shutdown();
     } finally {
@@ -664,6 +713,7 @@ public class TestCheckpoint extends Test
 
     // file2 is left behind.
 
+    testNameNodeImageSendFail(conf);
     testSecondaryNamenodeError1(conf);
     testSecondaryNamenodeError2(conf);
     testSecondaryNamenodeError3(conf);