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/trunk@1333286 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ca5b406ac0
commit
cbc2424290
|
@ -586,6 +586,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)
|
||||
|
|
|
@ -138,37 +138,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.
|
||||
*/
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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]++;
|
||||
|
|
|
@ -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;
|
||||
|
@ -94,10 +94,14 @@ public class TestCheckpoint extends TestCase {
|
|||
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
|
||||
|
|
Loading…
Reference in New Issue