hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From s..@apache.org
Subject svn commit: r646548 - in /hadoop/core/trunk: ./ src/java/org/apache/hadoop/dfs/ src/test/org/apache/hadoop/dfs/
Date Wed, 09 Apr 2008 21:28:21 GMT
Author: shv
Date: Wed Apr  9 14:28:18 2008
New Revision: 646548

URL: http://svn.apache.org/viewvc?rev=646548&view=rev
Log:
HADOOP-3069. Primary name-node should not truncate image when transfering it from the secondary.
Contributed by Konstantin Shvachko.

Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java
    hadoop/core/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java
    hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=646548&r1=646547&r2=646548&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Wed Apr  9 14:28:18 2008
@@ -565,6 +565,9 @@
     HADOOP-3195. Fix TestFileSystem to be deterministic.
     (Tsz Wo (Nicholas), SZE via cdouglas)
 
+    HADOOP-3069. Primary name-node should not truncate image when transferring
+    it from the secondary. (shv)
+
 Release 0.16.2 - 2008-04-02
 
   BUG FIXES

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java?rev=646548&r1=646547&r2=646548&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java Wed Apr  9 14:28:18
2008
@@ -61,12 +61,12 @@
                                       nn.getFsImageNameCheckpoint());
         nn.checkpointUploadDone();
       }
-    } catch (IOException ie) {
-      StringUtils.stringifyException(ie);
-      LOG.warn(ie);
-      String errMsg = "GetImage failed.";
+    } catch (Exception ie) {
+      String errMsg = "GetImage failed. " + StringUtils.stringifyException(ie);
       response.sendError(HttpServletResponse.SC_GONE, errMsg);
-      throw ie;
+      throw new IOException(errMsg);
+    } finally {
+      response.getOutputStream().close();
     }
   }
 }

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java?rev=646548&r1=646547&r2=646548&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java Wed Apr  9 14:28:18
2008
@@ -73,7 +73,35 @@
   private File destImage;
   private File editFile;
 
-  private boolean[] simulation = null; // error simulation events
+  /**
+   * Utility class to facilitate junit test error simulation.
+   */
+  static class ErrorSimulator {
+    private static boolean[] simulation = null; // error simulation events
+    static void initializeErrorSimulationEvent(int numberOfEvents) {
+      simulation = new boolean[numberOfEvents]; 
+      for (int i = 0; i < numberOfEvents; i++) {
+        simulation[i] = false;
+      }
+    }
+    
+    static boolean getErrorSimulation(int index) {
+      if(simulation == null)
+        return false;
+      assert(index < simulation.length);
+      return simulation[index];
+    }
+    
+    static void setErrorSimulation(int index) {
+      assert(index < simulation.length);
+      simulation[index] = true;
+    }
+    
+    static void clearErrorSimulation(int index) {
+      assert(index < simulation.length);
+      simulation[index] = false;
+    }
+  }
 
   /**
    * Create a connection to the primary namenode.
@@ -84,11 +112,6 @@
     JvmMetrics.init("SecondaryNameNode", conf.get("session.id"));
     
     //
-    // initialize error simulation code for junit test
-    //
-    initializeErrorSimulationEvent(2);
-
-    //
     // Create connection to the namenode.
     //
     shouldRun = true;
@@ -279,7 +302,7 @@
     //
     // error simulation code for junit test
     //
-    if (simulation != null && simulation[0]) {
+    if (ErrorSimulator.getErrorSimulation(0)) {
       throw new IOException("Simulating error0 " +
                             "after creating edits.new");
     }
@@ -297,7 +320,7 @@
     //
     // error simulation code for junit test
     //
-    if (simulation != null && simulation[1]) {
+    if (ErrorSimulator.getErrorSimulation(1)) {
       throw new IOException("Simulating error1 " +
                             "after uploading new image to NameNode");
     }
@@ -423,26 +446,6 @@
     }
   }
 
-  //
-  // utility method to facilitate junit test error simulation
-  //
-  void initializeErrorSimulationEvent(int numberOfEvents) {
-    simulation = new boolean[numberOfEvents]; 
-    for (int i = 0; i < numberOfEvents; i++) {
-      simulation[i] = false;
-    }
-  }
-
-  void setErrorSimulation(int index) {
-    assert(index < simulation.length);
-    simulation[index] = true;
-  }
-
-  void clearErrorSimulation(int index) {
-    assert(index < simulation.length);
-    simulation[index] = false;
-  }
-
   /**
    * This class is used in Namesystem's jetty to retrieve a file.
    * Typically used by the Secondary NameNode to retrieve image and
@@ -464,13 +467,12 @@
                                         nn.getNewImage());
         }
         LOG.info("New Image " + nn.getNewImage() + " retrieved by Namenode.");
-      } catch (IOException ie) {
-        StringUtils.stringifyException(ie);
-        LOG.error(ie);
-        String errMsg = "GetImage failed.";
+      } catch (Exception ie) {
+        String errMsg = "GetImage failed. " + StringUtils.stringifyException(ie);
         response.sendError(HttpServletResponse.SC_GONE, errMsg);
-        throw ie;
-
+        throw new IOException(errMsg);
+      } finally {
+        response.getOutputStream().close();
       }
     }
   }

Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java?rev=646548&r1=646547&r2=646548&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java Wed Apr  9 14:28:18
2008
@@ -24,6 +24,8 @@
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletRequest;
 
+import org.apache.hadoop.dfs.SecondaryNameNode.ErrorSimulator;
+
 /**
  * This class provides fetching a specified file from the NameNode.
  */
@@ -108,6 +110,12 @@
     FileInputStream infile = null;
     try {
       infile = new FileInputStream(localfile);
+      if (ErrorSimulator.getErrorSimulation(2)
+          && localfile.getAbsolutePath().contains("secondary")) {
+        // throw exception only when the secondary sends its image
+        throw new IOException("If this exception is not caught by the " +
+            "name-node fs image will be truncated.");
+      }
       int num = 1;
       while (num > 0) {
         num = infile.read(buf);
@@ -117,7 +125,6 @@
         outstream.write(buf, 0, num);
       }
     } finally {
-      outstream.close();
       if (infile != null) {
         infile.close();
       }

Modified: hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java?rev=646548&r1=646547&r2=646548&view=diff
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java (original)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java Wed Apr  9 14:28:18
2008
@@ -23,6 +23,7 @@
 import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.FSImage.NameNodeFile;
+import org.apache.hadoop.dfs.SecondaryNameNode.ErrorSimulator;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -132,18 +133,17 @@
     try {
       assertTrue(!fileSys.exists(file1));
       //
-      // Make the checkpoint fail after rolling the
-      // edit log.
+      // Make the checkpoint fail after rolling the edits log.
       //
       SecondaryNameNode secondary = new SecondaryNameNode(conf);
-      secondary.initializeErrorSimulationEvent(2);
-      secondary.setErrorSimulation(0);
+      ErrorSimulator.setErrorSimulation(0);
 
       try {
         secondary.doCheckpoint();  // this should fail
-        assertTrue(false);      
+        assertTrue(false);
       } catch (IOException e) {
       }
+      ErrorSimulator.clearErrorSimulation(0);
       secondary.shutdown();
 
       //
@@ -173,15 +173,7 @@
       assertFalse(image.getEditNewFile(idx).exists());
       File edits = image.getEditFile(idx);
       assertTrue(edits.exists()); // edits should exist and be empty
-      long editsLen = -1;
-      RandomAccessFile eF = null;
-      try {
-        eF = new RandomAccessFile(edits, "r");
-        editsLen = eF.length();
-      } finally {
-        if(eF != null)
-          eF.close();
-      }
+      long editsLen = edits.length();
       assertTrue(editsLen == Integer.SIZE/Byte.SIZE);
     }
     
@@ -215,14 +207,14 @@
       // Make the checkpoint fail after uploading the new fsimage.
       //
       SecondaryNameNode secondary = new SecondaryNameNode(conf);
-      secondary.initializeErrorSimulationEvent(2);
-      secondary.setErrorSimulation(1);
+      ErrorSimulator.setErrorSimulation(1);
 
       try {
         secondary.doCheckpoint();  // this should fail
-        assertTrue(false);      
+        assertTrue(false);
       } catch (IOException e) {
       }
+      ErrorSimulator.clearErrorSimulation(1);
       secondary.shutdown();
 
       //
@@ -273,14 +265,14 @@
       // Make the checkpoint fail after rolling the edit log.
       //
       SecondaryNameNode secondary = new SecondaryNameNode(conf);
-      secondary.initializeErrorSimulationEvent(2);
-      secondary.setErrorSimulation(0);
+      ErrorSimulator.setErrorSimulation(0);
 
       try {
         secondary.doCheckpoint();  // this should fail
-        assertTrue(false);      
+        assertTrue(false);
       } catch (IOException e) {
       }
+      ErrorSimulator.clearErrorSimulation(0);
       secondary.shutdown(); // secondary namenode crash!
 
       // start new instance of secondary and verify that 
@@ -323,6 +315,52 @@
   }
 
   /**
+   * Simulate a secondary node failure to transfer image
+   * back to the name-node.
+   * Used to truncate primary fsimage file.
+   */
+  void testSecondaryFailsToReturnImage(Configuration conf)
+    throws IOException {
+    System.out.println("Starting testSecondaryFailsToReturnImage");
+    Path file1 = new Path("checkpointRI.dat");
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, 
+                                                false, null);
+    cluster.waitActive();
+    FileSystem fileSys = cluster.getFileSystem();
+    FSImage image = cluster.getNameNode().getFSImage();
+    try {
+      assertTrue(!fileSys.exists(file1));
+      long fsimageLength = image.getImageFile(0, NameNodeFile.IMAGE).length();
+      //
+      // Make the checkpoint
+      //
+      SecondaryNameNode secondary = new SecondaryNameNode(conf);
+      ErrorSimulator.setErrorSimulation(2);
+
+      try {
+        secondary.doCheckpoint();  // this should fail
+        assertTrue(false);
+      } catch (IOException e) {
+        System.out.println("testSecondaryFailsToReturnImage: doCheckpoint() " +
+            "failed predictably - " + e);
+      }
+      ErrorSimulator.clearErrorSimulation(2);
+
+      // Verify that image file sizes did not change.
+      int nrDirs = image.getNumStorageDirs();
+      for(int idx = 0; idx < nrDirs; idx++) {
+        assertTrue(image.getImageFile(idx, 
+                                NameNodeFile.IMAGE).length() == fsimageLength);
+      }
+
+      secondary.shutdown();
+    } finally {
+      fileSys.close();
+      cluster.shutdown();
+    }
+  }
+
+  /**
    * Tests checkpoint in DFS.
    */
   public void testCheckpoint() throws IOException {
@@ -355,6 +393,7 @@
       // Take a checkpoint
       //
       SecondaryNameNode secondary = new SecondaryNameNode(conf);
+      ErrorSimulator.initializeErrorSimulationEvent(3);
       secondary.doCheckpoint();
       secondary.shutdown();
     } finally {
@@ -412,5 +451,6 @@
     testSecondaryNamenodeError2(conf);
     testSecondaryNamenodeError3(conf);
     testNamedirError(conf, namedirs);
+    testSecondaryFailsToReturnImage(conf);
   }
 }



Mime
View raw message