HDFS-4300. TransferFsImage.downloadEditsToStorage should use a tmp file for destination. Contributed by Andrew Wang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1481987 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Aaron Myers 2013-05-13 17:47:04 +00:00
parent 6521b5ee42
commit 09593530fb
6 changed files with 228 additions and 17 deletions

View File

@ -992,6 +992,9 @@ Release 2.0.5-beta - UNRELEASED
HDFS-4765. Permission check of symlink deletion incorrectly throws HDFS-4765. Permission check of symlink deletion incorrectly throws
UnresolvedLinkException. (Andrew Wang via atm) UnresolvedLinkException. (Andrew Wang via atm)
HDFS-4300. TransferFsImage.downloadEditsToStorage should use a tmp file for
destination. (Andrew Wang via atm)
Release 2.0.4-alpha - 2013-04-25 Release 2.0.4-alpha - 2013-04-25
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -45,4 +45,5 @@ public boolean shouldCorruptAByte(File localfile) {
} }
public void afterMD5Rename() throws IOException {} public void afterMD5Rename() throws IOException {}
public void beforeEditsRename() throws IOException {}
} }

View File

@ -76,7 +76,8 @@ enum NameNodeFile {
EDITS ("edits"), EDITS ("edits"),
IMAGE_NEW ("fsimage.ckpt"), IMAGE_NEW ("fsimage.ckpt"),
EDITS_NEW ("edits.new"), // from "old" pre-HDFS-1073 format EDITS_NEW ("edits.new"), // from "old" pre-HDFS-1073 format
EDITS_INPROGRESS ("edits_inprogress"); EDITS_INPROGRESS ("edits_inprogress"),
EDITS_TMP ("edits_tmp");
private String fileName = null; private String fileName = null;
private NameNodeFile(String name) { this.fileName = name; } private NameNodeFile(String name) { this.fileName = name; }
@ -681,6 +682,12 @@ static File getFinalizedEditsFile(StorageDirectory sd,
getFinalizedEditsFileName(startTxId, endTxId)); getFinalizedEditsFileName(startTxId, endTxId));
} }
static File getTemporaryEditsFile(StorageDirectory sd,
long startTxId, long endTxId, long timestamp) {
return new File(sd.getCurrentDir(),
getTemporaryEditsFileName(startTxId, endTxId, timestamp));
}
static File getImageFile(StorageDirectory sd, long txid) { static File getImageFile(StorageDirectory sd, long txid) {
return new File(sd.getCurrentDir(), return new File(sd.getCurrentDir(),
getImageFileName(txid)); getImageFileName(txid));
@ -692,6 +699,12 @@ public static String getFinalizedEditsFileName(long startTxId, long endTxId) {
startTxId, endTxId); startTxId, endTxId);
} }
public static String getTemporaryEditsFileName(long startTxId, long endTxId,
long timestamp) {
return String.format("%s_%019d-%019d_%019d", NameNodeFile.EDITS_TMP.getName(),
startTxId, endTxId, timestamp);
}
/** /**
* Return the first readable finalized edits file for the given txid. * Return the first readable finalized edits file for the given txid.
*/ */

View File

@ -17,7 +17,16 @@
*/ */
package org.apache.hadoop.hdfs.server.namenode; package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SECONDARY_NAMENODE_USER_NAME_KEY;
import static org.apache.hadoop.util.ExitUtil.terminate;
import java.io.File; import java.io.File;
import java.io.FilenameFilter;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.URI; import java.net.URI;
@ -42,23 +51,20 @@
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException; import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory; import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.Storage.StorageState; import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
import static org.apache.hadoop.util.ExitUtil.terminate;
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile; import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger; import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
@ -72,7 +78,6 @@
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
@ -242,6 +247,7 @@ private void initialize(final Configuration conf,
"/tmp/hadoop/dfs/namesecondary"); "/tmp/hadoop/dfs/namesecondary");
checkpointImage = new CheckpointStorage(conf, checkpointDirs, checkpointEditsDirs); checkpointImage = new CheckpointStorage(conf, checkpointDirs, checkpointEditsDirs);
checkpointImage.recoverCreate(commandLineOpts.shouldFormat()); checkpointImage.recoverCreate(commandLineOpts.shouldFormat());
checkpointImage.deleteTempEdits();
namesystem = new FSNamesystem(conf, checkpointImage); namesystem = new FSNamesystem(conf, checkpointImage);
@ -947,6 +953,31 @@ void ensureCurrentDirExists() throws IOException {
} }
} }
} }
void deleteTempEdits() throws IOException {
FilenameFilter filter = new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
return name.matches(NameNodeFile.EDITS_TMP.getName()
+ "_(\\d+)-(\\d+)_(\\d+)");
}
};
Iterator<StorageDirectory> it = storage.dirIterator(NameNodeDirType.EDITS);
for (;it.hasNext();) {
StorageDirectory dir = it.next();
File[] tempEdits = dir.getCurrentDir().listFiles(filter);
if (tempEdits != null) {
for (File t : tempEdits) {
boolean success = t.delete();
if (!success) {
LOG.warn("Failed to delete temporary edits file: "
+ t.getAbsolutePath());
}
}
}
}
}
} }
static void doMerge( static void doMerge(

View File

@ -42,6 +42,7 @@
import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.StorageErrorReporter; import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.util.DataTransferThrottler; import org.apache.hadoop.hdfs.util.DataTransferThrottler;
@ -96,25 +97,48 @@ static void downloadEditsToStorage(String fsName, RemoteEditLog log,
"bad log: " + log; "bad log: " + log;
String fileid = GetImageServlet.getParamStringForLog( String fileid = GetImageServlet.getParamStringForLog(
log, dstStorage); log, dstStorage);
String fileName = NNStorage.getFinalizedEditsFileName( String finalFileName = NNStorage.getFinalizedEditsFileName(
log.getStartTxId(), log.getEndTxId()); log.getStartTxId(), log.getEndTxId());
List<File> dstFiles = dstStorage.getFiles(NameNodeDirType.EDITS, fileName); List<File> finalFiles = dstStorage.getFiles(NameNodeDirType.EDITS,
assert !dstFiles.isEmpty() : "No checkpoint targets."; finalFileName);
assert !finalFiles.isEmpty() : "No checkpoint targets.";
for (File f : dstFiles) { for (File f : finalFiles) {
if (f.exists() && FileUtil.canRead(f)) { if (f.exists() && FileUtil.canRead(f)) {
LOG.info("Skipping download of remote edit log " + LOG.info("Skipping download of remote edit log " +
log + " since it already is stored locally at " + f); log + " since it already is stored locally at " + f);
return; return;
} else { } else if (LOG.isDebugEnabled()) {
LOG.debug("Dest file: " + f); LOG.debug("Dest file: " + f);
} }
} }
getFileClient(fsName, fileid, dstFiles, dstStorage, false); final long milliTime = System.currentTimeMillis();
LOG.info("Downloaded file " + dstFiles.get(0).getName() + " size " + String tmpFileName = NNStorage.getTemporaryEditsFileName(
dstFiles.get(0).length() + " bytes."); log.getStartTxId(), log.getEndTxId(), milliTime);
List<File> tmpFiles = dstStorage.getFiles(NameNodeDirType.EDITS,
tmpFileName);
getFileClient(fsName, fileid, tmpFiles, dstStorage, false);
LOG.info("Downloaded file " + tmpFiles.get(0).getName() + " size " +
finalFiles.get(0).length() + " bytes.");
CheckpointFaultInjector.getInstance().beforeEditsRename();
for (StorageDirectory sd : dstStorage.dirIterable(NameNodeDirType.EDITS)) {
File tmpFile = NNStorage.getTemporaryEditsFile(sd,
log.getStartTxId(), log.getEndTxId(), milliTime);
File finalizedFile = NNStorage.getFinalizedEditsFile(sd,
log.getStartTxId(), log.getEndTxId());
if (LOG.isDebugEnabled()) {
LOG.debug("Renaming " + tmpFile + " to " + finalizedFile);
}
boolean success = tmpFile.renameTo(finalizedFile);
if (!success) {
LOG.warn("Unable to rename edits file from " + tmpFile
+ " to " + finalizedFile);
}
}
} }
/** /**

View File

@ -28,7 +28,9 @@
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import java.io.File; import java.io.File;
import java.io.FilenameFilter;
import java.io.IOException; import java.io.IOException;
import java.io.RandomAccessFile;
import java.lang.management.ManagementFactory; import java.lang.management.ManagementFactory;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.URI; import java.net.URI;
@ -37,6 +39,7 @@
import java.util.List; import java.util.List;
import org.apache.commons.cli.ParseException; import org.apache.commons.cli.ParseException;
import org.apache.commons.io.filefilter.FileFilterUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger; import org.apache.commons.logging.impl.Log4JLogger;
@ -61,6 +64,7 @@
import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile; import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode.CheckpointStorage; import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode.CheckpointStorage;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@ -109,6 +113,13 @@ public class TestCheckpoint {
static final int numDatanodes = 3; static final int numDatanodes = 3;
short replication = 3; short replication = 3;
static FilenameFilter tmpEditsFilter = new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
return name.startsWith(NameNodeFile.EDITS_TMP.getName());
}
};
private CheckpointFaultInjector faultInjector; private CheckpointFaultInjector faultInjector;
@Before @Before
@ -266,7 +277,7 @@ public void testReloadOnEditReplayFailure () throws IOException {
/* /*
* Simulate 2NN exit due to too many merge failures. * Simulate 2NN exit due to too many merge failures.
*/ */
@Test(timeout=10000) @Test(timeout=30000)
public void testTooManyEditReplayFailures() throws IOException { public void testTooManyEditReplayFailures() throws IOException {
Configuration conf = new HdfsConfiguration(); Configuration conf = new HdfsConfiguration();
conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY, "1"); conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY, "1");
@ -1488,6 +1499,134 @@ public void testFailureBeforeRename () throws IOException {
} }
} }
/**
* Test that a fault while downloading edits does not prevent future
* checkpointing
*/
@Test(timeout = 30000)
public void testEditFailureBeforeRename() throws IOException {
Configuration conf = new HdfsConfiguration();
SecondaryNameNode secondary = null;
MiniDFSCluster cluster = null;
FileSystem fs = null;
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
.build();
cluster.waitActive();
fs = cluster.getFileSystem();
secondary = startSecondaryNameNode(conf);
DFSTestUtil.createFile(fs, new Path("tmpfile0"), 1024, (short) 1, 0l);
secondary.doCheckpoint();
// Cause edit rename to fail during next checkpoint
Mockito.doThrow(new IOException("Injecting failure before edit rename"))
.when(faultInjector).beforeEditsRename();
DFSTestUtil.createFile(fs, new Path("tmpfile1"), 1024, (short) 1, 0l);
try {
secondary.doCheckpoint();
fail("Fault injection failed.");
} catch (IOException ioe) {
GenericTestUtils.assertExceptionContains(
"Injecting failure before edit rename", ioe);
}
Mockito.reset(faultInjector);
// truncate the tmp edits file to simulate a partial download
for (StorageDirectory sd : secondary.getFSImage().getStorage()
.dirIterable(NameNodeDirType.EDITS)) {
File[] tmpEdits = sd.getCurrentDir().listFiles(tmpEditsFilter);
assertTrue(
"Expected a single tmp edits file in directory " + sd.toString(),
tmpEdits.length == 1);
RandomAccessFile randFile = new RandomAccessFile(tmpEdits[0], "rw");
randFile.setLength(0);
randFile.close();
}
// Next checkpoint should succeed
secondary.doCheckpoint();
} finally {
if (secondary != null) {
secondary.shutdown();
}
if (fs != null) {
fs.close();
}
if (cluster != null) {
cluster.shutdown();
}
Mockito.reset(faultInjector);
}
}
/**
* Test that the secondary namenode correctly deletes temporary edits
* on startup.
*/
@Test(timeout = 30000)
public void testDeleteTemporaryEditsOnStartup() throws IOException {
Configuration conf = new HdfsConfiguration();
SecondaryNameNode secondary = null;
MiniDFSCluster cluster = null;
FileSystem fs = null;
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
.build();
cluster.waitActive();
fs = cluster.getFileSystem();
secondary = startSecondaryNameNode(conf);
DFSTestUtil.createFile(fs, new Path("tmpfile0"), 1024, (short) 1, 0l);
secondary.doCheckpoint();
// Cause edit rename to fail during next checkpoint
Mockito.doThrow(new IOException("Injecting failure before edit rename"))
.when(faultInjector).beforeEditsRename();
DFSTestUtil.createFile(fs, new Path("tmpfile1"), 1024, (short) 1, 0l);
try {
secondary.doCheckpoint();
fail("Fault injection failed.");
} catch (IOException ioe) {
GenericTestUtils.assertExceptionContains(
"Injecting failure before edit rename", ioe);
}
Mockito.reset(faultInjector);
// Verify that a temp edits file is present
for (StorageDirectory sd : secondary.getFSImage().getStorage()
.dirIterable(NameNodeDirType.EDITS)) {
File[] tmpEdits = sd.getCurrentDir().listFiles(tmpEditsFilter);
assertTrue(
"Expected a single tmp edits file in directory " + sd.toString(),
tmpEdits.length == 1);
}
// Restart 2NN
secondary.shutdown();
secondary = startSecondaryNameNode(conf);
// Verify that tmp files were deleted
for (StorageDirectory sd : secondary.getFSImage().getStorage()
.dirIterable(NameNodeDirType.EDITS)) {
File[] tmpEdits = sd.getCurrentDir().listFiles(tmpEditsFilter);
assertTrue(
"Did not expect a tmp edits file in directory " + sd.toString(),
tmpEdits.length == 0);
}
// Next checkpoint should succeed
secondary.doCheckpoint();
} finally {
if (secondary != null) {
secondary.shutdown();
}
if (fs != null) {
fs.close();
}
if (cluster != null) {
cluster.shutdown();
}
Mockito.reset(faultInjector);
}
}
/** /**
* Test case where two secondary namenodes are checkpointing the same * Test case where two secondary namenodes are checkpointing the same
* NameNode. This differs from {@link #testMultipleSecondaryNamenodes()} * NameNode. This differs from {@link #testMultipleSecondaryNamenodes()}