Przeglądaj źródła

HDFS-3330. If GetImageServlet throws an Error or RTE, response should not have HTTP "OK" status. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1333285 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 lat temu
rodzic
commit
54d6cf02a9

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -462,6 +462,9 @@ Release 2.0.0 - UNRELEASED
     HDFS-3336. hdfs launcher script will be better off not special casing 
     namenode command with regards to hadoop.security.logger (rvs via tucu)
 
+    HDFS-3330. If GetImageServlet throws an Error or RTE, response should not
+    have HTTP "OK" status. (todd)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)

+ 0 - 31
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -146,37 +146,6 @@ public class DFSUtil {
     return true;
   }
 
-  /**
-   * Utility class to facilitate junit test error simulation.
-   */
-  @InterfaceAudience.Private
-  public static class ErrorSimulator {
-    private static boolean[] simulation = null; // error simulation events
-    public static void initializeErrorSimulationEvent(int numberOfEvents) {
-      simulation = new boolean[numberOfEvents]; 
-      for (int i = 0; i < numberOfEvents; i++) {
-        simulation[i] = false;
-      }
-    }
-    
-    public static boolean getErrorSimulation(int index) {
-      if(simulation == null)
-        return false;
-      assert(index < simulation.length);
-      return simulation[index];
-    }
-    
-    public static void setErrorSimulation(int index) {
-      assert(index < simulation.length);
-      simulation[index] = true;
-    }
-    
-    public static void clearErrorSimulation(int index) {
-      assert(index < simulation.length);
-      simulation[index] = false;
-    }
-  }
-
   /**
    * Converts a byte array to a string using UTF8 encoding.
    */

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointFaultInjector.java

@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Utility class to faciliate some fault injection tests for the checkpointing
+ * process.
+ */
+class CheckpointFaultInjector {
+  static CheckpointFaultInjector instance = new CheckpointFaultInjector();
+  
+  static CheckpointFaultInjector getInstance() {
+    return instance;
+  }
+  
+  public void beforeGetImageSetsHeaders() throws IOException {}
+  public void afterSecondaryCallsRollEditLog() throws IOException {}
+  public void afterSecondaryUploadsNewImage() throws IOException {}
+  public void aboutToSendFile(File localfile) throws IOException {}
+
+  public boolean shouldSendShortFile(File localfile) {
+    return false;
+  }
+  public boolean shouldCorruptAByte(File localfile) {
+    return false;
+  }
+  
+}

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java

@@ -119,6 +119,7 @@ public class GetImageServlet extends HttpServlet {
             if (imageFile == null) {
               throw new IOException(errorMessage);
             }
+            CheckpointFaultInjector.getInstance().beforeGetImageSetsHeaders();
             setFileNameHeaders(response, imageFile);
             setVerificationHeaders(response, imageFile);
             // send fsImage
@@ -189,8 +190,8 @@ public class GetImageServlet extends HttpServlet {
         }       
       });
       
-    } catch (Exception ie) {
-      String errMsg = "GetImage failed. " + StringUtils.stringifyException(ie);
+    } catch (Throwable t) {
+      String errMsg = "GetImage failed. " + StringUtils.stringifyException(t);
       response.sendError(HttpServletResponse.SC_GONE, errMsg);
       throw new IOException(errMsg);
     } finally {

+ 2 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -47,7 +47,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
-import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -476,10 +475,7 @@ public class SecondaryNameNode implements Runnable {
     }
 
     // error simulation code for junit test
-    if (ErrorSimulator.getErrorSimulation(0)) {
-      throw new IOException("Simulating error0 " +
-                            "after creating edits.new");
-    }
+    CheckpointFaultInjector.getInstance().afterSecondaryCallsRollEditLog();
 
     RemoteEditLogManifest manifest =
       namenode.getEditLogManifest(sig.mostRecentCheckpointTxId + 1);
@@ -497,10 +493,7 @@ public class SecondaryNameNode implements Runnable {
         dstStorage, txid);
 
     // error simulation code for junit test
-    if (ErrorSimulator.getErrorSimulation(1)) {
-      throw new IOException("Simulating error1 " +
-                            "after uploading new image to NameNode");
-    }
+    CheckpointFaultInjector.getInstance().afterSecondaryUploadsNewImage();
 
     LOG.warn("Checkpoint done. New Image Size: " 
              + dstStorage.getFsImageName(txid).length());

+ 7 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
-import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -153,15 +152,12 @@ public class TransferFsImage {
     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.");
-      }
+      CheckpointFaultInjector.getInstance()
+          .aboutToSendFile(localfile);
       
-      if (ErrorSimulator.getErrorSimulation(3)
-          && localfile.getAbsolutePath().contains("fsimage")) {
+
+      if (CheckpointFaultInjector.getInstance().
+            shouldSendShortFile(localfile)) {
           // Test sending image shorter than localfile
           long len = localfile.length();
           buf = new byte[(int)Math.min(len/2, HdfsConstants.IO_FILE_BUFFER_SIZE)];
@@ -175,8 +171,8 @@ public class TransferFsImage {
         if (num <= 0) {
           break;
         }
-
-        if (ErrorSimulator.getErrorSimulation(4)) {
+        if (CheckpointFaultInjector.getInstance()
+              .shouldCorruptAByte(localfile)) {
           // Simulate a corrupted byte on the wire
           LOG.warn("SIMULATING A CORRUPT BYTE IN IMAGE TRANSFER!");
           buf[0]++;

+ 73 - 21
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -42,7 +42,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -63,6 +62,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Level;
+import org.mockito.ArgumentMatcher;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -93,11 +93,15 @@ public class TestCheckpoint extends TestCase {
   static final int fileSize = 8192;
   static final int numDatanodes = 3;
   short replication = 3;
+
+  private CheckpointFaultInjector faultInjector;
     
   @Override
   public void setUp() throws IOException {
     FileUtil.fullyDeleteContents(new File(MiniDFSCluster.getBaseDirectory()));
-    ErrorSimulator.initializeErrorSimulationEvent(5);
+    
+    faultInjector = Mockito.mock(CheckpointFaultInjector.class);
+    CheckpointFaultInjector.instance = faultInjector;
   }
 
   static void writeFile(FileSystem fileSys, Path name, int repl)
@@ -222,14 +226,18 @@ public class TestCheckpoint extends TestCase {
       // Make the checkpoint fail after rolling the edits log.
       //
       SecondaryNameNode secondary = startSecondaryNameNode(conf);
-      ErrorSimulator.setErrorSimulation(0);
+      
+      Mockito.doThrow(new IOException(
+          "Injecting failure after rolling edit logs"))
+          .when(faultInjector).afterSecondaryCallsRollEditLog();
 
       try {
         secondary.doCheckpoint();  // this should fail
         assertTrue(false);
       } catch (IOException e) {
       }
-      ErrorSimulator.clearErrorSimulation(0);
+      
+      Mockito.reset(faultInjector);
       secondary.shutdown();
 
       //
@@ -282,14 +290,17 @@ public class TestCheckpoint extends TestCase {
       // Make the checkpoint fail after uploading the new fsimage.
       //
       SecondaryNameNode secondary = startSecondaryNameNode(conf);
-      ErrorSimulator.setErrorSimulation(1);
+      
+      Mockito.doThrow(new IOException(
+          "Injecting failure after uploading new image"))
+          .when(faultInjector).afterSecondaryUploadsNewImage();
 
       try {
         secondary.doCheckpoint();  // this should fail
         assertTrue(false);
       } catch (IOException e) {
       }
-      ErrorSimulator.clearErrorSimulation(1);
+      Mockito.reset(faultInjector);
       secondary.shutdown();
 
       //
@@ -341,14 +352,17 @@ public class TestCheckpoint extends TestCase {
       // Make the checkpoint fail after rolling the edit log.
       //
       SecondaryNameNode secondary = startSecondaryNameNode(conf);
-      ErrorSimulator.setErrorSimulation(0);
+
+      Mockito.doThrow(new IOException(
+          "Injecting failure after rolling edit logs"))
+          .when(faultInjector).afterSecondaryCallsRollEditLog();
 
       try {
         secondary.doCheckpoint();  // this should fail
         assertTrue(false);
       } catch (IOException e) {
       }
-      ErrorSimulator.clearErrorSimulation(0);
+      Mockito.reset(faultInjector);
       secondary.shutdown(); // secondary namenode crash!
 
       // start new instance of secondary and verify that 
@@ -395,6 +409,28 @@ public class TestCheckpoint extends TestCase {
    * Used to truncate primary fsimage file.
    */
   public void testSecondaryFailsToReturnImage() throws IOException {
+    Mockito.doThrow(new IOException("If this exception is not caught by the " +
+        "name-node, fs image will be truncated."))
+        .when(faultInjector).aboutToSendFile(filePathContaining("secondary"));
+
+    doSecondaryFailsToReturnImage();
+  }
+  
+  /**
+   * Similar to above test, but uses an unchecked Error, and causes it
+   * before even setting the length header. This used to cause image
+   * truncation. Regression test for HDFS-3330.
+   */
+  public void testSecondaryFailsWithErrorBeforeSettingHeaders()
+      throws IOException {
+    Mockito.doThrow(new Error("If this exception is not caught by the " +
+        "name-node, fs image will be truncated."))
+        .when(faultInjector).beforeGetImageSetsHeaders();
+
+    doSecondaryFailsToReturnImage();
+  }
+
+  private void doSecondaryFailsToReturnImage() throws IOException {
     LOG.info("Starting testSecondaryFailsToReturnImage");
     Configuration conf = new HdfsConfiguration();
     Path file1 = new Path("checkpointRI.dat");
@@ -414,7 +450,6 @@ public class TestCheckpoint extends TestCase {
       // Make the checkpoint
       //
       SecondaryNameNode secondary = startSecondaryNameNode(conf);
-      ErrorSimulator.setErrorSimulation(2);
 
       try {
         secondary.doCheckpoint();  // this should fail
@@ -424,7 +459,7 @@ public class TestCheckpoint extends TestCase {
         GenericTestUtils.assertExceptionContains(
             "If this exception is not caught", e);
       }
-      ErrorSimulator.clearErrorSimulation(2);
+      Mockito.reset(faultInjector);
 
       // Verify that image file sizes did not change.
       for (StorageDirectory sd2 :
@@ -442,6 +477,17 @@ public class TestCheckpoint extends TestCase {
     }
   }
 
+  private File filePathContaining(final String substring) {
+    return Mockito.<File>argThat(
+        new ArgumentMatcher<File>() {
+          @Override
+          public boolean matches(Object argument) {
+            String path = ((File)argument).getAbsolutePath();
+            return path.contains(substring);
+          }
+        });
+  }
+
   /**
    * Simulate 2NN failing to send the whole file (error type 3)
    * The length header in the HTTP transfer should prevent
@@ -450,7 +496,10 @@ public class TestCheckpoint extends TestCase {
   public void testNameNodeImageSendFailWrongSize()
       throws IOException {
     LOG.info("Starting testNameNodeImageSendFailWrongSize");
-    doSendFailTest(3, "is not of the advertised size");
+    
+    Mockito.doReturn(true).when(faultInjector)
+      .shouldSendShortFile(filePathContaining("fsimage"));
+    doSendFailTest("is not of the advertised size");
   }
 
   /**
@@ -461,19 +510,21 @@ public class TestCheckpoint extends TestCase {
   public void testNameNodeImageSendFailWrongDigest()
       throws IOException {
     LOG.info("Starting testNameNodeImageSendFailWrongDigest");
-    doSendFailTest(4, "does not match advertised digest");
+
+    Mockito.doReturn(true).when(faultInjector)
+        .shouldCorruptAByte(Mockito.any(File.class));
+    doSendFailTest("does not match advertised digest");
   }
 
   /**
    * Run a test where the 2NN runs into some kind of error when
    * sending the checkpoint back to the NN.
-   * @param errorType the ErrorSimulator type to trigger
    * @param exceptionSubstring an expected substring of the triggered exception
    */
-  private void doSendFailTest(int errorType, String exceptionSubstring)
+  private void doSendFailTest(String exceptionSubstring)
       throws IOException {
     Configuration conf = new HdfsConfiguration();
-    Path file1 = new Path("checkpoint-doSendFailTest-" + errorType + ".dat");
+    Path file1 = new Path("checkpoint-doSendFailTest-" + getName() + ".dat");
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
                                                .numDataNodes(numDatanodes)
                                                .build();
@@ -485,7 +536,6 @@ public class TestCheckpoint extends TestCase {
       // Make the checkpoint fail after rolling the edit log.
       //
       SecondaryNameNode secondary = startSecondaryNameNode(conf);
-      ErrorSimulator.setErrorSimulation(errorType);
 
       try {
         secondary.doCheckpoint();  // this should fail
@@ -494,7 +544,7 @@ public class TestCheckpoint extends TestCase {
         // We only sent part of the image. Have to trigger this exception
         GenericTestUtils.assertExceptionContains(exceptionSubstring, e);
       }
-      ErrorSimulator.clearErrorSimulation(errorType);
+      Mockito.reset(faultInjector);
       secondary.shutdown(); // secondary namenode crash!
 
       // start new instance of secondary and verify that 
@@ -1017,7 +1067,9 @@ public class TestCheckpoint extends TestCase {
   
       secondary = startSecondaryNameNode(conf);
 
-      ErrorSimulator.setErrorSimulation(1);
+      Mockito.doThrow(new IOException(
+          "Injecting failure after rolling edit logs"))
+          .when(faultInjector).afterSecondaryCallsRollEditLog();
       
       // Fail to checkpoint once
       try {
@@ -1025,7 +1077,7 @@ public class TestCheckpoint extends TestCase {
         fail("Should have failed upload");
       } catch (IOException ioe) {
         LOG.info("Got expected failure", ioe);
-        assertTrue(ioe.toString().contains("Simulating error1"));
+        assertTrue(ioe.toString().contains("Injecting failure"));
       }
 
       // Fail to checkpoint again
@@ -1034,9 +1086,9 @@ public class TestCheckpoint extends TestCase {
         fail("Should have failed upload");
       } catch (IOException ioe) {
         LOG.info("Got expected failure", ioe);
-        assertTrue(ioe.toString().contains("Simulating error1"));
+        assertTrue(ioe.toString().contains("Injecting failure"));
       } finally {
-        ErrorSimulator.clearErrorSimulation(1);
+        Mockito.reset(faultInjector);
       }
 
       // Now with the cleared error simulation, it should succeed