git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1152295 13f79535-47bb-0310-9956-ffa450edef68
@@ -11,6 +11,10 @@ Trunk (unreleased changes)
HDFS-2210. Remove hdfsproxy. (eli)
+ HDFS-1073. Redesign the NameNode's storage layout for image checkpoints
+ and edit logs to introduce transaction IDs and be more robust.
+ Please see HDFS-1073 section below for breakout of individual patches.
+
NEW FEATURES
HDFS-1359. Add BlockPoolID to Block. (suresh)
@@ -904,6 +908,57 @@ Trunk (unreleased changes)
HDFS-1776. Bug in Concat code. (Bharath Mundlapudi via Dmytro Molkov)
+ BREAKDOWN OF HDFS-1073 SUBTASKS
+ HDFS-1521. Persist transaction ID on disk between NN restarts.
+ (Ivan Kelly and Todd Lipcon via todd)
+ HDFS-1538. Refactor more startup and image loading code out of FSImage.
+ (todd)
+ HDFS-1729. Add code to detect valid length of an edits file. (todd)
+ HDFS-1793. Add code to inspect a storage directory with txid-based
+ filenames (todd)
+ HDFS-1794. Add code to list which edit logs are available on a remote NN
+ HDFS-1858. Add state management variables to FSEditLog (Ivan Kelly and Todd
+ Lipcon via todd)
+ HDFS-1859. Add some convenience functions to iterate over edit log streams
+ HDFS-1894. Add constants for LAYOUT_VERSIONs in edits log branch (todd)
+ HDFS-1892. Fix EditLogFileInputStream.getValidLength to be aware of
+ OP_INVALID filler (todd)
+ HDFS-1799. Refactor log rolling and filename management out of FSEditLog
+ HDFS-1801. Remove use of timestamps to identify checkpoints and logs (todd)
+ HDFS-1930. TestDFSUpgrade failing in HDFS-1073 branch (todd)
+ HDFS-1800. Extend image checksumming to function with multiple fsimage
+ files per directory. (todd)
+ HDFS-1725. Set storage directories only at FSImage construction (Ivan Kelly
+ via todd)
+ HDFS-1926. Remove references to StorageDirectory from JournalManager
+ interface (Ivan Kelly via todd)
+ HDFS-1893. Change edit logs and images to be named based on txid (todd)
+ HDFS-1985. Clean up image transfer servlet (todd)
+ HDFS-1984. Enable multiple secondary namenodes to run simultaneously (todd)
+ HDFS-1987. Re-enable TestCheckpoint.testSecondaryImageDownload which was
+ not running previously. (todd)
+ HDFS-1993. TestCheckpoint needs to clean up between cases (todd)
+ HDFS-1992. Remove vestiges of NNStorageListener. (todd)
+ HDFS-1991. Some refactoring of Secondary NameNode to be able to share more
+ code with the BackupNode or CheckpointNode. (todd)
+ HDFS-1994. Fix race conditions when running two rapidly checkpointing
+ Secondary NameNodes. (todd)
+ HDFS-2001. Remove use of previous.checkpoint and lastcheckpoint.tmp
+ directories (todd)
+ HDFS-2015. Remove checkpointTxId from VERSION file. (todd)
+ HDFS-2016. Add infrastructure to remove or archive old and unneeded storage
+ files within the name directories. (todd)
+ HDFS-2047. Improve TestNamespace and TestEditLog in HDFS-1073 branch.
+ HDFS-2048. Add upgrade tests and fix upgrade from 0.22 with corrupt image.
+ HDFS-2027. Image inspector should return finalized logs before unfinalized
+ logs. (todd)
Release 0.22.0 - Unreleased
INCOMPATIBLE CHANGES
@@ -106,9 +106,7 @@ elif [ "$COMMAND" = "getconf" ] ; then
elif [ "$COMMAND" = "groups" ] ; then
CLASS=org.apache.hadoop.hdfs.tools.GetGroups
else
- echo $COMMAND - invalid command
- print_usage
- exit
+ CLASS="$COMMAND"
fi
# for developers, add hdfs classes to CLASSPATH
@@ -67,6 +67,7 @@
<dependency org="commons-logging" name="commons-logging" rev="${commons-logging.version}" conf="compile->master"/>
<dependency org="commons-daemon" name="commons-daemon" rev="${commons-daemon.version}" conf="hdfs->default" />
<dependency org="log4j" name="log4j" rev="${log4j.version}" conf="common->master"/>
+ <dependency org="com.google.guava" name="guava" rev="${guava.version}" conf="hdfs->default" />
<dependency org="com.google.protobuf" name="protobuf-java" rev="2.4.0a" conf="common->master"/>
<dependency org="org.apache.hadoop" name="avro" rev="${avro.version}" conf="compile->master">
<exclude module="ant"/>
@@ -34,6 +34,8 @@ commons-net.version=1.4.1
core.version=3.1.1
coreplugin.version=1.3.2
+guava.version=r09
hadoop-common.version=0.23.0-SNAPSHOT
hadoop-hdfs.version=0.23.0-SNAPSHOT
@@ -271,9 +271,9 @@
the maximum delay between two consecutive checkpoints, and
</li>
<li>
- <code>dfs.namenode.checkpoint.size</code>, set to 64MB by default, defines the
- size of the edits log file that forces an urgent checkpoint even if
- the maximum checkpoint delay is not reached.
+ <code>dfs.namenode.checkpoint.txns</code>, set to 40000 default, defines the
+ number of uncheckpointed transactions on the NameNode which will force
+ an urgent checkpoint, even if the checkpoint period has not been reached.
</ul>
<p>
@@ -322,9 +322,9 @@
the maximum delay between two consecutive checkpoints
@@ -582,10 +582,30 @@ creations/deletions), or "all".</description>
</property>
<property>
- <name>dfs.namenode.checkpoint.size</name>
- <value>67108864</value>
- <description>The size of the current edit log (in bytes) that triggers
- a periodic checkpoint even if the dfs.namenode.checkpoint.period hasn't expired.
+ <name>dfs.namenode.checkpoint.txns</name>
+ <value>40000</value>
+ <description>The Secondary NameNode or CheckpointNode will create a checkpoint
+ of the namespace every 'dfs.namenode.checkpoint.txns' transactions, regardless
+ of whether 'dfs.namenode.checkpoint.period' has expired.
+ </description>
+</property>
+<property>
+ <name>dfs.namenode.checkpoint.check.period</name>
+ <value>60</value>
+ <description>The SecondaryNameNode and CheckpointNode will poll the NameNode
+ every 'dfs.namenode.checkpoint.check.period' seconds to query the number
+ of uncheckpointed transactions.
+ <name>dfs.namenode.num.checkpoints.retained</name>
+ <value>2</value>
+ <description>The number of image checkpoint files that will be retained by
+ the NameNode and Secondary NameNode in their storage directories. All edit
+ logs necessary to recover an up-to-date namespace from the oldest retained
+ checkpoint will also be retained.
</description>
@@ -73,10 +73,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final int DFS_NAMENODE_SAFEMODE_MIN_DATANODES_DEFAULT = 0;
public static final String DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY = "dfs.namenode.secondary.http-address";
public static final String DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50090";
+ public static final String DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY = "dfs.namenode.checkpoint.check.period";
+ public static final long DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT = 60;
public static final String DFS_NAMENODE_CHECKPOINT_PERIOD_KEY = "dfs.namenode.checkpoint.period";
public static final long DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT = 3600;
- public static final String DFS_NAMENODE_CHECKPOINT_SIZE_KEY = "dfs.namenode.checkpoint.size";
- public static final long DFS_NAMENODE_CHECKPOINT_SIZE_DEFAULT = 4194304;
+ public static final String DFS_NAMENODE_CHECKPOINT_TXNS_KEY = "dfs.namenode.checkpoint.txns";
+ public static final long DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT = 40000;
public static final String DFS_NAMENODE_UPGRADE_PERMISSION_KEY = "dfs.namenode.upgrade.permission";
public static final int DFS_NAMENODE_UPGRADE_PERMISSION_DEFAULT = 00777;
public static final String DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY = "dfs.namenode.heartbeat.recheck-interval";
@@ -110,6 +112,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final boolean DFS_NAMENODE_NAME_DIR_RESTORE_DEFAULT = false;
public static final String DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY = "dfs.namenode.support.allow.format";
public static final boolean DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_DEFAULT = true;
+ public static final String DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY = "dfs.namenode.num.checkpoints.retained";
+ public static final int DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_DEFAULT = 2;
public static final String DFS_LIST_LIMIT = "dfs.ls.limit";
public static final int DFS_LIST_LIMIT_DEFAULT = 1000;
public static final String DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY = "dfs.datanode.failed.volumes.tolerated";
@@ -85,7 +85,6 @@ public class HdfsConfiguration extends Configuration {
deprecate("fs.checkpoint.dir", DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY);
deprecate("fs.checkpoint.edits.dir", DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_EDITS_DIR_KEY);
deprecate("fs.checkpoint.period", DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY);
- deprecate("fs.checkpoint.size", DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_SIZE_KEY);
deprecate("dfs.upgrade.permission", DFSConfigKeys.DFS_NAMENODE_UPGRADE_PERMISSION_KEY);
deprecate("heartbeat.recheck.interval", DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY);
deprecate("StorageId", DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY);
@@ -65,6 +65,9 @@ public interface FSConstants {
// type of the datanode report
public static enum DatanodeReportType {ALL, LIVE, DEAD }
+ // An invalid transaction ID that will never be seen in a real namesystem.
+ public static final long INVALID_TXID = -12345;
/**
* Distributed upgrade actions:
@@ -78,7 +78,9 @@ public class LayoutVersion {
RESERVED_REL22(-33, -27, "Reserved for release 0.22"),
RESERVED_REL23(-34, -30, "Reserved for release 0.23"),
FEDERATION(-35, "Support for namenode federation"),
- LEASE_REASSIGNMENT(-36, "Support for persisting lease holder reassignment");
+ LEASE_REASSIGNMENT(-36, "Support for persisting lease holder reassignment"),
+ STORED_TXIDS(-37, "Transaction IDs are stored in edits log and image files"),
+ TXID_BASED_LAYOUT(-38, "File names in NN Storage are based on transaction IDs");
final int lv;
final int ancestorLV;
@@ -181,6 +181,16 @@ public abstract class Storage extends StorageInfo {
return new DirIterator(dirType);
}
+ public Iterable<StorageDirectory> dirIterable(final StorageDirType dirType) {
+ return new Iterable<StorageDirectory>() {
+ @Override
+ public Iterator<StorageDirectory> iterator() {
+ return dirIterator(dirType);
+ }
+ };
* generate storage list (debug line)
*/
@@ -568,13 +578,17 @@ public abstract class Storage extends StorageInfo {
LOG.info("Locking is disabled");
return;
- this.lock = tryLock();
- if (lock == null) {
+ FileLock newLock = tryLock();
+ if (newLock == null) {
String msg = "Cannot lock storage " + this.root
+ ". The directory is already locked.";
LOG.info(msg);
throw new IOException(msg);
+ // Don't overwrite lock until success - this way if we accidentally
+ // call lock twice, the internal state won't be cleared by the second
+ // (failed) lock attempt
+ lock = newLock;
@@ -614,6 +628,45 @@ public abstract class Storage extends StorageInfo {
lock.channel().close();
lock = null;
+ public String toString() {
+ return "Storage Directory " + this.root;
+ /**
+ * Check whether underlying file system supports file locking.
+ *
+ * @return <code>true</code> if exclusive locks are supported or
+ * <code>false</code> otherwise.
+ * @throws IOException
+ * @see StorageDirectory#lock()
+ */
+ public boolean isLockSupported() throws IOException {
+ FileLock firstLock = null;
+ FileLock secondLock = null;
+ try {
+ firstLock = lock;
+ if(firstLock == null) {
+ firstLock = tryLock();
+ if(firstLock == null)
+ return true;
+ secondLock = tryLock();
+ if(secondLock == null)
+ } finally {
+ if(firstLock != null && firstLock != lock) {
+ firstLock.release();
+ firstLock.channel().close();
+ if(secondLock != null) {
+ secondLock.release();
+ secondLock.channel().close();
+ return false;
@@ -829,41 +882,6 @@ public abstract class Storage extends StorageInfo {
- /**
- * Check whether underlying file system supports file locking.
- *
- * @return <code>true</code> if exclusive locks are supported or
- * <code>false</code> otherwise.
- * @throws IOException
- * @see StorageDirectory#lock()
- */
- public boolean isLockSupported(int idx) throws IOException {
- StorageDirectory sd = storageDirs.get(idx);
- FileLock firstLock = null;
- FileLock secondLock = null;
- try {
- firstLock = sd.lock;
- if(firstLock == null) {
- firstLock = sd.tryLock();
- if(firstLock == null)
- return true;
- }
- secondLock = sd.tryLock();
- if(secondLock == null)
- } finally {
- if(firstLock != null && firstLock != sd.lock) {
- firstLock.release();
- firstLock.channel().close();
- if(secondLock != null) {
- secondLock.release();
- secondLock.channel().close();
- return false;
-
public static String getBuildVersion() {
return VersionInfo.getRevision();
@@ -25,6 +25,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
+import com.google.common.base.Joiner;
* Common class for storage information.
*
@@ -105,4 +107,9 @@ public class StorageInfo implements Writable {
.append(";nsid=").append(namespaceID).append(";c=").append(cTime);
return sb.toString();
+ public String toColonSeparatedString() {
+ return Joiner.on(":").join(
+ layoutVersion, namespaceID, cTime, clusterID);
@@ -19,29 +19,21 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.BufferedInputStream;
import java.io.DataInputStream;
-import java.io.File;
import java.io.IOException;
-import java.net.URI;
-import java.util.Collection;
import java.util.Iterator;
-import java.util.zip.CheckedInputStream;
import java.util.zip.Checksum;
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
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.InconsistentFSStateException;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
-import org.apache.hadoop.hdfs.server.namenode.FSImage;
-import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
-import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
-import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
-import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.LogLoadPlan;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.StringUtils;
+import com.google.common.base.Preconditions;
* Extension of FSImage for the backup node.
@@ -50,29 +42,56 @@ import org.apache.hadoop.io.LongWritable;
@InterfaceAudience.Private
public class BackupImage extends FSImage {
- // Names of the journal spool directory and the spool file
- private static final String STORAGE_JSPOOL_DIR = "jspool";
- private static final String STORAGE_JSPOOL_FILE =
- NNStorage.NameNodeFile.EDITS_NEW.getName();
/** Backup input stream for loading edits into memory */
- private EditLogBackupInputStream backupInputStream;
- /** Is journal spooling in progress */
- volatile JSpoolState jsState;
- static enum JSpoolState {
- OFF,
- INPROGRESS,
- WAIT;
+ private EditLogBackupInputStream backupInputStream =
+ new EditLogBackupInputStream("Data from remote NameNode");
+ * Current state of the BackupNode. The BackupNode's state
+ * transitions are as follows:
+ * Initial: DROP_UNTIL_NEXT_ROLL
+ * - Transitions to JOURNAL_ONLY the next time the log rolls
+ * - Transitions to IN_SYNC in convergeJournalSpool
+ * - Transitions back to JOURNAL_ONLY if the log rolls while
+ * stopApplyingOnNextRoll is true.
+ volatile BNState bnState;
+ static enum BNState {
+ * Edits from the NN should be dropped. On the next log roll,
+ * transition to JOURNAL_ONLY state
+ DROP_UNTIL_NEXT_ROLL,
+ * Edits from the NN should be written to the local edits log
+ * but not applied to the namespace.
+ JOURNAL_ONLY,
+ * Edits should be written to the local edits log and applied
+ * to the local namespace.
+ IN_SYNC;
+ * Flag to indicate that the next time the NN rolls, the BN
+ * should transition from to JOURNAL_ONLY state.
+ * {@see #freezeNamespaceAtNextRoll()}
+ private boolean stopApplyingEditsOnNextRoll = false;
+ * Construct a backup image.
+ * @param conf Configuration
+ * @throws IOException if storage cannot be initialised.
- BackupImage() {
- super();
+ BackupImage(Configuration conf) throws IOException {
+ super(conf);
storage.setDisablePreUpgradableLayoutCheck(true);
- jsState = JSpoolState.OFF;
+ bnState = BNState.DROP_UNTIL_NEXT_ROLL;
+ editLog.initJournals();
@@ -81,14 +100,9 @@ public class BackupImage extends FSImage {
* Read VERSION and fstime files if exist.<br>
* Do not load image or edits.
- * @param imageDirs list of image directories as URI.
- * @param editsDirs list of edits directories URI.
* @throws IOException if the node should shutdown.
- void recoverCreateRead(Collection<URI> imageDirs,
- Collection<URI> editsDirs) throws IOException {
- storage.setStorageDirectories(imageDirs, editsDirs);
- storage.setCheckpointTime(0L);
+ void recoverCreateRead() throws IOException {
for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next();
StorageState curState;
@@ -123,282 +137,260 @@ public class BackupImage extends FSImage {
- * Reset storage directories.
- * <p>
- * Unlock the storage.
- * Rename <code>current</code> to <code>lastcheckpoint.tmp</code>
- * and recreate empty <code>current</code>.
+ * Save meta-data into fsimage files.
+ * and create empty edits.
- synchronized void reset() throws IOException {
- // reset NameSpace tree
- FSDirectory fsDir = getFSNamesystem().dir;
- fsDir.reset();
- // unlock, close and rename storage directories
- storage.unlockAll();
- // recover from unsuccessful checkpoint if necessary
- recoverCreateRead(storage.getImageDirectories(),
- storage.getEditsDirectories());
- // rename and recreate
- for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
- StorageDirectory sd = it.next();
- // rename current to lastcheckpoint.tmp
- storage.moveCurrent(sd);
+ void saveCheckpoint() throws IOException {
+ saveNamespace();
- * Load checkpoint from local files only if the memory state is empty.<br>
- * Set new checkpoint time received from the name-node.<br>
- * Move <code>lastcheckpoint.tmp</code> to <code>previous.checkpoint</code>.
+ * Receive a batch of edits from the NameNode.
+ * Depending on bnState, different actions are taken. See
+ * {@link BackupImage.BNState}
+ * @param firstTxId first txid in batch
+ * @param numTxns number of transactions
+ * @param data serialized journal records.
* @throws IOException
+ * @see #convergeJournalSpool()
- void loadCheckpoint(CheckpointSignature sig) throws IOException {
- // load current image and journal if it is not in memory already
- if(!editLog.isOpen())
- editLog.open();
- if(fsDir.isEmpty()) {
- Iterator<StorageDirectory> itImage
- = storage.dirIterator(NameNodeDirType.IMAGE);
- Iterator<StorageDirectory> itEdits
- = storage.dirIterator(NameNodeDirType.EDITS);
- if(!itImage.hasNext() || ! itEdits.hasNext())
- throw new IOException("Could not locate checkpoint directories");
- StorageDirectory sdName = itImage.next();
- StorageDirectory sdEdits = itEdits.next();
- getFSDirectoryRootLock().writeLock();
- try { // load image under rootDir lock
- loadFSImage(NNStorage.getStorageFile(sdName, NameNodeFile.IMAGE));
- getFSDirectoryRootLock().writeUnlock();
- loadFSEdits(sdEdits);
+ synchronized void journal(long firstTxId, int numTxns, byte[] data) throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Got journal, " +
+ "state = " + bnState +
+ "; firstTxId = " + firstTxId +
+ "; numTxns = " + numTxns);
+ switch(bnState) {
+ case DROP_UNTIL_NEXT_ROLL:
+ return;
- // set storage fields
- storage.setStorageInfo(sig);
- storage.setImageDigest(sig.imageDigest);
- storage.setCheckpointTime(sig.checkpointTime);
+ case IN_SYNC:
+ // update NameSpace in memory
+ applyEdits(firstTxId, numTxns, data);
+ break;
+ case JOURNAL_ONLY:
+ default:
+ throw new AssertionError("Unhandled state: " + bnState);
+ // write to BN's local edit log.
+ logEditsLocally(firstTxId, numTxns, data);
- * Save meta-data into fsimage files.
- * and create empty edits.
+ * Write the batch of edits to the local copy of the edit logs.
- void saveCheckpoint() throws IOException {
- saveNamespace(false);
- private FSDirectory getFSDirectoryRootLock() {
- return getFSNamesystem().dir;
- static File getJSpoolDir(StorageDirectory sd) {
- return new File(sd.getRoot(), STORAGE_JSPOOL_DIR);
- static File getJSpoolFile(StorageDirectory sd) {
- return new File(getJSpoolDir(sd), STORAGE_JSPOOL_FILE);
+ private void logEditsLocally(long firstTxId, int numTxns, byte[] data) {
+ long expectedTxId = editLog.getLastWrittenTxId() + 1;
+ Preconditions.checkState(firstTxId == expectedTxId,
+ "received txid batch starting at %s but expected txn %s",
+ firstTxId, expectedTxId);
+ editLog.setNextTxId(firstTxId + numTxns - 1);
+ editLog.logEdit(data.length, data);
+ editLog.logSync();
- * Journal writer journals new meta-data state.
- * <ol>
- * <li> If Journal Spool state is OFF then journal records (edits)
- * are applied directly to meta-data state in memory and are written
- * to the edits file(s).</li>
- * <li> If Journal Spool state is INPROGRESS then records are only
- * written to edits.new file, which is called Spooling.</li>
- * <li> Journal Spool state WAIT blocks journaling until the
- * Journal Spool reader finalizes merging of the spooled data and
- * switches to applying journal to memory.</li>
- * </ol>
- * @param length length of data.
- * @param data serialized journal records.
- * @see #convergeJournalSpool()
+ * Apply the batch of edits to the local namespace.
- synchronized void journal(int length, byte[] data) throws IOException {
+ private synchronized void applyEdits(long firstTxId, int numTxns, byte[] data)
+ throws IOException {
+ Preconditions.checkArgument(firstTxId == lastAppliedTxId + 1,
+ "Received txn batch starting at %s but expected %s",
+ firstTxId, lastAppliedTxId + 1);
assert backupInputStream.length() == 0 : "backup input stream is not empty";
try {
- switch(jsState) {
- case WAIT:
- case OFF:
- // wait until spooling is off
- waitSpoolEnd();
- // update NameSpace in memory
- backupInputStream.setBytes(data);
- FSEditLogLoader logLoader = new FSEditLogLoader(namesystem);
- int logVersion = storage.getLayoutVersion();
- BufferedInputStream bin = new BufferedInputStream(backupInputStream);
- DataInputStream in = new DataInputStream(bin);
- Checksum checksum = null;
- if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
- checksum = FSEditLog.getChecksum();
- in = new DataInputStream(new CheckedInputStream(bin, checksum));
- logLoader.loadEditRecords(logVersion, in, checksum, true);
- getFSNamesystem().dir.updateCountForINodeWithQuota(); // inefficient!
- break;
- case INPROGRESS:
+ LOG.debug("data:" + StringUtils.byteToHexString(data));
+ backupInputStream.setBytes(data);
+ FSEditLogLoader logLoader = new FSEditLogLoader(namesystem);
+ int logVersion = storage.getLayoutVersion();
+ BufferedInputStream bin = new BufferedInputStream(backupInputStream);
+ DataInputStream in = new DataInputStream(bin);
+ Checksum checksum = FSEditLog.getChecksum();
+ int numLoaded = logLoader.loadEditRecords(logVersion, in, checksum, true,
+ lastAppliedTxId + 1);
+ if (numLoaded != numTxns) {
+ throw new IOException("Batch of txns starting at txnid " +
+ firstTxId + " was supposed to contain " + numTxns +
+ " transactions but only was able to apply " + numLoaded);
- // write to files
- editLog.logEdit(length, data);
- editLog.logSync();
+ lastAppliedTxId += numTxns;
+ getFSNamesystem().dir.updateCountForINodeWithQuota(); // inefficient!
} finally {
backupInputStream.clear();
- private synchronized void waitSpoolEnd() {
- while(jsState == JSpoolState.WAIT) {
+ * Transition the BackupNode from JOURNAL_ONLY state to IN_SYNC state.
+ * This is done by repeated invocations of tryConvergeJournalSpool until
+ * we are caught up to the latest in-progress edits file.
+ void convergeJournalSpool() throws IOException {
+ Preconditions.checkState(bnState == BNState.JOURNAL_ONLY,
+ "bad state: %s", bnState);
+ while (!tryConvergeJournalSpool()) {
+ ;
+ assert bnState == BNState.IN_SYNC;
+ private boolean tryConvergeJournalSpool() throws IOException {
+ // This section is unsynchronized so we can continue to apply
+ // ahead of where we're reading, concurrently. Since the state
+ // is JOURNAL_ONLY at this point, we know that lastAppliedTxId
+ // doesn't change, and curSegmentTxId only increases
+ while (lastAppliedTxId < editLog.getCurSegmentTxId() - 1) {
+ long target = editLog.getCurSegmentTxId();
+ LOG.info("Loading edits into backupnode to try to catch up from txid "
+ + lastAppliedTxId + " to " + target);
+ FSImageTransactionalStorageInspector inspector =
+ new FSImageTransactionalStorageInspector();
+ storage.inspectStorageDirs(inspector);
+ LogLoadPlan logLoadPlan = inspector.createLogLoadPlan(lastAppliedTxId,
+ target - 1);
+ logLoadPlan.doRecovery();
+ loadEdits(logLoadPlan.getEditsFiles());
+ // now, need to load the in-progress file
+ synchronized (this) {
+ if (lastAppliedTxId != editLog.getCurSegmentTxId() - 1) {
+ LOG.debug("Logs rolled while catching up to current segment");
+ return false; // drop lock and try again to load local logs
+ EditLogInputStream stream = getEditLog().getInProgressFileInputStream();
- wait();
- } catch (InterruptedException e) {}
+ long remainingTxns = getEditLog().getLastWrittenTxId() - lastAppliedTxId;
+ LOG.info("Going to finish converging with remaining " + remainingTxns
+ + " txns from in-progress stream " + stream);
+ FSEditLogLoader loader = new FSEditLogLoader(namesystem);
+ int numLoaded = loader.loadFSEdits(stream, lastAppliedTxId + 1);
+ lastAppliedTxId += numLoaded;
+ assert numLoaded == remainingTxns :
+ "expected to load " + remainingTxns + " but loaded " +
+ numLoaded + " from " + stream;
+ IOUtils.closeStream(stream);
+ LOG.info("Successfully synced BackupNode with NameNode at txnid " +
+ lastAppliedTxId);
+ setState(BNState.IN_SYNC);
- // now spooling should be off, verifying just in case
- assert jsState == JSpoolState.OFF : "Unexpected JSpool state: " + jsState;
- * Start journal spool.
- * Switch to writing into edits.new instead of edits.
- * edits.new for spooling is in separate directory "spool" rather than in
- * "current" because the two directories should be independent.
- * While spooling a checkpoint can happen and current will first
- * move to lastcheckpoint.tmp and then to previous.checkpoint
- * spool/edits.new will remain in place during that.
+ * Transition edit log to a new state, logging as necessary.
- synchronized void startJournalSpool(NamenodeRegistration nnReg)
- throws IOException {
- return;
+ private synchronized void setState(BNState newState) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("State transition " + bnState + " -> " + newState,
+ new Exception("trace"));
+ bnState = newState;
- // create journal spool directories
- for (Iterator<StorageDirectory> it
- = storage.dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
- File jsDir = getJSpoolDir(sd);
- if (!jsDir.exists() && !jsDir.mkdirs()) {
- throw new IOException("Mkdirs failed to create "
- + jsDir.getCanonicalPath());
+ * Receive a notification that the NameNode has begun a new edit log.
+ * This causes the BN to also start the new edit log in its local
+ * directories.
+ synchronized void namenodeStartedLogSegment(long txid)
+ LOG.info("NameNode started a new log segment at txid " + txid);
+ if (editLog.isOpen()) {
+ if (editLog.getLastWrittenTxId() == txid - 1) {
+ // We are in sync with the NN, so end and finalize the current segment
+ editLog.endCurrentLogSegment(false);
+ } else {
+ // We appear to have missed some transactions -- the NN probably
+ // lost contact with us temporarily. So, mark the current segment
+ // as aborted.
+ LOG.warn("NN started new log segment at txid " + txid +
+ ", but BN had only written up to txid " +
+ editLog.getLastWrittenTxId() +
+ "in the log segment starting at " +
+ editLog.getCurSegmentTxId() + ". Aborting this " +
+ "log segment.");
+ editLog.abortCurrentLogSegment();
- // create edit file if missing
- File eFile = storage.getEditFile(sd);
- if(!eFile.exists()) {
- editLog.createEditLogFile(eFile);
+ editLog.setNextTxId(txid);
+ editLog.startLogSegment(txid, false);
+ if (bnState == BNState.DROP_UNTIL_NEXT_ROLL) {
+ setState(BNState.JOURNAL_ONLY);
+ if (stopApplyingEditsOnNextRoll) {
+ if (bnState == BNState.IN_SYNC) {
+ LOG.info("Stopped applying edits to prepare for checkpoint.");
+ stopApplyingEditsOnNextRoll = false;
+ notifyAll();
- // create streams pointing to the journal spool files
- // subsequent journal records will go directly to the spool
- editLog.divertFileStreams(STORAGE_JSPOOL_DIR + "/" + STORAGE_JSPOOL_FILE);
- setCheckpointState(CheckpointStates.ROLLED_EDITS);
- // set up spooling
- if(backupInputStream == null)
- backupInputStream = new EditLogBackupInputStream(nnReg.getAddress());
- jsState = JSpoolState.INPROGRESS;
- synchronized void setCheckpointTime(int length, byte[] data)
- assert backupInputStream.length() == 0 : "backup input stream is not empty";
- // unpack new checkpoint time
- DataInputStream in = backupInputStream.getDataInputStream();
- byte op = in.readByte();
- assert op == NamenodeProtocol.JA_CHECKPOINT_TIME;
- LongWritable lw = new LongWritable();
- lw.readFields(in);
- storage.setCheckpointTimeInStorage(lw.get());
- backupInputStream.clear();
+ * Request that the next time the BN receives a log roll, it should
+ * stop applying the edits log to the local namespace. This is
+ * typically followed on by a call to {@link #waitUntilNamespaceFrozen()}
+ synchronized void freezeNamespaceAtNextRoll() {
+ stopApplyingEditsOnNextRoll = true;
- * Merge Journal Spool to memory.<p>
- * Journal Spool reader reads journal records from edits.new.
- * When it reaches the end of the file it sets {@link JSpoolState} to WAIT.
- * This blocks journaling (see {@link #journal(int,byte[])}.
- * The reader
- * <ul>
- * <li> reads remaining journal records if any,</li>
- * <li> renames edits.new to edits,</li>
- * <li> sets {@link JSpoolState} to OFF,</li>
- * <li> and notifies the journaling thread.</li>
- * </ul>
- * Journaling resumes with applying new journal records to the memory state,
- * and writing them into edits file(s).
+ * After {@link #freezeNamespaceAtNextRoll()} has been called, wait until
+ * the BN receives notification of the next log roll.
- void convergeJournalSpool() throws IOException {
- if(! itEdits.hasNext())
- int numEdits = 0;
- File jSpoolFile = getJSpoolFile(sdEdits);
- long startTime = now();
- if(jSpoolFile.exists()) {
- // load edits.new
- EditLogFileInputStream edits = new EditLogFileInputStream(jSpoolFile);
- BufferedInputStream bin = new BufferedInputStream(edits);
- int logVersion = logLoader.readLogVersion(in);
+ synchronized void waitUntilNamespaceFrozen() throws IOException {
+ if (bnState != BNState.IN_SYNC) return;
+ LOG.info("Waiting until the NameNode rolls its edit logs in order " +
+ "to freeze the BackupNode namespace.");
+ while (bnState == BNState.IN_SYNC) {
+ Preconditions.checkState(stopApplyingEditsOnNextRoll,
+ "If still in sync, we should still have the flag set to " +
+ "freeze at next roll");
+ wait();
+ } catch (InterruptedException ie) {
+ LOG.warn("Interrupted waiting for namespace to freeze", ie);
+ throw new IOException(ie);
- numEdits += logLoader.loadEditRecords(logVersion, in, checksum, false);
- // first time reached the end of spool
- jsState = JSpoolState.WAIT;
- numEdits += logLoader.loadEditRecords(logVersion,
- in, checksum, true);
- getFSNamesystem().dir.updateCountForINodeWithQuota();
- edits.close();
- FSImage.LOG.info("Edits file " + jSpoolFile.getCanonicalPath()
- + " of size " + jSpoolFile.length() + " edits # " + numEdits
- + " loaded in " + (now()-startTime)/1000 + " seconds.");
- // rename spool edits.new to edits making it in sync with the active node
- // subsequent journal records will go directly to edits
- editLog.revertFileStreams(STORAGE_JSPOOL_DIR + "/" + STORAGE_JSPOOL_FILE);
- // write version file
- resetVersion(false, storage.getImageDigest());
- // wake up journal writer
- synchronized(this) {
- notifyAll();
+ LOG.info("BackupNode namespace frozen.");
- // Rename lastcheckpoint.tmp to previous.checkpoint
- storage.moveLastCheckpoint(sd);
+ * Override close() so that we don't finalize edit logs.
+ public synchronized void close() throws IOException {
+ storage.close();
@@ -0,0 +1,69 @@
+/**
+ * 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.IOException;
+import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+ * A JournalManager implementation that uses RPCs to log transactions
+ * to a BackupNode.
+class BackupJournalManager implements JournalManager {
+ private final NamenodeRegistration nnReg;
+ private final NamenodeRegistration bnReg;
+ BackupJournalManager(NamenodeRegistration bnReg,
+ NamenodeRegistration nnReg) {
+ this.bnReg = bnReg;
+ this.nnReg = nnReg;
+ public EditLogOutputStream startLogSegment(long txId) throws IOException {
+ EditLogBackupOutputStream stm = new EditLogBackupOutputStream(bnReg, nnReg);
+ stm.startLogSegment(txId);
+ return stm;
+ public void finalizeLogSegment(long firstTxId, long lastTxId)
+ public void setOutputBufferCapacity(int size) {
+ public void purgeLogsOlderThan(long minTxIdToKeep, StoragePurger purger)
+ public boolean matchesRegistration(NamenodeRegistration bnReg) {
+ return bnReg.getAddress().equals(this.bnReg.getAddress());
+ public EditLogInputStream getInProgressInputStream(long segmentStartsAtTxId) {
+ return null;
+}
@@ -22,21 +22,20 @@ import java.net.InetSocketAddress;
import java.net.SocketTimeoutException;
import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.hdfs.server.common.Storage;
-import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
-import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
-import org.apache.hadoop.hdfs.server.namenode.FSImage.CheckpointStates;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.net.NetUtils;
@@ -53,7 +52,7 @@ import org.apache.hadoop.net.NetUtils;
* </ol>
-public class BackupNode extends NameNode {
+public class BackupNode extends NameNode implements JournalProtocol {
private static final String BN_ADDRESS_NAME_KEY = DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY;
private static final String BN_ADDRESS_DEFAULT = DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_DEFAULT;
private static final String BN_HTTP_ADDRESS_NAME_KEY = DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
@@ -119,10 +118,9 @@ public class BackupNode extends NameNode {
@Override // NameNode
protected void loadNamesystem(Configuration conf) throws IOException {
- BackupImage bnImage = new BackupImage();
+ BackupImage bnImage = new BackupImage(conf);
this.namesystem = new FSNamesystem(conf, bnImage);
- bnImage.recoverCreateRead(FSNamesystem.getNamespaceDirs(conf),
- FSNamesystem.getNamespaceEditsDirs(conf));
+ bnImage.recoverCreateRead();
@@ -179,6 +177,17 @@ public class BackupNode extends NameNode {
super.stop();
+ public long getProtocolVersion(String protocol, long clientVersion)
+ if (protocol.equals(JournalProtocol.class.getName())) {
+ return JournalProtocol.versionID;
+ return super.getProtocolVersion(protocol, clientVersion);
/////////////////////////////////////////////////////
// NamenodeProtocol implementation for backup node.
@@ -205,34 +214,36 @@ public class BackupNode extends NameNode {
public void endCheckpoint(NamenodeRegistration registration,
CheckpointSignature sig) throws IOException {
throw new UnsupportedActionException("endCheckpoint");
- @Override // NamenodeProtocol
+ /////////////////////////////////////////////////////
+ // BackupNodeProtocol implementation for backup node.
public void journal(NamenodeRegistration nnReg,
- int jAction,
- int length,
- byte[] args) throws IOException {
+ long firstTxId, int numTxns,
+ byte[] records) throws IOException {
verifyRequest(nnReg);
if(!nnRpcAddress.equals(nnReg.getAddress()))
throw new IOException("Journal request from unexpected name-node: "
+ nnReg.getAddress() + " expecting " + nnRpcAddress);
- BackupImage bnImage = (BackupImage)getFSImage();
- switch(jAction) {
- case (int)JA_IS_ALIVE:
- case (int)JA_JOURNAL:
- bnImage.journal(length, args);
- case (int)JA_JSPOOL_START:
- bnImage.startJournalSpool(nnReg);
- case (int)JA_CHECKPOINT_TIME:
- bnImage.setCheckpointTime(length, args);
- setRegistration(); // keep registration up to date
- default:
- throw new IOException("Unexpected journal action: " + jAction);
+ getBNImage().journal(firstTxId, numTxns, records);
+ public void startLogSegment(NamenodeRegistration registration, long txid)
+ verifyRequest(registration);
+ getBNImage().namenodeStartedLogSegment(txid);
+ //////////////////////////////////////////////////////
+ BackupImage getBNImage() {
+ return (BackupImage)getFSImage();
boolean shouldCheckpointAtStartup() {
@@ -241,9 +252,9 @@ public class BackupNode extends NameNode {
assert fsImage.getStorage().getNumStorageDirs() > 0;
return ! fsImage.getStorage().getStorageDir(0).getVersionFile().exists();
- if(namesystem == null || namesystem.dir == null || getFSImage() == null)
- return fsImage.getEditLog().getNumEditStreams() == 0;
+ // BN always checkpoints on startup in order to get in sync with namespace
private NamespaceInfo handshake(Configuration conf) throws IOException {
@@ -287,14 +298,6 @@ public class BackupNode extends NameNode {
checkpointManager.doCheckpoint();
- CheckpointStates getCheckpointState() {
- return getFSImage().getCheckpointState();
- void setCheckpointState(CheckpointStates cs) {
- getFSImage().setCheckpointState(cs);
* Register this backup node with the active name-node.
* @param nsInfo
@@ -302,14 +305,15 @@ public class BackupNode extends NameNode {
private void registerWith(NamespaceInfo nsInfo) throws IOException {
BackupImage bnImage = (BackupImage)getFSImage();
+ NNStorage storage = bnImage.getStorage();
// verify namespaceID
- if(bnImage.getStorage().getNamespaceID() == 0) // new backup storage
- bnImage.getStorage().setStorageInfo(nsInfo);
- else if(bnImage.getStorage().getNamespaceID() != nsInfo.getNamespaceID())
- throw new IOException("Incompatible namespaceIDs"
- + ": active node namespaceID = " + nsInfo.getNamespaceID()
- + "; backup node namespaceID = " + bnImage.getStorage().getNamespaceID());
+ if (storage.getNamespaceID() == 0) { // new backup storage
+ storage.setStorageInfo(nsInfo);
+ storage.setBlockPoolID(nsInfo.getBlockPoolID());
+ storage.setClusterID(nsInfo.getClusterID());
+ nsInfo.validateStorage(storage);
setRegistration();
NamenodeRegistration nnReg = null;
while(!isStopRequested()) {
@@ -338,23 +342,6 @@ public class BackupNode extends NameNode {
nnRpcAddress = nnReg.getAddress();
- * Reset node namespace state in memory and in storage directories.
- void resetNamespace() throws IOException {
- ((BackupImage)getFSImage()).reset();
- * Get size of the local journal (edit log).
- * @return size of the current journal
- long journalSize() throws IOException {
- return namesystem.getEditLogSize();
// TODO: move to a common with DataNode util class
private static NamespaceInfo handshake(NamenodeProtocol namenode)
throws IOException, SocketTimeoutException {
@@ -24,10 +24,11 @@ import java.io.IOException;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.FSImage;
-import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.io.WritableComparable;
+import com.google.common.collect.ComparisonChain;
* A unique signature intended to identify checkpoint transactions.
@@ -35,41 +36,35 @@ import org.apache.hadoop.io.WritableUtils;
public class CheckpointSignature extends StorageInfo
implements WritableComparable<CheckpointSignature> {
private static final String FIELD_SEPARATOR = ":";
- long editsTime = -1L;
- long checkpointTime = -1L;
- MD5Hash imageDigest = null;
+ private static final int NUM_FIELDS = 7;
String blockpoolID = "";
+ long mostRecentCheckpointTxId;
+ long curSegmentTxId;
public CheckpointSignature() {}
CheckpointSignature(FSImage fsImage) {
super(fsImage.getStorage());
blockpoolID = fsImage.getBlockPoolID();
- editsTime = fsImage.getEditLog().getFsEditTime();
- checkpointTime = fsImage.getStorage().getCheckpointTime();
- imageDigest = fsImage.getStorage().getImageDigest();
+ mostRecentCheckpointTxId = fsImage.getStorage().getMostRecentCheckpointTxId();
+ curSegmentTxId = fsImage.getEditLog().getCurSegmentTxId();
CheckpointSignature(String str) {
String[] fields = str.split(FIELD_SEPARATOR);
- assert fields.length == 8 : "Must be 8 fields in CheckpointSignature";
- layoutVersion = Integer.valueOf(fields[0]);
- namespaceID = Integer.valueOf(fields[1]);
- cTime = Long.valueOf(fields[2]);
- editsTime = Long.valueOf(fields[3]);
- checkpointTime = Long.valueOf(fields[4]);
- imageDigest = new MD5Hash(fields[5]);
- clusterID = fields[6];
- blockpoolID = fields[7];
- * Get the MD5 image digest
- * @return the MD5 image digest
- MD5Hash getImageDigest() {
- return imageDigest;
+ assert fields.length == NUM_FIELDS :
+ "Must be " + NUM_FIELDS + " fields in CheckpointSignature";
+ int i = 0;
+ layoutVersion = Integer.valueOf(fields[i++]);
+ namespaceID = Integer.valueOf(fields[i++]);
+ cTime = Long.valueOf(fields[i++]);
+ mostRecentCheckpointTxId = Long.valueOf(fields[i++]);
+ curSegmentTxId = Long.valueOf(fields[i++]);
+ clusterID = fields[i++];
+ blockpoolID = fields[i++];
@@ -101,33 +96,26 @@ public class CheckpointSignature extends StorageInfo
return String.valueOf(layoutVersion) + FIELD_SEPARATOR
+ String.valueOf(namespaceID) + FIELD_SEPARATOR
+ String.valueOf(cTime) + FIELD_SEPARATOR
- + String.valueOf(editsTime) + FIELD_SEPARATOR
- + String.valueOf(checkpointTime) + FIELD_SEPARATOR
- + imageDigest.toString() + FIELD_SEPARATOR
+ + String.valueOf(mostRecentCheckpointTxId) + FIELD_SEPARATOR
+ + String.valueOf(curSegmentTxId) + FIELD_SEPARATOR
+ clusterID + FIELD_SEPARATOR
+ blockpoolID ;
void validateStorageInfo(FSImage si) throws IOException {
- if(layoutVersion != si.getLayoutVersion()
- || namespaceID != si.getNamespaceID()
- || cTime != si.getStorage().cTime
- || checkpointTime != si.getStorage().getCheckpointTime()
- || !imageDigest.equals(si.getStorage().imageDigest)
- || !clusterID.equals(si.getClusterID())
- || !blockpoolID.equals(si.getBlockPoolID())) {
- // checkpointTime can change when the image is saved - do not compare
+ if(layoutVersion != si.getStorage().layoutVersion
+ || namespaceID != si.getStorage().namespaceID
+ || cTime != si.getStorage().cTime
+ || !clusterID.equals(si.getClusterID())
+ || !blockpoolID.equals(si.getBlockPoolID())) {
throw new IOException("Inconsistent checkpoint fields.\n"
+ "LV = " + layoutVersion + " namespaceID = " + namespaceID
- + " cTime = " + cTime + "; checkpointTime = " + checkpointTime
- + " ; imageDigest = " + imageDigest
+ + " cTime = " + cTime
+ " ; clusterId = " + clusterID
+ " ; blockpoolId = " + blockpoolID
+ ".\nExpecting respectively: "
- + si.getLayoutVersion() + "; "
- + si.getNamespaceID() + "; " + si.getStorage().cTime
- + "; " + si.getStorage().getCheckpointTime() + "; "
- + si.getStorage().imageDigest
+ + si.getStorage().layoutVersion + "; "
+ + si.getStorage().namespaceID + "; " + si.getStorage().cTime
+ "; " + si.getClusterID() + "; "
+ si.getBlockPoolID() + ".");
@@ -137,19 +125,15 @@ public class CheckpointSignature extends StorageInfo
// Comparable interface
//
public int compareTo(CheckpointSignature o) {
- return
- (layoutVersion < o.layoutVersion) ? -1 :
- (layoutVersion > o.layoutVersion) ? 1 :
- (namespaceID < o.namespaceID) ? -1 : (namespaceID > o.namespaceID) ? 1 :
- (cTime < o.cTime) ? -1 : (cTime > o.cTime) ? 1 :
- (editsTime < o.editsTime) ? -1 : (editsTime > o.editsTime) ? 1 :
- (checkpointTime < o.checkpointTime) ? -1 :
- (checkpointTime > o.checkpointTime) ? 1 :
- (clusterID.compareTo(o.clusterID) < 0) ? -1 :
- (clusterID.compareTo(o.clusterID) > 0) ? 1 :
- (blockpoolID.compareTo(o.blockpoolID) < 0) ? -1 :
- (blockpoolID.compareTo(o.blockpoolID) > 0) ? 1 :
- imageDigest.compareTo(o.imageDigest);
+ return ComparisonChain.start()
+ .compare(layoutVersion, o.layoutVersion)
+ .compare(namespaceID, o.namespaceID)
+ .compare(cTime, o.cTime)
+ .compare(mostRecentCheckpointTxId, o.mostRecentCheckpointTxId)
+ .compare(curSegmentTxId, o.curSegmentTxId)
+ .compare(clusterID, o.clusterID)
+ .compare(blockpoolID, o.blockpoolID)
+ .result();
public boolean equals(Object o) {
@@ -161,9 +145,8 @@ public class CheckpointSignature extends StorageInfo
public int hashCode() {
return layoutVersion ^ namespaceID ^
- (int)(cTime ^ editsTime ^ checkpointTime) ^
- imageDigest.hashCode() ^ clusterID.hashCode()
- ^ blockpoolID.hashCode();
+ (int)(cTime ^ mostRecentCheckpointTxId ^ curSegmentTxId)
+ ^ clusterID.hashCode() ^ blockpoolID.hashCode();
/////////////////////////////////////////////////
@@ -172,17 +155,14 @@ public class CheckpointSignature extends StorageInfo
public void write(DataOutput out) throws IOException {
super.write(out);
WritableUtils.writeString(out, blockpoolID);
- out.writeLong(editsTime);
- out.writeLong(checkpointTime);
- imageDigest.write(out);
+ out.writeLong(mostRecentCheckpointTxId);
+ out.writeLong(curSegmentTxId);
public void readFields(DataInput in) throws IOException {
super.readFields(in);
blockpoolID = WritableUtils.readString(in);
- editsTime = in.readLong();
- checkpointTime = in.readLong();
- imageDigest = new MD5Hash();
- imageDigest.readFields(in);
+ mostRecentCheckpointTxId = in.readLong();
+ curSegmentTxId = in.readLong();
@@ -17,29 +17,30 @@
package org.apache.hadoop.hdfs.server.namenode;
-import java.io.IOException;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.File;
import java.net.InetSocketAddress;
+import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
+import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.util.Daemon;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT;
+import com.google.common.collect.Lists;
* The Checkpointer is responsible for supporting periodic checkpoints
@@ -59,7 +60,7 @@ class Checkpointer extends Daemon {
private BackupNode backupNode;
volatile boolean shouldRun;
private long checkpointPeriod; // in seconds
- private long checkpointSize; // size (in MB) of current Edit Log
+ private long checkpointTxnCount; // size (in MB) of current Edit Log
private String infoBindAddress;
@@ -79,6 +80,7 @@ class Checkpointer extends Daemon {
initialize(conf);
} catch(IOException e) {
+ LOG.warn("Checkpointer got exception", e);
shutdown();
throw e;
@@ -87,6 +89,7 @@ class Checkpointer extends Daemon {
* Initialize checkpoint.
+ @SuppressWarnings("deprecation")
private void initialize(Configuration conf) throws IOException {
// Create connection to the namenode.
shouldRun = true;
@@ -94,8 +97,9 @@ class Checkpointer extends Daemon {
// Initialize other scheduling parameters from the configuration
checkpointPeriod = conf.getLong(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY,
DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT);
- checkpointSize = conf.getLong(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_SIZE_KEY,
- DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_SIZE_DEFAULT);
+ checkpointTxnCount = conf.getLong(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
+ DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
+ SecondaryNameNode.warnForDeprecatedConfigs(conf);
// Pull out exact http address for posting url to avoid ip aliasing issues
String fullInfoAddr = conf.get(DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,
@@ -104,8 +108,7 @@ class Checkpointer extends Daemon {
LOG.info("Checkpoint Period : " + checkpointPeriod + " secs " +
"(" + checkpointPeriod/60 + " min)");
- LOG.info("Log Size Trigger : " + checkpointSize + " bytes " +
- "(" + checkpointSize/1024 + " KB)");
+ LOG.info("Log Size Trigger : " + checkpointTxnCount + " txns ");
@@ -128,8 +131,9 @@ class Checkpointer extends Daemon {
periodMSec *= 1000;
long lastCheckpointTime = 0;
- if(!backupNode.shouldCheckpointAtStartup())
+ if (!backupNode.shouldCheckpointAtStartup()) {
lastCheckpointTime = now();
while(shouldRun) {
long now = now();
@@ -137,8 +141,8 @@ class Checkpointer extends Daemon {
if(now >= lastCheckpointTime + periodMSec) {
shouldCheckpoint = true;
} else {
- long size = getJournalSize();
- if(size >= checkpointSize)
+ long txns = countUncheckpointedTxns();
+ if(txns >= checkpointTxnCount)
if(shouldCheckpoint) {
@@ -160,64 +164,24 @@ class Checkpointer extends Daemon {
- private long getJournalSize() throws IOException {
- // If BACKUP node has been loaded
- // get edits size from the local file. ACTIVE has the same.
- if(backupNode.isRole(NamenodeRole.BACKUP)
- && getFSImage().getEditLog().isOpen())
- return backupNode.journalSize();
- // Go to the ACTIVE node for its size
- return getNamenode().journalSize(backupNode.getRegistration());
- * Download <code>fsimage</code> and <code>edits</code>
- * files from the remote name-node.
- private void downloadCheckpoint(CheckpointSignature sig) throws IOException {
- // Retrieve image file
- String fileid = "getimage=1";
- Collection<File> list = getFSImage()
- .getStorage().getFiles(NameNodeFile.IMAGE, NameNodeDirType.IMAGE);
- File[] files = list.toArray(new File[list.size()]);
- assert files.length > 0 : "No checkpoint targets.";
- String nnHttpAddr = backupNode.nnHttpAddress;
- TransferFsImage.getFileClient(nnHttpAddr, fileid, files, false);
- LOG.info("Downloaded file " + files[0].getName() + " size " +
- files[0].length() + " bytes.");
- // Retrieve edits file
- fileid = "getedit=1";
- list = getFSImage()
- .getStorage().getFiles(NameNodeFile.EDITS, NameNodeDirType.EDITS);
- files = list.toArray(new File[list.size()]);
- * Copy the new image into remote name-node.
- private void uploadCheckpoint(CheckpointSignature sig) throws IOException {
- // Use the exact http addr as specified in config to deal with ip aliasing
- InetSocketAddress httpSocAddr = backupNode.getHttpAddress();
- int httpPort = httpSocAddr.getPort();
- String fileid = "putimage=1&port=" + httpPort +
- "&machine=" + infoBindAddress +
- "&token=" + sig.toString() +
- "&newChecksum=" + getFSImage().getStorage().getImageDigest().toString();
- LOG.info("Posted URL " + backupNode.nnHttpAddress + fileid);
- TransferFsImage.getFileClient(backupNode.nnHttpAddress,
- fileid, (File[])null, false);
+ private long countUncheckpointedTxns() throws IOException {
+ long curTxId = getNamenode().getTransactionID();
+ long uncheckpointedTxns = curTxId -
+ getFSImage().getStorage().getMostRecentCheckpointTxId();
+ assert uncheckpointedTxns >= 0;
+ return uncheckpointedTxns;
* Create a new checkpoint
void doCheckpoint() throws IOException {
+ BackupImage bnImage = getFSImage();
+ NNStorage bnStorage = bnImage.getStorage();
long startTime = now();
+ bnImage.freezeNamespaceAtNextRoll();
NamenodeCommand cmd =
getNamenode().startCheckpoint(backupNode.getRegistration());
CheckpointCommand cpCmd = null;
@@ -233,37 +197,94 @@ class Checkpointer extends Daemon {
throw new IOException("Unsupported NamenodeCommand: "+cmd.getAction());
+ bnImage.waitUntilNamespaceFrozen();
CheckpointSignature sig = cpCmd.getSignature();
- assert FSConstants.LAYOUT_VERSION == sig.getLayoutVersion() :
- "Signature should have current layout version. Expected: "
- + FSConstants.LAYOUT_VERSION + " actual "+ sig.getLayoutVersion();
- assert !backupNode.isRole(NamenodeRole.CHECKPOINT) ||
- cpCmd.isImageObsolete() : "checkpoint node should always download image.";
- backupNode.setCheckpointState(CheckpointStates.UPLOAD_START);
- if(cpCmd.isImageObsolete()) {
- // First reset storage on disk and memory state
- backupNode.resetNamespace();
- downloadCheckpoint(sig);
- BackupImage bnImage = getFSImage();
- bnImage.getStorage().setBlockPoolID(backupNode.getBlockPoolId());
- bnImage.getStorage().setClusterID(backupNode.getClusterId());
- bnImage.loadCheckpoint(sig);
+ // Make sure we're talking to the same NN!
sig.validateStorageInfo(bnImage);
- bnImage.saveCheckpoint();
- if(cpCmd.needToReturnImage())
- uploadCheckpoint(sig);
+ long lastApplied = bnImage.getLastAppliedTxId();
+ LOG.debug("Doing checkpoint. Last applied: " + lastApplied);
+ RemoteEditLogManifest manifest =
+ getNamenode().getEditLogManifest(bnImage.getLastAppliedTxId());
+ if (!manifest.getLogs().isEmpty()) {
+ RemoteEditLog firstRemoteLog = manifest.getLogs().get(0);
+ // we don't have enough logs to roll forward using only logs. Need
+ // to download and load the image.
+ if (firstRemoteLog.getStartTxId() > lastApplied + 1) {
+ LOG.info("Unable to roll forward using only logs. Downloading " +
+ "image with txid " + sig.mostRecentCheckpointTxId);
+ MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage(
+ backupNode.nnHttpAddress, sig.mostRecentCheckpointTxId,
+ bnStorage, true);
+ bnImage.saveDigestAndRenameCheckpointImage(
+ sig.mostRecentCheckpointTxId, downloadedHash);
+ LOG.info("Loading image with txid " + sig.mostRecentCheckpointTxId);
+ File file = bnStorage.findImageFile(sig.mostRecentCheckpointTxId);
+ bnImage.reloadFromImageFile(file);
+ lastApplied = bnImage.getLastAppliedTxId();
+ throw new IOException("No logs to roll forward from " + lastApplied);
+ // get edits files
+ for (RemoteEditLog log : manifest.getLogs()) {
+ TransferFsImage.downloadEditsToStorage(
+ backupNode.nnHttpAddress, log, bnStorage);
+ rollForwardByApplyingLogs(manifest, bnImage);
+ long txid = bnImage.getLastAppliedTxId();
+ bnImage.saveFSImageInAllDirs(txid);
+ bnStorage.writeAll();
+ if(cpCmd.needToReturnImage()) {
+ TransferFsImage.uploadImageFromStorage(
+ backupNode.nnHttpAddress, getImageListenAddress(),
+ bnStorage, txid);
getNamenode().endCheckpoint(backupNode.getRegistration(), sig);
- bnImage.convergeJournalSpool();
+ if (backupNode.getRole() == NamenodeRole.BACKUP) {
+ bnImage.convergeJournalSpool();
backupNode.setRegistration(); // keep registration up to date
- if(backupNode.isRole(NamenodeRole.CHECKPOINT))
- getFSImage().getEditLog().close();
+ long imageSize = bnImage.getStorage().getFsImageName(txid).length();
LOG.info("Checkpoint completed in "
+ (now() - startTime)/1000 + " seconds."
- + " New Image Size: " + bnImage.getStorage().getFsImageName().length());
+ + " New Image Size: " + imageSize);
+ private InetSocketAddress getImageListenAddress() {
+ InetSocketAddress httpSocAddr = backupNode.getHttpAddress();
+ int httpPort = httpSocAddr.getPort();
+ return new InetSocketAddress(infoBindAddress, httpPort);
+ static void rollForwardByApplyingLogs(
+ RemoteEditLogManifest manifest,
+ FSImage dstImage) throws IOException {
+ NNStorage dstStorage = dstImage.getStorage();
+ List<File> editsFiles = Lists.newArrayList();
+ File f = dstStorage.findFinalizedEditsFile(
+ log.getStartTxId(), log.getEndTxId());
+ if (log.getStartTxId() > dstImage.getLastAppliedTxId()) {
+ editsFiles.add(f);
+ LOG.info("Checkpointer about to load edits from " +
+ editsFiles.size() + " file(s).");
+ dstImage.loadEdits(editsFiles);
@@ -42,10 +42,6 @@ class EditLogBackupInputStream extends EditLogInputStream {
super(new byte[0]);
- byte[] getData() {
- return super.buf;
void setData(byte[] newBytes) {
super.buf = newBytes;
super.count = newBytes == null ? 0 : newBytes.length;
@@ -19,10 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode;
+import java.util.Arrays;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.io.DataOutputBuffer;
@@ -39,7 +40,7 @@ import org.apache.hadoop.net.NetUtils;
class EditLogBackupOutputStream extends EditLogOutputStream {
static int DEFAULT_BUFFER_SIZE = 256;
- private NamenodeProtocol backupNode; // RPC proxy to backup node
+ private JournalProtocol backupNode; // RPC proxy to backup node
private NamenodeRegistration bnRegistration; // backup node registration
private NamenodeRegistration nnRegistration; // active node registration
private EditsDoubleBuffer doubleBuf;
@@ -56,8 +57,8 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
Storage.LOG.info("EditLogBackupOutputStream connects to: " + bnAddress);
this.backupNode =
- (NamenodeProtocol) RPC.getProxy(NamenodeProtocol.class,
- NamenodeProtocol.versionID, bnAddress, new HdfsConfiguration());
+ RPC.getProxy(JournalProtocol.class,
+ JournalProtocol.versionID, bnAddress, new HdfsConfiguration());
Storage.LOG.error("Error connecting to: " + bnAddress, e);
@@ -65,7 +66,7 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
this.doubleBuf = new EditsDoubleBuffer(DEFAULT_BUFFER_SIZE);
this.out = new DataOutputBuffer(DEFAULT_BUFFER_SIZE);
@Override // JournalStream
public String getName() {
return bnRegistration.getAddress();
@@ -109,6 +110,12 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
doubleBuf = null;
+ public void abort() throws IOException {
+ RPC.stopProxy(backupNode);
+ doubleBuf = null;
@Override // EditLogOutputStream
void setReadyToFlush() throws IOException {
doubleBuf.setReadyToFlush();
@@ -116,11 +123,21 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
protected void flushAndSync() throws IOException {
- // XXX: this code won't work in trunk, but it's redone
- // in HDFS-1073 where it's simpler.
+ assert out.getLength() == 0 : "Output buffer is not empty";
+ int numReadyTxns = doubleBuf.countReadyTxns();
+ long firstTxToFlush = doubleBuf.getFirstReadyTxId();
doubleBuf.flushTo(out);
- if (out.size() > 0) {
- send(NamenodeProtocol.JA_JOURNAL);
+ if (out.getLength() > 0) {
+ assert numReadyTxns > 0;
+ byte[] data = Arrays.copyOf(out.getData(), out.getLength());
+ out.reset();
+ backupNode.journal(nnRegistration,
+ firstTxToFlush, numReadyTxns, data);
@@ -134,16 +151,6 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
return 0;
- private void send(int ja) throws IOException {
- int length = out.getLength();
- out.write(FSEditLogOpCodes.OP_INVALID.getOpCode());
- backupNode.journal(nnRegistration, ja, length, out.getData());
- out.reset();
* Get backup node registration.
@@ -151,17 +158,7 @@ class EditLogBackupOutputStream extends EditLogOutputStream {
return bnRegistration;
- * Verify that the backup node is alive.
- boolean isAlive() {
- send(NamenodeProtocol.JA_IS_ALIVE);
- } catch(IOException ei) {
- Storage.LOG.info(bnRegistration.getRole() + " "
- + bnRegistration.getAddress() + " is not alive. ", ei);
+ void startLogSegment(long txId) throws IOException {
+ backupNode.startLogSegment(nnRegistration, txId);
@@ -70,4 +70,10 @@ class EditLogFileInputStream extends EditLogInputStream {
// file size + size of both buffers
return file.length();
+ return getName();
@@ -25,6 +25,8 @@ import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.IOUtils;
@@ -35,6 +37,8 @@ import com.google.common.annotations.VisibleForTesting;
* stores edits in a local file.
class EditLogFileOutputStream extends EditLogOutputStream {
+ private static Log LOG = LogFactory.getLog(EditLogFileOutputStream.class);;
private static int EDITS_FILE_HEADER_SIZE_BYTES = Integer.SIZE / Byte.SIZE;
private File file;
@@ -85,7 +89,14 @@ class EditLogFileOutputStream extends EditLogOutputStream {
doubleBuf.writeOp(op);
- /** {@inheritDoc} */
+ * Write a transaction to the stream. The serialization format is:
+ * <ul>
+ * <li>the opcode (byte)</li>
+ * <li>the transaction id (long)</li>
+ * <li>the actual Writables for the transaction</li>
+ * </ul>
+ * */
@Override
void writeRaw(byte[] bytes, int offset, int length) throws IOException {
doubleBuf.writeRaw(bytes, offset, length);
@@ -105,6 +116,10 @@ class EditLogFileOutputStream extends EditLogOutputStream {
public void close() throws IOException {
+ if (fp == null) {
+ throw new IOException("Trying to use aborted output stream");
// close should have been called after all pending transactions
// have been flushed & synced.
@@ -130,6 +145,16 @@ class EditLogFileOutputStream extends EditLogOutputStream {
fc = null;
fp = null;
+ fp = null;
+ IOUtils.cleanup(LOG, fp);
@@ -148,6 +173,10 @@ class EditLogFileOutputStream extends EditLogOutputStream {
preallocate(); // preallocate file if necessary
doubleBuf.flushTo(fp);
fc.force(false); // metadata updates not needed because of preallocation
@@ -190,19 +219,17 @@ class EditLogFileOutputStream extends EditLogOutputStream {
- * Operations like OP_JSPOOL_START and OP_CHECKPOINT_TIME should not be
- * written into edits file.
+ * Returns the file associated with this stream.
- @Override
- boolean isOperationSupported(byte op) {
- return op < FSEditLogOpCodes.OP_JSPOOL_START.getOpCode() - 1;
+ File getFile() {
+ return file;
- * Returns the file associated with this stream.
+ * @return true if this stream is currently open.
- File getFile() {
- return file;
+ public boolean isOpen() {
+ return fp != null;
@VisibleForTesting
@@ -18,9 +18,13 @@
+import java.util.zip.Checksum;
import static org.apache.hadoop.hdfs.server.common.Util.now;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Writable;
* A generic abstract class to support journaling of edits logs into
* a persistent storage.
@@ -63,9 +67,19 @@ abstract class EditLogOutputStream implements JournalStream {
abstract void create() throws IOException;
+ * Close the journal.
+ * @throws IOException if the journal can't be closed,
+ * or if there are unflushed edits
abstract public void close() throws IOException;
+ * Close the stream without necessarily flushing any pending data.
+ * This may be called after a previous write or close threw an exception.
+ abstract public void abort() throws IOException;
* All data that has been written to the stream so far will be flushed.
* New data can be still written to the stream while flushing is performed.
@@ -108,10 +122,6 @@ abstract class EditLogOutputStream implements JournalStream {
return false;
* Return total time spent in {@link #flushAndSync()}
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.OutputStream;
+import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Writer;
@@ -35,20 +36,19 @@ import com.google.common.base.Preconditions;
class EditsDoubleBuffer {
- private DataOutputBuffer bufCurrent; // current buffer for writing
- private DataOutputBuffer bufReady; // buffer ready for flushing
+ private TxnBuffer bufCurrent; // current buffer for writing
+ private TxnBuffer bufReady; // buffer ready for flushing
private final int initBufferSize;
- private Writer writer;
public EditsDoubleBuffer(int defaultBufferSize) {
initBufferSize = defaultBufferSize;
- bufCurrent = new DataOutputBuffer(initBufferSize);
- bufReady = new DataOutputBuffer(initBufferSize);
- writer = new FSEditLogOp.Writer(bufCurrent);
+ bufCurrent = new TxnBuffer(initBufferSize);
+ bufReady = new TxnBuffer(initBufferSize);
public void writeOp(FSEditLogOp op) throws IOException {
- writer.writeOp(op);
+ bufCurrent.writeOp(op);
@@ -71,10 +71,9 @@ class EditsDoubleBuffer {
void setReadyToFlush() {
assert isFlushed() : "previous data not flushed yet";
- DataOutputBuffer tmp = bufReady;
+ TxnBuffer tmp = bufReady;
bufReady = bufCurrent;
bufCurrent = tmp;
@@ -102,4 +101,50 @@ class EditsDoubleBuffer {
return bufReady.size() + bufCurrent.size();
+ * @return the transaction ID of the first transaction ready to be flushed
+ public long getFirstReadyTxId() {
+ assert bufReady.firstTxId > 0;
+ return bufReady.firstTxId;
+ * @return the number of transactions that are ready to be flushed
+ public int countReadyTxns() {
+ return bufReady.numTxns;
+ private static class TxnBuffer extends DataOutputBuffer {
+ long firstTxId;
+ int numTxns;
+ private Writer writer;
+ public TxnBuffer(int initBufferSize) {
+ super(initBufferSize);
+ writer = new FSEditLogOp.Writer(this);
+ reset();
+ public void writeOp(FSEditLogOp op) throws IOException {
+ if (firstTxId == FSConstants.INVALID_TXID) {
+ firstTxId = op.txid;
+ assert op.txid > firstTxId;
+ writer.writeOp(op);
+ numTxns++;
+ public DataOutputBuffer reset() {
+ super.reset();
+ firstTxId = FSConstants.INVALID_TXID;
+ numTxns = 0;
+ return this;
@@ -22,9 +22,7 @@ import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
@@ -157,26 +155,33 @@ public class FSDirectory implements Closeable {
return getFSNamesystem().getBlockManager();
- void loadFSImage(Collection<URI> dataDirs,
- Collection<URI> editsDirs,
- StartupOption startOpt)
+ * Load the filesystem image into memory.
+ * @param startOpt Startup type as specified by the user.
+ * @throws IOException If image or editlog cannot be read.
+ void loadFSImage(StartupOption startOpt)
throws IOException {
// format before starting up if requested
if (startOpt == StartupOption.FORMAT) {
- fsImage.getStorage().setStorageDirectories(dataDirs, editsDirs);
- fsImage.getStorage().format(fsImage.getStorage().determineClusterId()); // reuse current id
+ fsImage.format(fsImage.getStorage().determineClusterId());// reuse current id
startOpt = StartupOption.REGULAR;
+ boolean success = false;
- if (fsImage.recoverTransitionRead(dataDirs, editsDirs, startOpt)) {
- fsImage.saveNamespace(true);
+ if (fsImage.recoverTransitionRead(startOpt)) {
+ fsImage.saveNamespace();
- FSEditLog editLog = fsImage.getEditLog();
- assert editLog != null : "editLog must be initialized";
+ fsImage.openEditLog();
fsImage.setCheckpointDirectories(null, null);
- } catch(IOException e) {
- fsImage.close();
- throw e;
+ success = true;
+ if (!success) {
+ fsImage.close();
writeLock();
@@ -17,10 +17,7 @@
-import java.io.DataOutputStream;
-import java.util.ArrayList;
@@ -33,22 +30,26 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
-import org.apache.hadoop.hdfs.server.namenode.NNStorage.NNStorageListener;
-import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.util.PureJavaCrc32;
+import com.google.common.annotations.VisibleForTesting;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.*;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
@@ -58,23 +59,42 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
@InterfaceStability.Evolving
-public class FSEditLog implements NNStorageListener {
+public class FSEditLog {
static final String NO_JOURNAL_STREAMS_WARNING = "!!! WARNING !!!" +
" File system changes are not persistent. No journal streams.";
- private static final Log LOG = LogFactory.getLog(FSEditLog.class);
+ static final Log LOG = LogFactory.getLog(FSEditLog.class);
- private volatile int sizeOutputFlushBuffer = 512*1024;
+ * State machine for edit log.
+ * The log starts in UNITIALIZED state upon construction. Once it's
+ * initialized, it is usually in IN_SEGMENT state, indicating that edits
+ * may be written. In the middle of a roll, or while saving the namespace,
+ * it briefly enters the BETWEEN_LOG_SEGMENTS state, indicating that the
+ * previous segment has been closed, but the new one has not yet been opened.
+ private enum State {
+ UNINITIALIZED,
+ BETWEEN_LOG_SEGMENTS,
+ IN_SEGMENT,
+ CLOSED;
+ private State state = State.UNINITIALIZED;
- private ArrayList<EditLogOutputStream> editStreams = null;
+ private List<JournalAndStream> journals = Lists.newArrayList();
// a monotonically increasing counter that represents transactionIds.
private long txid = 0;
// stores the last synced transactionId.
private long synctxid = 0;
+ // the first txid of the log that's currently open for writing.
+ // If this value is N, we are currently writing to edits_inprogress_N
+ private long curSegmentTxId = FSConstants.INVALID_TXID;
// the time of printing the statistics to the log file.
private long lastPrintTime;
@@ -83,6 +103,10 @@ public class FSEditLog implements NNStorageListener {
// is an automatic sync scheduled?
private volatile boolean isAutoSyncScheduled = false;
+ // Used to exit in the event of a failure to sync to all journals. It's a
+ // member variable so it can be swapped out for testing.
+ private Runtime runtime = Runtime.getRuntime();
// these are statistics counters.
private long numTransactions; // number of transactions
@@ -122,226 +146,90 @@ public class FSEditLog implements NNStorageListener {
FSEditLog(NNStorage storage) {
isSyncRunning = false;
this.storage = storage;
- this.storage.registerListener(this);
metrics = NameNode.getNameNodeMetrics();
lastPrintTime = now();
- private File getEditFile(StorageDirectory sd) {
- return storage.getEditFile(sd);
- private File getEditNewFile(StorageDirectory sd) {
- return storage.getEditNewFile(sd);
- private int getNumEditsDirs() {
- return storage.getNumStorageDirs(NameNodeDirType.EDITS);
- synchronized int getNumEditStreams() {
- return editStreams == null ? 0 : editStreams.size();
- * Return the currently active edit streams.
- * This should be used only by unit tests.
+ * Initialize the list of edit journals
- ArrayList<EditLogOutputStream> getEditStreams() {
- return editStreams;
- boolean isOpen() {
- return getNumEditStreams() > 0;
- * Create empty edit log files.
- * Initialize the output stream for logging.
- synchronized void open() throws IOException {
- numTransactions = totalTimeTransactions = numTransactionsBatchedInSync = 0;
- if (editStreams == null)
- editStreams = new ArrayList<EditLogOutputStream>();
+ synchronized void initJournals() {
+ assert journals.isEmpty();
+ Preconditions.checkState(state == State.UNINITIALIZED,
+ "Bad state: %s", state);
- ArrayList<StorageDirectory> al = null;
- File eFile = getEditFile(sd);
- addNewEditLogStream(eFile);
- } catch (IOException e) {
- LOG.warn("Unable to open edit log file " + eFile);
- // Remove the directory from list of storage directories
- if(al == null) al = new ArrayList<StorageDirectory>(1);
- al.add(sd);
+ for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.EDITS)) {
+ journals.add(new JournalAndStream(new FileJournalManager(sd)));
- if (al != null)
- storage.reportErrorsOnDirectories(al);
- // If there was an error in every storage dir, each one will have
- // been removed from the list of storage directories.
- if (storage.getNumStorageDirs(NameNodeDirType.EDITS) == 0) {
- throw new IOException(
- "Failed to initialize edits log in any storage directory.");
+ if (journals.isEmpty()) {
+ LOG.error("No edits directories configured!");
+ state = State.BETWEEN_LOG_SEGMENTS;
- synchronized void addNewEditLogStream(File eFile) throws IOException {
- EditLogOutputStream eStream = new EditLogFileOutputStream(eFile,
- sizeOutputFlushBuffer);
- editStreams.add(eStream);
- synchronized void createEditLogFile(File name) throws IOException {
- waitForSyncToFinish();
- EditLogOutputStream eStream = new EditLogFileOutputStream(name,
- eStream.create();
- eStream.close();
- * Shutdown the file store.
+ * Initialize the output stream for logging, opening the first
+ * log segment.
- synchronized void close() {
- if (editStreams == null || editStreams.isEmpty()) {
- printStatistics(true);
- ArrayList<EditLogOutputStream> errorStreams = null;
- Iterator<EditLogOutputStream> it = getOutputStreamIterator(null);
- while(it.hasNext()) {
- EditLogOutputStream eStream = it.next();
- closeStream(eStream);
- LOG.warn("FSEditLog:close - failed to close stream "
- + eStream.getName());
- if(errorStreams == null)
- errorStreams = new ArrayList<EditLogOutputStream>(1);
- errorStreams.add(eStream);
- disableAndReportErrorOnStreams(errorStreams);
- editStreams.clear();
+ synchronized void open() throws IOException {
+ Preconditions.checkState(state == State.UNINITIALIZED);
+ initJournals();
- * Close and remove edit log stream.
- * @param index of the stream
- synchronized private void removeStream(int index) {
- EditLogOutputStream eStream = editStreams.get(index);
- } catch (Exception e) {}
- editStreams.remove(index);
+ startLogSegment(getLastWrittenTxId() + 1, true);
+ assert state == State.IN_SEGMENT : "Bad state: " + state;
- * The specified streams have IO errors. Close and remove them.
- synchronized
- void disableAndReportErrorOnStreams(List<EditLogOutputStream> errorStreams) {
- if (errorStreams == null || errorStreams.size() == 0) {
- return; // nothing to do
- ArrayList<StorageDirectory> errorDirs = new ArrayList<StorageDirectory>();
- for (EditLogOutputStream e : errorStreams) {
- if (e.getType() == JournalType.FILE) {
- errorDirs.add(getStorageDirectoryForStream(e));
- } else {
- disableStream(e);
- storage.reportErrorsOnDirectories(errorDirs);
- } catch (IOException ioe) {
- LOG.error("Problem erroring streams " + ioe);
+ synchronized boolean isOpen() {
+ return state == State.IN_SEGMENT;
- * get an editStream corresponding to a sd
- * @param es - stream to remove
- * @return the matching stream
+ * Shutdown the file store.
- StorageDirectory getStorage(EditLogOutputStream es) {
- String parentStorageDir = ((EditLogFileOutputStream)es).getFile()
- .getParentFile().getParentFile().getAbsolutePath();
- Iterator<StorageDirectory> it = storage.dirIterator();
- while (it.hasNext()) {
- LOG.info("comparing: " + parentStorageDir + " and " + sd.getRoot().getAbsolutePath());
- if (parentStorageDir.equals(sd.getRoot().getAbsolutePath()))
- return sd;
+ synchronized void close() {
+ if (state == State.CLOSED) {
+ LOG.warn("Closing log when already closed", new Exception());
- return null;
- * @param sd
- synchronized EditLogOutputStream getEditsStream(StorageDirectory sd) {
- for (EditLogOutputStream es : editStreams) {
- File parentStorageDir = ((EditLogFileOutputStream)es).getFile()
- .getParentFile().getParentFile();
- if (parentStorageDir.getName().equals(sd.getRoot().getName()))
- return es;
+ if (state == State.IN_SEGMENT) {
+ assert !journals.isEmpty();
+ waitForSyncToFinish();
+ endCurrentLogSegment(true);
- * check if edits.new log exists in the specified stoorage directory
- boolean existsNew(StorageDirectory sd) {
- return getEditNewFile(sd).exists();
+ state = State.CLOSED;
* Write an operation to the edit log. Do not sync to persistent
* store yet.
- void logEdit(FSEditLogOp op) {
+ void logEdit(final FSEditLogOp op) {
synchronized (this) {
+ assert state != State.CLOSED;
// wait if an automatic sync is scheduled
waitIfAutoSyncScheduled();
- if(getNumEditStreams() == 0)
throw new java.lang.IllegalStateException(NO_JOURNAL_STREAMS_WARNING);
- long start = now();
- for(EditLogOutputStream eStream : editStreams) {
- if(!eStream.isOperationSupported(op.opCode.getOpCode()))
- continue;
- eStream.write(op);
- } catch (IOException ie) {
- LOG.error("logEdit: removing "+ eStream.getName(), ie);
- recordTransaction(start);
+ long start = beginTransaction();
+ op.setTransactionId(txid);
+ mapJournalsAndReportErrors(new JournalClosure() {
+ public void apply(JournalAndStream jas) throws IOException {
+ if (!jas.isActive()) return;
+ jas.stream.write(op);
+ }, "logging edit");
+ endTransaction(start);
// check if it is time to schedule an automatic sync
if (!shouldForceSync()) {
@@ -384,15 +272,18 @@ public class FSEditLog implements NNStorageListener {
* @return true if any of the edit stream says that it should sync
private boolean shouldForceSync() {
- for (EditLogOutputStream eStream : editStreams) {
- if (eStream.shouldForceSync()) {
+ for (JournalAndStream jas : journals) {
+ if (!jas.isActive()) continue;
+ if (jas.getCurrentStream().shouldForceSync()) {
return true;
- private void recordTransaction(long start) {
+ private long beginTransaction() {
+ assert Thread.holdsLock(this);
// get a new transactionId
txid++;
@@ -401,7 +292,12 @@ public class FSEditLog implements NNStorageListener {
TransactionId id = myTransactionId.get();
id.txid = txid;
+ return now();
+ private void endTransaction(long start) {
// update statistics
long end = now();
numTransactions++;
@@ -410,6 +306,35 @@ public class FSEditLog implements NNStorageListener {
metrics.addTransaction(end-start);
+ * Return the transaction ID of the last transaction written to the log.
+ synchronized long getLastWrittenTxId() {
+ return txid;
+ * @return the first transaction ID in the current log segment
+ synchronized long getCurSegmentTxId() {
+ Preconditions.checkState(state == State.IN_SEGMENT,
+ return curSegmentTxId;
+ * Set the transaction ID to use for the next transaction written.
+ synchronized void setNextTxId(long nextTxId) {
+ Preconditions.checkArgument(synctxid <= txid &&
+ nextTxId >= txid,
+ "May not decrease txid." +
+ " synctxid=%s txid=%s nextTxId=%s",
+ synctxid, txid, nextTxId);
+ txid = nextTxId - 1;
* Blocks until all ongoing edits have been synced to disk.
* This differs from logSync in that it waits for edits that have been
@@ -457,12 +382,15 @@ public class FSEditLog implements NNStorageListener {
* waitForSyncToFinish() before assuming they are running alone.
public void logSync() {
long syncStart = 0;
// Fetch the transactionId of this thread.
long mytxid = myTransactionId.get().txid;
- ArrayList<EditLogOutputStream> streams = new ArrayList<EditLogOutputStream>();
+ List<JournalAndStream> candidateJournals =
+ Lists.newArrayListWithCapacity(journals.size());
+ List<JournalAndStream> badJournals = Lists.newArrayList();
boolean sync = false;
@@ -493,20 +421,16 @@ public class FSEditLog implements NNStorageListener {
sync = true;
// swap buffers
- assert editStreams.size() > 0 : "no editlog streams";
+ assert !journals.isEmpty() : "no editlog streams";
- eStream.setReadyToFlush();
- streams.add(eStream);
+ jas.getCurrentStream().setReadyToFlush();
+ candidateJournals.add(jas);
} catch (IOException ie) {
LOG.error("Unable to get ready to flush.", ie);
- //
- // remember the streams that encountered an error.
- if (errorStreams == null) {
+ badJournals.add(jas);
@@ -517,29 +441,36 @@ public class FSEditLog implements NNStorageListener {
// do the sync
long start = now();
- for (EditLogOutputStream eStream : streams) {
+ for (JournalAndStream jas : candidateJournals) {
- eStream.flush();
+ jas.getCurrentStream().flush();
LOG.error("Unable to sync edit log.", ie);
// remember the streams that encountered an error.
long elapsed = now() - start;
+ disableAndReportErrorOnJournals(badJournals);
- if (metrics != null) // Metrics non-null only when used inside name node
+ if (metrics != null) { // Metrics non-null only when used inside name node
metrics.addSync(elapsed);
// Prevent RuntimeException from blocking other log edit sync
if (sync) {
+ if (badJournals.size() >= journals.size()) {
+ LOG.fatal("Could not sync any journal to persistent storage. " +
+ "Unsynced transactions: " + (txid - synctxid),
+ new Exception());
+ runtime.exit(1);
synctxid = syncStart;
@@ -556,7 +487,7 @@ public class FSEditLog implements NNStorageListener {
if (lastPrintTime + 60000 > now && !force) {
- if (editStreams == null || editStreams.size()==0) {
lastPrintTime = now;
@@ -568,12 +499,17 @@ public class FSEditLog implements NNStorageListener {
buf.append("Number of transactions batched in Syncs: ");
buf.append(numTransactionsBatchedInSync);
buf.append(" Number of syncs: ");
- buf.append(editStreams.get(0).getNumSync());
+ buf.append(jas.getCurrentStream().getNumSync());
buf.append(" SyncTimes(ms): ");
- int numEditStreams = editStreams.size();
- for (int idx = 0; idx < numEditStreams; idx++) {
- EditLogOutputStream eStream = editStreams.get(idx);
+ EditLogOutputStream eStream = jas.getCurrentStream();
buf.append(eStream.getTotalSyncTime());
buf.append(" ");
@@ -788,202 +724,191 @@ public class FSEditLog implements NNStorageListener {
- * Return the size of the current EditLog
+ * @return the number of active (non-failed) journals
- synchronized long getEditLogSize() throws IOException {
- assert getNumEditsDirs() <= getNumEditStreams() :
- "Number of edits directories should not exceed the number of streams.";
- long size = 0;
- ArrayList<EditLogOutputStream> al = null;
- for (int idx = 0; idx < getNumEditStreams(); idx++) {
- EditLogOutputStream es = editStreams.get(idx);
- long curSize = es.length();
- assert (size == 0 || size == curSize || curSize ==0) :
- "Wrong streams size";
- size = Math.max(size, curSize);
- LOG.error("getEditLogSize: editstream.length failed. removing editlog (" +
- idx + ") " + es.getName());
- if(al==null) al = new ArrayList<EditLogOutputStream>(1);
- al.add(es);
+ private int countActiveJournals() {
+ int count = 0;
+ if (jas.isActive()) {
+ count++;
- if(al!=null) disableAndReportErrorOnStreams(al);
- return size;
+ return count;
- * Closes the current edit log and opens edits.new.
+ * Used only by unit tests.
- synchronized void rollEditLog() throws IOException {
- Iterator<StorageDirectory> it = storage.dirIterator(NameNodeDirType.EDITS);
- if(!it.hasNext())
- // If edits.new already exists in some directory, verify it
- // exists in all directories.
- boolean alreadyExists = existsNew(it.next());
- if(alreadyExists != existsNew(sd))
- throw new IOException(getEditNewFile(sd)
- + "should " + (alreadyExists ? "" : "not ") + "exist.");
- if(alreadyExists)
- return; // nothing to do, edits.new exists!
- // check if any of failed storage is now available and put it back
- storage.attemptRestoreRemovedStorage();
- divertFileStreams(
- Storage.STORAGE_DIR_CURRENT + "/" + NameNodeFile.EDITS_NEW.getName());
+ @VisibleForTesting
+ List<JournalAndStream> getJournals() {
+ return journals;
- * Divert file streams from file edits to file edits.new.<p>
- * Close file streams, which are currently writing into edits files.
- * Create new streams based on file getRoot()/dest.
- * @param dest new stream path relative to the storage directory root.
- synchronized void divertFileStreams(String dest) throws IOException {
- assert getNumEditStreams() >= getNumEditsDirs() :
- "Inconsistent number of streams";
- EditStreamIterator itE =
- (EditStreamIterator)getOutputStreamIterator(JournalType.FILE);
- Iterator<StorageDirectory> itD =
- storage.dirIterator(NameNodeDirType.EDITS);
- while(itE.hasNext() && itD.hasNext()) {
- EditLogOutputStream eStream = itE.next();
- StorageDirectory sd = itD.next();
- if(!eStream.getName().startsWith(sd.getRoot().getPath()))
- throw new IOException("Inconsistent order of edit streams: " + eStream);
- // close old stream
- // create new stream
- eStream = new EditLogFileOutputStream(new File(sd.getRoot(), dest),
- // replace by the new stream
- itE.replace(eStream);
- LOG.warn("Error in editStream " + eStream.getName(), e);
+ synchronized void setRuntimeForTesting(Runtime runtime) {
+ this.runtime = runtime;
+ * Return a manifest of what finalized edit logs are available
+ public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
+ inspector.inspectDirectory(sd);
+ return inspector.getEditLogManifest(sinceTxId);
- * Removes the old edit log and renames edits.new to edits.
- * Reopens the edits file.
+ * Finalizes the current edit log and opens a new log segment.
+ * @return the transaction id of the BEGIN_LOG_SEGMENT transaction
+ * in the new log.
- synchronized void purgeEditLog() throws IOException {
- revertFileStreams(
+ synchronized long rollEditLog() throws IOException {
+ LOG.info("Rolling edit logs.");
+ long nextTxId = getLastWrittenTxId() + 1;
+ startLogSegment(nextTxId, true);
+ assert curSegmentTxId == nextTxId;
+ return nextTxId;
- * The actual sync activity happens while not synchronized on this object.
- * Thus, synchronized activities that require that they are not concurrent
- * with file operations should wait for any running sync to finish.
+ * Start writing to the log segment with the given txid.
+ * Transitions from BETWEEN_LOG_SEGMENTS state to IN_LOG_SEGMENT state.
- synchronized void waitForSyncToFinish() {
- while (isSyncRunning) {
- wait(1000);
- } catch (InterruptedException ie) {}
+ synchronized void startLogSegment(final long segmentTxId,
+ boolean writeHeaderTxn) throws IOException {
+ LOG.info("Starting log segment at " + segmentTxId);
+ Preconditions.checkArgument(segmentTxId > 0,
+ "Bad txid: %s", segmentTxId);
+ Preconditions.checkState(state == State.BETWEEN_LOG_SEGMENTS,
+ Preconditions.checkState(segmentTxId > curSegmentTxId,
+ "Cannot start writing to log segment " + segmentTxId +
+ " when previous log segment started at " + curSegmentTxId);
+ Preconditions.checkArgument(segmentTxId == txid + 1,
+ "Cannot start log segment at txid %s when next expected " +
+ "txid is %s", segmentTxId, txid + 1);
+ numTransactions = totalTimeTransactions = numTransactionsBatchedInSync = 0;
+ // TODO no need to link this back to storage anymore!
+ // See HDFS-2174.
+ storage.attemptRestoreRemovedStorage();
+ jas.startLogSegment(segmentTxId);
+ }, "starting log segment " + segmentTxId);
+ if (countActiveJournals() == 0) {
+ throw new IOException("Unable to start log segment " +
+ segmentTxId + ": no journals successfully started.");
+ curSegmentTxId = segmentTxId;
+ state = State.IN_SEGMENT;
+ if (writeHeaderTxn) {
+ logEdit(LogSegmentOp.getInstance(
+ FSEditLogOpCodes.OP_START_LOG_SEGMENT));
+ logSync();
- * Revert file streams from file edits.new back to file edits.<p>
- * Close file streams, which are currently writing into getRoot()/source.
- * Rename getRoot()/source to edits.
- * Reopen streams so that they start writing into edits files.
+ * Finalize the current log segment.
+ * Transitions from IN_SEGMENT state to BETWEEN_LOG_SEGMENTS state.
- synchronized void revertFileStreams(String source) throws IOException {
- throw new IOException("Inconsistent order of edit streams: " + eStream +
- " does not start with " + sd.getRoot().getPath());
- // rename edits.new to edits
- File editFile = getEditFile(sd);
- File prevEditFile = new File(sd.getRoot(), source);
- if(prevEditFile.exists()) {
- if(!prevEditFile.renameTo(editFile)) {
- // renameTo() fails on Windows if the destination
- // file exists.
- if(!editFile.delete() || !prevEditFile.renameTo(editFile)) {
- throw new IOException("Rename failed for " + sd.getRoot());
+ synchronized void endCurrentLogSegment(boolean writeEndTxn) {
+ LOG.info("Ending log segment " + curSegmentTxId);
+ if (writeEndTxn) {
+ FSEditLogOpCodes.OP_END_LOG_SEGMENT));
+ printStatistics(true);
+ final long lastTxId = getLastWrittenTxId();
+ jas.close(lastTxId);
- // open new stream
- eStream = new EditLogFileOutputStream(editFile, sizeOutputFlushBuffer);
+ }, "ending log segment");
+ * Abort all current logs. Called from the backup node.
+ synchronized void abortCurrentLogSegment() {
+ jas.abort();
+ }, "aborting all streams");
- * Return the name of the edit file
+ * Archive any log files that are older than the given txid.
- synchronized File getFsEditName() {
- StorageDirectory sd = null;
- for (Iterator<StorageDirectory> it =
- storage.dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
- sd = it.next();
- if(sd.getRoot().canRead())
- return getEditFile(sd);
+ public void purgeLogsOlderThan(
+ final long minTxIdToKeep, final StoragePurger purger) {
+ // synchronized to prevent findbugs warning about inconsistent
+ // synchronization. This will be JIT-ed out if asserts are
+ // off.
+ assert curSegmentTxId == FSConstants.INVALID_TXID || // on format this is no-op
+ minTxIdToKeep <= curSegmentTxId :
+ "cannot purge logs older than txid " + minTxIdToKeep +
+ " when current segment starts at " + curSegmentTxId;
+ jas.manager.purgeLogsOlderThan(minTxIdToKeep, purger);
+ }, "purging logs older than " + minTxIdToKeep);
- * Returns the timestamp of the edit log
+ * The actual sync activity happens while not synchronized on this object.
+ * Thus, synchronized activities that require that they are not concurrent
+ * with file operations should wait for any running sync to finish.
- synchronized long getFsEditTime() {
- if(it.hasNext())
- return getEditFile(it.next()).lastModified();
- return 0;
+ synchronized void waitForSyncToFinish() {
+ while (isSyncRunning) {
+ wait(1000);
+ } catch (InterruptedException ie) {}
@@ -996,273 +921,224 @@ public class FSEditLog implements NNStorageListener {
// sets the initial capacity of the flush buffer.
- public void setBufferCapacity(int size) {
- sizeOutputFlushBuffer = size;
- boolean isEmpty() throws IOException {
- return getEditLogSize() <= 0;
+ jas.manager.setOutputBufferCapacity(size);
* Create (or find if already exists) an edit output stream, which
* streams journal records (edits) to the specified backup node.<br>
- * Send a record, prescribing to start journal spool.<br>
- * This should be sent via regular stream of journal records so that
- * the backup node new exactly after which record it should start spooling.
+ * The new BackupNode will start receiving edits the next time this
+ * NameNode's logs roll.
* @param bnReg the backup node registration information.
* @param nnReg this (active) name-node registration.
- synchronized void logJSpoolStart(NamenodeRegistration bnReg, // backup node
- NamenodeRegistration nnReg) // active name-node
+ synchronized void registerBackupNode(
+ NamenodeRegistration bnReg, // backup node
+ NamenodeRegistration nnReg) // active name-node
if(bnReg.isRole(NamenodeRole.CHECKPOINT))
return; // checkpoint node does not stream edits
- if(editStreams == null)
- EditLogOutputStream boStream = null;
- if(eStream.getName().equals(bnReg.getAddress())) {
- boStream = eStream; // already there
- if(boStream == null) {
- boStream = new EditLogBackupOutputStream(bnReg, nnReg);
- editStreams.add(boStream);
+ JournalAndStream jas = findBackupJournalAndStream(bnReg);
+ if (jas != null) {
+ // already registered
+ LOG.info("Backup node " + bnReg + " re-registers");
- logEdit(JSpoolStartOp.getInstance());
+ LOG.info("Registering new backup node: " + bnReg);
+ BackupJournalManager bjm = new BackupJournalManager(bnReg, nnReg);
+ journals.add(new JournalAndStream(bjm));
- * Write an operation to the edit log. Do not sync to persistent
- * store yet.
- synchronized void logEdit(int length, byte[] data) {
- throw new java.lang.IllegalStateException(NO_JOURNAL_STREAMS_WARNING);
- eStream.writeRaw(data, 0, length);
- LOG.warn("Error in editStream " + eStream.getName(), ie);
+ synchronized void releaseBackupStream(NamenodeRegistration registration) {
+ for (Iterator<JournalAndStream> iter = journals.iterator();
+ iter.hasNext();) {
+ JournalAndStream jas = iter.next();
+ if (jas.manager instanceof BackupJournalManager &&
+ ((BackupJournalManager)jas.manager).matchesRegistration(
+ registration)) {
+ LOG.info("Removing backup journal " + jas);
+ iter.remove();
- * Iterates output streams based of the same type.
- * Type null will iterate over all streams.
+ * Find the JournalAndStream associated with this BackupNode.
+ * @return null if it cannot be found
- private class EditStreamIterator implements Iterator<EditLogOutputStream> {
- JournalType type;
- int prevIndex; // for remove()
- int nextIndex; // for next()
- EditStreamIterator(JournalType streamType) {
- this.type = streamType;
- this.nextIndex = 0;
- this.prevIndex = 0;
- public boolean hasNext() {
- synchronized(FSEditLog.this) {
- if(editStreams == null ||
- editStreams.isEmpty() || nextIndex >= editStreams.size())
- while(nextIndex < editStreams.size()
- && !editStreams.get(nextIndex).getType().isOfType(type))
- nextIndex++;
- return nextIndex < editStreams.size();
- public EditLogOutputStream next() {
- EditLogOutputStream stream = null;
- stream = editStreams.get(nextIndex);
- prevIndex = nextIndex;
- return stream;
- public void remove() {
- nextIndex = prevIndex; // restore previous state
- removeStream(prevIndex); // remove last returned element
- hasNext(); // reset nextIndex to correct place
- void replace(EditLogOutputStream newStream) {
- synchronized (FSEditLog.this) {
- assert 0 <= prevIndex && prevIndex < editStreams.size() :
- "Index out of bound.";
- editStreams.set(prevIndex, newStream);
+ private synchronized JournalAndStream findBackupJournalAndStream(
+ NamenodeRegistration bnReg) {
+ if (jas.manager instanceof BackupJournalManager) {
+ BackupJournalManager bjm = (BackupJournalManager)jas.manager;
+ if (bjm.matchesRegistration(bnReg)) {
+ return jas;
- * Get stream iterator for the specified type.
- public Iterator<EditLogOutputStream>
- getOutputStreamIterator(JournalType streamType) {
- return new EditStreamIterator(streamType);
+ * Write an operation to the edit log. Do not sync to persistent
+ * store yet.
+ synchronized void logEdit(final int length, final byte[] data) {
+ jas.getCurrentStream().writeRaw(data, 0, length); // TODO writeRaw
+ }, "Logging edit");
- private void closeStream(EditLogOutputStream eStream) throws IOException {
- void incrementCheckpointTime() {
- storage.incrementCheckpointTime();
- CheckpointTimeOp op = CheckpointTimeOp.getInstance()
- .setCheckpointTime(storage.getCheckpointTime());
- logEdit(op);
+ //// Iteration across journals
+ private interface JournalClosure {
+ public void apply(JournalAndStream jas) throws IOException;
- synchronized void releaseBackupStream(NamenodeRegistration registration) {
- Iterator<EditLogOutputStream> it =
- getOutputStreamIterator(JournalType.BACKUP);
- NamenodeRegistration backupNode = null;
- EditLogBackupOutputStream eStream = (EditLogBackupOutputStream)it.next();
- backupNode = eStream.getRegistration();
- if(backupNode.getAddress().equals(registration.getAddress()) &&
- backupNode.isRole(registration.getRole())) {
+ * Apply the given function across all of the journal managers, disabling
+ * any for which the closure throws an IOException.
+ * @param status message used for logging errors (e.g. "opening journal")
+ private void mapJournalsAndReportErrors(
+ JournalClosure closure, String status) {
+ List<JournalAndStream> badJAS = Lists.newLinkedList();
+ closure.apply(jas);
+ } catch (Throwable t) {
+ LOG.error("Error " + status + " (journal " + jas + ")", t);
+ badJAS.add(jas);
- assert backupNode == null || backupNode.isRole(NamenodeRole.BACKUP) :
- "Not a backup node corresponds to a backup stream";
- synchronized boolean checkBackupRegistration(
- NamenodeRegistration registration) {
- boolean regAllowed = !it.hasNext();
- regAllowed = true; // same node re-registers
- if(!eStream.isAlive()) {
- regAllowed = true; // previous backup node failed
- return regAllowed;
+ disableAndReportErrorOnJournals(badJAS);
- * Get the StorageDirectory for a stream
- * @param es Stream whose StorageDirectory we wish to know
- * @return the matching StorageDirectory
+ * Called when some journals experience an error in some operation.
+ * This propagates errors to the storage level.
- StorageDirectory getStorageDirectoryForStream(EditLogOutputStream es) {
- String parentStorageDir = ((EditLogFileOutputStream)es).getFile().getParentFile().getParentFile().getAbsolutePath();
- FSNamesystem.LOG.info("comparing: " + parentStorageDir
- + " and " + sd.getRoot().getAbsolutePath());
- private synchronized void disableStream(EditLogOutputStream stream) {
- try { stream.close(); } catch (IOException e) {
- // nothing to do.
- LOG.warn("Failed to close eStream " + stream.getName()
- + " before removing it (might be ok)");
+ private void disableAndReportErrorOnJournals(List<JournalAndStream> badJournals) {
+ if (badJournals == null || badJournals.isEmpty()) {
+ return; // nothing to do
- editStreams.remove(stream);
- if (editStreams.size() <= 0) {
- String msg = "Fatal Error: All storage directories are inaccessible.";
- LOG.fatal(msg, new IOException(msg));
- Runtime.getRuntime().exit(-1);
+ for (JournalAndStream j : badJournals) {
+ LOG.error("Disabling journal " + j);
+ j.abort();
- * Error Handling on a storageDirectory
+ * Container for a JournalManager paired with its currently
+ * active stream.
+ * If a Journal gets disabled due to an error writing to its
+ * stream, then the stream will be aborted and set to null.
- // NNStorageListener Interface
- @Override // NNStorageListener
- public synchronized void errorOccurred(StorageDirectory sd)
- if (editStreams == null) {
- //errors can occur on storage directories
- //before edit streams have been set up
+ static class JournalAndStream {
+ private final JournalManager manager;
+ private EditLogOutputStream stream;
+ private long segmentStartsAtTxId = FSConstants.INVALID_TXID;
+ private JournalAndStream(JournalManager manager) {
+ this.manager = manager;
- ArrayList<EditLogOutputStream> errorStreams
- = new ArrayList<EditLogOutputStream>();
- LOG.error("Unable to log edits to " + eStream.getName()
- + "; removing it");
+ private void startLogSegment(long txId) throws IOException {
+ Preconditions.checkState(stream == null);
+ stream = manager.startLogSegment(txId);
+ segmentStartsAtTxId = txId;
- StorageDirectory streamStorageDir = getStorageDirectoryForStream(eStream);
- if (sd == streamStorageDir) {
+ private void close(long lastTxId) throws IOException {
+ Preconditions.checkArgument(lastTxId >= segmentStartsAtTxId,
+ "invalid segment: lastTxId %s >= " +
+ "segment starting txid %s", lastTxId, segmentStartsAtTxId);
+ if (stream == null) return;
+ stream.close();
+ manager.finalizeLogSegment(segmentStartsAtTxId, lastTxId);
+ stream = null;
+ private void abort() {
+ stream.abort();
+ } catch (IOException ioe) {
+ LOG.error("Unable to abort stream " + stream, ioe);
+ segmentStartsAtTxId = FSConstants.INVALID_TXID;
- for (EditLogOutputStream eStream : errorStreams) {
- disableStream(eStream);
+ private boolean isActive() {
+ return stream != null;
- public synchronized void formatOccurred(StorageDirectory sd)
- if (sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
- createEditLogFile(NNStorage.getStorageFile(sd, NameNodeFile.EDITS));
+ EditLogOutputStream getCurrentStream() {
+ return stream;
- };
+ return "JournalAndStream(mgr=" + manager +
+ ", " + "stream=" + stream + ")";
+ void setCurrentStreamForTests(EditLogOutputStream stream) {
+ this.stream = stream;
+ JournalManager getManager() {
+ return manager;
+ private EditLogInputStream getInProgressInputStream() throws IOException {
+ return manager.getInProgressInputStream(segmentStartsAtTxId);
- public synchronized void directoryAvailable(StorageDirectory sd)
+ * @return an EditLogInputStream that reads from the same log that
+ * the edit log is currently writing. This is used from the BackupNode
+ * during edits synchronization.
+ * @throws IOException if no valid logs are available.
+ synchronized EditLogInputStream getInProgressFileInputStream()
+ EditLogInputStream in = jas.getInProgressInputStream();
+ if (in != null) return in;
+ LOG.warn("Unable to get the in-progress input stream from " + jas,
+ ioe);
+ throw new IOException("No in-progress stream provided edits");
@@ -21,12 +21,12 @@ import static org.apache.hadoop.hdfs.server.common.Util.now;
-import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
import java.io.FilterInputStream;
import java.io.InputStream;
import java.util.Arrays;
import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -37,6 +37,8 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogHeader;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Reader;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
@@ -71,75 +73,42 @@ public class FSEditLogLoader {
* This is where we apply edits that we've been writing to disk all
* along.
- int loadFSEdits(EditLogInputStream edits) throws IOException {
+ int loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
- int numEdits = loadFSEdits(edits, true);
+ int numEdits = loadFSEdits(edits, true, expectedStartingTxId);
FSImage.LOG.info("Edits file " + edits.getName()
+ " of size " + edits.length() + " edits # " + numEdits
+ " loaded in " + (now()-startTime)/1000 + " seconds.");
return numEdits;
- * Read the header of fsedit log
- * @param in fsedit stream
- * @return the edit log version number
- * @throws IOException if error occurs
- int readLogVersion(DataInputStream in) throws IOException {
- int logVersion = 0;
- // Read log file version. Could be missing.
- in.mark(4);
- // If edits log is greater than 2G, available method will return negative
- // numbers, so we avoid having to call available
- boolean available = true;
- logVersion = in.readByte();
- } catch (EOFException e) {
- available = false;
- if (available) {
- in.reset();
- logVersion = in.readInt();
- if (logVersion < FSConstants.LAYOUT_VERSION) // future version
- "Unexpected version of the file system log file: "
- + logVersion + ". Current version = "
- + FSConstants.LAYOUT_VERSION + ".");
- assert logVersion <= Storage.LAST_UPGRADABLE_LAYOUT_VERSION :
- "Unsupported version " + logVersion;
- return logVersion;
- int loadFSEdits(EditLogInputStream edits, boolean closeOnExit) throws IOException {
+ int loadFSEdits(EditLogInputStream edits, boolean closeOnExit,
+ long expectedStartingTxId)
BufferedInputStream bin = new BufferedInputStream(edits);
DataInputStream in = new DataInputStream(bin);
int numEdits = 0;
- logVersion = readLogVersion(in);
- numEdits = loadEditRecords(logVersion, in, checksum, false);
+ LogHeader header = LogHeader.read(in);
+ numEdits = loadEditRecords(
+ header.logVersion, in, header.checksum, false,
+ expectedStartingTxId);
if(closeOnExit)
in.close();
- if (logVersion != FSConstants.LAYOUT_VERSION) // other version
- numEdits++; // save this image asap
@SuppressWarnings("deprecation")
int loadEditRecords(int logVersion, DataInputStream in,
- Checksum checksum, boolean closeOnExit) throws IOException {
+ Checksum checksum, boolean closeOnExit,
FSDirectory fsDir = fsNamesys.dir;
@@ -162,6 +131,8 @@ public class FSEditLogLoader {
Arrays.fill(recentOpcodeOffsets, -1);
+ long txId = expectedStartingTxId - 1;
FSEditLogOp.Reader reader = new FSEditLogOp.Reader(in, logVersion,
checksum);
@@ -169,6 +140,15 @@ public class FSEditLogLoader {
while ((op = reader.readOp()) != null) {
recentOpcodeOffsets[numEdits % recentOpcodeOffsets.length] =
tracker.getPos();
+ if (LayoutVersion.supports(Feature.STORED_TXIDS, logVersion)) {
+ long thisTxId = op.txid;
+ if (thisTxId != txId + 1) {
+ throw new IOException("Expected transaction ID " +
+ (txId + 1) + " but got " + thisTxId);
+ txId = thisTxId;
numEdits++;
switch (op.opCode) {
case OP_ADD:
@@ -417,6 +397,12 @@ public class FSEditLogLoader {
reassignLeaseOp.path, reassignLeaseOp.newHolder, pendingFile);
break;
+ case OP_START_LOG_SEGMENT:
+ case OP_END_LOG_SEGMENT: {
+ // no data in here currently.
+ numOpOther++;
case OP_DATANODE_ADD:
case OP_DATANODE_REMOVE:
numOpOther++;
@@ -494,6 +480,61 @@ public class FSEditLogLoader {
+ * Return the number of valid transactions in the file. If the file is
+ * truncated during the header, returns a value indicating that there are
+ * 0 valid transactions.
+ * @throws IOException if the file cannot be read due to an IO error (eg
+ * if the log does not exist)
+ static EditLogValidation validateEditLog(File f) throws IOException {
+ FileInputStream fis = new FileInputStream(f);
+ PositionTrackingInputStream tracker = new PositionTrackingInputStream(
+ new BufferedInputStream(fis));
+ DataInputStream dis = new DataInputStream(tracker);
+ LogHeader header;
+ header = LogHeader.read(dis);
+ FSImage.LOG.debug("Unable to read header from " + f +
+ " -> no valid transactions in this file.");
+ return new EditLogValidation(0, 0);
+ Reader reader = new FSEditLogOp.Reader(dis, header.logVersion, header.checksum);
+ long numValid = 0;
+ long lastPos = 0;
+ while (true) {
+ lastPos = tracker.getPos();
+ if (reader.readOp() == null) {
+ numValid++;
+ // Catch Throwable and not just IOE, since bad edits may generate
+ // NumberFormatExceptions, AssertionErrors, OutOfMemoryErrors, etc.
+ FSImage.LOG.debug("Caught exception after reading " + numValid +
+ " ops from " + f + " while determining its valid length.", t);
+ return new EditLogValidation(lastPos, numValid);
+ fis.close();
+ static class EditLogValidation {
+ long validLength;
+ long numTransactions;
+ EditLogValidation(long validLength, long numTransactions) {
+ this.validLength = validLength;
+ this.numTransactions = numTransactions;
* Stream wrapper that keeps track of the current file position.
@@ -17,6 +17,7 @@
+import java.util.zip.CheckedInputStream;
import java.util.EnumMap;
@@ -29,10 +30,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.common.GenerationStamp;
@@ -61,6 +64,8 @@ import java.io.EOFException;
@InterfaceStability.Unstable
public abstract class FSEditLogOp {
final FSEditLogOpCodes opCode;
+ long txid;
private static ThreadLocal<EnumMap<FSEditLogOpCodes, FSEditLogOp>> opInstances =
@@ -93,8 +98,10 @@ public abstract class FSEditLogOp {
instances.put(OP_CANCEL_DELEGATION_TOKEN,
new CancelDelegationTokenOp());
instances.put(OP_UPDATE_MASTER_KEY, new UpdateMasterKeyOp());
- instances.put(OP_CHECKPOINT_TIME, new CheckpointTimeOp());
- instances.put(OP_JSPOOL_START, new JSpoolStartOp());
+ instances.put(OP_START_LOG_SEGMENT,
+ new LogSegmentOp(OP_START_LOG_SEGMENT));
+ instances.put(OP_END_LOG_SEGMENT,
+ new LogSegmentOp(OP_END_LOG_SEGMENT));
return instances;
};
@@ -105,6 +112,11 @@ public abstract class FSEditLogOp {
private FSEditLogOp(FSEditLogOpCodes opCode) {
this.opCode = opCode;
+ this.txid = 0;
+ public void setTransactionId(long txid) {
+ this.txid = txid;
abstract void readFields(DataInputStream in, int logVersion)
@@ -1207,72 +1219,46 @@ public abstract class FSEditLogOp {
this.key.readFields(in);
- static class InvalidOp extends FSEditLogOp {
- private InvalidOp() {
- super(OP_INVALID);
+ static class LogSegmentOp extends FSEditLogOp {
+ private LogSegmentOp(FSEditLogOpCodes code) {
+ super(code);
+ assert code == OP_START_LOG_SEGMENT ||
+ code == OP_END_LOG_SEGMENT : "Bad op: " + code;
- static InvalidOp getInstance() {
- return (InvalidOp)opInstances.get().get(OP_INVALID);
+ static LogSegmentOp getInstance(FSEditLogOpCodes code) {
+ return (LogSegmentOp)opInstances.get().get(code);
- void writeFields(DataOutputStream out) throws IOException {
- void readFields(DataInputStream in, int logVersion)
+ public void readFields(DataInputStream in, int logVersion)
- // nothing to read
- static class JSpoolStartOp extends FSEditLogOp {
- private JSpoolStartOp() {
- super(OP_JSPOOL_START);
- static JSpoolStartOp getInstance() {
- return (JSpoolStartOp)opInstances.get().get(OP_JSPOOL_START);
+ // no data stored in these ops yet
+ void writeFields(DataOutputStream out) throws IOException {
+ // no data stored
- static class CheckpointTimeOp extends FSEditLogOp {
- long checkpointTime;
- private CheckpointTimeOp() {
- super(OP_CHECKPOINT_TIME);
- CheckpointTimeOp setCheckpointTime(long time) {
- this.checkpointTime = time;
- return this;
+ static class InvalidOp extends FSEditLogOp {
+ private InvalidOp() {
+ super(OP_INVALID);
- static CheckpointTimeOp getInstance() {
- return (CheckpointTimeOp)opInstances.get()
- .get(OP_CHECKPOINT_TIME);
+ static InvalidOp getInstance() {
+ return (InvalidOp)opInstances.get().get(OP_INVALID);
void writeFields(DataOutputStream out) throws IOException {
- new LongWritable(checkpointTime).write(out);
void readFields(DataInputStream in, int logVersion)
- this.checkpointTime = readLong(in);
+ // nothing to read
@@ -1337,6 +1323,62 @@ public abstract class FSEditLogOp {
return longWritable.get();
+ * Class to encapsulate the header at the top of a log file.
+ static class LogHeader {
+ final int logVersion;
+ final Checksum checksum;
+ public LogHeader(int logVersion, Checksum checksum) {
+ this.logVersion = logVersion;
+ this.checksum = checksum;
+ static LogHeader read(DataInputStream in) throws IOException {
+ int logVersion = 0;
+ logVersion = FSEditLogOp.LogHeader.readLogVersion(in);
+ Checksum checksum = null;
+ if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
+ checksum = FSEditLog.getChecksum();
+ return new LogHeader(logVersion, checksum);
+ * Read the header of fsedit log
+ * @param in fsedit stream
+ * @return the edit log version number
+ * @throws IOException if error occurs
+ private static int readLogVersion(DataInputStream in) throws IOException {
+ // Read log file version. Could be missing.
+ in.mark(4);
+ // If edits log is greater than 2G, available method will return negative
+ // numbers, so we avoid having to call available
+ boolean available = true;
+ logVersion = in.readByte();
+ } catch (EOFException e) {
+ available = false;
+ if (available) {
+ in.reset();
+ logVersion = in.readInt();
+ if (logVersion < FSConstants.LAYOUT_VERSION) // future version
+ throw new IOException(
+ "Unexpected version of the file system log file: "
+ + logVersion + ". Current version = "
+ + FSConstants.LAYOUT_VERSION + ".");
+ assert logVersion <= Storage.LAST_UPGRADABLE_LAYOUT_VERSION :
+ "Unsupported version " + logVersion;
+ return logVersion;
* Class for writing editlog ops
@@ -1357,6 +1399,7 @@ public abstract class FSEditLogOp {
int start = buf.getLength();
buf.writeByte(op.opCode.getOpCode());
+ buf.writeLong(op.txid);
op.writeFields(buf);
int end = buf.getLength();
Checksum checksum = FSEditLog.getChecksum();
@@ -1384,7 +1427,12 @@ public abstract class FSEditLogOp {
public Reader(DataInputStream in, int logVersion,
Checksum checksum) {
- this.in = in;
+ if (checksum != null) {
+ this.in = new DataInputStream(
+ new CheckedInputStream(in, checksum));
+ this.in = in;
this.logVersion = logVersion;
this.checksum = checksum;
@@ -1423,9 +1471,15 @@ public abstract class FSEditLogOp {
if (op == null) {
throw new IOException("Read invalid opcode " + opCode);
+ // Read the txid
+ op.setTransactionId(in.readLong());
op.readFields(in, logVersion);
- validateChecksum(in, checksum);
+ validateChecksum(in, checksum, op.txid);
return op;
@@ -1433,7 +1487,8 @@ public abstract class FSEditLogOp {
* Validate a transaction's checksum
private void validateChecksum(DataInputStream in,
- Checksum checksum)
+ Checksum checksum,
+ long txid)
if (checksum != null) {
int calculatedChecksum = (int)checksum.getValue();
@@ -1441,7 +1496,7 @@ public abstract class FSEditLogOp {
if (readChecksum != calculatedChecksum) {
throw new ChecksumException(
"Transaction is corrupt. Calculated checksum is " +
- calculatedChecksum + " but read checksum " + readChecksum, -1);
+ calculatedChecksum + " but read checksum " + readChecksum, txid);
@@ -54,10 +54,8 @@ public enum FSEditLogOpCodes {
OP_CANCEL_DELEGATION_TOKEN ((byte) 20),
OP_UPDATE_MASTER_KEY ((byte) 21),
OP_REASSIGN_LEASE ((byte) 22),
- // must be same as NamenodeProtocol.JA_JSPOOL_START
- OP_JSPOOL_START ((byte)102),
- // must be same as NamenodeProtocol.JA_CHECKPOINT_TIME
- OP_CHECKPOINT_TIME ((byte)103);
+ OP_END_LOG_SEGMENT ((byte) 23),
+ OP_START_LOG_SEGMENT ((byte) 24);
private byte opCode;
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.io.Text;
@@ -71,10 +72,8 @@ class FSImageFormat {
/** Set to true once a file has been loaded using this loader. */
private boolean loaded = false;
- /** The image version of the loaded file */
- private int imgVersion;
- /** The namespace ID of the loaded file */
- private int imgNamespaceID;
+ /** The transaction ID of the last edit represented by the loaded file */
+ private long imgTxId;
/** The MD5 sum of the loaded file */
private MD5Hash imgDigest;
@@ -83,15 +82,6 @@ class FSImageFormat {
this.namesystem = namesystem;
- * Return the version number of the image that has been loaded.
- * @throws IllegalStateException if load() has not yet been called.
- int getLoadedImageVersion() {
- checkLoaded();
- return imgVersion;
* Return the MD5 checksum of the image that has been loaded.
* @throws IllegalStateException if load() has not yet been called.
@@ -101,13 +91,9 @@ class FSImageFormat {
return imgDigest;
- * Return the namespace ID of the image that has been loaded.
- int getLoadedNamespaceID() {
+ long getLoadedImageTxId() {
checkLoaded();
- return imgNamespaceID;
+ return imgTxId;
@@ -156,10 +142,14 @@ class FSImageFormat {
* it should not contain version and namespace fields
// read image version: first appeared in version -1
- imgVersion = in.readInt();
+ int imgVersion = in.readInt();
+ if(getLayoutVersion() != imgVersion)
+ throw new InconsistentFSStateException(curFile,
+ "imgVersion " + imgVersion +
+ " expected to be " + getLayoutVersion());
// read namespaceID: first appeared in version -2
- imgNamespaceID = in.readInt();
+ in.readInt();
// read number of files
long numFiles = readNumFiles(in);
@@ -169,6 +159,15 @@ class FSImageFormat {
long genstamp = in.readLong();
namesystem.setGenerationStamp(genstamp);
+ // read the transaction ID of the last edit represented by
+ // this image
+ if (LayoutVersion.supports(Feature.STORED_TXIDS, imgVersion)) {
+ imgTxId = in.readLong();
+ imgTxId = 0;
// read compression related info
FSImageCompression compression;
@@ -234,7 +233,7 @@ class FSImageFormat {
private void loadLocalNameINodes(long numFiles, DataInputStream in)
assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
- imgVersion);
+ getLayoutVersion());
assert numFiles > 0;
// load root
@@ -324,11 +323,12 @@ class FSImageFormat {
* @return an inode
private INode loadINode(DataInputStream in)
long modificationTime = 0;
long atime = 0;
long blockSize = 0;
+ int imgVersion = getLayoutVersion();
short replication = in.readShort();
replication = namesystem.adjustReplication(replication);
modificationTime = in.readLong();
@@ -396,7 +396,10 @@ class FSImageFormat {
modificationTime, atime, nsQuota, dsQuota, blockSize);
- private void loadDatanodes(DataInputStream in) throws IOException {
+ private void loadDatanodes(DataInputStream in)
if (imgVersion > -3) // pre datanode image version
if (imgVersion <= -12) {
@@ -412,6 +415,7 @@ class FSImageFormat {
private void loadFilesUnderConstruction(DataInputStream in)
FSDirectory fsDir = namesystem.dir;
if (imgVersion > -13) // pre lease image version
int size = in.readInt();
@@ -437,7 +441,10 @@ class FSImageFormat {
- private void loadSecretManagerState(DataInputStream in) throws IOException {
+ private void loadSecretManagerState(DataInputStream in)
if (!LayoutVersion.supports(Feature.DELEGATION_TOKEN, imgVersion)) {
//SecretManagerState is not available.
//This must not happen if security is turned on.
@@ -446,8 +453,14 @@ class FSImageFormat {
namesystem.loadSecretManagerState(in);
+ private int getLayoutVersion() {
+ return namesystem.getFSImage().getStorage().getLayoutVersion();
+ private long readNumFiles(DataInputStream in)
- private long readNumFiles(DataInputStream in) throws IOException {
if (LayoutVersion.supports(Feature.NAMESPACE_QUOTA, imgVersion)) {
return in.readLong();
@@ -526,6 +539,7 @@ class FSImageFormat {
void save(File newFile,
+ long txid,
FSNamesystem sourceNamesystem,
FSImageCompression compression)
@@ -542,9 +556,11 @@ class FSImageFormat {
DataOutputStream out = new DataOutputStream(fos);
out.writeInt(FSConstants.LAYOUT_VERSION);
- out.writeInt(sourceNamesystem.getFSImage().getStorage().getNamespaceID()); // TODO bad dependency
+ out.writeInt(sourceNamesystem.getFSImage()
+ .getStorage().getNamespaceID()); // TODO bad dependency
out.writeLong(fsDir.rootDir.numItemsInTree());
out.writeLong(sourceNamesystem.getGenerationStamp());
+ out.writeLong(txid);
// write compression info and set up compressed stream
out = compression.writeHeaderAndWrapStream(fos);
@@ -0,0 +1,272 @@
+import java.io.DataInputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+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.NameNodeFile;
+ * Inspects a FSImage storage directory in the "old" (pre-HDFS-1073) format.
+ * This format has the following data files:
+ * - fsimage
+ * - fsimage.ckpt (when checkpoint is being uploaded)
+ * - edits
+ * - edits.new (when logs are "rolled")
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
+ private static final Log LOG =
+ LogFactory.getLog(FSImagePreTransactionalStorageInspector.class);
+ /* Flag if there is at least one storage dir that doesn't contain the newest
+ * fstime */
+ private boolean hasOutOfDateStorageDirs = false;
+ /* Flag set false if there are any "previous" directories found */
+ private boolean isUpgradeFinalized = true;
+ // Track the name and edits dir with the latest times
+ private long latestNameCheckpointTime = Long.MIN_VALUE;
+ private long latestEditsCheckpointTime = Long.MIN_VALUE;
+ private StorageDirectory latestNameSD = null;
+ private StorageDirectory latestEditsSD = null;
+ /** Set to determine if all of storageDirectories share the same checkpoint */
+ Set<Long> checkpointTimes = new HashSet<Long>();
+ private List<String> imageDirs = new ArrayList<String>();
+ private List<String> editsDirs = new ArrayList<String>();
+ void inspectDirectory(StorageDirectory sd) throws IOException {
+ // Was the file just formatted?
+ if (!sd.getVersionFile().exists()) {
+ hasOutOfDateStorageDirs = true;
+ boolean imageExists = false;
+ boolean editsExists = false;
+ // Determine if sd is image, edits or both
+ if (sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
+ imageExists = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE).exists();
+ imageDirs.add(sd.getRoot().getCanonicalPath());
+ if (sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
+ editsExists = NNStorage.getStorageFile(sd, NameNodeFile.EDITS).exists();
+ editsDirs.add(sd.getRoot().getCanonicalPath());
+ long checkpointTime = readCheckpointTime(sd);
+ checkpointTimes.add(checkpointTime);
+ if (sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE) &&
+ (latestNameCheckpointTime < checkpointTime) && imageExists) {
+ latestNameCheckpointTime = checkpointTime;
+ latestNameSD = sd;
+ if (sd.getStorageDirType().isOfType(NameNodeDirType.EDITS) &&
+ (latestEditsCheckpointTime < checkpointTime) && editsExists) {
+ latestEditsCheckpointTime = checkpointTime;
+ latestEditsSD = sd;
+ // check that we have a valid, non-default checkpointTime
+ if (checkpointTime <= 0L)
+ // set finalized flag
+ isUpgradeFinalized = isUpgradeFinalized && !sd.getPreviousDir().exists();
+ * Determine the checkpoint time of the specified StorageDirectory
+ * @param sd StorageDirectory to check
+ * @return If file exists and can be read, last checkpoint time. If not, 0L.
+ * @throws IOException On errors processing file pointed to by sd
+ static long readCheckpointTime(StorageDirectory sd) throws IOException {
+ File timeFile = NNStorage.getStorageFile(sd, NameNodeFile.TIME);
+ long timeStamp = 0L;
+ if (timeFile.exists() && timeFile.canRead()) {
+ DataInputStream in = new DataInputStream(new FileInputStream(timeFile));
+ timeStamp = in.readLong();
+ in.close();
+ return timeStamp;
+ boolean isUpgradeFinalized() {
+ return isUpgradeFinalized;
+ LoadPlan createLoadPlan() throws IOException {
+ // We should have at least one image and one edits dirs
+ if (latestNameSD == null)
+ throw new IOException("Image file is not found in " + imageDirs);
+ if (latestEditsSD == null)
+ throw new IOException("Edits file is not found in " + editsDirs);
+ // Make sure we are loading image and edits from same checkpoint
+ if (latestNameCheckpointTime > latestEditsCheckpointTime
+ && latestNameSD != latestEditsSD
+ && latestNameSD.getStorageDirType() == NameNodeDirType.IMAGE
+ && latestEditsSD.getStorageDirType() == NameNodeDirType.EDITS) {
+ // This is a rare failure when NN has image-only and edits-only
+ // storage directories, and fails right after saving images,
+ // in some of the storage directories, but before purging edits.
+ // See -NOTE- in saveNamespace().
+ LOG.error("This is a rare failure scenario!!!");
+ LOG.error("Image checkpoint time " + latestNameCheckpointTime +
+ " > edits checkpoint time " + latestEditsCheckpointTime);
+ LOG.error("Name-node will treat the image as the latest state of " +
+ "the namespace. Old edits will be discarded.");
+ } else if (latestNameCheckpointTime != latestEditsCheckpointTime) {
+ throw new IOException("Inconsistent storage detected, " +
+ "image and edits checkpoint times do not match. " +
+ "image checkpoint time = " + latestNameCheckpointTime +
+ "edits checkpoint time = " + latestEditsCheckpointTime);
+ return new PreTransactionalLoadPlan();
+ boolean needToSave() {
+ return hasOutOfDateStorageDirs ||
+ checkpointTimes.size() != 1 ||
+ latestNameCheckpointTime > latestEditsCheckpointTime;
+ private class PreTransactionalLoadPlan extends LoadPlan {
+ boolean doRecovery() throws IOException {
+ LOG.debug(
+ "Performing recovery in "+ latestNameSD + " and " + latestEditsSD);
+ boolean needToSave = false;
+ File curFile =
+ NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE);
+ File ckptFile =
+ NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE_NEW);
+ //
+ // If we were in the midst of a checkpoint
+ if (ckptFile.exists()) {
+ needToSave = true;
+ if (NNStorage.getStorageFile(latestEditsSD, NameNodeFile.EDITS_NEW)
+ .exists()) {
+ // checkpointing migth have uploaded a new
+ // merged image, but we discard it here because we are
+ // not sure whether the entire merged image was uploaded
+ // before the namenode crashed.
+ if (!ckptFile.delete()) {
+ throw new IOException("Unable to delete " + ckptFile);
+ // checkpointing was in progress when the namenode
+ // shutdown. The fsimage.ckpt was created and the edits.new
+ // file was moved to edits. We complete that checkpoint by
+ // moving fsimage.new to fsimage. There is no need to
+ // update the fstime file here. renameTo fails on Windows
+ // if the destination file already exists.
+ if (!ckptFile.renameTo(curFile)) {
+ if (!curFile.delete())
+ LOG.warn("Unable to delete dir " + curFile + " before rename");
+ throw new IOException("Unable to rename " + ckptFile +
+ " to " + curFile);
+ return needToSave;
+ File getImageFile() {
+ return NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE);
+ List<File> getEditsFiles() {
+ if (latestNameCheckpointTime > latestEditsCheckpointTime) {
+ // the image is already current, discard edits
+ "Name checkpoint time is newer than edits, not loading edits.");
+ return Collections.<File>emptyList();
+ return getEditsInStorageDir(latestEditsSD);
+ StorageDirectory getStorageDirectoryForProperties() {
+ return latestNameSD;
+ * @return a list with the paths to EDITS and EDITS_NEW (if it exists)
+ * in a given storage directory.
+ static List<File> getEditsInStorageDir(StorageDirectory sd) {
+ ArrayList<File> files = new ArrayList<File>();
+ File edits = NNStorage.getStorageFile(sd, NameNodeFile.EDITS);
+ assert edits.exists() : "Expected edits file at " + edits;
+ files.add(edits);
+ File editsNew = NNStorage.getStorageFile(sd, NameNodeFile.EDITS_NEW);
+ if (editsNew.exists()) {
+ files.add(editsNew);
+ return files;
@@ -0,0 +1,99 @@
+ * Interface responsible for inspecting a set of storage directories and devising
+ * a plan to load the namespace from them.
+abstract class FSImageStorageInspector {
+ * Inspect the contents of the given storage directory.
+ abstract void inspectDirectory(StorageDirectory sd) throws IOException;
+ * @return false if any of the storage directories have an unfinalized upgrade
+ abstract boolean isUpgradeFinalized();
+ * Create a plan to load the image from the set of inspected storage directories.
+ * @throws IOException if not enough files are available (eg no image found in any directory)
+ abstract LoadPlan createLoadPlan() throws IOException;
+ * @return true if the directories are in such a state that the image should be re-saved
+ * following the load
+ abstract boolean needToSave();
+ * A plan to load the namespace from disk, providing the locations from which to load
+ * the image and a set of edits files.
+ abstract static class LoadPlan {
+ * Execute atomic move sequence in the chosen storage directories,
+ * in order to recover from an interrupted checkpoint.
+ * @return true if some recovery action was taken
+ abstract boolean doRecovery() throws IOException;
+ * @return the file from which to load the image data
+ abstract File getImageFile();
+ * @return a list of flies containing edits to replay
+ abstract List<File> getEditsFiles();
+ * @return the storage directory containing the VERSION file that should be
+ * loaded.
+ abstract StorageDirectory getStorageDirectoryForProperties();
+ StringBuilder sb = new StringBuilder();
+ sb.append("Will load image file: ").append(getImageFile()).append("\n");
+ sb.append("Will load edits files:").append("\n");
+ for (File f : getEditsFiles()) {
+ sb.append(" ").append(f).append("\n");
+ sb.append("Will load metadata from: ")
+ .append(getStorageDirectoryForProperties())
+ .append("\n");
+ return sb.toString();
@@ -0,0 +1,688 @@
+import java.io.FileNotFoundException;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
+import com.google.common.collect.ImmutableList;
+class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
+ public static final Log LOG = LogFactory.getLog(
+ FSImageTransactionalStorageInspector.class);
+ private boolean needToSave = false;
+ List<FoundFSImage> foundImages = new ArrayList<FoundFSImage>();
+ List<FoundEditLog> foundEditLogs = new ArrayList<FoundEditLog>();
+ SortedMap<Long, LogGroup> logGroups = new TreeMap<Long, LogGroup>();
+ long maxSeenTxId = 0;
+ private static final Pattern IMAGE_REGEX = Pattern.compile(
+ NameNodeFile.IMAGE.getName() + "_(\\d+)");
+ private static final Pattern EDITS_REGEX = Pattern.compile(
+ NameNodeFile.EDITS.getName() + "_(\\d+)-(\\d+)");
+ private static final Pattern EDITS_INPROGRESS_REGEX = Pattern.compile(
+ NameNodeFile.EDITS_INPROGRESS.getName() + "_(\\d+)");
+ public void inspectDirectory(StorageDirectory sd) throws IOException {
+ // Was the directory just formatted?
+ LOG.info("No version file in " + sd.getRoot());
+ needToSave |= true;
+ File currentDir = sd.getCurrentDir();
+ File filesInStorage[];
+ filesInStorage = FileUtil.listFiles(currentDir);
+ LOG.warn("Unable to inspect storage directory " + currentDir,
+ for (File f : filesInStorage) {
+ LOG.debug("Checking file " + f);
+ String name = f.getName();
+ // Check for fsimage_*
+ Matcher imageMatch = IMAGE_REGEX.matcher(name);
+ if (imageMatch.matches()) {
+ long txid = Long.valueOf(imageMatch.group(1));
+ foundImages.add(new FoundFSImage(sd, f, txid));
+ } catch (NumberFormatException nfe) {
+ LOG.error("Image file " + f + " has improperly formatted " +
+ "transaction ID");
+ // skip
+ LOG.warn("Found image file at " + f + " but storage directory is " +
+ "not configured to contain images.");
+ // Check for a seen_txid file, which marks a minimum transaction ID that
+ // must be included in our load plan.
+ maxSeenTxId = Math.max(maxSeenTxId, NNStorage.readTransactionIdFile(sd));
+ LOG.warn("Unable to determine the max transaction ID seen by " + sd, ioe);
+ List<FoundEditLog> editLogs = matchEditLogs(filesInStorage);
+ for (FoundEditLog log : editLogs) {
+ addEditLog(log);
+ } else if (!editLogs.isEmpty()){
+ LOG.warn("Found the following edit log file(s) in " + sd +
+ " even though it was not configured to store edits:\n" +
+ " " + Joiner.on("\n ").join(editLogs));
+ static List<FoundEditLog> matchEditLogs(File[] filesInStorage) {
+ List<FoundEditLog> ret = Lists.newArrayList();
+ // Check for edits
+ Matcher editsMatch = EDITS_REGEX.matcher(name);
+ if (editsMatch.matches()) {
+ long startTxId = Long.valueOf(editsMatch.group(1));
+ long endTxId = Long.valueOf(editsMatch.group(2));
+ ret.add(new FoundEditLog(f, startTxId, endTxId));
+ LOG.error("Edits file " + f + " has improperly formatted " +
+ // Check for in-progress edits
+ Matcher inProgressEditsMatch = EDITS_INPROGRESS_REGEX.matcher(name);
+ if (inProgressEditsMatch.matches()) {
+ long startTxId = Long.valueOf(inProgressEditsMatch.group(1));
+ ret.add(
+ new FoundEditLog(f, startTxId, FoundEditLog.UNKNOWN_END));
+ LOG.error("In-progress edits file " + f + " has improperly " +
+ "formatted transaction ID");
+ return ret;
+ private void addEditLog(FoundEditLog foundEditLog) {
+ foundEditLogs.add(foundEditLog);
+ LogGroup group = logGroups.get(foundEditLog.startTxId);
+ if (group == null) {
+ group = new LogGroup(foundEditLog.startTxId);
+ logGroups.put(foundEditLog.startTxId, group);
+ group.add(foundEditLog);
+ public boolean isUpgradeFinalized() {
+ * @return the image that has the most recent associated transaction ID.
+ * If there are multiple storage directories which contain equal images
+ * the storage directory that was inspected first will be preferred.
+ * Returns null if no images were found.
+ FoundFSImage getLatestImage() {
+ FoundFSImage ret = null;
+ for (FoundFSImage img : foundImages) {
+ if (ret == null || img.txId > ret.txId) {
+ ret = img;
+ public List<FoundFSImage> getFoundImages() {
+ return ImmutableList.copyOf(foundImages);
+ public List<FoundEditLog> getFoundEditLogs() {
+ return ImmutableList.copyOf(foundEditLogs);
+ public LoadPlan createLoadPlan() throws IOException {
+ if (foundImages.isEmpty()) {
+ throw new FileNotFoundException("No valid image files found");
+ FoundFSImage recoveryImage = getLatestImage();
+ LogLoadPlan logPlan = createLogLoadPlan(recoveryImage.txId, Long.MAX_VALUE);
+ return new TransactionalLoadPlan(recoveryImage,
+ logPlan);
+ * Plan which logs to load in order to bring the namespace up-to-date.
+ * Transactions will be considered in the range (sinceTxId, maxTxId]
+ * @param sinceTxId the highest txid that is already loaded
+ * (eg from the image checkpoint)
+ * @param maxStartTxId ignore any log files that start after this txid
+ LogLoadPlan createLogLoadPlan(long sinceTxId, long maxStartTxId) throws IOException {
+ long expectedTxId = sinceTxId + 1;
+ List<FoundEditLog> recoveryLogs = new ArrayList<FoundEditLog>();
+ SortedMap<Long, LogGroup> tailGroups = logGroups.tailMap(expectedTxId);
+ if (logGroups.size() > tailGroups.size()) {
+ LOG.debug("Excluded " + (logGroups.size() - tailGroups.size()) +
+ " groups of logs because they start with a txid less than image " +
+ "txid " + sinceTxId);
+ SortedMap<Long, LogGroup> usefulGroups;
+ if (maxStartTxId > sinceTxId) {
+ usefulGroups = tailGroups.headMap(maxStartTxId);
+ usefulGroups = new TreeMap<Long, LogGroup>();
+ if (usefulGroups.size() > tailGroups.size()) {
+ LOG.debug("Excluded " + (tailGroups.size() - usefulGroups.size()) +
+ " groups of logs because they start with a txid higher than max " +
+ for (Map.Entry<Long, LogGroup> entry : usefulGroups.entrySet()) {
+ long logStartTxId = entry.getKey();
+ LogGroup logGroup = entry.getValue();
+ logGroup.planRecovery();
+ if (expectedTxId != FSConstants.INVALID_TXID && logStartTxId != expectedTxId) {
+ throw new IOException("Expected next log group would start at txid " +
+ expectedTxId + " but starts at txid " + logStartTxId);
+ // We can pick any of the non-corrupt logs here
+ recoveryLogs.add(logGroup.getBestNonCorruptLog());
+ // If this log group was finalized, we know to expect the next
+ // log group to start at the following txid (ie no gaps)
+ if (logGroup.hasKnownLastTxId()) {
+ expectedTxId = logGroup.getLastTxId() + 1;
+ // the log group was in-progress so we don't know what ID
+ // the next group should start from.
+ expectedTxId = FSConstants.INVALID_TXID;
+ long lastLogGroupStartTxId = usefulGroups.isEmpty() ?
+ 0 : usefulGroups.lastKey();
+ if (maxSeenTxId > sinceTxId &&
+ maxSeenTxId > lastLogGroupStartTxId) {
+ String msg = "At least one storage directory indicated it has seen a " +
+ "log segment starting at txid " + maxSeenTxId;
+ if (usefulGroups.isEmpty()) {
+ msg += " but there are no logs to load.";
+ msg += " but the most recent log file found starts with txid " +
+ lastLogGroupStartTxId;
+ throw new IOException(msg);
+ return new LogLoadPlan(recoveryLogs,
+ Lists.newArrayList(usefulGroups.values()));
+ public boolean needToSave() {
+ RemoteEditLogManifest getEditLogManifest(long sinceTxId) {
+ List<RemoteEditLog> logs = Lists.newArrayList();
+ for (LogGroup g : logGroups.values()) {
+ if (!g.hasFinalized) continue;
+ FoundEditLog fel = g.getBestNonCorruptLog();
+ if (fel.getLastTxId() < sinceTxId) continue;
+ logs.add(new RemoteEditLog(fel.getStartTxId(),
+ fel.getLastTxId()));
+ return new RemoteEditLogManifest(logs);
+ * A group of logs that all start at the same txid.
+ * Handles determining which logs are corrupt and which should be considered
+ * candidates for loading.
+ static class LogGroup {
+ long startTxId;
+ List<FoundEditLog> logs = new ArrayList<FoundEditLog>();;
+ private Set<Long> endTxIds = new TreeSet<Long>();
+ private boolean hasInProgress = false;
+ private boolean hasFinalized = false;
+ LogGroup(long startTxId) {
+ this.startTxId = startTxId;
+ FoundEditLog getBestNonCorruptLog() {
+ // First look for non-corrupt finalized logs
+ for (FoundEditLog log : logs) {
+ if (!log.isCorrupt() && !log.isInProgress()) {
+ return log;
+ // Then look for non-corrupt in-progress logs
+ if (!log.isCorrupt()) {
+ // We should never get here, because we don't get to the planning stage
+ // without calling planRecovery first, and if we've called planRecovery,
+ // we would have already thrown if there were no non-corrupt logs!
+ throw new IllegalStateException(
+ "No non-corrupt logs for txid " + startTxId);
+ * @return true if we can determine the last txid in this log group.
+ boolean hasKnownLastTxId() {
+ if (!log.isInProgress()) {
+ * @return the last txid included in the logs in this group
+ * @throws IllegalStateException if it is unknown -
+ * {@see #hasKnownLastTxId()}
+ long getLastTxId() {
+ return log.lastTxId;
+ throw new IllegalStateException("LogGroup only has in-progress logs");
+ void add(FoundEditLog log) {
+ assert log.getStartTxId() == startTxId;
+ logs.add(log);
+ if (log.isInProgress()) {
+ hasInProgress = true;
+ hasFinalized = true;
+ endTxIds.add(log.lastTxId);
+ void planRecovery() throws IOException {
+ assert hasInProgress || hasFinalized;
+ checkConsistentEndTxIds();
+ if (hasFinalized && hasInProgress) {
+ planMixedLogRecovery();
+ } else if (!hasFinalized && hasInProgress) {
+ planAllInProgressRecovery();
+ } else if (hasFinalized && !hasInProgress) {
+ LOG.debug("No recovery necessary for logs starting at txid " +
+ startTxId);
+ * Recovery case for when some logs in the group were in-progress, and
+ * others were finalized. This happens when one of the storage
+ * directories fails.
+ * The in-progress logs in this case should be considered corrupt.
+ private void planMixedLogRecovery() throws IOException {
+ LOG.warn("Log at " + log.getFile() + " is in progress, but " +
+ "other logs starting at the same txid " + startTxId +
+ " are finalized. Moving aside.");
+ log.markCorrupt();
+ * Recovery case for when all of the logs in the group were in progress.
+ * This happens if the NN completely crashes and restarts. In this case
+ * we check the non-zero lengths of each log file, and any logs that are
+ * less than the max of these lengths are considered corrupt.
+ private void planAllInProgressRecovery() throws IOException {
+ // We only have in-progress logs. We need to figure out which logs have
+ // the latest data to reccover them
+ LOG.warn("Logs beginning at txid " + startTxId + " were are all " +
+ "in-progress (probably truncated due to a previous NameNode " +
+ "crash)");
+ if (logs.size() == 1) {
+ // Only one log, it's our only choice!
+ FoundEditLog log = logs.get(0);
+ if (log.validateLog().numTransactions == 0) {
+ // If it has no transactions, we should consider it corrupt just
+ // to be conservative.
+ // See comment below for similar case
+ LOG.warn("Marking log at " + log.getFile() + " as corrupt since " +
+ "it has no transactions in it.");
+ long maxValidTxnCount = Long.MIN_VALUE;
+ long validTxnCount = log.validateLog().numTransactions;
+ LOG.warn(" Log " + log.getFile() +
+ " valid txns=" + validTxnCount +
+ " valid len=" + log.validateLog().validLength);
+ maxValidTxnCount = Math.max(maxValidTxnCount, validTxnCount);
+ long txns = log.validateLog().numTransactions;
+ if (txns < maxValidTxnCount) {
+ "it is has only " + txns + " valid txns whereas another " +
+ "log has " + maxValidTxnCount);
+ } else if (txns == 0) {
+ // this can happen if the NN crashes right after rolling a log
+ // but before the START_LOG_SEGMENT txn is written. Since the log
+ // is empty, we can just move it aside to its corrupt name.
+ * Check for the case when we have multiple finalized logs and they have
+ * different ending transaction IDs. This violates an invariant that all
+ * log directories should roll together. We should abort in this case.
+ private void checkConsistentEndTxIds() throws IOException {
+ if (hasFinalized && endTxIds.size() > 1) {
+ throw new IOException("More than one ending txid was found " +
+ "for logs starting at txid " + startTxId + ". " +
+ "Found: " + StringUtils.join(endTxIds, ','));
+ void recover() throws IOException {
+ if (log.isCorrupt()) {
+ log.moveAsideCorruptFile();
+ } else if (log.isInProgress()) {
+ log.finalizeLog();
+ * Record of an image that has been located and had its filename parsed.
+ static class FoundFSImage {
+ final StorageDirectory sd;
+ final long txId;
+ private final File file;
+ FoundFSImage(StorageDirectory sd, File file, long txId) {
+ assert txId >= 0 : "Invalid txid on " + file +": " + txId;
+ this.sd = sd;
+ this.txId = txId;
+ this.file = file;
+ public long getTxId() {
+ return txId;
+ return file.toString();
+ * Record of an edit log that has been located and had its filename parsed.
+ static class FoundEditLog {
+ File file;
+ final long startTxId;
+ long lastTxId;
+ private EditLogValidation cachedValidation = null;
+ private boolean isCorrupt = false;
+ static final long UNKNOWN_END = -1;
+ FoundEditLog(File file,
+ long startTxId, long endTxId) {
+ assert endTxId == UNKNOWN_END || endTxId >= startTxId;
+ assert startTxId > 0;
+ assert file != null;
+ this.lastTxId = endTxId;
+ public void finalizeLog() throws IOException {
+ long numTransactions = validateLog().numTransactions;
+ long lastTxId = startTxId + numTransactions - 1;
+ File dst = new File(file.getParentFile(),
+ NNStorage.getFinalizedEditsFileName(startTxId, lastTxId));
+ LOG.info("Finalizing edits log " + file + " by renaming to "
+ + dst.getName());
+ if (!file.renameTo(dst)) {
+ throw new IOException("Couldn't finalize log " +
+ file + " to " + dst);
+ this.lastTxId = lastTxId;
+ file = dst;
+ long getStartTxId() {
+ return startTxId;
+ return lastTxId;
+ EditLogValidation validateLog() throws IOException {
+ if (cachedValidation == null) {
+ cachedValidation = FSEditLogLoader.validateEditLog(file);
+ return cachedValidation;
+ boolean isInProgress() {
+ return (lastTxId == UNKNOWN_END);
+ void markCorrupt() {
+ isCorrupt = true;
+ boolean isCorrupt() {
+ return isCorrupt;
+ void moveAsideCorruptFile() throws IOException {
+ assert isCorrupt;
+ File src = file;
+ File dst = new File(src.getParent(), src.getName() + ".corrupt");
+ boolean success = src.renameTo(dst);
+ "Couldn't rename corrupt log " + src + " to " + dst);
+ static class TransactionalLoadPlan extends LoadPlan {
+ final FoundFSImage image;
+ final LogLoadPlan logPlan;
+ public TransactionalLoadPlan(FoundFSImage image,
+ LogLoadPlan logPlan) {
+ super();
+ this.image = image;
+ this.logPlan = logPlan;
+ logPlan.doRecovery();
+ return image.getFile();
+ return logPlan.getEditsFiles();
+ return image.sd;
+ static class LogLoadPlan {
+ final List<FoundEditLog> editLogs;
+ final List<LogGroup> logGroupsToRecover;
+ LogLoadPlan(List<FoundEditLog> editLogs,
+ List<LogGroup> logGroupsToRecover) {
+ this.editLogs = editLogs;
+ this.logGroupsToRecover = logGroupsToRecover;
+ public void doRecovery() throws IOException {
+ for (LogGroup g : logGroupsToRecover) {
+ g.recover();
+ public List<File> getEditsFiles() {
+ List<File> ret = new ArrayList<File>();
+ ret.add(log.getFile());
@@ -105,6 +105,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
@@ -124,6 +125,7 @@ import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
@@ -323,8 +325,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
if(fsImage == null) {
this.dir = new FSDirectory(this, conf);
StartupOption startOpt = NameNode.getStartupOption(conf);
- this.dir.loadFSImage(getNamespaceDirs(conf),
- getNamespaceEditsDirs(conf), startOpt);
+ this.dir.loadFSImage(startOpt);
long timeTakenToLoadFSImage = now() - systemStart;
LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
NameNode.getNameNodeMetrics().setFsImageLoadTime(
@@ -392,8 +393,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
+ propertyName + "\" in hdfs-site.xml;" +
"\n\t\t- use Backup Node as a persistent and up-to-date storage " +
"of the file system meta-data.");
- } else if (dirNames.isEmpty())
- dirNames.add("file:///tmp/hadoop/dfs/name");
+ } else if (dirNames.isEmpty()) {
+ dirNames = Collections.singletonList("file:///tmp/hadoop/dfs/name");
return Util.stringCollectionAsURIs(dirNames);
@@ -3258,7 +3260,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
throw new IOException("Safe mode should be turned ON " +
"in order to create namespace image.");
- getFSImage().saveNamespace(true);
+ getFSImage().saveNamespace();
LOG.info("New namespace image has been created.");
readUnlock();
@@ -4003,8 +4005,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
- long getEditLogSize() throws IOException {
- return getEditLog().getEditLogSize();
+ public long getTransactionID() {
+ return getEditLog().getSyncTxId();
CheckpointSignature rollEditLog() throws IOException {
@@ -4019,24 +4021,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
writeUnlock();
- * Moves fsimage.ckpt to fsImage and edits.new to edits
- * Reopens the new edits file.
- * @param sig the signature of this checkpoint (old image)
- void rollFSImage(CheckpointSignature sig) throws IOException {
- writeLock();
- if (isInSafeMode()) {
- throw new SafeModeException("Image not rolled", safeMode);
- LOG.info("Roll FSImage from " + Server.getRemoteAddress());
- getFSImage().rollFSImage(sig, true);
- writeUnlock();
+ public RemoteEditLogManifest getEditLogManifest(long sinceTxId) throws IOException {
+ return getEditLog().getEditLogManifest(sinceTxId);
NamenodeCommand startCheckpoint(
@@ -4516,31 +4503,29 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
- * Register a name-node.
- * Registration is allowed if there is no ongoing streaming to
- * another backup node.
- * We currently allow only one backup node, but multiple chackpointers
- * if there are no backups.
+ * Register a Backup name-node, verifying that it belongs
+ * to the correct namespace, and adding it to the set of
+ * active journals if necessary.
- * @param registration
+ * @param bnReg registration of the new BackupNode
+ * @param nnReg registration of this NameNode
+ * @throws IOException if the namespace IDs do not match
- void registerBackupNode(NamenodeRegistration registration)
+ void registerBackupNode(NamenodeRegistration bnReg,
+ NamenodeRegistration nnReg) throws IOException {
if(getFSImage().getStorage().getNamespaceID()
- != registration.getNamespaceID())
+ != bnReg.getNamespaceID())
throw new IOException("Incompatible namespaceIDs: "
+ " Namenode namespaceID = "
+ getFSImage().getStorage().getNamespaceID() + "; "
- + registration.getRole() +
- " node namespaceID = " + registration.getNamespaceID());
- boolean regAllowed = getEditLog().checkBackupRegistration(registration);
- if(!regAllowed)
- throw new IOException("Registration is not allowed. " +
- "Another node is registered as a backup.");
+ + bnReg.getRole() +
+ " node namespaceID = " + bnReg.getNamespaceID());
+ if (bnReg.getRole() == NamenodeRole.BACKUP) {
+ getFSImage().getEditLog().registerBackupNode(
+ bnReg, nnReg);
@@ -5080,4 +5065,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
void removeDecomNodeFromList(List<DatanodeDescriptor> nodeList) {
getBlockManager().getDatanodeManager().removeDecomNodeFromList(nodeList);
@@ -0,0 +1,114 @@
+import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundEditLog;
+ * Journal manager for the common case of edits files being written
+ * to a storage directory.
+ * Note: this class is not thread-safe and should be externally
+ * synchronized.
+class FileJournalManager implements JournalManager {
+ private static final Log LOG = LogFactory.getLog(FileJournalManager.class);
+ private final StorageDirectory sd;
+ private int outputBufferCapacity = 512*1024;
+ public FileJournalManager(StorageDirectory sd) {
+ public EditLogOutputStream startLogSegment(long txid) throws IOException {
+ File newInProgress = NNStorage.getInProgressEditsFile(sd, txid);
+ EditLogOutputStream stm = new EditLogFileOutputStream(newInProgress,
+ outputBufferCapacity);
+ stm.create();
+ File inprogressFile = NNStorage.getInProgressEditsFile(
+ sd, firstTxId);
+ File dstFile = NNStorage.getFinalizedEditsFile(
+ sd, firstTxId, lastTxId);
+ LOG.debug("Finalizing edits file " + inprogressFile + " -> " + dstFile);
+ Preconditions.checkState(!dstFile.exists(),
+ "Can't finalize edits file " + inprogressFile + " since finalized file " +
+ "already exists");
+ if (!inprogressFile.renameTo(dstFile)) {
+ throw new IOException("Unable to finalize edits file " + inprogressFile);
+ public StorageDirectory getStorageDirectory() {
+ return sd;
+ return "FileJournalManager for storage directory " + sd;
+ this.outputBufferCapacity = size;
+ File[] files = FileUtil.listFiles(sd.getCurrentDir());
+ List<FoundEditLog> editLogs =
+ FSImageTransactionalStorageInspector.matchEditLogs(files);
+ if (log.getStartTxId() < minTxIdToKeep &&
+ log.getLastTxId() < minTxIdToKeep) {
+ purger.purgeLog(log);
+ public EditLogInputStream getInProgressInputStream(long segmentStartsAtTxId)
+ File f = NNStorage.getInProgressEditsFile(sd, segmentStartsAtTxId);
+ return new EditLogFileInputStream(f);
@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.security.PrivilegedExceptionAction;
import java.util.*;
import java.io.*;
+import java.net.InetSocketAddress;
import javax.servlet.ServletContext;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
@@ -34,11 +36,16 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
+import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
* This class is used in Namesystem's jetty to retrieve a file.
* Typically used by the Secondary NameNode to retrieve image and
@@ -50,15 +57,21 @@ public class GetImageServlet extends HttpServlet {
private static final Log LOG = LogFactory.getLog(GetImageServlet.class);
- @SuppressWarnings("unchecked")
+ private static final String TXID_PARAM = "txid";
+ private static final String START_TXID_PARAM = "startTxId";
+ private static final String END_TXID_PARAM = "endTxId";
+ private static final String STORAGEINFO_PARAM = "storageInfo";
+ private static Set<Long> currentlyDownloadingCheckpoints =
+ Collections.<Long>synchronizedSet(new HashSet<Long>());
public void doGet(final HttpServletRequest request,
final HttpServletResponse response
) throws ServletException, IOException {
- Map<String,String[]> pmap = request.getParameterMap();
ServletContext context = getServletContext();
final FSImage nnImage = NameNodeHttpServer.getFsImageFromContext(context);
- final TransferFsImage ff = new TransferFsImage(pmap, request, response);
+ final GetImageParams parsedParams = new GetImageParams(request, response);
final Configuration conf =
(Configuration)getServletContext().getAttribute(JspHelper.CURRENT_CONF);
@@ -70,45 +83,77 @@ public class GetImageServlet extends HttpServlet {
+ request.getRemoteHost());
+ String myStorageInfoString = nnImage.getStorage().toColonSeparatedString();
+ String theirStorageInfoString = parsedParams.getStorageInfoString();
+ if (theirStorageInfoString != null &&
+ !myStorageInfoString.equals(theirStorageInfoString)) {
+ response.sendError(HttpServletResponse.SC_FORBIDDEN,
+ "This namenode has storage info " + myStorageInfoString +
+ " but the secondary expected " + theirStorageInfoString);
+ LOG.warn("Received an invalid request file transfer request " +
+ "from a secondary with storage info " + theirStorageInfoString);
UserGroupInformation.getCurrentUser().doAs(new PrivilegedExceptionAction<Void>() {
public Void run() throws Exception {
- if (ff.getImage()) {
- response.setHeader(TransferFsImage.CONTENT_LENGTH,
- String.valueOf(nnImage.getStorage()
- .getFsImageName().length()));
+ if (parsedParams.isGetImage()) {
+ long txid = parsedParams.getTxId();
+ File imageFile = nnImage.getStorage().getFsImageName(txid);
+ if (imageFile == null) {
+ throw new IOException("Could not find image with txid " + txid);
+ setVerificationHeaders(response, imageFile);
// send fsImage
- TransferFsImage.getFileServer(response.getOutputStream(),
- nnImage.getStorage().getFsImageName(),
+ TransferFsImage.getFileServer(response.getOutputStream(), imageFile,
getThrottler(conf));
- } else if (ff.getEdit()) {
- .getFsEditName().length()));
+ } else if (parsedParams.isGetEdit()) {
+ long startTxId = parsedParams.getStartTxId();
+ long endTxId = parsedParams.getEndTxId();
+ File editFile = nnImage.getStorage()
+ .findFinalizedEditsFile(startTxId, endTxId);
+ setVerificationHeaders(response, editFile);
// send edits
- nnImage.getStorage().getFsEditName(),
+ TransferFsImage.getFileServer(response.getOutputStream(), editFile,
- } else if (ff.putImage()) {
- // issue a HTTP get request to download the new fsimage
- nnImage.validateCheckpointUpload(ff.getToken());
- nnImage.newImageDigest = ff.getNewChecksum();
- MD5Hash downloadImageDigest = reloginIfNecessary().doAs(
- new PrivilegedExceptionAction<MD5Hash>() {
- public MD5Hash run() throws Exception {
- return TransferFsImage.getFileClient(
- ff.getInfoServer(), "getimage=1",
- nnImage.getStorage().getFsImageNameCheckpoint(), true);
- });
- if (!nnImage.newImageDigest.equals(downloadImageDigest)) {
- throw new IOException("The downloaded image is corrupt," +
- " expecting a checksum " + nnImage.newImageDigest +
- " but received a checksum " + downloadImageDigest);
+ } else if (parsedParams.isPutImage()) {
+ final long txid = parsedParams.getTxId();
+ if (! currentlyDownloadingCheckpoints.add(txid)) {
+ "Another checkpointer is already in the process of uploading a" +
+ " checkpoint made at transaction ID " + txid);
+ if (nnImage.getStorage().findImageFile(txid) != null) {
+ "Another checkpointer already uploaded an checkpoint " +
+ "for txid " + txid);
+ // issue a HTTP get request to download the new fsimage
+ MD5Hash downloadImageDigest = reloginIfNecessary().doAs(
+ new PrivilegedExceptionAction<MD5Hash>() {
+ public MD5Hash run() throws Exception {
+ return TransferFsImage.downloadImageToStorage(
+ parsedParams.getInfoServer(), txid,
+ nnImage.getStorage(), true);
+ });
+ nnImage.saveDigestAndRenameCheckpointImage(txid, downloadImageDigest);
+ // Now that we have a new checkpoint, we might be able to
+ // remove some old ones.
+ nnImage.purgeOldStorage();
+ currentlyDownloadingCheckpoints.remove(txid);
- nnImage.checkpointUploadDone();
return null;
@@ -182,4 +227,148 @@ public class GetImageServlet extends HttpServlet {
if(LOG.isDebugEnabled()) LOG.debug("isValidRequestor is rejecting: " + remoteUser);
+ * Set headers for content length, and, if available, md5.
+ private void setVerificationHeaders(HttpServletResponse response, File file)
+ response.setHeader(TransferFsImage.CONTENT_LENGTH,
+ String.valueOf(file.length()));
+ MD5Hash hash = MD5FileUtils.readStoredMd5ForFile(file);
+ if (hash != null) {
+ response.setHeader(TransferFsImage.MD5_HEADER, hash.toString());
+ static String getParamStringForImage(long txid,
+ StorageInfo remoteStorageInfo) {
+ return "getimage=1&" + TXID_PARAM + "=" + txid
+ + "&" + STORAGEINFO_PARAM + "=" +
+ remoteStorageInfo.toColonSeparatedString();
+ static String getParamStringForLog(RemoteEditLog log,
+ return "getedit=1&" + START_TXID_PARAM + "=" + log.getStartTxId()
+ + "&" + END_TXID_PARAM + "=" + log.getEndTxId()
+ static String getParamStringToPutImage(long txid,
+ InetSocketAddress imageListenAddress, NNStorage storage) {
+ return "putimage=1" +
+ "&" + TXID_PARAM + "=" + txid +
+ "&port=" + imageListenAddress.getPort() +
+ "&machine=" + imageListenAddress.getHostName()
+ storage.toColonSeparatedString();
+ static class GetImageParams {
+ private boolean isGetImage;
+ private boolean isGetEdit;
+ private boolean isPutImage;
+ private int remoteport;
+ private String machineName;
+ private long startTxId, endTxId, txId;
+ private String storageInfoString;
+ * @param request the object from which this servlet reads the url contents
+ * @param response the object into which this servlet writes the url contents
+ * @throws IOException if the request is bad
+ public GetImageParams(HttpServletRequest request,
+ HttpServletResponse response
+ ) throws IOException {
+ @SuppressWarnings("unchecked")
+ Map<String, String[]> pmap = request.getParameterMap();
+ isGetImage = isGetEdit = isPutImage = false;
+ remoteport = 0;
+ machineName = null;
+ for (Map.Entry<String, String[]> entry : pmap.entrySet()) {
+ String key = entry.getKey();
+ String[] val = entry.getValue();
+ if (key.equals("getimage")) {
+ isGetImage = true;
+ txId = parseLongParam(request, TXID_PARAM);
+ } else if (key.equals("getedit")) {
+ isGetEdit = true;
+ startTxId = parseLongParam(request, START_TXID_PARAM);
+ endTxId = parseLongParam(request, END_TXID_PARAM);
+ } else if (key.equals("putimage")) {
+ isPutImage = true;
+ } else if (key.equals("port")) {
+ remoteport = new Integer(val[0]).intValue();
+ } else if (key.equals("machine")) {
+ machineName = val[0];
+ } else if (key.equals(STORAGEINFO_PARAM)) {
+ storageInfoString = val[0];
+ int numGets = (isGetImage?1:0) + (isGetEdit?1:0);
+ if ((numGets > 1) || (numGets == 0) && !isPutImage) {
+ throw new IOException("Illegal parameters to TransferFsImage");
+ public String getStorageInfoString() {
+ return storageInfoString;
+ Preconditions.checkState(isGetImage || isPutImage);
+ public long getStartTxId() {
+ Preconditions.checkState(isGetEdit);
+ public long getEndTxId() {
+ return endTxId;
+ boolean isGetEdit() {
+ return isGetEdit;
+ boolean isGetImage() {
+ return isGetImage;
+ boolean isPutImage() {
+ return isPutImage;
+ String getInfoServer() throws IOException{
+ if (machineName == null || remoteport == 0) {
+ throw new IOException ("MachineName and port undefined");
+ return machineName + ":" + remoteport;
+ private static long parseLongParam(HttpServletRequest request, String param)
+ // Parse the 'txid' parameter which indicates which image is to be
+ // fetched.
+ String paramStr = request.getParameter(param);
+ if (paramStr == null) {
+ throw new IOException("Invalid request has no " + param + " parameter");
+ return Long.valueOf(paramStr);
@@ -0,0 +1,68 @@
+ * A JournalManager is responsible for managing a single place of storing
+ * edit logs. It may correspond to multiple files, a backup node, etc.
+ * Even when the actual underlying storage is rolled, or failed and restored,
+ * each conceptual place of storage corresponds to exactly one instance of
+ * this class, which is created when the EditLog is first opened.
+interface JournalManager {
+ * Begin writing to a new segment of the log stream, which starts at
+ * the given transaction ID.
+ EditLogOutputStream startLogSegment(long txId) throws IOException;
+ * Mark the log segment that spans from firstTxId to lastTxId
+ * as finalized and complete.
+ void finalizeLogSegment(long firstTxId, long lastTxId) throws IOException;
+ * Set the amount of memory that this stream should use to buffer edits
+ void setOutputBufferCapacity(int size);
+ * The JournalManager may archive/purge any logs for transactions less than
+ * or equal to minImageTxId.
+ * @param minTxIdToKeep the earliest txid that must be retained after purging
+ * old logs
+ * @param purger the purging implementation to use
+ * @throws IOException if purging fails
+ void purgeLogsOlderThan(long minTxIdToKeep, StoragePurger purger)
+ throws IOException;
+ * the edit log is currently writing. May return null if this journal
+ * manager does not support this operation.
+ EditLogInputStream getInProgressInputStream(long segmentStartsAtTxId)
@@ -17,22 +17,20 @@
+import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
+import java.io.FileReader;
import java.io.RandomAccessFile;
+import java.io.OutputStream;
import java.net.URI;
import java.net.UnknownHostException;
import java.security.NoSuchAlgorithmException;
import java.security.SecureRandom;
import java.util.Collection;
+import java.util.HashMap;
import java.util.Properties;
@@ -51,13 +49,17 @@ import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
-import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.UpgradeManager;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType;
+import org.apache.hadoop.hdfs.util.AtomicFileOutputStream;
import org.apache.hadoop.net.DNS;
* NNStorage is responsible for management of the StorageDirectories used by
* the NameNode.
@@ -66,17 +68,19 @@ import org.apache.hadoop.net.DNS;
public class NNStorage extends Storage implements Closeable {
private static final Log LOG = LogFactory.getLog(NNStorage.class.getName());
- static final String MESSAGE_DIGEST_PROPERTY = "imageMD5Digest";
+ static final String DEPRECATED_MESSAGE_DIGEST_PROPERTY = "imageMD5Digest";
// The filenames used for storing the images
enum NameNodeFile {
IMAGE ("fsimage"),
- TIME ("fstime"),
+ TIME ("fstime"), // from "old" pre-HDFS-1073 format
+ SEEN_TXID ("seen_txid"),
EDITS ("edits"),
IMAGE_NEW ("fsimage.ckpt"),
- EDITS_NEW ("edits.new");
+ EDITS_NEW ("edits.new"), // from "old" pre-HDFS-1073 format
+ EDITS_INPROGRESS ("edits_inprogress");
private String fileName = null;
private NameNodeFile(String name) { this.fileName = name; }
@@ -106,42 +110,9 @@ public class NNStorage extends Storage implements Closeable {
- * Interface to be implemented by classes which make use of storage
- * directories. They are notified when a StorageDirectory is causing errors,
- * becoming available or being formatted.
- * This allows the implementors of the interface take their own specific
- * action on the StorageDirectory when this occurs.
- interface NNStorageListener {
- * An error has occurred with a StorageDirectory.
- * @param sd The storage directory causing the error.
- void errorOccurred(StorageDirectory sd) throws IOException;
- * A storage directory has been formatted.
- * @param sd The storage directory being formatted.
- void formatOccurred(StorageDirectory sd) throws IOException;
- * A storage directory is now available use.
- * @param sd The storage directory which has become available.
- void directoryAvailable(StorageDirectory sd) throws IOException;
- final private List<NNStorageListener> listeners;
private UpgradeManager upgradeManager = null;
- protected MD5Hash imageDigest = null;
protected String blockpoolID = ""; // id of the block pool
* flag that controls if we try to restore failed storages
@@ -149,7 +120,13 @@ public class NNStorage extends Storage implements Closeable {
private Object restorationLock = new Object();
private boolean disablePreUpgradableLayoutCheck = false;
- private long checkpointTime = -1L; // The age of the image
+ * TxId of the last transaction that was included in the most
+ * recent fsimage file. This does not include any transactions
+ * that have since been written to the edit log.
+ protected long mostRecentCheckpointTxId = FSConstants.INVALID_TXID;
* list of failed (and thus removed) storages
@@ -158,27 +135,26 @@ public class NNStorage extends Storage implements Closeable {
= new CopyOnWriteArrayList<StorageDirectory>();
- * Construct the NNStorage.
- * @param conf Namenode configuration.
+ * Properties from old layout versions that may be needed
+ * during upgrade only.
- public NNStorage(Configuration conf) {
- super(NodeType.NAME_NODE);
- storageDirs = new CopyOnWriteArrayList<StorageDirectory>();
- this.listeners = new CopyOnWriteArrayList<NNStorageListener>();
+ private HashMap<String, String> deprecatedProperties;
* Construct the NNStorage.
- * @param storageInfo storage information
- * @param bpid block pool Id
+ * @param conf Namenode configuration.
+ * @param imageDirs Directories the image can be stored in.
+ * @param editsDirs Directories the editlog can be stored in.
+ * @throws IOException if any directories are inaccessible.
- public NNStorage(StorageInfo storageInfo, String bpid) {
- super(NodeType.NAME_NODE, storageInfo);
+ public NNStorage(Configuration conf,
+ Collection<URI> imageDirs, Collection<URI> editsDirs)
+ super(NodeType.NAME_NODE);
storageDirs = new CopyOnWriteArrayList<StorageDirectory>();
- this.blockpoolID = bpid;
+ setStorageDirectories(imageDirs, editsDirs);
@Override // Storage
@@ -207,7 +183,6 @@ public class NNStorage extends Storage implements Closeable {
@Override // Closeable
- listeners.clear();
unlockAll();
storageDirs.clear();
@@ -232,10 +207,7 @@ public class NNStorage extends Storage implements Closeable {
* See if any of removed storages is "writable" again, and can be returned
- * into service. If saveNamespace is set, then this method is being
- * called from saveNamespace.
- * @param saveNamespace Whether method is being called from saveNamespace()
+ * into service.
void attemptRestoreRemovedStorage() {
// if directory is "alive" - copy the images there...
@@ -253,23 +225,10 @@ public class NNStorage extends Storage implements Closeable {
LOG.info("currently disabled dir " + root.getAbsolutePath() +
"; type="+sd.getStorageDirType()
+ ";canwrite="+root.canWrite());
- if(root.exists() && root.canWrite()) {
- // when we try to restore we just need to remove all the data
- // without saving current in-memory state (which could've changed).
- sd.clearDirectory();
- LOG.info("restoring dir " + sd.getRoot().getAbsolutePath());
- for (NNStorageListener listener : listeners) {
- listener.directoryAvailable(sd);
- this.addStorageDir(sd); // restore
- this.removedStorageDirs.remove(sd);
- LOG.warn("failed to restore " + sd.getRoot().getAbsolutePath(), e);
+ if(root.exists() && root.canWrite()) {
+ LOG.info("restoring dir " + sd.getRoot().getAbsolutePath());
+ this.addStorageDir(sd); // restore
+ this.removedStorageDirs.remove(sd);
@@ -283,9 +242,11 @@ public class NNStorage extends Storage implements Closeable {
- * Set the storage directories which will be used. NNStorage.close() should
- * be called before this to ensure any previous storage directories have been
- * freed.
+ * Set the storage directories which will be used. This should only ever be
+ * called from inside NNStorage. However, it needs to remain package private
+ * for testing, as StorageDirectories need to be reinitialised after using
+ * Mockito.spy() on this class, as Mockito doesn't work well with inner
+ * classes, such as StorageDirectory in this case.
* Synchronized due to initialization of storageDirs and removedStorageDirs.
@@ -293,6 +254,7 @@ public class NNStorage extends Storage implements Closeable {
* @param fsEditsDirs Locations to store edit logs.
synchronized void setStorageDirectories(Collection<URI> fsNameDirs,
Collection<URI> fsEditsDirs)
@@ -411,110 +373,84 @@ public class NNStorage extends Storage implements Closeable {
return list;
- * Determine the checkpoint time of the specified StorageDirectory
+ * Determine the last transaction ID noted in this storage directory.
+ * This txid is stored in a special seen_txid file since it might not
+ * correspond to the latest image or edit log. For example, an image-only
+ * directory will have this txid incremented when edits logs roll, even
+ * though the edits logs are in a different directory.
* @param sd StorageDirectory to check
- * @return If file exists and can be read, last checkpoint time. If not, 0L.
+ * @return If file exists and can be read, last recorded txid. If not, 0L.
* @throws IOException On errors processing file pointed to by sd
- long readCheckpointTime(StorageDirectory sd) throws IOException {
- File timeFile = getStorageFile(sd, NameNodeFile.TIME);
- long timeStamp = 0L;
- if (timeFile.exists() && timeFile.canRead()) {
- DataInputStream in = new DataInputStream(new FileInputStream(timeFile));
+ static long readTransactionIdFile(StorageDirectory sd) throws IOException {
+ File txidFile = getStorageFile(sd, NameNodeFile.SEEN_TXID);
+ long txid = 0L;
+ if (txidFile.exists() && txidFile.canRead()) {
+ BufferedReader br = new BufferedReader(new FileReader(txidFile));
- timeStamp = in.readLong();
+ txid = Long.valueOf(br.readLine());
- in.close();
+ IOUtils.cleanup(LOG, br);
- return timeStamp;
* Write last checkpoint time into a separate file.
* @param sd
- public void writeCheckpointTime(StorageDirectory sd) throws IOException {
- if (checkpointTime < 0L)
- return; // do not write negative time
- if (timeFile.exists() && ! timeFile.delete()) {
- LOG.error("Cannot delete chekpoint time file: "
- + timeFile.getCanonicalPath());
- FileOutputStream fos = new FileOutputStream(timeFile);
- DataOutputStream out = new DataOutputStream(fos);
+ void writeTransactionIdFile(StorageDirectory sd, long txid) throws IOException {
+ Preconditions.checkArgument(txid >= 0, "bad txid: " + txid);
+ File txIdFile = getStorageFile(sd, NameNodeFile.SEEN_TXID);
+ OutputStream fos = new AtomicFileOutputStream(txIdFile);
- out.flush();
- fos.getChannel().force(true);
+ fos.write(String.valueOf(txid).getBytes());
+ fos.write('\n');
- out.close();
+ IOUtils.cleanup(LOG, fos);
- * Record new checkpoint time in order to
- * distinguish healthy directories from the removed ones.
- * If there is an error writing new checkpoint time, the corresponding
- * storage directory is removed from the list.
+ * Set the transaction ID of the last checkpoint
- public void incrementCheckpointTime() {
- setCheckpointTimeInStorage(checkpointTime + 1);
+ void setMostRecentCheckpointTxId(long txid) {
+ this.mostRecentCheckpointTxId = txid;
- * The age of the namespace state.<p>
- * Reflects the latest time the image was saved.
- * Modified with every save or a checkpoint.
- * Persisted in VERSION file.
- * @return the current checkpoint time.
+ * Return the transaction ID of the last checkpoint.
- public long getCheckpointTime() {
- return checkpointTime;
+ long getMostRecentCheckpointTxId() {
+ return mostRecentCheckpointTxId;
- * Set the checkpoint time.
- * This method does not persist the checkpoint time to storage immediately.
+ * Write a small file in all available storage directories that
+ * indicates that the namespace has reached some given transaction ID.
- * @see #setCheckpointTimeInStorage
- * @param newCpT the new checkpoint time.
- public void setCheckpointTime(long newCpT) {
- checkpointTime = newCpT;
- * Set the current checkpoint time. Writes the new checkpoint
- * time to all available storage directories.
- * @param newCpT The new checkpoint time.
+ * This is used when the image is loaded to avoid accidental rollbacks
+ * in the case where an edit log is fully deleted but there is no
+ * checkpoint. See TestNameEditsConfigs.testNameEditsConfigsFailure()
+ * @param txid the txid that has been reached
- public void setCheckpointTimeInStorage(long newCpT) {
- // Write new checkpoint time in all storage directories
- for(Iterator<StorageDirectory> it =
- dirIterator(); it.hasNext();) {
+ public void writeTransactionIdFileToStorage(long txid) {
+ // Write txid marker in all storage directories
+ for (StorageDirectory sd : storageDirs) {
- writeCheckpointTime(sd);
+ writeTransactionIdFile(sd, txid);
// Close any edits stream associated with this dir and remove directory
- LOG.warn("incrementCheckpointTime failed on "
- + sd.getRoot().getPath() + ";type="+sd.getStorageDirType());
- reportErrorsOnDirectory(sd);
- LOG.error("Failed to report and remove NN storage directory "
- + sd.getRoot().getPath(), ioe);
+ LOG.warn("writeTransactionIdToStorage failed on " + sd,
+ e);
+ reportErrorsOnDirectory(sd);
@@ -525,11 +461,11 @@ public class NNStorage extends Storage implements Closeable {
* @return List of filenames to save checkpoints to.
- public File[] getFsImageNameCheckpoint() {
+ public File[] getFsImageNameCheckpoint(long txid) {
ArrayList<File> list = new ArrayList<File>();
for (Iterator<StorageDirectory> it =
dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
- list.add(getStorageFile(it.next(), NameNodeFile.IMAGE_NEW));
+ list.add(getStorageFile(it.next(), NameNodeFile.IMAGE_NEW, txid));
return list.toArray(new File[list.size()]);
@@ -538,51 +474,24 @@ public class NNStorage extends Storage implements Closeable {
* Return the name of the image file.
* @return The name of the first image file.
- public File getFsImageName() {
+ public File getFsImageName(long txid) {
StorageDirectory sd = null;
sd = it.next();
- File fsImage = getStorageFile(sd, NameNodeFile.IMAGE);
+ File fsImage = getStorageFile(sd, NameNodeFile.IMAGE, txid);
if(sd.getRoot().canRead() && fsImage.exists())
return fsImage;
- * @return The name of the first editlog file.
- public File getFsEditName() throws IOException {
- = dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
- * @return The name of the first time file.
- public File getFsTimeName() {
- // NameNodeFile.TIME shoul be same on all directories
- dirIterator(); it.hasNext();)
- return getStorageFile(sd, NameNodeFile.TIME);
/** Create new dfs name directory. Caution: this destroys all files
* in this filesystem. */
private void format(StorageDirectory sd) throws IOException {
sd.clearDirectory(); // create currrent dir
- listener.formatOccurred(sd);
writeProperties(sd);
+ writeTransactionIdFile(sd, 0);
LOG.info("Storage directory " + sd.getRoot()
+ " has been successfully formatted.");
@@ -597,7 +506,6 @@ public class NNStorage extends Storage implements Closeable {
this.clusterID = clusterId;
this.blockpoolID = newBlockPoolID();
this.cTime = 0L;
- this.setCheckpointTime(now());
dirIterator(); it.hasNext();) {
@@ -624,50 +532,6 @@ public class NNStorage extends Storage implements Closeable {
return newID;
- * Move {@code current} to {@code lastcheckpoint.tmp} and
- * recreate empty {@code current}.
- * {@code current} is moved only if it is well formatted,
- * that is contains VERSION file.
- * @see org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory#getLastCheckpointTmp()
- * @see org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory#getPreviousCheckpoint()
- protected void moveCurrent(StorageDirectory sd)
- File curDir = sd.getCurrentDir();
- File tmpCkptDir = sd.getLastCheckpointTmp();
- // mv current -> lastcheckpoint.tmp
- // only if current is formatted - has VERSION file
- if(sd.getVersionFile().exists()) {
- assert curDir.exists() : curDir + " directory must exist.";
- assert !tmpCkptDir.exists() : tmpCkptDir + " directory must not exist.";
- rename(curDir, tmpCkptDir);
- // recreate current
- if(!curDir.exists() && !curDir.mkdir())
- throw new IOException("Cannot create directory " + curDir);
- * Move {@code lastcheckpoint.tmp} to {@code previous.checkpoint}
- protected void moveLastCheckpoint(StorageDirectory sd)
- File prevCkptDir = sd.getPreviousCheckpoint();
- // remove previous.checkpoint
- if (prevCkptDir.exists())
- deleteDir(prevCkptDir);
- // mv lastcheckpoint.tmp -> previous.checkpoint
- if(tmpCkptDir.exists())
- rename(tmpCkptDir, prevCkptDir);
protected void setFieldsFromProperties(
Properties props, StorageDirectory sd) throws IOException {
@@ -689,26 +553,35 @@ public class NNStorage extends Storage implements Closeable {
setDistributedUpgradeState(
sDUS == null? false : Boolean.parseBoolean(sDUS),
sDUV == null? getLayoutVersion() : Integer.parseInt(sDUV));
+ setDeprecatedPropertiesForUpgrade(props);
- String sMd5 = props.getProperty(MESSAGE_DIGEST_PROPERTY);
- if (LayoutVersion.supports(Feature.FSIMAGE_CHECKSUM, layoutVersion)) {
- if (sMd5 == null) {
- throw new InconsistentFSStateException(sd.getRoot(),
- "file " + STORAGE_FILE_VERSION
- + " does not have MD5 image digest.");
- this.imageDigest = new MD5Hash(sMd5);
- } else if (sMd5 != null) {
- "file " + STORAGE_FILE_VERSION +
- " has image MD5 digest when version is " + layoutVersion);
+ * Pull any properties out of the VERSION file that are from older
+ * versions of HDFS and only necessary during upgrade.
+ private void setDeprecatedPropertiesForUpgrade(Properties props) {
+ deprecatedProperties = new HashMap<String, String>();
+ String md5 = props.getProperty(DEPRECATED_MESSAGE_DIGEST_PROPERTY);
+ if (md5 != null) {
+ deprecatedProperties.put(DEPRECATED_MESSAGE_DIGEST_PROPERTY, md5);
- this.setCheckpointTime(readCheckpointTime(sd));
+ * Return a property that was stored in an earlier version of HDFS.
+ * This should only be used during upgrades.
+ String getDeprecatedProperty(String prop) {
+ assert getLayoutVersion() > FSConstants.LAYOUT_VERSION :
+ "getDeprecatedProperty should only be done when loading " +
+ "storage from past versions during upgrade.";
+ return deprecatedProperties.get(prop);
- * Write last checkpoint time and version file into the storage directory.
+ * Write version file into the storage directory.
* The version file should always be written last.
* Missing or corrupted version file indicates that
@@ -733,50 +606,109 @@ public class NNStorage extends Storage implements Closeable {
props.setProperty("distributedUpgradeVersion",
Integer.toString(uVersion));
- // Though the current NN supports this feature, this function
- // is called with old layoutVersions from the upgrade tests.
- if (imageDigest == null) {
- // May be null on the first save after an upgrade.
- imageDigest = MD5Hash.digest(
- new FileInputStream(getStorageFile(sd, NameNodeFile.IMAGE)));
- props.setProperty(MESSAGE_DIGEST_PROPERTY, imageDigest.toString());
+ static File getStorageFile(StorageDirectory sd, NameNodeFile type, long imageTxId) {
+ return new File(sd.getCurrentDir(),
+ String.format("%s_%019d", type.getName(), imageTxId));
- * @return A File of 'type' in storage directory 'sd'.
+ * Get a storage file for one of the files that doesn't need a txid associated
+ * (e.g version, seen_txid)
static File getStorageFile(StorageDirectory sd, NameNodeFile type) {
return new File(sd.getCurrentDir(), type.getName());
+ public static String getCheckpointImageFileName(long txid) {
+ return String.format("%s_%019d",
+ NameNodeFile.IMAGE_NEW.getName(), txid);
+ public static String getImageFileName(long txid) {
+ NameNodeFile.IMAGE.getName(), txid);
+ public static String getInProgressEditsFileName(long startTxId) {
+ return String.format("%s_%019d", NameNodeFile.EDITS_INPROGRESS.getName(),
+ static File getInProgressEditsFile(StorageDirectory sd, long startTxId) {
+ return new File(sd.getCurrentDir(), getInProgressEditsFileName(startTxId));
+ static File getFinalizedEditsFile(StorageDirectory sd,
+ getFinalizedEditsFileName(startTxId, endTxId));
+ static File getImageFile(StorageDirectory sd, long txid) {
+ getImageFileName(txid));
+ public static String getFinalizedEditsFileName(long startTxId, long endTxId) {
+ return String.format("%s_%019d-%019d", NameNodeFile.EDITS.getName(),
+ startTxId, endTxId);
- * @return A editlog File in storage directory 'sd'.
+ * Return the first readable finalized edits file for the given txid.
- File getEditFile(StorageDirectory sd) {
- return getStorageFile(sd, NameNodeFile.EDITS);
+ File findFinalizedEditsFile(long startTxId, long endTxId)
+ File ret = findFile(NameNodeDirType.EDITS,
+ if (ret == null) {
+ "No edits file for txid " + startTxId + "-" + endTxId + " exists!");
+ * Return the first readable image file for the given txid, or null
+ * if no such image can be found
+ File findImageFile(long txid) throws IOException {
+ return findFile(NameNodeDirType.IMAGE,
- * @return A temporary editlog File in storage directory 'sd'.
+ * Return the first readable storage file of the given name
+ * across any of the 'current' directories in SDs of the
+ * given type, or null if no such file exists.
- File getEditNewFile(StorageDirectory sd) {
- return getStorageFile(sd, NameNodeFile.EDITS_NEW);
+ private File findFile(NameNodeDirType dirType, String name) {
+ for (StorageDirectory sd : dirIterable(dirType)) {
+ File candidate = new File(sd.getCurrentDir(), name);
+ if (sd.getCurrentDir().canRead() &&
+ candidate.exists()) {
+ return candidate;
- * @return A list of all Files of 'type' in available storage directories.
+ * @return A list of the given File in every available storage directory,
+ * regardless of whether it might exist.
- Collection<File> getFiles(NameNodeFile type, NameNodeDirType dirType) {
+ List<File> getFiles(NameNodeDirType dirType, String fileName) {
Iterator<StorageDirectory> it =
(dirType == null) ? dirIterator() : dirIterator(dirType);
for ( ;it.hasNext(); ) {
- list.add(getStorageFile(it.next(), type));
+ list.add(new File(it.next().getCurrentDir(), fileName));
@@ -809,7 +741,9 @@ public class NNStorage extends Storage implements Closeable {
* @param uVersion the new version.
private void setDistributedUpgradeState(boolean uState, int uVersion) {
- upgradeManager.setUpgradeState(uState, uVersion);
+ if (upgradeManager != null) {
+ upgradeManager.setUpgradeState(uState, uVersion);
@@ -849,33 +783,6 @@ public class NNStorage extends Storage implements Closeable {
+ FSConstants.LAYOUT_VERSION + " is initialized.");
- * Set the digest for the latest image stored by NNStorage.
- * @param digest The digest for the image.
- void setImageDigest(MD5Hash digest) {
- this.imageDigest = digest;
- * Get the digest for the latest image storage by NNStorage.
- * @return The digest for the latest image.
- * Register a listener. The listener will be notified of changes to the list
- * of available storage directories.
- * @see NNStorageListener
- * @param sel A storage listener.
- void registerListener(NNStorageListener sel) {
- listeners.add(sel);
* Disable the check for pre-upgradable layouts. Needed for BackupImage.
* @param val Whether to disable the preupgradeable layout check.
@@ -890,7 +797,7 @@ public class NNStorage extends Storage implements Closeable {
* @param sds A list of storage directories to mark as errored.
- void reportErrorsOnDirectories(List<StorageDirectory> sds) throws IOException {
+ void reportErrorsOnDirectories(List<StorageDirectory> sds) {
for (StorageDirectory sd : sds) {
reportErrorsOnDirectory(sd);
@@ -904,17 +811,12 @@ public class NNStorage extends Storage implements Closeable {
* @param sd A storage directory to mark as errored.
- void reportErrorsOnDirectory(StorageDirectory sd)
+ void reportErrorsOnDirectory(StorageDirectory sd) {
LOG.error("Error reported on storage directory " + sd);
String lsd = listStorageDirectories();
LOG.debug("current list of storage dirs:" + lsd);
- listener.errorOccurred(sd);
LOG.warn("About to remove corresponding storage: "
+ sd.getRoot().getAbsolutePath());
@@ -927,8 +829,7 @@ public class NNStorage extends Storage implements Closeable {
if (this.storageDirs.remove(sd)) {
this.removedStorageDirs.add(sd);
- incrementCheckpointTime();
lsd = listStorageDirectories();
LOG.debug("at the end current list of storage dirs:" + lsd);
@@ -967,6 +868,29 @@ public class NNStorage extends Storage implements Closeable {
+ * Report that an IOE has occurred on some file which may
+ * or may not be within one of the NN image storage directories.
+ void reportErrorOnFile(File f) {
+ // We use getAbsolutePath here instead of getCanonicalPath since we know
+ // that there is some IO problem on that drive.
+ // getCanonicalPath may need to call stat() or readlink() and it's likely
+ // those calls would fail due to the same underlying IO problem.
+ String absPath = f.getAbsolutePath();
+ String dirPath = sd.getRoot().getAbsolutePath();
+ if (!dirPath.endsWith("/")) {
+ dirPath += "/";
+ if (absPath.startsWith(dirPath)) {
* Generate new clusterID.
@@ -1065,4 +989,67 @@ public class NNStorage extends Storage implements Closeable {
public String getBlockPoolID() {
return blockpoolID;
+ * Iterate over all current storage directories, inspecting them
+ * with the given inspector.
+ void inspectStorageDirs(FSImageStorageInspector inspector)
+ // Process each of the storage directories to find the pair of
+ // newest image file and edit file
+ for (Iterator<StorageDirectory> it = dirIterator(); it.hasNext();) {
+ StorageDirectory sd = it.next();
+ * Iterate over all of the storage dirs, reading their contents to determine
+ * their layout versions. Returns an FSImageStorageInspector which has
+ * inspected each directory.
+ * <b>Note:</b> this can mutate the storage info fields (ctime, version, etc).
+ * @throws IOException if no valid storage dirs are found
+ FSImageStorageInspector readAndInspectDirs()
+ int minLayoutVersion = Integer.MAX_VALUE; // the newest
+ int maxLayoutVersion = Integer.MIN_VALUE; // the oldest
+ // First determine what range of layout versions we're going to inspect
+ for (Iterator<StorageDirectory> it = dirIterator();
+ it.hasNext();) {
+ FSImage.LOG.warn("Storage directory " + sd + " contains no VERSION file. Skipping...");
+ continue;
+ readProperties(sd); // sets layoutVersion
+ minLayoutVersion = Math.min(minLayoutVersion, getLayoutVersion());
+ maxLayoutVersion = Math.max(maxLayoutVersion, getLayoutVersion());
+ if (minLayoutVersion > maxLayoutVersion) {
+ throw new IOException("No storage directories contained VERSION information");
+ assert minLayoutVersion <= maxLayoutVersion;
+ // If we have any storage directories with the new layout version
+ // (ie edits_<txnid>) then use the new inspector, which will ignore
+ // the old format dirs.
+ FSImageStorageInspector inspector;
+ if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, minLayoutVersion)) {
+ inspector = new FSImageTransactionalStorageInspector();
+ if (!LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, maxLayoutVersion)) {
+ FSImage.LOG.warn("Ignoring one or more storage directories with old layouts");
+ inspector = new FSImagePreTransactionalStorageInspector();
+ inspectStorageDirs(inspector);
+ return inspector;
@@ -0,0 +1,151 @@
+import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundFSImage;
+import com.google.common.collect.Sets;
+ * The NNStorageRetentionManager is responsible for inspecting the storage
+ * directories of the NN and enforcing a retention policy on checkpoints
+ * and edit logs.
+ * It delegates the actual removal of files to a StoragePurger
+ * implementation, which might delete the files or instead copy them to
+ * a filer or HDFS for later analysis.
+public class NNStorageRetentionManager {
+ private final int numCheckpointsToRetain;
+ private static final Log LOG = LogFactory.getLog(
+ NNStorageRetentionManager.class);
+ private final NNStorage storage;
+ private final StoragePurger purger;
+ private final FSEditLog editLog;
+ public NNStorageRetentionManager(
+ Configuration conf,
+ NNStorage storage,
+ FSEditLog editLog,
+ StoragePurger purger) {
+ this.numCheckpointsToRetain = conf.getInt(
+ DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY,
+ DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_DEFAULT);
+ this.storage = storage;
+ this.editLog = editLog;
+ this.purger = purger;
+ public NNStorageRetentionManager(Configuration conf, NNStorage storage,
+ FSEditLog editLog) {
+ this(conf, storage, editLog, new DeletionStoragePurger());
+ public void purgeOldStorage() throws IOException {
+ long minImageTxId = getImageTxIdToRetain(inspector);
+ purgeCheckpointsOlderThan(inspector, minImageTxId);
+ // If fsimage_N is the image we want to keep, then we need to keep
+ // all txns > N. We can remove anything < N+1, since fsimage_N
+ // reflects the state up to and including N.
+ editLog.purgeLogsOlderThan(minImageTxId + 1, purger);
+ private void purgeCheckpointsOlderThan(
+ FSImageTransactionalStorageInspector inspector,
+ long minTxId) {
+ for (FoundFSImage image : inspector.getFoundImages()) {
+ if (image.getTxId() < minTxId) {
+ LOG.info("Purging old image " + image);
+ purger.purgeImage(image);
+ * @param inspector inspector that has already inspected all storage dirs
+ * @return the transaction ID corresponding to the oldest checkpoint
+ * that should be retained.
+ private long getImageTxIdToRetain(FSImageTransactionalStorageInspector inspector) {
+ List<FoundFSImage> images = inspector.getFoundImages();
+ TreeSet<Long> imageTxIds = Sets.newTreeSet();
+ for (FoundFSImage image : images) {
+ imageTxIds.add(image.getTxId());
+ List<Long> imageTxIdsList = Lists.newArrayList(imageTxIds);
+ if (imageTxIdsList.isEmpty()) {
+ return 0;
+ Collections.reverse(imageTxIdsList);
+ int toRetain = Math.min(numCheckpointsToRetain, imageTxIdsList.size());
+ long minTxId = imageTxIdsList.get(toRetain - 1);
+ LOG.info("Going to retain " + toRetain + " images with txid >= " +
+ minTxId);
+ return minTxId;
+ * Interface responsible for disposing of old checkpoints and edit logs.
+ static interface StoragePurger {
+ void purgeLog(FoundEditLog log);
+ void purgeImage(FoundFSImage image);
+ static class DeletionStoragePurger implements StoragePurger {
+ public void purgeLog(FoundEditLog log) {
+ deleteOrWarn(log.getFile());
+ public void purgeImage(FoundFSImage image) {
+ deleteOrWarn(image.getFile());
+ deleteOrWarn(MD5FileUtils.getDigestFileForFile(image.getFile()));
+ private static void deleteOrWarn(File file) {
+ if (!file.delete()) {
+ // It's OK if we fail to delete something -- we'll catch it
+ // next time we swing through this directory.
+ LOG.warn("Could not delete " + file);
@@ -78,6 +78,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.server.protocol.NodeRegistration;
import org.apache.hadoop.io.EnumSetWritable;
@@ -382,7 +383,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
nodeRegistration = new NamenodeRegistration(
getHostPortString(rpcAddress),
getHostPortString(getHttpAddress()),
- getFSImage().getStorage(), getRole(), getFSImage().getStorage().getCheckpointTime());
+ getFSImage().getStorage(), getRole());
return nodeRegistration;
@@ -647,8 +648,9 @@ public class NameNode implements NamenodeProtocols, FSConstants {
public NamenodeRegistration register(NamenodeRegistration registration)
verifyVersion(registration.getVersion());
- namesystem.registerBackupNode(registration);
- return setRegistration();
+ NamenodeRegistration myRegistration = setRegistration();
+ namesystem.registerBackupNode(registration, myRegistration);
+ return myRegistration;
@Override // NamenodeProtocol
@@ -669,22 +671,6 @@ public class NameNode implements NamenodeProtocols, FSConstants {
namesystem.endCheckpoint(registration, sig);
- public long journalSize(NamenodeRegistration registration)
- verifyRequest(registration);
- public void journal(NamenodeRegistration registration,
- // Active name-node cannot journal.
- throw new UnsupportedActionException("journal");
@Override // ClientProtocol
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
@@ -1056,21 +1042,20 @@ public class NameNode implements NamenodeProtocols, FSConstants {
namesystem.refreshNodes(new HdfsConfiguration());
- @Deprecated // NamenodeProtocol
- public long getEditLogSize() throws IOException {
+ @Override // NamenodeProtocol
+ return namesystem.getTransactionID();
- @Deprecated
public CheckpointSignature rollEditLog() throws IOException {
return namesystem.rollEditLog();
- public void rollFsImage(CheckpointSignature sig) throws IOException {
- namesystem.rollFSImage(sig);
+ return namesystem.getEditLogManifest(sinceTxId);
@@ -1279,26 +1264,11 @@ public class NameNode implements NamenodeProtocols, FSConstants {
if (version != LAYOUT_VERSION)
throw new IncorrectVersionException(version, "data node");
- * Returns the name of the fsImage file
- public File getFsImageName() throws IOException {
- return getFSImage().getStorage().getFsImageName();
public FSImage getFSImage() {
return namesystem.dir.fsImage;
- * Returns the name of the fsImage file uploaded by periodic
- * checkpointing
- public File[] getFsImageNameCheckpoint() throws IOException {
- return getFSImage().getStorage().getFsImageNameCheckpoint();
* Returns the address on which the NameNodes is listening to.
* @return namenode rpc address
@@ -1374,20 +1344,16 @@ public class NameNode implements NamenodeProtocols, FSConstants {
System.out.println("Formatting using clusterid: " + clusterId);
- FSImage fsImage = new FSImage(dirsToFormat, editDirsToFormat);
+ FSImage fsImage = new FSImage(conf, null, dirsToFormat, editDirsToFormat);
FSNamesystem nsys = new FSNamesystem(fsImage, conf);
- nsys.dir.fsImage.getStorage().format(clusterId);
+ nsys.dir.fsImage.format(clusterId);
private static boolean finalize(Configuration conf,
boolean isConfirmationNeeded
) throws IOException {
- Collection<URI> dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
- Collection<URI> editDirsToFormat =
- FSNamesystem.getNamespaceEditsDirs(conf);
- FSNamesystem nsys = new FSNamesystem(new FSImage(dirsToFormat,
- editDirsToFormat), conf);
+ FSNamesystem nsys = new FSNamesystem(new FSImage(conf), conf);
System.err.print(
"\"finalize\" will remove the previous state of the files system.\n"
+ "Recent upgrade will become permanent.\n"
@@ -23,11 +23,19 @@ import java.net.InetSocketAddress;
import java.security.PrivilegedAction;
import java.util.Date;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
@@ -43,10 +51,11 @@ import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -60,6 +69,9 @@ import org.apache.hadoop.security.authorize.AccessControlList;
/**********************************************************
* The Secondary NameNode is a helper to the primary NameNode.
* The Secondary is responsible for supporting periodic checkpoints
@@ -98,11 +110,18 @@ public class SecondaryNameNode implements Runnable {
private int imagePort;
- private FSNamesystem namesystem;
private Collection<URI> checkpointDirs;
private Collection<URI> checkpointEditsDirs;
+ /** How often to checkpoint regardless of number of txns */
- private long checkpointSize; // size (in bytes) of current Edit Log
+ /** How often to poll the NN to check checkpointTxnCount */
+ private long checkpointCheckPeriod; // in seconds
+ /** checkpoint once every this many transactions, regardless of time */
+ private long checkpointTxnCount;
/** {@inheritDoc} */
public String toString() {
@@ -111,23 +130,49 @@ public class SecondaryNameNode implements Runnable {
+ "\nStart Time : " + new Date(starttime)
+ "\nLast Checkpoint Time : " + (lastCheckpointTime == 0? "--": new Date(lastCheckpointTime))
+ "\nCheckpoint Period : " + checkpointPeriod + " seconds"
- + "\nCheckpoint Size : " + StringUtils.byteDesc(checkpointSize)
- + " (= " + checkpointSize + " bytes)"
+ + "\nCheckpoint Size : " + StringUtils.byteDesc(checkpointTxnCount)
+ + " (= " + checkpointTxnCount + " bytes)"
+ "\nCheckpoint Dirs : " + checkpointDirs
+ "\nCheckpoint Edits Dirs: " + checkpointEditsDirs;
FSImage getFSImage() {
return checkpointImage;
+ void setFSImage(CheckpointStorage image) {
+ this.checkpointImage = image;
+ NamenodeProtocol getNameNode() {
+ return namenode;
+ void setNameNode(NamenodeProtocol namenode) {
+ this.namenode = namenode;
+ List<URI> getCheckpointDirs() {
+ return ImmutableList.copyOf(checkpointDirs);
* Create a connection to the primary namenode.
public SecondaryNameNode(Configuration conf) throws IOException {
+ this(conf, new CommandLineOpts());
+ public SecondaryNameNode(Configuration conf,
+ CommandLineOpts commandLineOpts) throws IOException {
NameNode.initializeGenericKeys(conf);
- initialize(conf);
+ initialize(conf, commandLineOpts);
LOG.fatal("Failed to start secondary namenode. ", e);
@@ -143,8 +188,10 @@ public class SecondaryNameNode implements Runnable {
* Initialize SecondaryNameNode.
+ * @param commandLineOpts
- private void initialize(final Configuration conf) throws IOException {
+ private void initialize(final Configuration conf,
final InetSocketAddress infoSocAddr = getHttpAddress(conf);
infoBindAddress = infoSocAddr.getHostName();
UserGroupInformation.setConfiguration(conf);
@@ -171,14 +218,19 @@ public class SecondaryNameNode implements Runnable {
"/tmp/hadoop/dfs/namesecondary");
checkpointEditsDirs = FSImage.getCheckpointEditsDirs(conf,
- checkpointImage = new CheckpointStorage(conf);
- checkpointImage.recoverCreate(checkpointDirs, checkpointEditsDirs);
+ checkpointImage = new CheckpointStorage(conf, checkpointDirs, checkpointEditsDirs);
+ checkpointImage.recoverCreate(commandLineOpts.shouldFormat());
+ checkpointCheckPeriod = conf.getLong(
+ DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY,
+ DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT);
checkpointPeriod = conf.getLong(DFS_NAMENODE_CHECKPOINT_PERIOD_KEY,
DFS_NAMENODE_CHECKPOINT_PERIOD_DEFAULT);
- checkpointSize = conf.getLong(DFS_NAMENODE_CHECKPOINT_SIZE_KEY,
- DFS_NAMENODE_CHECKPOINT_SIZE_DEFAULT);
+ checkpointTxnCount = conf.getLong(DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
+ DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
+ warnForDeprecatedConfigs(conf);
// initialize the webserver for uploading files.
// Kerberized SSL servers must be run from the host principal...
@@ -204,8 +256,8 @@ public class SecondaryNameNode implements Runnable {
System.setProperty("https.cipherSuites",
Krb5AndCertsSslSocketConnector.KRB5_CIPHER_SUITES.get(0));
InetSocketAddress secInfoSocAddr =
- NetUtils.createSocketAddr(infoBindAddress + ":"+ conf.get(
- "dfs.secondary.https.port", infoBindAddress + ":" + 0));
+ NetUtils.createSocketAddr(infoBindAddress + ":"+ conf.getInt(
+ "dfs.secondary.https.port", 443));
imagePort = secInfoSocAddr.getPort();
infoServer.addSslListener(secInfoSocAddr, conf, false, true);
@@ -227,15 +279,28 @@ public class SecondaryNameNode implements Runnable {
// The web-server port can be ephemeral... ensure we have the correct info
infoPort = infoServer.getPort();
- if(!UserGroupInformation.isSecurityEnabled())
+ if (!UserGroupInformation.isSecurityEnabled()) {
imagePort = infoPort;
conf.set(DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, infoBindAddress + ":" +infoPort);
LOG.info("Secondary Web-server up at: " + infoBindAddress + ":" +infoPort);
LOG.info("Secondary image servlet up at: " + infoBindAddress + ":" + imagePort);
- LOG.warn("Checkpoint Period :" + checkpointPeriod + " secs " +
+ LOG.info("Checkpoint Period :" + checkpointPeriod + " secs " +
- LOG.warn("Log Size Trigger :" + checkpointSize + " bytes " +
+ LOG.info("Log Size Trigger :" + checkpointTxnCount + " txns");
+ static void warnForDeprecatedConfigs(Configuration conf) {
+ for (String key : ImmutableList.of(
+ "fs.checkpoint.size",
+ "dfs.namenode.checkpoint.size")) {
+ if (conf.get(key) != null) {
+ LOG.warn("Configuration key " + key + " is deprecated! Ignoring..." +
+ " Instead please specify a value for " +
+ DFS_NAMENODE_CHECKPOINT_TXNS_KEY);
@@ -283,13 +348,10 @@ public class SecondaryNameNode implements Runnable {
public void doWork() {
- // Poll the Namenode (once every 5 minutes) to find the size of the
- // pending edit log.
+ // Poll the Namenode (once every checkpointCheckPeriod seconds) to find the
+ // number of transactions in the edit log that haven't yet been checkpointed.
- long period = 5 * 60; // 5 minutes
- if (checkpointPeriod < period) {
- period = checkpointPeriod;
+ long period = Math.min(checkpointCheckPeriod, checkpointPeriod);
while (shouldRun) {
@@ -307,8 +369,7 @@ public class SecondaryNameNode implements Runnable {
long now = System.currentTimeMillis();
- long size = namenode.getEditLogSize();
- if (size >= checkpointSize ||
+ if (shouldCheckpointBasedOnCount() ||
now >= lastCheckpointTime + 1000 * checkpointPeriod) {
doCheckpoint();
lastCheckpointTime = now;
@@ -316,7 +377,6 @@ public class SecondaryNameNode implements Runnable {
} catch (IOException e) {
LOG.error("Exception in doCheckpoint", e);
e.printStackTrace();
- checkpointImage.getStorage().imageDigest = null;
} catch (Throwable e) {
LOG.error("Throwable Exception in doCheckpoint", e);
@@ -331,49 +391,53 @@ public class SecondaryNameNode implements Runnable {
* @return true if a new image has been downloaded and needs to be loaded
- private boolean downloadCheckpointFiles(final CheckpointSignature sig
- ) throws IOException {
+ static boolean downloadCheckpointFiles(
+ final String nnHostPort,
+ final FSImage dstImage,
+ final CheckpointSignature sig,
+ final RemoteEditLogManifest manifest
+ // Sanity check manifest - these could happen if, eg, someone on the
+ // NN side accidentally rmed the storage directories
+ if (manifest.getLogs().isEmpty()) {
+ throw new IOException("Found no edit logs to download on NN since txid "
+ + sig.mostRecentCheckpointTxId);
+ long expectedTxId = sig.mostRecentCheckpointTxId + 1;
+ if (manifest.getLogs().get(0).getStartTxId() != expectedTxId) {
+ throw new IOException("Bad edit log manifest (expected txid = " +
+ expectedTxId + ": " + manifest);
Boolean b = UserGroupInformation.getCurrentUser().doAs(
new PrivilegedExceptionAction<Boolean>() {
public Boolean run() throws Exception {
- checkpointImage.getStorage().cTime = sig.cTime;
- checkpointImage.getStorage().setCheckpointTime(sig.checkpointTime);
+ dstImage.getStorage().cTime = sig.cTime;
// get fsimage
- String fileid;
- Collection<File> list;
- File[] srcNames;
boolean downloadImage = true;
- if (sig.imageDigest.equals(
- checkpointImage.getStorage().imageDigest)) {
+ if (sig.mostRecentCheckpointTxId ==
+ dstImage.getStorage().getMostRecentCheckpointTxId()) {
downloadImage = false;
LOG.info("Image has not changed. Will not download image.");
- fileid = "getimage=1";
- list = checkpointImage.getStorage().getFiles(
- NameNodeFile.IMAGE, NameNodeDirType.IMAGE);
- srcNames = list.toArray(new File[list.size()]);
- assert srcNames.length > 0 : "No checkpoint targets.";
- TransferFsImage.getFileClient(fsName, fileid, srcNames, false);
- checkpointImage.getStorage().imageDigest = sig.imageDigest;
- LOG.info("Downloaded file " + srcNames[0].getName() + " size " +
- srcNames[0].length() + " bytes.");
+ nnHostPort, sig.mostRecentCheckpointTxId, dstImage.getStorage(), true);
+ dstImage.saveDigestAndRenameCheckpointImage(
// get edits file
- list = getFSImage().getStorage().getFiles(
- NameNodeFile.EDITS, NameNodeDirType.EDITS);
- srcNames = list.toArray(new File[list.size()]);;
+ nnHostPort, log, dstImage.getStorage());
- checkpointImage.checkpointUploadDone();
return Boolean.valueOf(downloadImage);
});
@@ -387,18 +451,6 @@ public class SecondaryNameNode implements Runnable {
return nameNodeAddr;
- * Copy the new fsimage into the NameNode
- private void putFSImage(CheckpointSignature sig) throws IOException {
- String fileid = "putimage=1&port=" + imagePort +
- "&newChecksum=" + checkpointImage.getStorage().getImageDigest();
- LOG.info("Posted URL " + fsName + fileid);
- TransferFsImage.getFileClient(fsName, fileid, (File[])null, false);
* Returns the Jetty server that the Namenode is listening on.
@@ -423,19 +475,39 @@ public class SecondaryNameNode implements Runnable {
return configuredAddress;
+ * Return the host:port of where this SecondaryNameNode is listening
+ * for image transfers
+ return new InetSocketAddress(infoBindAddress, imagePort);
* @return if the image is fetched from primary or not
boolean doCheckpoint() throws IOException {
- // Do the required initialization of the merge work area.
- startCheckpoint();
+ checkpointImage.ensureCurrentDirExists();
+ NNStorage dstStorage = checkpointImage.getStorage();
// Tell the namenode to start logging transactions in a new edit file
// Returns a token that would be used to upload the merged image.
CheckpointSignature sig = namenode.rollEditLog();
+ if (checkpointImage.getNamespaceID() != 0) {
+ // If the image actually has some data, make sure we're talking
+ // to the same NN as we did before.
+ sig.validateStorageInfo(checkpointImage);
+ // if we're a fresh 2NN, just take the storage info from the server
+ // we first talk to.
+ dstStorage.setStorageInfo(sig);
+ dstStorage.setClusterID(sig.getClusterID());
+ dstStorage.setBlockPoolID(sig.getBlockpoolID());
// error simulation code for junit test
if (ErrorSimulator.getErrorSimulation(0)) {
@@ -443,14 +515,20 @@ public class SecondaryNameNode implements Runnable {
"after creating edits.new");
- boolean loadImage = downloadCheckpointFiles(sig); // Fetch fsimage and edits
- doMerge(sig, loadImage); // Do the merge
+ namenode.getEditLogManifest(sig.mostRecentCheckpointTxId + 1);
+ boolean loadImage = downloadCheckpointFiles(
+ fsName, checkpointImage, sig, manifest); // Fetch fsimage and edits
+ doMerge(sig, manifest, loadImage, checkpointImage);
// Upload the new image into the NameNode. Then tell the Namenode
// to make this new uploaded image as the most current image.
- putFSImage(sig);
+ long txid = checkpointImage.getLastAppliedTxId();
+ TransferFsImage.uploadImageFromStorage(fsName, getImageListenAddress(),
+ dstStorage, txid);
if (ErrorSimulator.getErrorSimulation(1)) {
@@ -458,91 +536,53 @@ public class SecondaryNameNode implements Runnable {
"after uploading new image to NameNode");
- namenode.rollFsImage(sig);
- checkpointImage.endCheckpoint();
LOG.warn("Checkpoint done. New Image Size: "
- + checkpointImage.getStorage().getFsImageName().length());
+ + dstStorage.getFsImageName(txid).length());
+ // Since we've successfully checkpointed, we can remove some old
+ // image files
+ checkpointImage.purgeOldStorage();
return loadImage;
- private void startCheckpoint() throws IOException {
- checkpointImage.getStorage().unlockAll();
- checkpointImage.getEditLog().close();
- checkpointImage.startCheckpoint();
- * Merge downloaded image and edits and write the new image into
- * current storage directory.
- private void doMerge(CheckpointSignature sig, boolean loadImage)
- if (loadImage) {
- namesystem = new FSNamesystem(checkpointImage, conf);
- assert namesystem.dir.fsImage == checkpointImage;
- checkpointImage.doMerge(sig, loadImage);
* @param argv The parameters passed to this program.
* @exception Exception if the filesystem does not exist.
* @return 0 on success, non zero on error.
- private int processArgs(String[] argv) throws Exception {
- if (argv.length < 1) {
- printUsage("");
- return -1;
- int exitCode = -1;
- int i = 0;
- String cmd = argv[i++];
- // verify that we have enough command line parameters
- if ("-geteditsize".equals(cmd)) {
- if (argv.length != 1) {
- printUsage(cmd);
- return exitCode;
- } else if ("-checkpoint".equals(cmd)) {
- if (argv.length != 1 && argv.length != 2) {
- if (argv.length == 2 && !"force".equals(argv[i])) {
+ private int processStartupCommand(CommandLineOpts opts) throws Exception {
+ if (opts.getCommand() == null) {
- exitCode = 0;
+ String cmd = opts.getCommand().toString().toLowerCase();
+ int exitCode = 0;
- if ("-checkpoint".equals(cmd)) {
- argv.length == 2 && "force".equals(argv[i])) {
+ switch (opts.getCommand()) {
+ case CHECKPOINT:
+ long count = countUncheckpointedTxns();
+ if (count > checkpointTxnCount ||
+ opts.shouldForceCheckpoint()) {
- System.err.println("EditLog size " + size + " bytes is " +
+ System.err.println("EditLog size " + count + " transactions is " +
"smaller than configured checkpoint " +
- "size " + checkpointSize + " bytes.");
+ "interval " + checkpointTxnCount + " transactions.");
System.err.println("Skipping checkpoint.");
- } else if ("-geteditsize".equals(cmd)) {
- System.out.println("EditLog size is " + size + " bytes");
- exitCode = -1;
- LOG.error(cmd.substring(1) + ": Unknown command");
+ case GETEDITSIZE:
+ long uncheckpointed = countUncheckpointedTxns();
+ System.out.println("NameNode has " + uncheckpointed +
+ " uncheckpointed transactions");
+ throw new AssertionError("bad command enum: " + opts.getCommand());
} catch (RemoteException e) {
// This is a error returned by hadoop server. Print
@@ -551,41 +591,32 @@ public class SecondaryNameNode implements Runnable {
String[] content;
content = e.getLocalizedMessage().split("\n");
- LOG.error(cmd.substring(1) + ": "
- + content[0]);
+ LOG.error(cmd + ": " + content[0]);
} catch (Exception ex) {
- + ex.getLocalizedMessage());
+ LOG.error(cmd + ": " + ex.getLocalizedMessage());
// IO exception encountered locally.
exitCode = -1;
- + e.getLocalizedMessage());
+ LOG.error(cmd + ": " + e.getLocalizedMessage());
// Does the RPC connection need to be closed?
return exitCode;
- * Displays format of commands.
- * @param cmd The command that is being executed.
- private void printUsage(String cmd) {
- System.err.println("Usage: java SecondaryNameNode"
- + " [-geteditsize]");
- + " [-checkpoint [force]]");
- System.err.println("Usage: java SecondaryNameNode " +
- "[-checkpoint [force]] " +
- "[-geteditsize] ");
+ long curTxId = namenode.getTransactionID();
+ checkpointImage.getStorage().getMostRecentCheckpointTxId();
+ boolean shouldCheckpointBasedOnCount() throws IOException {
+ return countUncheckpointedTxns() >= checkpointTxnCount;
@@ -594,41 +625,151 @@ public class SecondaryNameNode implements Runnable {
public static void main(String[] argv) throws Exception {
+ CommandLineOpts opts = SecondaryNameNode.parseArgs(argv);
+ if (opts == null) {
+ System.exit(-1);
StringUtils.startupShutdownMessage(SecondaryNameNode.class, argv, LOG);
Configuration tconf = new HdfsConfiguration();
- if (argv.length >= 1) {
- SecondaryNameNode secondary = new SecondaryNameNode(tconf);
- int ret = secondary.processArgs(argv);
+ SecondaryNameNode secondary = new SecondaryNameNode(tconf, opts);
+ if (opts.getCommand() != null) {
+ int ret = secondary.processStartupCommand(opts);
System.exit(ret);
// Create a never ending deamon
- Daemon checkpointThread = new Daemon(new SecondaryNameNode(tconf));
+ Daemon checkpointThread = new Daemon(secondary);
checkpointThread.start();
+ * Container for parsed command-line options.
+ @SuppressWarnings("static-access")
+ static class CommandLineOpts {
+ private final Options options = new Options();
+ private final Option geteditsizeOpt;
+ private final Option checkpointOpt;
+ private final Option formatOpt;
+ Command cmd;
+ enum Command {
+ GETEDITSIZE,
+ CHECKPOINT;
+ private boolean shouldForce;
+ private boolean shouldFormat;
+ CommandLineOpts() {
+ geteditsizeOpt = new Option("geteditsize",
+ "return the number of uncheckpointed transactions on the NameNode");
+ checkpointOpt = OptionBuilder.withArgName("force")
+ .hasOptionalArg().withDescription("checkpoint on startup").create("checkpoint");;
+ formatOpt = new Option("format", "format the local storage during startup");
+ options.addOption(geteditsizeOpt);
+ options.addOption(checkpointOpt);
+ options.addOption(formatOpt);
+ public boolean shouldFormat() {
+ return shouldFormat;
+ public void parse(String ... argv) throws ParseException {
+ CommandLineParser parser = new PosixParser();
+ CommandLine cmdLine = parser.parse(options, argv);
+ boolean hasGetEdit = cmdLine.hasOption(geteditsizeOpt.getOpt());
+ boolean hasCheckpoint = cmdLine.hasOption(checkpointOpt.getOpt());
+ if (hasGetEdit && hasCheckpoint) {
+ throw new ParseException("May not pass both "
+ + geteditsizeOpt.getOpt() + " and "
+ + checkpointOpt.getOpt());
+ if (hasGetEdit) {
+ cmd = Command.GETEDITSIZE;
+ } else if (hasCheckpoint) {
+ cmd = Command.CHECKPOINT;
+ String arg = cmdLine.getOptionValue(checkpointOpt.getOpt());
+ if ("force".equals(arg)) {
+ shouldForce = true;
+ } else if (arg != null) {
+ throw new ParseException("-checkpoint may only take 'force' as an "
+ + "argument");
+ if (cmdLine.hasOption(formatOpt.getOpt())) {
+ shouldFormat = true;
+ public Command getCommand() {
+ return cmd;
+ public boolean shouldForceCheckpoint() {
+ return shouldForce;
+ void usage() {
+ HelpFormatter formatter = new HelpFormatter();
+ formatter.printHelp("secondarynamenode", options);
+ private static CommandLineOpts parseArgs(String[] argv) {
+ CommandLineOpts opts = new CommandLineOpts();
+ opts.parse(argv);
+ } catch (ParseException pe) {
+ LOG.error(pe.getMessage());
+ opts.usage();
+ return opts;
static class CheckpointStorage extends FSImage {
+ * Construct a checkpoint image.
+ * @param conf Node configuration.
+ * @param imageDirs URIs of storage for image.
+ * @param editDirs URIs of storage for edit logs.
+ * @throws IOException If storage cannot be access.
- CheckpointStorage(Configuration conf) throws IOException {
- super(conf);
+ CheckpointStorage(Configuration conf,
+ Collection<URI> imageDirs,
+ Collection<URI> editsDirs) throws IOException {
+ super(conf, (FSNamesystem)null, imageDirs, editsDirs);
+ setFSNamesystem(new FSNamesystem(this, conf));
+ // the 2NN never writes edits -- it only downloads them. So
+ // we shouldn't have any editLog instance. Setting to null
+ // makes sure we don't accidentally depend on it.
+ editLog = null;
* Analyze checkpoint directories.
* Create directories if they do not exist.
- * Recover from an unsuccessful checkpoint is necessary.
- * @param dataDirs
- * @param editsDirs
+ * Recover from an unsuccessful checkpoint is necessary.
- void recoverCreate(Collection<URI> dataDirs,
- Collection<URI> tempDataDirs = new ArrayList<URI>(dataDirs);
- Collection<URI> tempEditsDirs = new ArrayList<URI>(editsDirs);
- storage.close();
- storage.setStorageDirectories(tempDataDirs, tempEditsDirs);
+ void recoverCreate(boolean format) throws IOException {
+ storage.unlockAll();
storage.dirIterator(); it.hasNext();) {
@@ -643,6 +784,13 @@ public class SecondaryNameNode implements Runnable {
if(!isAccessible)
throw new InconsistentFSStateException(sd.getRoot(),
"cannot access checkpoint directory.");
+ if (format) {
+ // Don't confirm, since this is just the secondary namenode.
+ LOG.info("Formatting storage directory " + sd);
+ sd.clearDirectory();
curState = sd.analyzeStorage(HdfsConstants.StartupOption.REGULAR, storage);
@@ -655,6 +803,11 @@ public class SecondaryNameNode implements Runnable {
case NOT_FORMATTED:
break; // it's ok since initially there is no current and VERSION
case NORMAL:
+ // Read the VERSION file. This verifies that:
+ // (a) the VERSION file for each of the directories is the same,
+ // and (b) when we connect to a NN, we can verify that the remote
+ // node matches the same namespace that we ran on previously.
+ storage.readProperties(sd);
default: // recovery is possible
sd.doRecover(curState);
@@ -665,63 +818,41 @@ public class SecondaryNameNode implements Runnable {
- * Prepare directories for a new checkpoint.
- * and recreate <code>current</code>.
+ * Ensure that the current/ directory exists in all storage
+ * directories
- void startCheckpoint() throws IOException {
+ void ensureCurrentDirExists() throws IOException {
for (Iterator<StorageDirectory> it
= storage.dirIterator(); it.hasNext();) {
- void endCheckpoint() throws IOException {
- = storage.dirIterator(); it.hasNext();) {
- * Merge image and edits, and verify consistency with the signature.
- getEditLog().open();
- StorageDirectory sdName = null;
- StorageDirectory sdEdits = null;
- Iterator<StorageDirectory> it = null;
- it = getStorage().dirIterator(NameNodeDirType.IMAGE);
- if (it.hasNext())
- sdName = it.next();
- if (sdName == null) {
- throw new IOException("Could not locate checkpoint fsimage");
+ File curDir = sd.getCurrentDir();
+ if (!curDir.exists() && !curDir.mkdirs()) {
+ throw new IOException("Could not create directory " + curDir);
- it = getStorage().dirIterator(NameNodeDirType.EDITS);
- sdEdits = it.next();
- if (sdEdits == null)
- throw new IOException("Could not locate checkpoint edits");
- // to avoid assert in loadFSImage()
- this.getStorage().layoutVersion = -1;
- getStorage();
+ static void doMerge(
+ CheckpointSignature sig, RemoteEditLogManifest manifest,
+ boolean loadImage, FSImage dstImage) throws IOException {
+ if (loadImage) {
+ File file = dstStorage.findImageFile(sig.mostRecentCheckpointTxId);
+ if (file == null) {
+ throw new IOException("Couldn't find image file at txid " +
+ sig.mostRecentCheckpointTxId + " even though it should have " +
+ "just been downloaded");
- storage.setClusterID(sig.getClusterID());
- storage.setBlockPoolID(sig.getBlockpoolID());
- sig.validateStorageInfo(this);
+ dstImage.reloadFromImageFile(file);
+ Checkpointer.rollForwardByApplyingLogs(manifest, dstImage);
+ dstImage.saveFSImageInAllDirs(dstImage.getLastAppliedTxId());
+ dstStorage.writeAll();
@@ -21,19 +21,22 @@ import java.io.*;
import java.net.*;
import java.security.DigestInputStream;
import java.security.MessageDigest;
-import java.util.Iterator;
-import java.util.Map;
import java.lang.Math;
-import javax.servlet.http.HttpServletResponse;
-import javax.servlet.http.HttpServletRequest;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
* This class provides fetching a specified file from the NameNode.
@@ -41,88 +44,77 @@ import org.apache.hadoop.security.UserGroupInformation;
class TransferFsImage implements FSConstants {
public final static String CONTENT_LENGTH = "Content-Length";
- private boolean isGetImage;
- private boolean isGetEdit;
- private boolean isPutImage;
- private int remoteport;
- private String machineName;
- private CheckpointSignature token;
- private MD5Hash newChecksum = null;
- * File downloader.
- * @param pmap key=value[] map that is passed to the http servlet as
- * url parameters
- * @param request the object from which this servelet reads the url contents
- * @param response the object into which this servelet writes the url contents
- public TransferFsImage(Map<String,String[]> pmap,
- HttpServletRequest request,
- HttpServletResponse response
- isGetImage = isGetEdit = isPutImage = false;
- remoteport = 0;
- machineName = null;
- token = null;
- for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) {
- String key = it.next();
- if (key.equals("getimage")) {
- isGetImage = true;
- } else if (key.equals("getedit")) {
- isGetEdit = true;
- } else if (key.equals("putimage")) {
- isPutImage = true;
- } else if (key.equals("port")) {
- remoteport = new Integer(pmap.get("port")[0]).intValue();
- } else if (key.equals("machine")) {
- machineName = pmap.get("machine")[0];
- } else if (key.equals("token")) {
- token = new CheckpointSignature(pmap.get("token")[0]);
- } else if (key.equals("newChecksum")) {
- newChecksum = new MD5Hash(pmap.get("newChecksum")[0]);
- int numGets = (isGetImage?1:0) + (isGetEdit?1:0);
- if ((numGets > 1) || (numGets == 0) && !isPutImage) {
- throw new IOException("Illegal parameters to TransferFsImage");
+ public final static String MD5_HEADER = "X-MD5-Digest";
- boolean getEdit() {
- return isGetEdit;
+ private static final Log LOG = LogFactory.getLog(TransferFsImage.class);
- boolean getImage() {
- return isGetImage;
+ static MD5Hash downloadImageToStorage(
+ String fsName, long imageTxId, NNStorage dstStorage, boolean needDigest)
+ String fileid = GetImageServlet.getParamStringForImage(
+ imageTxId, dstStorage);
+ String fileName = NNStorage.getCheckpointImageFileName(imageTxId);
+ List<File> dstFiles = dstStorage.getFiles(
+ NameNodeDirType.IMAGE, fileName);
+ if (dstFiles.isEmpty()) {
+ throw new IOException("No targets in destination storage!");
+ MD5Hash hash = getFileClient(fsName, fileid, dstFiles, dstStorage, needDigest);
+ LOG.info("Downloaded file " + dstFiles.get(0).getName() + " size " +
+ dstFiles.get(0).length() + " bytes.");
+ return hash;
+ static void downloadEditsToStorage(String fsName, RemoteEditLog log,
+ NNStorage dstStorage) throws IOException {
+ String fileid = GetImageServlet.getParamStringForLog(
+ log, dstStorage);
+ String fileName = NNStorage.getFinalizedEditsFileName(
- boolean putImage() {
- return isPutImage;
+ List<File> dstFiles = dstStorage.getFiles(NameNodeDirType.EDITS, fileName);
+ assert !dstFiles.isEmpty() : "No checkpoint targets.";
+ for (File f : dstFiles) {
+ if (f.exists() && f.canRead()) {
+ LOG.info("Skipping download of remote edit log " +
+ log + " since it already is stored locally at " + f);
+ LOG.debug("Dest file: " + f);
- CheckpointSignature getToken() {
- return token;
+ getFileClient(fsName, fileid, dstFiles, dstStorage, false);
- * Get the MD5 digest of the new image
- * @return the MD5 digest of the new image
+ * Requests that the NameNode download an image from this node.
+ * @param fsName the http address for the remote NN
+ * @param imageListenAddress the host/port where the local node is running an
+ * HTTPServer hosting GetImageServlet
+ * @param storage the storage directory to transfer the image from
+ * @param txid the transaction ID of the image to be uploaded
- MD5Hash getNewChecksum() {
- return newChecksum;
- String getInfoServer() throws IOException{
- if (machineName == null || remoteport == 0) {
- throw new IOException ("MachineName and port undefined");
- return machineName + ":" + remoteport;
+ static void uploadImageFromStorage(String fsName,
+ InetSocketAddress imageListenAddress,
+ NNStorage storage, long txid) throws IOException {
+ String fileid = GetImageServlet.getParamStringToPutImage(
+ txid, imageListenAddress, storage);
+ // this doesn't directly upload an image, but rather asks the NN
+ // to connect back to the 2NN to download the specified image.
+ TransferFsImage.getFileClient(fsName, fileid, null, null, false);
+ LOG.info("Uploaded image with txid " + txid + " to namenode at " +
+ fsName);
* A server-side method to respond to a getfile http request
* Copies the contents of the local file into the output stream.
@@ -156,6 +148,13 @@ class TransferFsImage implements FSConstants {
if (num <= 0) {
+ if (ErrorSimulator.getErrorSimulation(4)) {
+ // Simulate a corrupted byte on the wire
+ LOG.warn("SIMULATING A CORRUPT BYTE IN IMAGE TRANSFER!");
+ buf[0]++;
outstream.write(buf, 0, num);
if (throttler != null) {
throttler.throttle(num);
@@ -171,16 +170,17 @@ class TransferFsImage implements FSConstants {
* Client-side Method to fetch file from a server
* Copies the response from the URL to a list of local files.
+ * @param dstStorage if an error occurs writing to one of the files,
+ * this storage object will be notified.
* @Return a digest of the received file if getChecksum is true
- static MD5Hash getFileClient(String fsName, String id, File[] localPath,
- boolean getChecksum)
+ static MD5Hash getFileClient(String nnHostPort,
+ String queryString, List<File> localPaths,
+ NNStorage dstStorage, boolean getChecksum) throws IOException {
byte[] buf = new byte[BUFFER_SIZE];
String proto = UserGroupInformation.isSecurityEnabled() ? "https://" : "http://";
- StringBuilder str = new StringBuilder(proto+fsName+"/getimage?");
- str.append(id);
+ StringBuilder str = new StringBuilder(proto+nnHostPort+"/getimage?");
+ str.append(queryString);
// open connection to remote server
@@ -189,7 +189,15 @@ class TransferFsImage implements FSConstants {
// Avoid Krb bug with cross-realm hosts
SecurityUtil.fetchServiceTicket(url);
- URLConnection connection = url.openConnection();
+ HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+ if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
+ "Image transfer servlet at " + url +
+ " failed with status code " + connection.getResponseCode() +
+ "\nResponse message:\n" + connection.getResponseMessage());
long advertisedSize;
String contentLength = connection.getHeaderField(CONTENT_LENGTH);
if (contentLength != null) {
@@ -198,6 +206,9 @@ class TransferFsImage implements FSConstants {
throw new IOException(CONTENT_LENGTH + " header is not provided " +
"by the namenode when trying to fetch " + str);
+ MD5Hash advertisedDigest = parseMD5Header(connection);
long received = 0;
InputStream stream = connection.getInputStream();
MessageDigest digester = null;
@@ -205,36 +216,47 @@ class TransferFsImage implements FSConstants {
digester = MD5Hash.getDigester();
stream = new DigestInputStream(stream, digester);
- FileOutputStream[] output = null;
boolean finishedReceiving = false;
+ List<FileOutputStream> outputStreams = Lists.newArrayList();
- if (localPath != null) {
- output = new FileOutputStream[localPath.length];
- for (int i = 0; i < output.length; i++) {
- output[i] = new FileOutputStream(localPath[i]);
+ if (localPaths != null) {
+ for (File f : localPaths) {
+ if (f.exists()) {
+ LOG.warn("Overwriting existing file " + f
+ + " with file downloaded from " + str);
+ outputStreams.add(new FileOutputStream(f));
+ LOG.warn("Unable to download file " + f, ioe);
+ dstStorage.reportErrorOnFile(f);
+ if (outputStreams.isEmpty()) {
+ "Unable to download to any storage directory");
int num = 1;
while (num > 0) {
num = stream.read(buf);
- if (num > 0 && localPath != null) {
+ if (num > 0) {
received += num;
- output[i].write(buf, 0, num);
+ for (FileOutputStream fos : outputStreams) {
+ fos.write(buf, 0, num);
finishedReceiving = true;
stream.close();
- if (output != null) {
- if (output[i] != null) {
- output[i].getChannel().force(true);
- output[i].close();
+ fos.getChannel().force(true);
+ fos.close();
if (finishedReceiving && received != advertisedSize) {
// only throw this exception if we think we read all of it on our end
@@ -245,6 +267,25 @@ class TransferFsImage implements FSConstants {
advertisedSize);
- return digester==null ? null : new MD5Hash(digester.digest());
+ if (digester != null) {
+ MD5Hash computedDigest = new MD5Hash(digester.digest());
+ if (advertisedDigest != null &&
+ !computedDigest.equals(advertisedDigest)) {
+ throw new IOException("File " + str + " computed digest " +
+ computedDigest + " does not match advertised digest " +
+ advertisedDigest);
+ return computedDigest;
+ private static MD5Hash parseMD5Header(HttpURLConnection connection) {
+ String header = connection.getHeaderField(MD5_HEADER);
+ return (header != null) ? new MD5Hash(header) : null;
@@ -47,19 +47,16 @@ import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
public class CheckpointCommand extends NamenodeCommand {
private CheckpointSignature cSig;
- private boolean isImageObsolete;
private boolean needToReturnImage;
public CheckpointCommand() {
- this(null, false, false);
+ this(null, false);
public CheckpointCommand(CheckpointSignature sig,
- boolean isImgObsolete,
boolean needToReturnImg) {
super(NamenodeProtocol.ACT_CHECKPOINT);
this.cSig = sig;
- this.isImageObsolete = isImgObsolete;
this.needToReturnImage = needToReturnImg;
@@ -71,16 +68,6 @@ public class CheckpointCommand extends NamenodeCommand {
return cSig;
- * Indicates whether current backup image is obsolete, and therefore
- * need to be discarded?
- * @return true if current image should be discarded.
- public boolean isImageObsolete() {
- return isImageObsolete;
* Indicates whether the new checkpoint image needs to be transfered
* back to the name-node after the checkpoint is done.
@@ -104,7 +91,6 @@ public class CheckpointCommand extends NamenodeCommand {
cSig.write(out);
- out.writeBoolean(isImageObsolete);
out.writeBoolean(needToReturnImage);
@@ -112,7 +98,6 @@ public class CheckpointCommand extends NamenodeCommand {
cSig = new CheckpointSignature();
cSig.readFields(in);
- isImageObsolete = in.readBoolean();
needToReturnImage = in.readBoolean();
@@ -0,0 +1,62 @@
+package org.apache.hadoop.hdfs.server.protocol;
+import org.apache.hadoop.ipc.VersionedProtocol;
+import org.apache.hadoop.security.KerberosInfo;
+ * Protocol used to journal edits to a remote node. Currently,
+ * this is used to publish edits from the NameNode to a BackupNode.
+@KerberosInfo(
+ serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
+ clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
+public interface JournalProtocol extends VersionedProtocol {
+ public static final long versionID = 1L;
+ * Journal edit records.
+ * This message is sent by the active name-node to the backup node
+ * via {@code EditLogBackupOutputStream} in order to synchronize meta-data
+ * changes with the backup namespace image.
+ * @param registration active node registration
+ * @param firstTxnId the first transaction of this batch
+ * @param records byte array containing serialized journal records
+ public void journal(NamenodeRegistration registration,
+ long firstTxnId,
+ int numTxns,
+ byte[] records) throws IOException;
+ * Notify the BackupNode that the NameNode has rolled its edit logs
+ * and is now writing a new log segment.
+ * @param registration the registration of the active NameNode
+ * @param txid the first txid in the new log
+ public void startLogSegment(NamenodeRegistration registration,
+ long txid) throws IOException;
@@ -42,21 +42,14 @@ public interface NamenodeProtocol extends VersionedProtocol {
* (Only the latest change is reflected.
* The log of historical changes can be retrieved from the svn).
- * 5: Added one parameter to rollFSImage() and
- * changed the definition of CheckpointSignature
+ * 6: Switch to txid-based file naming for image and edits
- public static final long versionID = 5L;
+ public static final long versionID = 6L;
// Error codes passed by errorReport().
final static int NOTIFY = 0;
final static int FATAL = 1;
- // Journal action codes. See journal().
- public static byte JA_IS_ALIVE = 100; // check whether the journal is alive
- public static byte JA_JOURNAL = 101; // just journal
- public static byte JA_JSPOOL_START = 102; // = FSEditLogOpCodes.OP_JSPOOL_START
- public static byte JA_CHECKPOINT_TIME = 103; // = FSEditLogOpCodes.OP_CHECKPOINT_TIME
public final static int ACT_UNKNOWN = 0; // unknown action
public final static int ACT_SHUTDOWN = 50; // shutdown node
public final static int ACT_CHECKPOINT = 51; // do checkpoint
@@ -84,14 +77,11 @@ public interface NamenodeProtocol extends VersionedProtocol {
public ExportedBlockKeys getBlockKeys() throws IOException;
- * Get the size of the current edit log (in bytes).
- * @return The number of bytes in the current edit log.
+ * @return The most recent transaction ID that has been synced to
+ * persistent storage.
- * @deprecated
- * See {@link org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode}
- public long getEditLogSize() throws IOException;
+ public long getTransactionID() throws IOException;
* Closes the current edit log and opens a new one. The
@@ -104,20 +94,6 @@ public interface NamenodeProtocol extends VersionedProtocol {
@Deprecated
public CheckpointSignature rollEditLog() throws IOException;
- * Rolls the fsImage log. It removes the old fsImage, copies the
- * new image to fsImage, removes the old edits and renames edits.new
- * to edits. The call fails if any of the four files are missing.
- * @param sig the signature of this checkpoint (old fsimage)
- public void rollFsImage(CheckpointSignature sig)
- throws IOException;
* Request name-node version and storage information.
@@ -177,31 +153,14 @@ public interface NamenodeProtocol extends VersionedProtocol {
CheckpointSignature sig) throws IOException;
- * Get the size of the active name-node journal (edit log) in bytes.
- * @param registration the requesting node
- * @return The number of bytes in the journal.
- public long journalSize(NamenodeRegistration registration) throws IOException;
- * Journal edit records.
- * This message is sent by the active name-node to the backup node
- * via {@code EditLogBackupOutputStream} in order to synchronize meta-data
- * changes with the backup namespace image.
- * @param registration active node registration
- * @param jAction journal action
- * @param length length of the byte array
- * @param records byte array containing serialized journal records
+ * Return a structure containing details about all edit logs
+ * available to be fetched from the NameNode.
+ * @param sinceTxId return only logs that contain transactions >= sinceTxId
- byte[] records) throws IOException;
@@ -43,7 +43,6 @@ implements NodeRegistration {
String rpcAddress; // RPC address of the node
String httpAddress; // HTTP address of the node
NamenodeRole role; // node role
- long checkpointTime = -1L; // the age of the image
public NamenodeRegistration() {
super();
@@ -52,14 +51,12 @@ implements NodeRegistration {
public NamenodeRegistration(String address,
String httpAddress,
StorageInfo storageInfo,
- NamenodeRole role,
- long checkpointTime) {
+ NamenodeRole role) {
this.rpcAddress = address;
this.httpAddress = httpAddress;
this.setStorageInfo(storageInfo);
this.role = role;
- this.checkpointTime = checkpointTime;
@Override // NodeRegistration
@@ -96,13 +93,6 @@ implements NodeRegistration {
return role.equals(that);
- * Get the age of the image.
// Writable
@@ -120,7 +110,6 @@ implements NodeRegistration {
Text.writeString(out, httpAddress);
Text.writeString(out, role.name());
@Override // Writable
@@ -129,6 +118,5 @@ implements NodeRegistration {
httpAddress = Text.readString(in);
role = NamenodeRole.valueOf(Text.readString(in));
@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.hdfs.DeprecatedUTF8;
import org.apache.hadoop.io.WritableFactories;
@@ -98,4 +99,26 @@ public class NamespaceInfo extends StorageInfo {
public String toString(){
return super.toString() + ";bpid=" + blockPoolID;
+ public void validateStorage(NNStorage storage) throws IOException {
+ if (layoutVersion != storage.getLayoutVersion() ||
+ namespaceID != storage.getNamespaceID() ||
+ cTime != storage.cTime ||
+ !clusterID.equals(storage.getClusterID()) ||
+ !blockPoolID.equals(storage.getBlockPoolID())) {
+ throw new IOException("Inconsistent namespace information:\n" +
+ "NamespaceInfo has:\n" +
+ "LV=" + layoutVersion + ";" +
+ "NS=" + namespaceID + ";" +
+ "cTime=" + cTime + ";" +
+ "CID=" + clusterID + ";" +
+ "BPID=" + blockPoolID +
+ ".\nStorage has:\n" +
+ "LV=" + storage.getLayoutVersion() + ";" +
+ "NS=" + storage.getNamespaceID() + ";" +
+ "cTime=" + storage.getCTime() + ";" +
+ "CID=" + storage.getClusterID() + ";" +
+ "BPID=" + storage.getBlockPoolID() + ".");
@@ -0,0 +1,64 @@
+import java.io.DataInput;
+import java.io.DataOutput;
+public class RemoteEditLog implements Writable {
+ private long startTxId = FSConstants.INVALID_TXID;
+ private long endTxId = FSConstants.INVALID_TXID;
+ public RemoteEditLog() {
+ public RemoteEditLog(long startTxId, long endTxId) {
+ this.endTxId = endTxId;
+ return "[" + startTxId + "," + endTxId + "]";
+ public void write(DataOutput out) throws IOException {
+ out.writeLong(startTxId);
+ out.writeLong(endTxId);
+ public void readFields(DataInput in) throws IOException {
+ startTxId = in.readLong();
+ endTxId = in.readLong();
+ * An enumeration of logs available on a remote NameNode.
+public class RemoteEditLogManifest implements Writable {
+ private List<RemoteEditLog> logs;
+ public RemoteEditLogManifest() {
+ public RemoteEditLogManifest(List<RemoteEditLog> logs) {
+ this.logs = logs;
+ checkState();
+ * Check that the logs are contiguous and non-overlapping
+ * sequences of transactions, in sorted order
+ * @throws IllegalStateException if incorrect
+ private void checkState() {
+ Preconditions.checkNotNull(logs);
+ RemoteEditLog prev = null;
+ for (RemoteEditLog log : logs) {
+ if (prev != null) {
+ if (log.getStartTxId() != prev.getEndTxId() + 1) {
+ throw new IllegalStateException("Invalid log manifest:" + this);
+ prev = log;
+ public List<RemoteEditLog> getLogs() {
+ return Collections.unmodifiableList(logs);
+ return "[" + Joiner.on(", ").join(logs) + "]";
+ out.writeInt(logs.size());
+ log.write(out);
+ int numLogs = in.readInt();
+ logs = Lists.newArrayList();
+ for (int i = 0; i < numLogs; i++) {
+ RemoteEditLog log = new RemoteEditLog();
+ log.readFields(in);
@@ -31,6 +31,7 @@ public enum EditsElement {
EDITS_VERSION,
RECORD,
OPCODE,
+ TRANSACTION_ID,
DATA,
// elements in the data part of the editLog records
LENGTH,
package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
+import java.io.EOFException;
import org.apache.hadoop.classification.InterfaceStability;
@@ -40,7 +41,7 @@ import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.VIntToke
class EditsLoaderCurrent implements EditsLoader {
private static int[] supportedVersions = { -18, -19, -20, -21, -22, -23, -24,
- -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36 };
+ -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38};
private EditsVisitor v;
private int editsVersion = 0;
@@ -64,11 +65,19 @@ class EditsLoaderCurrent implements EditsLoader {
+ * Visit a transaction ID, if the log version supports it.
+ private void visitTxId() throws IOException {
+ if (LayoutVersion.supports(Feature.STORED_TXIDS, editsVersion)) {
+ v.visitLong(EditsElement.TRANSACTION_ID);
* Visit OP_INVALID
private void visit_OP_INVALID() throws IOException {
- ; // nothing to do, this op code has no data
@@ -92,6 +101,7 @@ class EditsLoaderCurrent implements EditsLoader {
private void visit_OP_ADD_or_OP_CLOSE(FSEditLogOpCodes editsOpCode)
+ visitTxId();
IntToken opAddLength = v.visitInt(EditsElement.LENGTH);
// this happens if the edits is not properly ended (-1 op code),
@@ -135,6 +145,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_RENAME_OLD
private void visit_OP_RENAME_OLD() throws IOException {
v.visitInt( EditsElement.LENGTH);
v.visitStringUTF8( EditsElement.SOURCE);
v.visitStringUTF8( EditsElement.DESTINATION);
@@ -145,6 +157,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_DELETE
private void visit_OP_DELETE() throws IOException {
v.visitStringUTF8( EditsElement.PATH);
v.visitStringUTF8( EditsElement.TIMESTAMP);
@@ -154,6 +168,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_MKDIR
private void visit_OP_MKDIR() throws IOException {
@@ -172,6 +188,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_SET_REPLICATION
private void visit_OP_SET_REPLICATION() throws IOException {
v.visitStringUTF8(EditsElement.PATH);
v.visitStringUTF8(EditsElement.REPLICATION);
@@ -180,6 +198,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_SET_PERMISSIONS
private void visit_OP_SET_PERMISSIONS() throws IOException {
v.visitShort( EditsElement.FS_PERMISSIONS);
@@ -188,6 +208,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_SET_OWNER
private void visit_OP_SET_OWNER() throws IOException {
v.visitStringUTF8(EditsElement.USERNAME);
v.visitStringUTF8(EditsElement.GROUPNAME);
@@ -197,6 +219,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_SET_GENSTAMP
private void visit_OP_SET_GENSTAMP() throws IOException {
v.visitLong(EditsElement.GENERATION_STAMP);
@@ -204,6 +228,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_TIMES
private void visit_OP_TIMES() throws IOException {
v.visitStringUTF8( EditsElement.MTIME);
@@ -214,6 +240,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_SET_QUOTA
private void visit_OP_SET_QUOTA() throws IOException {
v.visitLong( EditsElement.NS_QUOTA);
v.visitLong( EditsElement.DS_QUOTA);
@@ -223,6 +251,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_RENAME
private void visit_OP_RENAME() throws IOException {
@@ -234,6 +264,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_CONCAT_DELETE
private void visit_OP_CONCAT_DELETE() throws IOException {
IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
v.visitStringUTF8(EditsElement.CONCAT_TARGET);
// all except of CONCAT_TARGET and TIMESTAMP
@@ -248,6 +280,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_SYMLINK
private void visit_OP_SYMLINK() throws IOException {
@@ -267,6 +301,8 @@ class EditsLoaderCurrent implements EditsLoader {
* Visit OP_GET_DELEGATION_TOKEN
private void visit_OP_GET_DELEGATION_TOKEN() throws IOException {
v.visitByte( EditsElement.T_VERSION);
v.visitStringText( EditsElement.T_OWNER);
v.visitStringText( EditsElement.T_RENEWER);
@@ -283,6 +319,8 @@ class EditsLoaderCurrent implements EditsLoader {
private void visit_OP_RENEW_DELEGATION_TOKEN()
@@ -299,6 +337,8 @@ class EditsLoaderCurrent implements EditsLoader {
private void visit_OP_CANCEL_DELEGATION_TOKEN()
@@ -314,6 +354,8 @@ class EditsLoaderCurrent implements EditsLoader {
private void visit_OP_UPDATE_MASTER_KEY()
v.visitVInt( EditsElement.KEY_ID);
v.visitVLong( EditsElement.KEY_EXPIRY_DATE);
VIntToken blobLengthToken = v.visitVInt(EditsElement.KEY_LENGTH);
@@ -322,11 +364,29 @@ class EditsLoaderCurrent implements EditsLoader {
private void visit_OP_REASSIGN_LEASE()
v.visitStringUTF8(EditsElement.CLIENT_NAME);
+ * Visit OP_BEGIN_LOG_SEGMENT
+ private void visit_OP_BEGIN_LOG_SEGMENT()
+ * Visit OP_END_LOG_SEGMENT
+ private void visit_OP_END_LOG_SEGMENT()
private void visitOpCode(FSEditLogOpCodes editsOpCode)
@@ -391,6 +451,12 @@ class EditsLoaderCurrent implements EditsLoader {
case OP_REASSIGN_LEASE: // 22
visit_OP_REASSIGN_LEASE();
+ case OP_END_LOG_SEGMENT: // 23
+ visit_OP_END_LOG_SEGMENT();
+ case OP_START_LOG_SEGMENT: // 24
+ visit_OP_BEGIN_LOG_SEGMENT();
default:
{
throw new IOException("Unknown op code " + editsOpCode);
@@ -419,7 +485,17 @@ class EditsLoaderCurrent implements EditsLoader {
do {
v.visitEnclosingElement(EditsElement.RECORD);
- ByteToken opCodeToken = v.visitByte(EditsElement.OPCODE);
+ ByteToken opCodeToken;
+ opCodeToken = v.visitByte(EditsElement.OPCODE);
+ } catch (EOFException eof) {
+ // Getting EOF when reading the opcode is fine --
+ // it's just a finalized edits file
+ // Just fake the OP_INVALID here.
+ opCodeToken = new ByteToken(EditsElement.OPCODE);
+ opCodeToken.fromByte(FSEditLogOpCodes.OP_INVALID.getOpCode());
+ v.visit(opCodeToken);
editsOpCode = FSEditLogOpCodes.fromByte(opCodeToken.value);
v.visitEnclosingElement(EditsElement.DATA);
@@ -17,18 +17,12 @@
-import java.io.BufferedInputStream;
import java.io.EOFException;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@@ -104,6 +104,10 @@ interface Tokenizer {
public void fromBinary(DataInputStream in) throws IOException {
value = in.readByte();
+ public void fromByte(byte b) {
+ value = b;
@@ -122,7 +122,7 @@ class ImageLoaderCurrent implements ImageLoader {
protected final DateFormat dateFormat =
new SimpleDateFormat("yyyy-MM-dd HH:mm");
private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
- -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36 };
+ -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38};
private int imageVersion = 0;
/* (non-Javadoc)
@@ -157,6 +157,10 @@ class ImageLoaderCurrent implements ImageLoader {
v.visit(ImageElement.GENERATION_STAMP, in.readLong());
+ if (LayoutVersion.supports(Feature.STORED_TXIDS, imageVersion)) {
+ v.visit(ImageElement.TRANSACTION_ID, in.readLong());
if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
boolean isCompressed = in.readBoolean();
v.visit(ImageElement.IS_COMPRESSED, imageVersion);
@@ -71,7 +71,8 @@ abstract class ImageVisitor {
NUM_DELEGATION_TOKENS,
DELEGATION_TOKENS,
DELEGATION_TOKEN_IDENTIFIER,
- DELEGATION_TOKEN_EXPIRY_TIME
+ DELEGATION_TOKEN_EXPIRY_TIME,
+ TRANSACTION_ID
@@ -0,0 +1,94 @@
+package org.apache.hadoop.hdfs.util;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+ * A FileOutputStream that has the property that it will only show
+ * up at its destination once it has been entirely written and flushed
+ * to disk. While being written, it will use a .tmp suffix.
+ * When the output stream is closed, it is flushed, fsynced, and
+ * will be moved into place, overwriting any file that already
+ * exists at that location.
+ * <b>NOTE</b>: on Windows platforms, it will not atomically
+ * replace the target file - instead the target file is deleted
+ * before this one is moved into place.
+public class AtomicFileOutputStream extends FilterOutputStream {
+ private static final String TMP_EXTENSION = ".tmp";
+ private final static Log LOG = LogFactory.getLog(
+ AtomicFileOutputStream.class);
+ private final File origFile;
+ private final File tmpFile;
+ public AtomicFileOutputStream(File f) throws FileNotFoundException {
+ // Code unfortunately must be duplicated below since we can't assign anything
+ // before calling super
+ super(new FileOutputStream(new File(f.getParentFile(), f.getName() + TMP_EXTENSION)));
+ origFile = f.getAbsoluteFile();
+ tmpFile = new File(f.getParentFile(), f.getName() + TMP_EXTENSION).getAbsoluteFile();
+ public void close() throws IOException {
+ boolean triedToClose = false, success = false;
+ flush();
+ ((FileOutputStream)out).getChannel().force(true);
+ triedToClose = true;
+ super.close();
+ if (success) {
+ boolean renamed = tmpFile.renameTo(origFile);
+ if (!renamed) {
+ // On windows, renameTo does not replace.
+ if (!origFile.delete() || !tmpFile.renameTo(origFile)) {
+ throw new IOException("Could not rename temporary file " +
+ tmpFile + " to " + origFile);
+ if (!triedToClose) {
+ // If we failed when flushing, try to close it to not leak an FD
+ IOUtils.closeStream(out);
+ // close wasn't successful, try to delete the tmp file
+ if (!tmpFile.delete()) {
+ LOG.warn("Unable to delete tmp file " + tmpFile);
@@ -0,0 +1,153 @@
+import java.io.InputStream;
+import java.security.DigestInputStream;
+import java.security.MessageDigest;
+ * Static functions for dealing with files of the same format
+ * that the Unix "md5sum" utility writes.
+public abstract class MD5FileUtils {
+ MD5FileUtils.class);
+ private static final String MD5_SUFFIX = ".md5";
+ private static final Pattern LINE_REGEX =
+ Pattern.compile("([0-9a-f]{32}) [ \\*](.+)");
+ * Verify that the previously saved md5 for the given file matches
+ * expectedMd5.
+ public static void verifySavedMD5(File dataFile, MD5Hash expectedMD5)
+ MD5Hash storedHash = readStoredMd5ForFile(dataFile);
+ // Check the hash itself
+ if (!expectedMD5.equals(storedHash)) {
+ "File " + dataFile + " did not match stored MD5 checksum " +
+ " (stored: " + storedHash + ", computed: " + expectedMD5);
+ * Read the md5 checksum stored alongside the given file, or null
+ * if no md5 is stored.
+ * @param dataFile the file containing data
+ * @return the checksum stored in dataFile.md5
+ public static MD5Hash readStoredMd5ForFile(File dataFile) throws IOException {
+ File md5File = getDigestFileForFile(dataFile);
+ String md5Line;
+ if (!md5File.exists()) {
+ BufferedReader reader =
+ new BufferedReader(new FileReader(md5File));
+ md5Line = reader.readLine();
+ if (md5Line == null) { md5Line = ""; }
+ md5Line = md5Line.trim();
+ throw new IOException("Error reading md5 file at " + md5File, ioe);
+ IOUtils.cleanup(LOG, reader);
+ Matcher matcher = LINE_REGEX.matcher(md5Line);
+ if (!matcher.matches()) {
+ throw new IOException("Invalid MD5 file at " + md5File
+ + " (does not match expected pattern)");
+ String storedHash = matcher.group(1);
+ File referencedFile = new File(matcher.group(2));
+ // Sanity check: Make sure that the file referenced in the .md5 file at
+ // least has the same name as the file we expect
+ if (!referencedFile.getName().equals(dataFile.getName())) {
+ "MD5 file at " + md5File + " references file named " +
+ referencedFile.getName() + " but we expected it to reference " +
+ dataFile);
+ return new MD5Hash(storedHash);
+ * Read dataFile and compute its MD5 checksum.
+ public static MD5Hash computeMd5ForFile(File dataFile) throws IOException {
+ InputStream in = new FileInputStream(dataFile);
+ MessageDigest digester = MD5Hash.getDigester();
+ DigestInputStream dis = new DigestInputStream(in, digester);
+ IOUtils.copyBytes(dis, new IOUtils.NullOutputStream(), 128*1024);
+ return new MD5Hash(digester.digest());
+ IOUtils.closeStream(in);
+ * Save the ".md5" file that lists the md5sum of another file.
+ * @param dataFile the original file whose md5 was computed
+ * @param digest the computed digest
+ public static void saveMD5File(File dataFile, MD5Hash digest)
+ String digestString = StringUtils.byteToHexString(
+ digest.getDigest());
+ String md5Line = digestString + " *" + dataFile.getName() + "\n";
+ AtomicFileOutputStream afos = new AtomicFileOutputStream(md5File);
+ afos.write(md5Line.getBytes());
+ afos.close();
+ LOG.debug("Saved MD5 " + digest + " to " + md5File);
+ * @return a reference to the file with .md5 suffix that will
+ * contain the md5 checksum for the given data file.
+ public static File getDigestFileForFile(File file) {
+ return new File(file.getParentFile(), file.getName() + MD5_SUFFIX);
@@ -229,6 +229,15 @@
<Bug pattern="REC_CATCH_EXCEPTION" />
</Match>
+ <!--
+ lastAppliedTxid is carefully unsynchronized in the BackupNode in a couple spots.
+ See the comments in BackupImage for justification.
+ -->
+ <Match>
+ <Class name="org.apache.hadoop.hdfs.server.namenode.FSImage" />
+ <Field name="lastAppliedTxId" />
+ <Bug pattern="IS2_INCONSISTENT_SYNC" />
+ </Match>
<!--
Findbugs doesn't realize that closing a FilterOutputStream pushes the close down to
wrapped streams, too.
@@ -95,6 +95,7 @@ public class MiniDFSCluster {
public static class Builder {
private int nameNodePort = 0;
+ private int nameNodeHttpPort = 0;
private final Configuration conf;
private int numNameNodes = 1;
private int numDataNodes = 1;
@@ -130,6 +131,14 @@ public class MiniDFSCluster {
this.nameNodePort = val;
return this;
+ * Default: 0
+ public Builder nameNodeHttpPort(int val) {
+ this.nameNodeHttpPort = val;
* Default: 1
@@ -247,6 +256,7 @@ public class MiniDFSCluster {
builder.federation = true;
initMiniDFSCluster(builder.nameNodePort,
+ builder.nameNodeHttpPort,
builder.conf,
builder.numDataNodes,
builder.format,
@@ -473,12 +483,13 @@ public class MiniDFSCluster {
String[] racks, String hosts[],
long[] simulatedCapacities) throws IOException {
this.nameNodes = new NameNodeInfo[1]; // Single namenode in the cluster
- initMiniDFSCluster(nameNodePort, conf, numDataNodes, format,
+ initMiniDFSCluster(nameNodePort, 0, conf, numDataNodes, format,
manageNameDfsDirs, manageDataDfsDirs, operation, racks, hosts,
simulatedCapacities, null, true, false, false);
- private void initMiniDFSCluster(int nameNodePort, Configuration conf,
+ private void initMiniDFSCluster(int nameNodePort, int nameNodeHttpPort,
int numDataNodes, boolean format, boolean manageNameDfsDirs,
boolean manageDataDfsDirs, StartupOption operation, String[] racks,
String[] hosts, long[] simulatedCapacities, String clusterId,
@@ -526,7 +537,8 @@ public class MiniDFSCluster {
if (!federation) {
conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "127.0.0.1:" + nameNodePort);
- conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+ conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:"
+ + nameNodeHttpPort);
NameNode nn = createNameNode(0, conf, numDataNodes, manageNameDfsDirs,
format, operation, clusterId);
nameNodes[0] = new NameNodeInfo(nn, conf);
@@ -18,14 +18,18 @@
package org.apache.hadoop.hdfs;
import junit.framework.TestCase;
-import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_NODE;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
* This test ensures the appropriate response from the system when
@@ -57,14 +61,17 @@ public class TestDFSFinalize extends TestCase {
* because its removal is asynchronous therefore we have no reliable
* way to know when it will happen.
- void checkResult(String[] nameNodeDirs, String[] dataNodeDirs) throws IOException {
+ static void checkResult(String[] nameNodeDirs, String[] dataNodeDirs) throws Exception {
+ List<File> dirs = Lists.newArrayList();
for (int i = 0; i < nameNodeDirs.length; i++) {
- assertTrue(new File(nameNodeDirs[i],"current").isDirectory());
- assertTrue(new File(nameNodeDirs[i],"current/VERSION").isFile());
- assertTrue(new File(nameNodeDirs[i],"current/edits").isFile());
- assertTrue(new File(nameNodeDirs[i],"current/fsimage").isFile());
- assertTrue(new File(nameNodeDirs[i],"current/fstime").isFile());
+ File curDir = new File(nameNodeDirs[i], "current");
+ dirs.add(curDir);
+ FSImageTestUtil.assertReasonableNameCurrentDir(curDir);
+ FSImageTestUtil.assertParallelFilesAreIdentical(
+ dirs, Collections.<String>emptySet());
for (int i = 0; i < dataNodeDirs.length; i++) {
assertEquals(
UpgradeUtilities.checksumContents(
@@ -22,6 +22,8 @@ import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_N
@@ -32,8 +34,11 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
* This test ensures the appropriate response (successful or failure) from
* the system when the system is rolled back under various storage state and
@@ -61,26 +66,26 @@ public class TestDFSRollback extends TestCase {
* Verify that the new current directory is the old previous.
* It is assumed that the server has recovered and rolled back.
- void checkResult(NodeType nodeType, String[] baseDirs) throws IOException {
- switch (nodeType) {
- case NAME_NODE:
- for (int i = 0; i < baseDirs.length; i++) {
- assertTrue(new File(baseDirs[i],"current").isDirectory());
- assertTrue(new File(baseDirs[i],"current/VERSION").isFile());
- assertTrue(new File(baseDirs[i],"current/edits").isFile());
- assertTrue(new File(baseDirs[i],"current/fsimage").isFile());
- assertTrue(new File(baseDirs[i],"current/fstime").isFile());
- case DATA_NODE:
+ void checkResult(NodeType nodeType, String[] baseDirs) throws Exception {
+ List<File> curDirs = Lists.newArrayList();
+ for (String baseDir : baseDirs) {
+ File curDir = new File(baseDir, "current");
+ curDirs.add(curDir);
+ switch (nodeType) {
+ case NAME_NODE:
+ case DATA_NODE:
- UpgradeUtilities.checksumContents(
- nodeType, new File(baseDirs[i],"current")),
- UpgradeUtilities.checksumMasterDataNodeContents());
+ UpgradeUtilities.checksumContents(nodeType, curDir),
+ UpgradeUtilities.checksumMasterDataNodeContents());
+ curDirs, Collections.<String>emptySet());
for (int i = 0; i < baseDirs.length; i++) {
assertFalse(new File(baseDirs[i],"previous").isDirectory());
@@ -241,21 +246,17 @@ public class TestDFSRollback extends TestCase {
log("NameNode rollback with no edits file", numDirs);
UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
- for (File f : baseDirs) {
- FileUtil.fullyDelete(new File(f,"edits"));
+ deleteMatchingFiles(baseDirs, "edits.*");
startNameNodeShouldFail(StartupOption.ROLLBACK,
- "Edits file is not found");
+ "but there are no logs to load");
UpgradeUtilities.createEmptyDirs(nameNodeDirs);
log("NameNode rollback with no image file", numDirs);
- FileUtil.fullyDelete(new File(f,"fsimage"));
+ deleteMatchingFiles(baseDirs, "fsimage_.*");
- "Image file is not found");
+ "No valid image files found");
log("NameNode rollback with corrupt version file", numDirs);
@@ -284,6 +285,16 @@ public class TestDFSRollback extends TestCase {
} // end numDir loop
+ private void deleteMatchingFiles(File[] baseDirs, String regex) {
+ for (File baseDir : baseDirs) {
+ for (File f : baseDir.listFiles()) {
+ if (f.getName().matches(regex)) {
+ f.delete();
protected void tearDown() throws Exception {
LOG.info("Shutting down MiniDFSCluster");
if (cluster != null) cluster.shutdown();
@@ -25,6 +25,7 @@ import org.apache.commons.logging.LogFactory;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
@@ -42,47 +43,54 @@ public class TestDFSStorageStateRecovery extends TestCase {
private int testCounter = 0;
private MiniDFSCluster cluster = null;
+ // Constants for indexes into test case table below.
+ private static final int CURRENT_EXISTS = 0;
+ private static final int PREVIOUS_EXISTS = 1;
+ private static final int PREVIOUS_TMP_EXISTS = 2;
+ private static final int REMOVED_TMP_EXISTS = 3;
+ private static final int SHOULD_RECOVER = 4;
+ private static final int CURRENT_SHOULD_EXIST_AFTER_RECOVER = 5;
+ private static final int PREVIOUS_SHOULD_EXIST_AFTER_RECOVER = 6;
* The test case table. Each row represents a test case. This table is
* taken from the table in Apendix A of the HDFS Upgrade Test Plan
* (TestPlan-HdfsUpgrade.html) attached to
* http://issues.apache.org/jira/browse/HADOOP-702
+ * It has been slightly modified since previouscheckpoint.tmp no longer
+ * exists.
* The column meanings are:
* 0) current directory exists
* 1) previous directory exists
* 2) previous.tmp directory exists
* 3) removed.tmp directory exists
- * 4) lastcheckpoint.tmp directory exists
- * 5) node should recover and startup
- * 6) current directory should exist after recovery but before startup
- * 7) previous directory should exist after recovery but before startup
+ * 4) node should recover and startup
+ * 5) current directory should exist after recovery but before startup
+ * 6) previous directory should exist after recovery but before startup
static boolean[][] testCases = new boolean[][] {
- new boolean[] {true, false, false, false, false, true, true, false}, // 1
- new boolean[] {true, true, false, false, false, true, true, true }, // 2
- new boolean[] {true, false, true, false, false, true, true, true }, // 3
- new boolean[] {true, true, true, true, false, false, false, false}, // 4
- new boolean[] {true, true, true, false, false, false, false, false}, // 4
- new boolean[] {false, true, true, true, false, false, false, false}, // 4
- new boolean[] {false, true, true, false, false, false, false, false}, // 4
- new boolean[] {false, false, false, false, false, false, false, false}, // 5
- new boolean[] {false, true, false, false, false, false, false, false}, // 6
- new boolean[] {false, false, true, false, false, true, true, false}, // 7
- new boolean[] {true, false, false, true, false, true, true, false}, // 8
- new boolean[] {true, true, false, true, false, false, false, false}, // 9
- new boolean[] {true, true, true, true, false, false, false, false}, // 10
- new boolean[] {true, false, true, true, false, false, false, false}, // 10
- new boolean[] {false, true, true, true, false, false, false, false}, // 10
- new boolean[] {false, false, true, true, false, false, false, false}, // 10
- new boolean[] {false, false, false, true, false, false, false, false}, // 11
- new boolean[] {false, true, false, true, false, true, true, true }, // 12
+ new boolean[] {true, false, false, false, true, true, false}, // 1
+ new boolean[] {true, true, false, false, true, true, true }, // 2
+ new boolean[] {true, false, true, false, true, true, true }, // 3
+ new boolean[] {true, true, true, true, false, false, false}, // 4
+ new boolean[] {true, true, true, false, false, false, false}, // 4
+ new boolean[] {false, true, true, true, false, false, false}, // 4
+ new boolean[] {false, true, true, false, false, false, false}, // 4
+ new boolean[] {false, false, false, false, false, false, false}, // 5
+ new boolean[] {false, true, false, false, false, false, false}, // 6
+ new boolean[] {false, false, true, false, true, true, false}, // 7
+ new boolean[] {true, false, false, true, true, true, false}, // 8
+ new boolean[] {true, true, false, true, false, false, false}, // 9
+ new boolean[] {true, true, true, true, false, false, false}, // 10
+ new boolean[] {true, false, true, true, false, false, false}, // 10
+ new boolean[] {false, true, true, true, false, false, false}, // 10
+ new boolean[] {false, false, true, true, false, false, false}, // 10
+ new boolean[] {false, false, false, true, false, false, false}, // 11
+ new boolean[] {false, true, false, true, true, true, true }, // 12
// name-node specific cases
- new boolean[] {true, false, false, false, true, true, true, false}, // 13
- new boolean[] {true, true, false, false, true, true, true, false}, // 13
- new boolean[] {false, false, false, false, true, true, true, false}, // 14
- new boolean[] {false, true, false, false, true, true, true, false}, // 14
- new boolean[] {true, false, true, false, true, false, false, false}, // 15
- new boolean[] {true, true, false, true, true, false, false, false} // 16
+ new boolean[] {true, true, false, false, true, true, false}, // 13
private static final int NUM_NN_TEST_CASES = testCases.length;
@@ -98,14 +106,13 @@ public class TestDFSStorageStateRecovery extends TestCase {
+ label + ":"
+ " numDirs="+numDirs
+ " testCase="+testCaseNum
- + " current="+state[0]
- + " previous="+state[1]
- + " previous.tmp="+state[2]
- + " removed.tmp="+state[3]
- + " lastcheckpoint.tmp="+state[4]
- + " should recover="+state[5]
- + " current exists after="+state[6]
- + " previous exists after="+state[7]);
+ + " current="+state[CURRENT_EXISTS]
+ + " previous="+state[PREVIOUS_EXISTS]
+ + " previous.tmp="+state[PREVIOUS_TMP_EXISTS]
+ + " removed.tmp="+state[REMOVED_TMP_EXISTS]
+ + " should recover="+state[SHOULD_RECOVER]
+ + " current exists after="+state[CURRENT_SHOULD_EXIST_AFTER_RECOVER]
+ + " previous exists after="+state[PREVIOUS_SHOULD_EXIST_AFTER_RECOVER]);
@@ -125,16 +132,15 @@ public class TestDFSStorageStateRecovery extends TestCase {
String[] createNameNodeStorageState(boolean[] state) throws Exception {
String[] baseDirs = conf.getStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
UpgradeUtilities.createEmptyDirs(baseDirs);
- if (state[0]) // current
+ if (state[CURRENT_EXISTS]) // current
UpgradeUtilities.createNameNodeStorageDirs(baseDirs, "current");
- if (state[1]) // previous
+ if (state[PREVIOUS_EXISTS]) // previous
UpgradeUtilities.createNameNodeStorageDirs(baseDirs, "previous");
- if (state[2]) // previous.tmp
+ if (state[PREVIOUS_TMP_EXISTS]) // previous.tmp
UpgradeUtilities.createNameNodeStorageDirs(baseDirs, "previous.tmp");
- if (state[3]) // removed.tmp
+ if (state[REMOVED_TMP_EXISTS]) // removed.tmp
UpgradeUtilities.createNameNodeStorageDirs(baseDirs, "removed.tmp");
- if (state[4]) // lastcheckpoint.tmp
- UpgradeUtilities.createNameNodeStorageDirs(baseDirs, "lastcheckpoint.tmp");
return baseDirs;
@@ -154,16 +160,15 @@ public class TestDFSStorageStateRecovery extends TestCase {
String[] createDataNodeStorageState(boolean[] state) throws Exception {
String[] baseDirs = conf.getStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "current");
UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "previous");
UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "previous.tmp");
UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "removed.tmp");
- UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "lastcheckpoint.tmp");
@@ -189,19 +194,16 @@ public class TestDFSStorageStateRecovery extends TestCase {
// After copying the storage directories from master datanode, empty
// the block pool storage directories
String[] bpDirs = UpgradeUtilities.createEmptyBPDirs(baseDirs, bpid);
UpgradeUtilities.createBlockPoolStorageDirs(baseDirs, "current", bpid);
UpgradeUtilities.createBlockPoolStorageDirs(baseDirs, "previous", bpid);
UpgradeUtilities.createBlockPoolStorageDirs(baseDirs, "previous.tmp",
bpid);
UpgradeUtilities
.createBlockPoolStorageDirs(baseDirs, "removed.tmp", bpid);
- UpgradeUtilities.createBlockPoolStorageDirs(baseDirs,
- "lastcheckpoint.tmp", bpid);
return bpDirs;
@@ -220,9 +222,9 @@ public class TestDFSStorageStateRecovery extends TestCase {
assertTrue(new File(baseDirs[i],"current").isDirectory());
assertTrue(new File(baseDirs[i],"current/VERSION").isFile());
+ assertNotNull(FSImageTestUtil.findNewestImageFile(
+ baseDirs[i] + "/current"));
+ assertTrue(new File(baseDirs[i],"current/seen_txid").isFile());
if (previousShouldExist) {
@@ -318,9 +320,9 @@ public class TestDFSStorageStateRecovery extends TestCase {
conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
for (int i = 0; i < NUM_NN_TEST_CASES; i++) {
boolean[] testCase = testCases[i];
- boolean shouldRecover = testCase[5];
- boolean curAfterRecover = testCase[6];
- boolean prevAfterRecover = testCase[7];
+ boolean shouldRecover = testCase[SHOULD_RECOVER];
+ boolean curAfterRecover = testCase[CURRENT_SHOULD_EXIST_AFTER_RECOVER];
+ boolean prevAfterRecover = testCase[PREVIOUS_SHOULD_EXIST_AFTER_RECOVER];
log("NAME_NODE recovery", numDirs, i, testCase);
baseDirs = createNameNodeStorageState(testCase);
@@ -336,8 +338,8 @@ public class TestDFSStorageStateRecovery extends TestCase {
// the exception is expected
// check that the message says "not formatted"
// when storage directory is empty (case #5)
- if(!testCases[i][0] && !testCases[i][2]
- && !testCases[i][1] && !testCases[i][3] && !testCases[i][4]) {
+ if(!testCases[i][CURRENT_EXISTS] && !testCases[i][PREVIOUS_TMP_EXISTS]
+ && !testCases[i][PREVIOUS_EXISTS] && !testCases[i][REMOVED_TMP_EXISTS]) {
assertTrue(expected.getLocalizedMessage().contains(
"NameNode is not formatted"));
@@ -362,16 +364,16 @@ public class TestDFSStorageStateRecovery extends TestCase {
for (int i = 0; i < NUM_DN_TEST_CASES; i++) {
log("DATA_NODE recovery", numDirs, i, testCase);
createNameNodeStorageState(new boolean[] { true, true, false, false,
false });
cluster = createCluster(conf);
baseDirs = createDataNodeStorageState(testCase);
- if (!testCase[0] && !testCase[1] && !testCase[2] && !testCase[3]) {
+ if (!testCase[CURRENT_EXISTS] && !testCase[PREVIOUS_EXISTS] && !testCase[PREVIOUS_TMP_EXISTS] && !testCase[REMOVED_TMP_EXISTS]) {
// DataNode will create and format current if no directories exist
cluster.startDataNodes(conf, 1, false, StartupOption.REGULAR, null);
@@ -403,16 +405,16 @@ public class TestDFSStorageStateRecovery extends TestCase {
log("BLOCK_POOL recovery", numDirs, i, testCase);
baseDirs = createBlockPoolStorageState(bpid, testCase);
@@ -27,15 +27,20 @@ import java.util.regex.Pattern;
-import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.server.namenode.TestParallelImageWrite;
+import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
+import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import static org.junit.Assert.*;
@@ -45,6 +50,7 @@ import static org.junit.Assert.*;
public class TestDFSUpgrade {
+ private static final int EXPECTED_TXID = 17;
private static final Log LOG = LogFactory.getLog(TestDFSUpgrade.class.getName());
private Configuration conf;
@@ -66,15 +72,22 @@ public class TestDFSUpgrade {
* its files with their original checksum. It is assumed that the
* server has recovered and upgraded.
- void checkNameNode(String[] baseDirs) throws IOException {
+ void checkNameNode(String[] baseDirs, long imageTxId) throws IOException {
+ LOG.info("Checking namenode directory " + baseDir);
+ LOG.info("==== Contents ====:\n " +
+ Joiner.on(" \n").join(new File(baseDir, "current").list()));
+ LOG.info("==================");
+ assertTrue(new File(baseDir,"current").isDirectory());
+ assertTrue(new File(baseDir,"current/VERSION").isFile());
+ assertTrue(new File(baseDir,"current/"
+ + getInProgressEditsFileName(imageTxId + 1)).isFile());
+ + getImageFileName(imageTxId)).isFile());
+ assertTrue(new File(baseDir,"current/seen_txid").isFile());
- File previous = new File(baseDirs[i], "previous");
+ File previous = new File(baseDir, "previous");
assertTrue(previous.isDirectory());
assertEquals(UpgradeUtilities.checksumContents(NAME_NODE, previous),
UpgradeUtilities.checksumMasterNameNodeContents());
@@ -200,7 +213,6 @@ public class TestDFSUpgrade {
StorageInfo storageInfo = null;
for (int numDirs = 1; numDirs <= 2; numDirs++) {
conf = new HdfsConfiguration();
- conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
String[] nameNodeDirs = conf.getStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
String[] dataNodeDirs = conf.getStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
@@ -208,7 +220,7 @@ public class TestDFSUpgrade {
log("Normal NameNode upgrade", numDirs);
cluster = createCluster();
- checkNameNode(nameNodeDirs);
+ checkNameNode(nameNodeDirs, EXPECTED_TXID);
if (numDirs > 1)
TestParallelImageWrite.checkImages(cluster.getNamesystem(), numDirs);
cluster.shutdown();
@@ -277,25 +289,21 @@ public class TestDFSUpgrade {
UpgradeUtilities.createEmptyDirs(dataNodeDirs);
log("NameNode upgrade with no edits file", numDirs);
- baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
+ UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
+ deleteStorageFilesWithPrefix(nameNodeDirs, "edits_");
startNameNodeShouldFail(StartupOption.UPGRADE);
log("NameNode upgrade with no image file", numDirs);
+ deleteStorageFilesWithPrefix(nameNodeDirs, "fsimage_");
log("NameNode upgrade with corrupt version file", numDirs);
baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
for (File f : baseDirs) {
- UpgradeUtilities.corruptFile(new File(f,"VERSION"));
+ UpgradeUtilities.corruptFile(new File (f,"VERSION"));
@@ -338,7 +346,7 @@ public class TestDFSUpgrade {
@@ -369,6 +377,19 @@ public class TestDFSUpgrade {
+ private void deleteStorageFilesWithPrefix(String[] nameNodeDirs, String prefix)
+ throws Exception {
+ for (String baseDirStr : nameNodeDirs) {
+ File baseDir = new File(baseDirStr);
+ File currentDir = new File(baseDir, "current");
+ for (File f : currentDir.listFiles()) {
+ if (f.getName().startsWith(prefix)) {
+ assertTrue("Deleting " + f, f.delete());
@Test(expected=IOException.class)
public void testUpgradeFromPreUpgradeLVFails() throws IOException {
// Upgrade from versions prior to Storage#LAST_UPGRADABLE_LAYOUT_VERSION
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.test.GenericTestUtils;
@@ -183,38 +184,6 @@ public class TestDFSUpgradeFromImage extends TestCase {
- public void testUpgradeFromRel14Image() throws IOException {
- unpackStorage();
- MiniDFSCluster cluster = null;
- Configuration conf = new HdfsConfiguration();
- if (System.getProperty("test.build.data") == null) { // to allow test to be run outside of Ant
- System.setProperty("test.build.data", "build/test/data");
- conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // block scanning off
- cluster = new MiniDFSCluster.Builder(conf)
- .numDataNodes(numDataNodes)
- .format(false)
- .startupOption(StartupOption.UPGRADE)
- .clusterId("testClusterId")
- .build();
- cluster.waitActive();
- DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
- DFSClient dfsClient = dfs.dfs;
- //Safemode will be off only after upgrade is complete. Wait for it.
- while ( dfsClient.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_GET) ) {
- LOG.info("Waiting for SafeMode to be OFF.");
- Thread.sleep(1000);
- } catch (InterruptedException ignored) {}
- verifyFileSystem(dfs);
- if (cluster != null) { cluster.shutdown(); }
* Test that sets up a fake image from Hadoop 0.3.0 and tries to start a
* NN, verifying that the correct error message is thrown.
@@ -259,11 +228,51 @@ public class TestDFSUpgradeFromImage extends TestCase {
+ * Test upgrade from an 0.14 image
+ public void testUpgradeFromRel14Image() throws IOException {
+ unpackStorage();
+ upgradeAndVerify();
* Test upgrade from 0.22 image
public void testUpgradeFromRel22Image() throws IOException {
unpackStorage(HADOOP22_IMAGE);
+ * Test upgrade from 0.22 image with corrupt md5, make sure it
+ * fails to upgrade
+ public void testUpgradeFromCorruptRel22Image() throws IOException {
+ unpackStorage(HADOOP22_IMAGE);
+ // Overwrite the md5 stored in the VERSION files
+ File baseDir = new File(MiniDFSCluster.getBaseDirectory());
+ FSImageTestUtil.corruptVersionFile(
+ new File(baseDir, "name1/current/VERSION"),
+ "imageMD5Digest", "22222222222222222222222222222222");
+ new File(baseDir, "name2/current/VERSION"),
+ // Upgrade should now fail
+ fail("Upgrade did not fail with bad MD5");
+ String msg = StringUtils.stringifyException(ioe);
+ if (!msg.contains("is corrupt with MD5 checksum")) {
+ throw ioe;
+ private void upgradeAndVerify() throws IOException {
MiniDFSCluster cluster = null;
Configuration conf = new HdfsConfiguration();
@@ -287,8 +296,12 @@ public class TestDFSUpgradeFromImage extends TestCase {
Thread.sleep(1000);
} catch (InterruptedException ignored) {}
+ verifyFileSystem(dfs);
if (cluster != null) { cluster.shutdown(); }
@@ -27,7 +27,6 @@ import static org.mockito.Mockito.spy;
-import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
@@ -45,11 +44,10 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.junit.Before;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
/* File Append tests for HDFS-200 & HDFS-142, specifically focused on:
* using append()/sync() to recover block information
@@ -157,7 +155,7 @@ public class TestFileAppend4 {
NameNode spyNN = spy(preSpyNN);
// Delay completeFile
- DelayAnswer delayer = new DelayAnswer();
+ GenericTestUtils.DelayAnswer delayer = new GenericTestUtils.DelayAnswer(LOG);
doAnswer(delayer).when(spyNN).complete(
anyString(), anyString(), (ExtendedBlock)anyObject());
@@ -228,7 +226,8 @@ public class TestFileAppend4 {
+ GenericTestUtils.DelayAnswer delayer =
+ new GenericTestUtils.DelayAnswer(LOG);
doAnswer(delayer).when(spyNN).complete(anyString(), anyString(),
(ExtendedBlock) anyObject());
@@ -290,42 +289,5 @@ public class TestFileAppend4 {
- * Mockito answer helper that triggers one latch as soon as the
- * method is called, then waits on another before continuing.
- private static class DelayAnswer implements Answer<Object> {
- private final CountDownLatch fireLatch = new CountDownLatch(1);
- private final CountDownLatch waitLatch = new CountDownLatch(1);
- * Wait until the method is called.
- public void waitForCall() throws InterruptedException {
- fireLatch.await();
- * Tell the method to proceed.
- * This should only be called after waitForCall()
- public void proceed() {
- waitLatch.countDown();
- public Object answer(InvocationOnMock invocation) throws Throwable {
- LOG.info("DelayAnswer firing fireLatch");
- fireLatch.countDown();
- LOG.info("DelayAnswer waiting on waitLatch");
- waitLatch.await();
- LOG.info("DelayAnswer delay complete");
- } catch (InterruptedException ie) {
- throw new IOException("Interrupted waiting on latch", ie);
- return invocation.callRealMethod();
@@ -25,8 +25,10 @@ import java.io.FileInputStream;
+import java.net.URI;
import java.util.Random;
import java.util.zip.CRC32;
import org.apache.hadoop.fs.FileSystem;
@@ -407,7 +409,9 @@ public class UpgradeUtilities {
public static File[] createNameNodeVersionFile(Configuration conf,
File[] parent, StorageInfo version, String bpid) throws IOException {
- Storage storage = new NNStorage(conf);
+ Storage storage = new NNStorage(conf,
+ Collections.<URI>emptyList(),
+ Collections.<URI>emptyList());
storage.setStorageInfo(version);
File[] versionFiles = new File[parent.length];
for (int i = 0; i < parent.length; i++) {
@@ -0,0 +1,39 @@
+package org.apache.hadoop.hdfs.server.common;
+import org.mockito.Mockito;
+ * Test methods that need to access package-private parts of
+ * Storage
+public abstract class StorageAdapter {
+ * Inject and return a spy on a storage directory
+ public static StorageDirectory spyOnStorageDirectory(
+ Storage s, int idx) {
+ StorageDirectory dir = Mockito.spy(s.getStorageDir(idx));
+ s.storageDirs.set(idx, dir);
+ return dir;
@@ -190,11 +190,9 @@ public class CreateEditsLog {
- FSImage fsImage = new FSImage(editsLogDir.getAbsoluteFile().toURI());
- FileNameGenerator nameGenerator = new FileNameGenerator(BASE_PATH, 100);
- editLog.createEditLogFile(fsImage.getStorage().getFsEditName());
+ FileNameGenerator nameGenerator = new FileNameGenerator(BASE_PATH, 100);
+ FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
editLog.open();
addFiles(editLog, numFiles, replication, numBlocksPerFile, startingBlockId,
nameGenerator);
@@ -0,0 +1,375 @@
+import java.io.RandomAccessFile;
+import java.util.Comparator;
+import java.util.Properties;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+ * Utility functions for testing fsimage storage.
+public abstract class FSImageTestUtil {
+ * The position in the fsimage header where the txid is
+ * written.
+ private static final long IMAGE_TXID_POS = 24;
+ * This function returns a md5 hash of a file.
+ * @param file input file
+ * @return The md5 string
+ public static String getFileMD5(File file) throws IOException {
+ return MD5FileUtils.computeMd5ForFile(file).toString();
+ * Calculate the md5sum of an image after zeroing out the transaction ID
+ * field in the header. This is useful for tests that want to verify
+ * that two checkpoints have identical namespaces.
+ public static String getImageFileMD5IgnoringTxId(File imageFile)
+ File tmpFile = File.createTempFile("hadoop_imagefile_tmp", "fsimage");
+ tmpFile.deleteOnExit();
+ Files.copy(imageFile, tmpFile);
+ RandomAccessFile raf = new RandomAccessFile(tmpFile, "rw");
+ raf.seek(IMAGE_TXID_POS);
+ raf.writeLong(0);
+ IOUtils.closeStream(raf);
+ return getFileMD5(tmpFile);
+ tmpFile.delete();
+ public static StorageDirectory mockStorageDirectory(
+ File currentDir, NameNodeDirType type) {
+ // Mock the StorageDirectory interface to just point to this file
+ StorageDirectory sd = Mockito.mock(StorageDirectory.class);
+ Mockito.doReturn(type)
+ .when(sd).getStorageDirType();
+ Mockito.doReturn(currentDir).when(sd).getCurrentDir();
+ Mockito.doReturn(mockFile(true)).when(sd).getVersionFile();
+ Mockito.doReturn(mockFile(false)).when(sd).getPreviousDir();
+ static File mockFile(boolean exists) {
+ File mockFile = mock(File.class);
+ doReturn(exists).when(mockFile).exists();
+ return mockFile;
+ public static FSImageTransactionalStorageInspector inspectStorageDirectory(
+ File dir, NameNodeDirType dirType) throws IOException {
+ inspector.inspectDirectory(mockStorageDirectory(dir, dirType));
+ * Return a standalone instance of FSEditLog that will log into the given
+ * log directory. The returned instance is not yet opened.
+ public static FSEditLog createStandaloneEditLog(File logDir)
+ assertTrue(logDir.mkdirs() || logDir.exists());
+ Files.deleteDirectoryContents(logDir);
+ NNStorage storage = Mockito.mock(NNStorage.class);
+ List<StorageDirectory> sds = Lists.newArrayList(
+ FSImageTestUtil.mockStorageDirectory(logDir, NameNodeDirType.EDITS));
+ Mockito.doReturn(sds).when(storage).dirIterable(NameNodeDirType.EDITS);
+ return new FSEditLog(storage);
+ * Assert that all of the given directories have the same newest filename
+ * for fsimage that they hold the same data.
+ public static void assertSameNewestImage(List<File> dirs) throws Exception {
+ if (dirs.size() < 2) return;
+ long imageTxId = -1;
+ List<File> imageFiles = new ArrayList<File>();
+ for (File dir : dirs) {
+ inspectStorageDirectory(dir, NameNodeDirType.IMAGE);
+ FoundFSImage latestImage = inspector.getLatestImage();
+ assertNotNull("No image in " + dir, latestImage);
+ long thisTxId = latestImage.getTxId();
+ if (imageTxId != -1 && thisTxId != imageTxId) {
+ fail("Storage directory " + dir + " does not have the same " +
+ "last image index " + imageTxId + " as another");
+ imageTxId = thisTxId;
+ imageFiles.add(inspector.getLatestImage().getFile());
+ assertFileContentsSame(imageFiles.toArray(new File[0]));
+ * Given a list of directories, assert that any files that are named
+ * the same thing have the same contents. For example, if a file
+ * named "fsimage_1" shows up in more than one directory, then it must
+ * be the same.
+ * @throws Exception
+ public static void assertParallelFilesAreIdentical(List<File> dirs,
+ Set<String> ignoredFileNames) throws Exception {
+ HashMap<String, List<File>> groupedByName = new HashMap<String, List<File>>();
+ for (File f : dir.listFiles()) {
+ if (ignoredFileNames.contains(f.getName())) {
+ List<File> fileList = groupedByName.get(f.getName());
+ if (fileList == null) {
+ fileList = new ArrayList<File>();
+ groupedByName.put(f.getName(), fileList);
+ fileList.add(f);
+ for (List<File> sameNameList : groupedByName.values()) {
+ if (sameNameList.get(0).isDirectory()) {
+ // recurse
+ assertParallelFilesAreIdentical(sameNameList, ignoredFileNames);
+ assertFileContentsSame(sameNameList.toArray(new File[0]));
+ * Assert that all of the given paths have the exact same
+ * contents
+ public static void assertFileContentsSame(File... files) throws Exception {
+ if (files.length < 2) return;
+ Map<File, String> md5s = getFileMD5s(files);
+ if (Sets.newHashSet(md5s.values()).size() > 1) {
+ fail("File contents differed:\n " +
+ Joiner.on("\n ")
+ .withKeyValueSeparator("=")
+ .join(md5s));
+ * Assert that the given files are not all the same, and in fact that
+ * they have <code>expectedUniqueHashes</code> unique contents.
+ public static void assertFileContentsDifferent(
+ int expectedUniqueHashes,
+ File... files) throws Exception
+ {
+ if (Sets.newHashSet(md5s.values()).size() != expectedUniqueHashes) {
+ fail("Expected " + expectedUniqueHashes + " different hashes, got:\n " +
+ public static Map<File, String> getFileMD5s(File... files) throws Exception {
+ Map<File, String> ret = Maps.newHashMap();
+ for (File f : files) {
+ assertTrue("Must exist: " + f, f.exists());
+ ret.put(f, getFileMD5(f));
+ * @return a List which contains the "current" dir for each storage
+ * directory of the given type.
+ public static List<File> getCurrentDirs(NNStorage storage,
+ NameNodeDirType type) {
+ List<File> ret = Lists.newArrayList();
+ for (StorageDirectory sd : storage.dirIterable(type)) {
+ ret.add(sd.getCurrentDir());
+ * @return the fsimage file with the most recent transaction ID in the
+ * given storage directory.
+ public static File findLatestImageFile(StorageDirectory sd)
+ return inspector.getLatestImage().getFile();
+ * given 'current/' directory.
+ public static File findNewestImageFile(String currentDirPath) throws IOException {
+ StorageDirectory sd = FSImageTestUtil.mockStorageDirectory(
+ new File(currentDirPath), NameNodeDirType.IMAGE);
+ return (latestImage == null) ? null : latestImage.getFile();
+ * Assert that the NameNode has checkpoints at the expected
+ * transaction IDs.
+ static void assertNNHasCheckpoints(MiniDFSCluster cluster,
+ List<Integer> txids) {
+ for (File nameDir : getNameNodeCurrentDirs(cluster)) {
+ // Should have fsimage_N for the three checkpoints
+ for (long checkpointTxId : txids) {
+ File image = new File(nameDir,
+ NNStorage.getImageFileName(checkpointTxId));
+ assertTrue("Expected non-empty " + image, image.length() > 0);
+ static List<File> getNameNodeCurrentDirs(MiniDFSCluster cluster) {
+ List<File> nameDirs = Lists.newArrayList();
+ for (URI u : cluster.getNameDirs(0)) {
+ nameDirs.add(new File(u.getPath(), "current"));
+ return nameDirs;
+ * @return the latest edits log, finalized or otherwise, from the given
+ * storage directory.
+ public static FoundEditLog findLatestEditsLog(StorageDirectory sd)
+ List<FoundEditLog> foundEditLogs = Lists.newArrayList(
+ inspector.getFoundEditLogs());
+ return Collections.max(foundEditLogs, new Comparator<FoundEditLog>() {
+ public int compare(FoundEditLog a, FoundEditLog b) {
+ .compare(a.getStartTxId(), b.getStartTxId())
+ .compare(a.getLastTxId(), b.getLastTxId())
+ * Corrupt the given VERSION file by replacing a given
+ * key with a new value and re-writing the file.
+ * @param versionFile the VERSION file to corrupt
+ * @param key the key to replace
+ * @param value the new value for this key
+ public static void corruptVersionFile(File versionFile, String key, String value)
+ Properties props = new Properties();
+ FileInputStream fis = new FileInputStream(versionFile);
+ FileOutputStream out = null;
+ props.load(fis);
+ IOUtils.closeStream(fis);
+ props.setProperty(key, value);
+ out = new FileOutputStream(versionFile);
+ props.store(out, null);
+ IOUtils.cleanup(null, fis, out);
+ public static void assertReasonableNameCurrentDir(File curDir)
+ assertTrue(curDir.isDirectory());
+ assertTrue(new File(curDir, "VERSION").isFile());
+ assertTrue(new File(curDir, "seen_txid").isFile());
+ File image = findNewestImageFile(curDir.toString());
+ assertNotNull(image);
@@ -69,8 +69,8 @@ public class OfflineEditsViewerHelper {
* @param editsFilename where to copy the edits
public String generateEdits() throws IOException {
- runOperations();
- return getEditsFilename();
+ CheckpointSignature signature = runOperations();
+ return getEditsFilename(signature);
@@ -78,13 +78,16 @@ public class OfflineEditsViewerHelper {
* @return edits file name for cluster
- private String getEditsFilename() throws IOException {
+ private String getEditsFilename(CheckpointSignature sig) throws IOException {
FSImage image = cluster.getNameNode().getFSImage();
// it was set up to only have ONE StorageDirectory
Iterator<StorageDirectory> it
= image.getStorage().dirIterator(NameNodeDirType.EDITS);
- return image.getStorage().getEditFile(sd).getAbsolutePath();
+ File ret = NNStorage.getFinalizedEditsFile(
+ sd, 1, sig.curSegmentTxId - 1);
+ assert ret.exists() : "expected " + ret + " exists";
+ return ret.getAbsolutePath();
@@ -131,7 +134,7 @@ public class OfflineEditsViewerHelper {
* OP_SET_NS_QUOTA (11)
* OP_CLEAR_NS_QUOTA (12)
- private void runOperations() throws IOException {
+ private CheckpointSignature runOperations() throws IOException {
LOG.info("Creating edits by performing fs operations");
// no check, if it's not it throws an exception which is what we want
@@ -238,5 +241,8 @@ public class OfflineEditsViewerHelper {
locatedBlocks = DFSClientAdapter.callGetBlockLocations(
cluster.getNameNode(), filePath, 0L, bytes.length);
} while (locatedBlocks.isUnderConstruction());
+ // Force a roll so we get an OP_END_LOG_SEGMENT txn
+ return cluster.getNameNode().rollEditLog();
@@ -19,9 +19,12 @@ package org.apache.hadoop.hdfs.server.namenode;
+import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.fs.FileUtil;
@@ -29,14 +32,28 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.log4j.Level;
+import com.google.common.base.Supplier;
+import com.google.common.collect.ImmutableSet;
public class TestBackupNode extends TestCase {
public static final Log LOG = LogFactory.getLog(TestBackupNode.class);
+ static {
+ ((Log4JLogger)Checkpointer.LOG).getLogger().setLevel(Level.ALL);
+ ((Log4JLogger)BackupImage.LOG).getLogger().setLevel(Level.ALL);
static final String BASE_DIR = MiniDFSCluster.getBaseDirectory();
protected void setUp() throws Exception {
@@ -53,73 +70,185 @@ public class TestBackupNode extends TestCase {
dirB.mkdirs();
- protected void tearDown() throws Exception {
- super.tearDown();
- File baseDir = new File(BASE_DIR);
- if(!(FileUtil.fullyDelete(baseDir)))
- throw new IOException("Cannot remove directory: " + baseDir);
- static void writeFile(FileSystem fileSys, Path name, int repl)
- TestCheckpoint.writeFile(fileSys, name, repl);
- static void checkFile(FileSystem fileSys, Path name, int repl)
- TestCheckpoint.checkFile(fileSys, name, repl);
- void cleanupFile(FileSystem fileSys, Path name)
- TestCheckpoint.cleanupFile(fileSys, name);
- static String getBackupNodeDir(StartupOption t, int i) {
- return BASE_DIR + "name" + t.getName() + i + "/";
+ static String getBackupNodeDir(StartupOption t, int idx) {
+ return BASE_DIR + "name" + t.getName() + idx + "/";
BackupNode startBackupNode(Configuration conf,
- StartupOption t, int i) throws IOException {
+ StartupOption startupOpt,
+ int idx) throws IOException {
Configuration c = new HdfsConfiguration(conf);
- String dirs = getBackupNodeDir(t, i);
+ String dirs = getBackupNodeDir(startupOpt, idx);
c.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, dirs);
c.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
"${" + DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY + "}");
- return (BackupNode)NameNode.createNameNode(new String[]{t.getName()}, c);
+ c.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY,
+ "127.0.0.1:0");
+ return (BackupNode)NameNode.createNameNode(new String[]{startupOpt.getName()}, c);
- void waitCheckpointDone(BackupNode backup) {
+ void waitCheckpointDone(
+ MiniDFSCluster cluster, BackupNode backup, long txid) {
+ long thisCheckpointTxId;
- LOG.info("Waiting checkpoint to complete...");
+ LOG.info("Waiting checkpoint to complete... " +
+ "checkpoint txid should increase above " + txid);
} catch (Exception e) {}
- } while(backup.getCheckpointState() != CheckpointStates.START);
+ thisCheckpointTxId = backup.getFSImage().getStorage()
+ .getMostRecentCheckpointTxId();
+ } while (thisCheckpointTxId < txid);
+ // Check that the checkpoint got uploaded to NN successfully
+ FSImageTestUtil.assertNNHasCheckpoints(cluster,
+ Collections.singletonList((int)thisCheckpointTxId));
- public void testCheckpoint() throws IOException {
+ public void testCheckpointNode() throws Exception {
testCheckpoint(StartupOption.CHECKPOINT);
- testCheckpoint(StartupOption.BACKUP);
+ * Ensure that the backupnode will tail edits from the NN
+ * and keep in sync, even while the NN rolls, checkpoints
+ * occur, etc.
+ public void testBackupNodeTailsEdits() throws Exception {
+ Configuration conf = new HdfsConfiguration();
+ MiniDFSCluster cluster = null;
+ FileSystem fileSys = null;
+ BackupNode backup = null;
- void testCheckpoint(StartupOption op) throws IOException {
+ cluster = new MiniDFSCluster.Builder(conf)
+ .numDataNodes(0).build();
+ fileSys = cluster.getFileSystem();
+ backup = startBackupNode(conf, StartupOption.BACKUP, 1);
+ BackupImage bnImage = backup.getBNImage();
+ testBNInSync(cluster, backup, 1);
+ // Force a roll -- BN should roll with NN.
+ NameNode nn = cluster.getNameNode();
+ nn.rollEditLog();
+ assertEquals(bnImage.getEditLog().getCurSegmentTxId(),
+ nn.getFSImage().getEditLog().getCurSegmentTxId());
+ // BN should stay in sync after roll
+ testBNInSync(cluster, backup, 2);
+ long nnImageBefore =
+ nn.getFSImage().getStorage().getMostRecentCheckpointTxId();
+ // BN checkpoint
+ backup.doCheckpoint();
+ // NN should have received a new image
+ long nnImageAfter =
+ assertTrue("nn should have received new checkpoint. before: " +
+ nnImageBefore + " after: " + nnImageAfter,
+ nnImageAfter > nnImageBefore);
+ // BN should stay in sync after checkpoint
+ testBNInSync(cluster, backup, 3);
+ // Stop BN
+ StorageDirectory sd = bnImage.getStorage().getStorageDir(0);
+ backup.stop();
+ backup = null;
+ // When shutting down the BN, it shouldn't finalize logs that are
+ // still open on the NN
+ FoundEditLog editsLog = FSImageTestUtil.findLatestEditsLog(sd);
+ assertEquals(editsLog.getStartTxId(),
+ assertTrue("Should not have finalized " + editsLog,
+ editsLog.isInProgress());
+ // do some edits
+ assertTrue(fileSys.mkdirs(new Path("/edit-while-bn-down")));
+ // start a new backup node
+ testBNInSync(cluster, backup, 4);
+ assertNotNull(backup.getNamesystem().getFileInfo("/edit-while-bn-down", false));
+ LOG.info("Shutting down...");
+ if (backup != null) backup.stop();
+ if (fileSys != null) fileSys.close();
+ if (cluster != null) cluster.shutdown();
+ assertStorageDirsMatch(cluster.getNameNode(), backup);
+ private void testBNInSync(MiniDFSCluster cluster, final BackupNode backup,
+ int testIdx) throws Exception {
+ final NameNode nn = cluster.getNameNode();
+ final FileSystem fs = cluster.getFileSystem();
+ // Do a bunch of namespace operations, make sure they're replicated
+ // to the BN.
+ for (int i = 0; i < 10; i++) {
+ final String src = "/test_" + testIdx + "_" + i;
+ LOG.info("Creating " + src + " on NN");
+ Path p = new Path(src);
+ assertTrue(fs.mkdirs(p));
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
+ public Boolean get() {
+ LOG.info("Checking for " + src + " on BN");
+ boolean hasFile = backup.getNamesystem().getFileInfo(src, false) != null;
+ boolean txnIdMatch = backup.getTransactionID() == nn.getTransactionID();
+ return hasFile && txnIdMatch;
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }, 30, 10000);
+ assertStorageDirsMatch(nn, backup);
+ private void assertStorageDirsMatch(final NameNode nn, final BackupNode backup)
+ // Check that the stored files in the name dirs are identical
+ List<File> dirs = Lists.newArrayList(
+ FSImageTestUtil.getCurrentDirs(nn.getFSImage().getStorage(),
+ null));
+ dirs.addAll(FSImageTestUtil.getCurrentDirs(backup.getFSImage().getStorage(),
+ FSImageTestUtil.assertParallelFilesAreIdentical(dirs, ImmutableSet.of("VERSION"));
+ public void testBackupNode() throws Exception {
+ testCheckpoint(StartupOption.BACKUP);
+ void testCheckpoint(StartupOption op) throws Exception {
Path file1 = new Path("checkpoint.dat");
Path file2 = new Path("checkpoint2.dat");
- short replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 3);
conf.set(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, "0");
conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // disable block scanner
- int numDatanodes = Math.max(3, replication);
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 1);
FileSystem fileSys = null;
BackupNode backup = null;
cluster = new MiniDFSCluster.Builder(conf)
- .numDataNodes(numDatanodes).build();
fileSys = cluster.getFileSystem();
// verify that 'format' really blew away all pre-existing files
@@ -130,14 +259,14 @@ public class TestBackupNode extends TestCase {
// Create file1
- writeFile(fileSys, file1, replication);
- checkFile(fileSys, file1, replication);
+ assertTrue(fileSys.mkdirs(file1));
// Take a checkpoint
+ long txid = cluster.getNameNode().getTransactionID();
backup = startBackupNode(conf, op, 1);
- waitCheckpointDone(backup);
+ waitCheckpointDone(cluster, backup, txid);
LOG.error("Error in TestBackupNode:", e);
assertTrue(e.getLocalizedMessage(), false);
@@ -146,32 +275,46 @@ public class TestBackupNode extends TestCase {
if(fileSys != null) fileSys.close();
if(cluster != null) cluster.shutdown();
- File imageFileNN = new File(BASE_DIR, "name1/current/fsimage");
- File imageFileBN = new File(getBackupNodeDir(op, 1), "/current/fsimage");
- LOG.info("NameNode fsimage length = " + imageFileNN.length());
- LOG.info("Backup Node fsimage length = " + imageFileBN.length());
- assertTrue(imageFileNN.length() == imageFileBN.length());
+ File nnCurDir = new File(BASE_DIR, "name1/current/");
+ File bnCurDir = new File(getBackupNodeDir(op, 1), "/current/");
+ ImmutableList.of(bnCurDir, nnCurDir),
+ ImmutableSet.<String>of("VERSION"));
// Restart cluster and verify that file1 still exist.
- cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
.format(false).build();
// check that file1 still exists
- cleanupFile(fileSys, file1);
+ assertTrue(fileSys.exists(file1));
+ fileSys.delete(file1, true);
// create new file file2
- writeFile(fileSys, file2, replication);
- checkFile(fileSys, file2, replication);
+ fileSys.mkdirs(file2);
+ fileSys.mkdirs(new Path("file_" + i));
+ txid = cluster.getNameNode().getTransactionID();
@@ -180,22 +323,22 @@ public class TestBackupNode extends TestCase {
// Restart cluster and verify that file2 exists and
// file1 does not exist.
- cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes).format(false).build();
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build();
assertTrue(!fileSys.exists(file1));
// verify that file2 exists
+ assertTrue(fileSys.exists(file2));
@@ -204,52 +347,4 @@ public class TestBackupNode extends TestCase {
- * Test that only one backup node can register.
- public void testBackupRegistration() throws IOException {
- Configuration conf1 = new HdfsConfiguration();
- Configuration conf2 = null;
- BackupNode backup1 = null;
- BackupNode backup2 = null;
- // start name-node and backup node 1
- cluster = new MiniDFSCluster.Builder(conf1).numDataNodes(0).build();
- conf1.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY, "0.0.0.0:7771");
- conf1.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, "0.0.0.0:7775");
- backup1 = startBackupNode(conf1, StartupOption.BACKUP, 1);
- // try to start backup node 2
- conf2 = new HdfsConfiguration(conf1);
- conf2.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY, "0.0.0.0:7772");
- conf2.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY, "0.0.0.0:7776");
- backup2 = startBackupNode(conf2, StartupOption.BACKUP, 2);
- backup2.stop();
- backup2 = null;
- assertTrue("Only one backup node should be able to start", false);
- assertTrue(
- e.getLocalizedMessage().contains("Registration is not allowed"));
- // should fail - doing good
- // stop backup node 1; backup node 2 should be able to start
- backup1.stop();
- backup1 = null;
- assertTrue("Backup node 2 should be able to start", false);
- LOG.error("Error in TestBackupNode:", e);
- assertTrue(e.getLocalizedMessage(), false);
- if(backup1 != null) backup1.stop();
- if(backup2 != null) backup2.stop();
- if(cluster != null) cluster.shutdown();
@@ -17,10 +17,9 @@
import junit.framework.Assert;
import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -28,9 +27,9 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.security.token.Token;
@@ -47,20 +46,6 @@ public class TestCheckPointForSecurityTokens {
short replication = 3;
- NameNode startNameNode( Configuration conf,
- String imageDirs,
- String editsDirs,
- StartupOption start) throws IOException {
- conf.set(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "hdfs://localhost:0");
- conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
- conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, imageDirs);
- conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, editsDirs);
- String[] args = new String[]{start.getName()};
- NameNode nn = NameNode.createNameNode(args, conf);
- Assert.assertTrue(nn.isInSafeMode());
- return nn;
private void cancelToken(Token<DelegationTokenIdentifier> token)
cluster.getNamesystem().cancelDelegationToken(token);
@@ -95,10 +80,12 @@ public class TestCheckPointForSecurityTokens {
String[] args = new String[]{"-saveNamespace"};
// verify that the edits file is NOT empty
- Collection<URI> editsDirs = cluster.getNameEditsDirs(0);
- for(URI uri : editsDirs) {
- File ed = new File(uri.getPath());
- Assert.assertTrue(new File(ed, "current/edits").length() > Integer.SIZE/Byte.SIZE);
+ for (StorageDirectory sd : nn.getFSImage().getStorage().dirIterable(null)) {
+ FoundEditLog log = FSImageTestUtil.findLatestEditsLog(sd);
+ assertTrue(log.isInProgress());
+ assertEquals("In-progress log " + log + " should have 5 transactions",
+ 5, log.validateLog().numTransactions);
// Saving image in safe mode should succeed
@@ -108,10 +95,12 @@ public class TestCheckPointForSecurityTokens {
} catch(Exception e) {
throw new IOException(e.getMessage());
- // verify that the edits file is empty
- Assert.assertTrue(new File(ed, "current/edits").length() == Integer.SIZE/Byte.SIZE);
+ // verify that the edits file is empty except for the START txn
+ assertEquals("In-progress log " + log + " should only have START txn",
+ 1, log.validateLog().numTransactions);
// restart cluster
@@ -50,7 +50,7 @@ public class TestClusterId {
// see if cluster id not empty.
Collection<URI> dirsToFormat = FSNamesystem.getNamespaceDirs(config);
Collection<URI> editsToFormat = new ArrayList<URI>(0);
- FSImage fsImage = new FSImage(dirsToFormat, editsToFormat);
+ FSImage fsImage = new FSImage(config, null, dirsToFormat, editsToFormat);
Iterator<StorageDirectory> sdit =
fsImage.getStorage().dirIterator(NNStorage.NameNodeDirType.IMAGE);
+import java.util.Collection;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
@@ -27,44 +28,75 @@ import java.util.concurrent.Executors;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
-import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.aspectj.util.FileUtil;
import org.mockito.Mockito;
import static org.apache.hadoop.test.MetricsAsserts.*;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.anyObject;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.spy;
* This class tests the creation and validation of a checkpoint.
public class TestEditLog extends TestCase {
- private static final Log LOG = LogFactory.getLog(TestEditLog.class);
+ ((Log4JLogger)FSEditLog.LOG).getLogger().setLevel(Level.ALL);
+ static final Log LOG = LogFactory.getLog(TestEditLog.class);
static final int NUM_DATA_NODES = 0;
// This test creates NUM_THREADS threads and each thread does
// 2 * NUM_TRANSACTIONS Transactions concurrently.
static final int NUM_TRANSACTIONS = 100;
static final int NUM_THREADS = 100;
+ private static final File TEST_DIR = new File(
+ System.getProperty("test.build.data","build/test/data"));
+ /** An edits log with 3 edits from 0.20 - the result of
+ * a fresh namesystem followed by hadoop fs -touchz /myfile */
+ static final byte[] HADOOP20_SOME_EDITS =
+ StringUtils.hexStringToByte((
+ "ffff ffed 0a00 0000 0000 03fa e100 0000" +
+ "0005 0007 2f6d 7966 696c 6500 0133 000d" +
+ "3132 3932 3331 3634 3034 3138 3400 0d31" +
+ "3239 3233 3136 3430 3431 3834 0009 3133" +
+ "3432 3137 3732 3800 0000 0004 746f 6464" +
+ "0a73 7570 6572 6772 6f75 7001 a400 1544" +
+ "4653 436c 6965 6e74 5f2d 3136 3136 3535" +
+ "3738 3931 000b 3137 322e 3239 2e35 2e33" +
+ "3209 0000 0005 0007 2f6d 7966 696c 6500" +
+ "0133 000d 3132 3932 3331 3634 3034 3138" +
+ "3400 0d31 3239 3233 3136 3430 3431 3834" +
+ "0009 3133 3432 3137 3732 3800 0000 0004" +
+ "746f 6464 0a73 7570 6572 6772 6f75 7001" +
+ "a4ff 0000 0000 0000 0000 0000 0000 0000"
+ ).replace(" ",""));
+ static final byte TRAILER_BYTE = FSEditLogOpCodes.OP_INVALID.getOpCode();
+ private static final int CHECKPOINT_ON_STARTUP_MIN_TXNS = 100;
// an object that does a bunch of transactions
@@ -95,15 +127,111 @@ public class TestEditLog extends TestCase {
+ * Test case for an empty edit log from a prior version of Hadoop.
+ public void testPreTxIdEditLogNoEdits() throws Exception {
+ FSNamesystem namesys = Mockito.mock(FSNamesystem.class);
+ namesys.dir = Mockito.mock(FSDirectory.class);
+ int numEdits = testLoad(
+ StringUtils.hexStringToByte("ffffffed"), // just version number
+ namesys);
+ assertEquals(0, numEdits);
+ * Test case for loading a very simple edit log from a format
+ * prior to the inclusion of edit transaction IDs in the log.
+ public void testPreTxidEditLogWithEdits() throws Exception {
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+ cluster.waitActive();
+ final FSNamesystem namesystem = cluster.getNamesystem();
+ int numEdits = testLoad(HADOOP20_SOME_EDITS, namesystem);
+ assertEquals(3, numEdits);
+ // Sanity check the edit
+ HdfsFileStatus fileInfo = namesystem.getFileInfo("/myfile", false);
+ assertEquals("supergroup", fileInfo.getGroup());
+ assertEquals(3, fileInfo.getReplication());
+ if (cluster != null) { cluster.shutdown(); }
+ private int testLoad(byte[] data, FSNamesystem namesys) throws IOException {
+ FSEditLogLoader loader = new FSEditLogLoader(namesys);
+ return loader.loadFSEdits(new EditLogByteInputStream(data), 1);
+ * Simple test for writing to and rolling the edit log.
+ public void testSimpleEditLog() throws IOException {
+ // start a cluster
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
+ FSImage fsimage = namesystem.getFSImage();
+ final FSEditLog editLog = fsimage.getEditLog();
+ assertExistsInStorageDirs(
+ cluster, NameNodeDirType.EDITS,
+ NNStorage.getInProgressEditsFileName(1));
+ editLog.logSetReplication("fakefile", (short) 1);
+ editLog.rollEditLog();
+ NNStorage.getFinalizedEditsFileName(1,3));
+ NNStorage.getInProgressEditsFileName(4));
+ editLog.logSetReplication("fakefile", (short) 2);
+ editLog.close();
+ if(fileSys != null) fileSys.close();
+ if(cluster != null) cluster.shutdown();
* Tests transaction logging in dfs.
- public void testEditLog() throws IOException {
+ public void testMultiThreadedEditLog() throws IOException {
testEditLog(2048);
// force edit buffer to automatically sync on each log of edit log entry
testEditLog(1);
+ private void assertExistsInStorageDirs(MiniDFSCluster cluster,
+ NameNodeDirType dirType,
+ String filename) {
+ NNStorage storage = cluster.getNamesystem().getFSImage().getStorage();
+ for (StorageDirectory sd : storage.dirIterable(dirType)) {
+ File f = new File(sd.getCurrentDir(), filename);
+ assertTrue("Expect that " + f + " exists", f.exists());
* Test edit log with different initial buffer size
@@ -132,9 +260,11 @@ public class TestEditLog extends TestCase {
FSEditLog editLog = fsimage.getEditLog();
// set small size of flush buffer
- editLog.setBufferCapacity(initialSize);
- editLog.close();
+ editLog.setOutputBufferCapacity(initialSize);
+ // Roll log so new output buffer size takes effect
+ // we should now be writing to edits_inprogress_3
+ fsimage.rollEditLog();
// Create threads and make them run transactions concurrently.
Thread threadId[] = new Thread[NUM_THREADS];
@@ -153,32 +283,42 @@ public class TestEditLog extends TestCase {
+ // Roll another time to finalize edits_inprogress_3
+ long expectedTxns = (NUM_THREADS * 2 * NUM_TRANSACTIONS) + 2; // +2 for start/end txns
// Verify that we can read in all the transactions that we have written.
// If there were any corruptions, it is likely that the reading in
// of these transactions will throw an exception.
- FSEditLogLoader loader = new FSEditLogLoader(namesystem);
fsimage.getStorage().dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
- File editFile = NNStorage.getStorageFile(it.next(), NameNodeFile.EDITS);
+ File editFile = NNStorage.getFinalizedEditsFile(it.next(), 3,
+ 3 + expectedTxns - 1);
+ assertTrue("Expect " + editFile + " exists", editFile.exists());
System.out.println("Verifying file: " + editFile);
int numEdits = loader.loadFSEdits(
- new EditLogFileInputStream(editFile));
+ new EditLogFileInputStream(editFile), 3);
int numLeases = namesystem.leaseManager.countLease();
System.out.println("Number of outstanding leases " + numLeases);
assertEquals(0, numLeases);
assertTrue("Verification for " + editFile + " failed. " +
- "Expected " + (NUM_THREADS * 2 * NUM_TRANSACTIONS) + " transactions. "+
+ "Expected " + expectedTxns + " transactions. "+
"Found " + numEdits + " transactions.",
- numEdits == NUM_THREADS * 2 * NUM_TRANSACTIONS);
+ numEdits == expectedTxns);
- if(fileSys != null) fileSys.close();
+ LOG.error("Couldn't shut down cleanly", t);
@@ -231,29 +371,29 @@ public class TestEditLog extends TestCase {
FSImage fsimage = namesystem.getFSImage();
final FSEditLog editLog = fsimage.getEditLog();
- assertEquals("should start with no txids synced",
- 0, editLog.getSyncTxId());
+ assertEquals("should start with only the BEGIN_LOG_SEGMENT txn synced",
+ 1, editLog.getSyncTxId());
// Log an edit from thread A
doLogEdit(threadA, editLog, "thread-a 1");
assertEquals("logging edit without syncing should do not affect txid",
// Log an edit from thread B
doLogEdit(threadB, editLog, "thread-b 1");
// Now ask to sync edit from B, which should sync both edits.
doCallLogSync(threadB, editLog);
assertEquals("logSync from second thread should bump txid up to 2",
- 2, editLog.getSyncTxId());
+ 3, editLog.getSyncTxId());
// Now ask to sync edit from A, which was already batched in - thus
// it should increment the batch count metric
doCallLogSync(threadA, editLog);
assertEquals("logSync from first thread shouldn't change txid",
//Should have incremented the batch count exactly once
assertCounter("TransactionsBatchedInSync", 1L,
@@ -296,12 +436,12 @@ public class TestEditLog extends TestCase {
// logSyncAll in Thread B
doCallLogSyncAll(threadB, editLog);
assertEquals("logSyncAll should sync thread A's transaction",
- 1, editLog.getSyncTxId());
+ 2, editLog.getSyncTxId());
// Close edit log
editLog.close();
@@ -329,10 +469,14 @@ public class TestEditLog extends TestCase {
fileSys.mkdirs(new Path("/tmp"));
- File editFile = editLog.getFsEditName();
+ StorageDirectory sd = fsimage.getStorage().dirIterator(NameNodeDirType.EDITS).next();
- long fileLen = editFile.length();
+ File editFile = NNStorage.getFinalizedEditsFile(sd, 1, 3);
+ assertTrue(editFile.exists());
+ long fileLen = editFile.length();
System.out.println("File name: " + editFile + " len: " + fileLen);
RandomAccessFile rwf = new RandomAccessFile(editFile, "rw");
rwf.seek(fileLen-4); // seek to checksum bytes
@@ -350,54 +494,239 @@ public class TestEditLog extends TestCase {
e.getCause().getClass(), ChecksumException.class);
+ * Test what happens if the NN crashes when it has has started but
+ * had no transactions written.
+ public void testCrashRecoveryNoTransactions() throws Exception {
+ testCrashRecovery(0);
- public void testFailedOpen() throws Exception {
+ * Test what happens if the NN crashes when it has has started and
+ * had a few transactions written
+ public void testCrashRecoveryWithTransactions() throws Exception {
+ testCrashRecovery(150);
+ * Do a test to make sure the edit log can recover edits even after
+ * a non-clean shutdown. This does a simulated crash by copying over
+ * the edits directory while the NN is still running, then shutting it
+ * down, and restoring that edits directory.
+ private void testCrashRecovery(int numTransactions) throws Exception {
- cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
- final FSNamesystem fsn = cluster.getNamesystem();
- // Set up spys
- final FSImage originalImage = fsn.getFSImage();
- NNStorage storage = originalImage.getStorage();
- NNStorage spyStorage = spy(storage);
- originalImage.storage = spyStorage;
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
+ CHECKPOINT_ON_STARTUP_MIN_TXNS);
- final FSEditLog editLog = originalImage.getEditLog();
- FSEditLog spyLog = spy(editLog);
- FSImage spyImage = spy(originalImage);
- fsn.dir.fsImage = spyImage;
- spyImage.storage.setStorageDirectories(
- FSNamesystem.getNamespaceDirs(conf),
+ LOG.info("\n===========================================\n" +
+ "Starting empty cluster");
- // Fail every attempt to open a new edit file
- doThrow(new IOException("Injected fault: open")).
- when(spyLog).addNewEditLogStream((File)anyObject());
+ .numDataNodes(NUM_DATA_NODES)
+ .format(true)
+ .build();
+ FileSystem fs = cluster.getFileSystem();
+ for (int i = 0; i < numTransactions; i++) {
+ fs.mkdirs(new Path("/test" + i));
+ // Directory layout looks like:
+ // test/data/dfs/nameN/current/{fsimage_N,edits_...}
+ File nameDir = new File(cluster.getNameDirs(0).iterator().next().getPath());
+ File dfsDir = nameDir.getParentFile();
+ assertEquals(dfsDir.getName(), "dfs"); // make sure we got right dir
+ LOG.info("Copying data directory aside to a hot backup");
+ File backupDir = new File(dfsDir.getParentFile(), "dfs.backup-while-running");
+ FileUtil.copyDir(dfsDir, backupDir);;
+ LOG.info("Shutting down cluster #1");
+ cluster.shutdown();
+ cluster = null;
+ // Now restore the backup
+ FileUtil.deleteContents(dfsDir);
+ backupDir.renameTo(dfsDir);
+ File currentDir = new File(nameDir, "current");
+ // We should see the file as in-progress
+ File editsFile = new File(currentDir,
+ assertTrue("Edits file " + editsFile + " should exist", editsFile.exists());
+ File imageFile = FSImageTestUtil.findNewestImageFile(
+ currentDir.getAbsolutePath());
+ assertNotNull("No image found in " + nameDir, imageFile);
+ assertEquals(NNStorage.getImageFileName(0), imageFile.getName());
+ // Try to start a new cluster
+ "Starting same cluster after simulated crash");
+ .format(false)
+ // We should still have the files we wrote prior to the simulated crash
+ fs = cluster.getFileSystem();
+ assertTrue(fs.exists(new Path("/test" + i)));
+ long expectedTxId;
+ if (numTransactions > CHECKPOINT_ON_STARTUP_MIN_TXNS) {
+ // It should have saved a checkpoint on startup since there
+ // were more unfinalized edits than configured
+ expectedTxId = numTransactions + 1;
+ // otherwise, it shouldn't have made a checkpoint
+ expectedTxId = 0;
+ imageFile = FSImageTestUtil.findNewestImageFile(
+ assertEquals(NNStorage.getImageFileName(expectedTxId),
+ imageFile.getName());
+ // Started successfully. Shut it down and make sure it can restart.
+ if (cluster != null) {
+ public void testCrashRecoveryEmptyLogOneDir() throws Exception {
+ doTestCrashRecoveryEmptyLog(false);
+ public void testCrashRecoveryEmptyLogBothDirs() throws Exception {
+ doTestCrashRecoveryEmptyLog(true);
+ * Test that the NN handles the corruption properly
+ * after it crashes just after creating an edit log
+ * (ie before writing START_LOG_SEGMENT). In the case
+ * that all logs have this problem, it should mark them
+ * as corrupt instead of trying to finalize them.
+ * @param inBothDirs if true, there will be a truncated log in
+ * both of the edits directories. If false, the truncated log
+ * will only be in one of the directories. In both cases, the
+ * NN should fail to start up, because it's aware that txid 3
+ * was reached, but unable to find a non-corrupt log starting there.
+ private void doTestCrashRecoveryEmptyLog(boolean inBothDirs) throws Exception {
+ .numDataNodes(NUM_DATA_NODES).build();
+ Collection<URI> editsDirs = cluster.getNameEditsDirs(0);
+ for (URI uri : editsDirs) {
+ File dir = new File(uri.getPath());
+ File currentDir = new File(dir, "current");
+ // We should start with only the finalized edits_1-2
+ GenericTestUtils.assertGlobEquals(currentDir, "edits_.*",
+ NNStorage.getFinalizedEditsFileName(1, 2));
+ // Make a truncated edits_3_inprogress
+ File log = new File(currentDir,
+ NNStorage.getInProgressEditsFileName(3));
+ new EditLogFileOutputStream(log, 1024).create();
+ if (!inBothDirs) {
- spyLog.close();
- spyLog.open();
- fail("open did not fail even when all directories failed!");
- } catch(IOException ioe) {
- LOG.info("Got expected exception", ioe);
+ .numDataNodes(NUM_DATA_NODES).format(false).build();
+ fail("Did not fail to start with all-corrupt logs");
+ } catch (IllegalStateException ise) {
+ GenericTestUtils.assertExceptionContains(
+ "No non-corrupt logs for txid 3", ise);
+ private static class EditLogByteInputStream extends EditLogInputStream {
+ private InputStream input;
+ private long len;
+ public EditLogByteInputStream(byte[] data) {
+ len = data.length;
+ input = new ByteArrayInputStream(data);
+ public int available() throws IOException {
+ return input.available();
+ public int read() throws IOException {
+ return input.read();
- // Reset and try it with a working open
- Mockito.reset(spyLog);
+ public long length() throws IOException {
+ return len;
- // Close everything off
- originalImage.close();
- fsn.close();
+ public int read(byte[] b, int off, int len) throws IOException {
+ return input.read(b, off, len);
+ input.close();
+ @Override // JournalStream
+ public String getName() {
+ return "AnonEditLogByteInputStream";
+ public JournalType getType() {
+ return JournalType.FILE;
+ public void testFailedOpen() throws Exception {
+ File logDir = new File(TEST_DIR, "testFailedOpen");
+ logDir.mkdirs();
+ FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir);
+ logDir.setWritable(false);
+ log.open();
+ fail("Did no throw exception on only having a bad dir");
+ "no journals successfully started", ioe);
+ logDir.setWritable(true);
+ log.close();
@@ -26,40 +26,64 @@ import java.io.File;
import org.apache.hadoop.fs.DU;
+import org.junit.Before;
public class TestEditLogFileOutputStream {
+ private final static int HEADER_LEN = 17;
+ private static final File TEST_EDITS =
+ new File(System.getProperty("test.build.data","/tmp"),
+ "editLogStream.dat");
+ @Before
+ public void deleteEditsFile() {
+ TEST_EDITS.delete();
@Test
public void testPreallocation() throws IOException {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
.build();
- File editLog = cluster.getNameNode().getFSImage().getEditLog()
- .getFsEditName();
+ StorageDirectory sd = cluster.getNameNode().getFSImage()
+ .getStorage().getStorageDir(0);
+ File editLog = NNStorage.getInProgressEditsFile(sd, 1);
- assertEquals("Edit log should only be 4 bytes long",
- 4, editLog.length());
- assertEquals("Edit log disk space used should be one block",
- 4096, new DU(editLog, conf).getUsed());
+ EditLogValidation validation = FSEditLogLoader.validateEditLog(editLog);
+ assertEquals("Edit log should contain a header as valid length",
+ HEADER_LEN, validation.validLength);
+ assertEquals(1, validation.numTransactions);
+ assertEquals("Edit log should have 1MB of bytes allocated",
+ 1024*1024, editLog.length());
cluster.getFileSystem().mkdirs(new Path("/tmp"),
new FsPermission((short)777));
- assertEquals("Edit log should be 1MB + 4 bytes long",
- (1024 * 1024) + 4, editLog.length());
- // 256 blocks for the 1MB of preallocation space, 1 block for the original
- // 4 bytes
+ long oldLength = validation.validLength;
+ validation = FSEditLogLoader.validateEditLog(editLog);
+ assertTrue("Edit log should have more valid data after writing a txn " +
+ "(was: " + oldLength + " now: " + validation.validLength + ")",
+ validation.validLength > oldLength);
+ assertEquals(2, validation.numTransactions);
+ assertEquals("Edit log should be 1MB long",
+ 1024 * 1024, editLog.length());
+ // 256 blocks for the 1MB of preallocation space
assertTrue("Edit log disk space used should be at least 257 blocks",
- 257 * 4096 <= new DU(editLog, conf).getUsed());
+ 256 * 4096 <= new DU(editLog, conf).getUsed());
@@ -86,4 +110,48 @@ public class TestEditLogFileOutputStream {
assertEquals("fc was not nulled when elos.close() failed", elos.getFileChannelForTesting(), null);
+ * Tests EditLogFileOutputStream doesn't throw NullPointerException on
+ * close/abort sequence. See HDFS-2011.
+ @Test
+ public void testEditLogFileOutputStreamCloseAbort() throws IOException {
+ // abort after a close should just ignore
+ EditLogFileOutputStream editLogStream =
+ new EditLogFileOutputStream(TEST_EDITS, 0);
+ editLogStream.close();
+ editLogStream.abort();
+ * close/close sequence. See HDFS-2011.
+ public void testEditLogFileOutputStreamCloseClose() throws IOException {
+ // close after a close should result in an IOE
+ assertTrue(msg, msg.contains("Trying to use aborted output stream"));
+ * Tests EditLogFileOutputStream doesn't throw NullPointerException on being
+ * abort/abort sequence. See HDFS-2011.
+ public void testEditLogFileOutputStreamAbortAbort() throws IOException {
@@ -0,0 +1,176 @@
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.junit.After;
+import org.junit.Test;
+public class TestEditLogJournalFailures {
+ private int editsPerformed = 0;
+ private Configuration conf;
+ private MiniDFSCluster cluster;
+ private FileSystem fs;
+ private Runtime runtime;
+ * Create the mini cluster for testing and sub in a custom runtime so that
+ * edit log journal failures don't actually cause the JVM to exit.
+ public void setUpMiniCluster() throws IOException {
+ conf = new HdfsConfiguration();
+ runtime = Runtime.getRuntime();
+ runtime = spy(runtime);
+ doNothing().when(runtime).exit(anyInt());
+ cluster.getNameNode().getFSImage().getEditLog().setRuntimeForTesting(runtime);
+ @After
+ public void shutDownMiniCluster() throws IOException {
+ fs.close();
+ public void testSingleFailedEditsDirOnFlush() throws IOException {
+ assertTrue(doAnEdit());
+ // Invalidate one edits journal.
+ invalidateEditsDirAtIndex(0, true);
+ // Make sure runtime.exit(...) hasn't been called at all yet.
+ assertExitInvocations(0);
+ // A single journal failure should not result in a call to runtime.exit(...).
+ assertFalse(cluster.getNameNode().isInSafeMode());
+ public void testAllEditsDirsFailOnFlush() throws IOException {
+ // Invalidate both edits journals.
+ invalidateEditsDirAtIndex(1, true);
+ // The previous edit could not be synced to any persistent storage, should
+ // have halted the NN.
+ assertExitInvocations(1);
+ public void testSingleFailedEditsDirOnSetReadyToFlush() throws IOException {
+ invalidateEditsDirAtIndex(0, false);
+ * Replace the journal at index <code>index</code> with one that throws an
+ * exception on flush.
+ * @param index the index of the journal to take offline.
+ * @return the original <code>EditLogOutputStream</code> of the journal.
+ private EditLogOutputStream invalidateEditsDirAtIndex(int index,
+ boolean failOnFlush) throws IOException {
+ FSImage fsimage = cluster.getNamesystem().getFSImage();
+ FSEditLog editLog = fsimage.getEditLog();
+ FSEditLog.JournalAndStream jas = editLog.getJournals().get(index);
+ EditLogFileOutputStream elos =
+ (EditLogFileOutputStream) jas.getCurrentStream();
+ EditLogFileOutputStream spyElos = spy(elos);
+ if (failOnFlush) {
+ doThrow(new IOException("fail on flush()")).when(spyElos).flush();
+ doThrow(new IOException("fail on setReadyToFlush()")).when(spyElos)
+ .setReadyToFlush();
+ doNothing().when(spyElos).abort();
+ jas.setCurrentStreamForTests(spyElos);
+ return elos;
+ * Restore the journal at index <code>index</code> with the passed
+ * {@link EditLogOutputStream}.
+ * @param index index of the journal to restore.
+ * @param elos the {@link EditLogOutputStream} to put at that index.
+ private void restoreEditsDirAtIndex(int index, EditLogOutputStream elos) {
+ jas.setCurrentStreamForTests(elos);
+ * Do a mutative metadata operation on the file system.
+ * @return true if the operation was successful, false otherwise.
+ private boolean doAnEdit() throws IOException {
+ return fs.mkdirs(new Path("/tmp", Integer.toString(editsPerformed++)));
+ * Make sure that Runtime.exit(...) has been called
+ * <code>expectedExits<code> number of times.
+ * @param expectedExits the number of times Runtime.exit(...) should have been called.
+ private void assertExitInvocations(int expectedExits) {
+ verify(runtime, times(expectedExits)).exit(anyInt());
@@ -18,11 +18,12 @@
import java.util.concurrent.CountDownLatch;
@@ -40,8 +41,7 @@ import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
-import org.apache.hadoop.test.GenericTestUtils;
@@ -55,6 +55,10 @@ import static org.mockito.Mockito.*;
* and namespace saving.
public class TestEditLogRace {
private static final Log LOG = LogFactory.getLog(TestEditLogRace.class);
private static final String NAME_DIR =
@@ -181,27 +185,29 @@ public class TestEditLogRace {
- // set small size of flush buffer
- editLog.setBufferCapacity(2048);
+ StorageDirectory sd = fsimage.getStorage().getStorageDir(0);
startTransactionWorkers(namesystem, caughtErr);
+ long previousLogTxId = 1;
for (int i = 0; i < NUM_ROLLS && caughtErr.get() == null; i++) {
Thread.sleep(20);
} catch (InterruptedException e) {}
LOG.info("Starting roll " + i + ".");
- editLog.rollEditLog();
- LOG.info("Roll complete " + i + ".");
- verifyEditLogs(namesystem, fsimage);
- LOG.info("Starting purge " + i + ".");
- editLog.purgeEditLog();
- LOG.info("Complete purge " + i + ".");
+ CheckpointSignature sig = namesystem.rollEditLog();
+ long nextLog = sig.curSegmentTxId;
+ String logFileName = NNStorage.getFinalizedEditsFileName(
+ previousLogTxId, nextLog - 1);
+ previousLogTxId += verifyEditLogs(namesystem, fsimage, logFileName, previousLogTxId);
+ assertEquals(previousLogTxId, nextLog);
+ File expectedLog = NNStorage.getInProgressEditsFile(sd, previousLogTxId);
+ assertTrue("Expect " + expectedLog + " to exist", expectedLog.exists());
stopTransactionWorkers();
@@ -214,19 +220,32 @@ public class TestEditLogRace {
- private void verifyEditLogs(FSNamesystem namesystem, FSImage fsimage)
+ private long verifyEditLogs(FSNamesystem namesystem, FSImage fsimage,
+ String logFileName, long startTxId)
+ long numEdits = -1;
- fsimage.getStorage().dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
- File editFile = fsimage.getStorage().getStorageFile(it.next(), NameNodeFile.EDITS);
+ for (StorageDirectory sd :
+ fsimage.getStorage().dirIterable(NameNodeDirType.EDITS)) {
+ File editFile = new File(sd.getCurrentDir(), logFileName);
- int numEdits = new FSEditLogLoader(namesystem).loadFSEdits(
- System.out.println("Number of edits: " + numEdits);
+ int numEditsThisLog = loader.loadFSEdits(new EditLogFileInputStream(editFile),
+ System.out.println("Number of edits: " + numEditsThisLog);
+ assertTrue(numEdits == -1 || numEditsThisLog == numEdits);
+ numEdits = numEditsThisLog;
+ assertTrue(numEdits != -1);
+ return numEdits;
@@ -249,11 +268,6 @@ public class TestEditLogRace {
for (int i = 0; i < NUM_SAVE_IMAGE && caughtErr.get() == null; i++) {
@@ -266,14 +280,28 @@ public class TestEditLogRace {
namesystem.enterSafeMode(false);
// Verify edit logs before the save
+ // They should start with the first edit after the checkpoint
+ long logStartTxId = fsimage.getStorage().getMostRecentCheckpointTxId() + 1;
+ verifyEditLogs(namesystem, fsimage,
+ NNStorage.getInProgressEditsFileName(logStartTxId),
+ logStartTxId);
LOG.info("Save " + i + ": saving namespace");
namesystem.saveNamespace();
LOG.info("Save " + i + ": leaving safemode");
- // Verify that edit logs post save are also not corrupt
+ long savedImageTxId = fsimage.getStorage().getMostRecentCheckpointTxId();
+ // Verify that edit logs post save got finalized and aren't corrupt
+ NNStorage.getFinalizedEditsFileName(logStartTxId, savedImageTxId),
+ // The checkpoint id should be 1 less than the last written ID, since
+ // the log roll writes the "BEGIN" transaction to the new log.
+ assertEquals(fsimage.getStorage().getMostRecentCheckpointTxId(),
+ editLog.getLastWrittenTxId() - 1);
namesystem.leaveSafeMode(false);
LOG.info("Save " + i + ": complete");
@@ -328,9 +356,10 @@ public class TestEditLogRace {
- ArrayList<EditLogOutputStream> streams = editLog.getEditStreams();
- EditLogOutputStream spyElos = spy(streams.get(0));
- streams.set(0, spyElos);
+ FSEditLog.JournalAndStream jas = editLog.getJournals().get(0);
+ EditLogFileOutputStream spyElos =
+ spy((EditLogFileOutputStream)jas.getCurrentStream());
final AtomicReference<Throwable> deferredException =
new AtomicReference<Throwable>();
@@ -393,7 +422,14 @@ public class TestEditLogRace {
doAnEditThread.join();
assertNull(deferredException.get());
+ // We did 3 edits: begin, txn, and end
+ assertEquals(3, verifyEditLogs(namesystem, fsimage,
+ NNStorage.getFinalizedEditsFileName(1, 3),
+ 1));
+ // after the save, just the one "begin"
+ assertEquals(1, verifyEditLogs(namesystem, fsimage,
+ NNStorage.getInProgressEditsFileName(4),
+ 4));
LOG.info("Closing namesystem");
if(namesystem != null) namesystem.close();
@@ -478,7 +514,14 @@ public class TestEditLogRace {
@@ -20,9 +20,14 @@ package org.apache.hadoop.hdfs.server.namenode;
+import java.nio.channels.FileChannel;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -31,10 +36,25 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
public class TestFSEditLogLoader {
+ ((Log4JLogger)FSImage.LOG).getLogger().setLevel(Level.ALL);
private static final int NUM_DATA_NODES = 0;
@@ -50,14 +70,15 @@ public class TestFSEditLogLoader {
final FSNamesystem namesystem = cluster.getNamesystem();
- final FSEditLog editLog = fsimage.getEditLog();
for (int i = 0; i < 20; i++) {
fileSys.mkdirs(new Path("/tmp/tmp" + i));
+ File editFile = FSImageTestUtil.findLatestEditsLog(sd).getFile();
+ assertTrue("Should exist: " + editFile, editFile.exists());
// Corrupt the edits file.
long fileLen = editFile.length();
@@ -124,4 +145,168 @@ public class TestFSEditLogLoader {
+ * Test that the valid number of transactions can be counted from a file.
+ public void testCountValidTransactions() throws IOException {
+ File testDir = new File(TEST_DIR, "testCountValidTransactions");
+ File logFile = new File(testDir,
+ // Create a log file, and return the offsets at which each
+ // transaction starts.
+ FSEditLog fsel = null;
+ final int NUM_TXNS = 30;
+ SortedMap<Long, Long> offsetToTxId = Maps.newTreeMap();
+ fsel = FSImageTestUtil.createStandaloneEditLog(testDir);
+ fsel.open();
+ assertTrue("should exist: " + logFile, logFile.exists());
+ for (int i = 0; i < NUM_TXNS; i++) {
+ long trueOffset = getNonTrailerLength(logFile);
+ long thisTxId = fsel.getLastWrittenTxId() + 1;
+ offsetToTxId.put(trueOffset, thisTxId);
+ System.err.println("txid " + thisTxId + " at offset " + trueOffset);
+ fsel.logDelete("path" + i, i);
+ fsel.logSync();
+ if (fsel != null) {
+ fsel.close();
+ // The file got renamed when the log was closed.
+ logFile = testDir.listFiles()[0];
+ long validLength = getNonTrailerLength(logFile);
+ // Make sure that uncorrupted log has the expected length and number
+ // of transactions.
+ EditLogValidation validation = FSEditLogLoader.validateEditLog(logFile);
+ assertEquals(NUM_TXNS + 2, validation.numTransactions);
+ assertEquals(validLength, validation.validLength);
+ // Back up the uncorrupted log
+ File logFileBak = new File(testDir, logFile.getName() + ".bak");
+ Files.copy(logFile, logFileBak);
+ // Corrupt the log file in various ways for each txn
+ for (Map.Entry<Long, Long> entry : offsetToTxId.entrySet()) {
+ long txOffset = entry.getKey();
+ long txid = entry.getValue();
+ // Restore backup, truncate the file exactly before the txn
+ Files.copy(logFileBak, logFile);
+ truncateFile(logFile, txOffset);
+ validation = FSEditLogLoader.validateEditLog(logFile);
+ assertEquals("Failed when truncating to length " + txOffset,
+ txid - 1, validation.numTransactions);
+ assertEquals(txOffset, validation.validLength);
+ // Restore backup, truncate the file with one byte in the txn,
+ // also isn't valid
+ truncateFile(logFile, txOffset + 1);
+ assertEquals("Failed when truncating to length " + (txOffset + 1),
+ // Restore backup, corrupt the txn opcode
+ corruptByteInFile(logFile, txOffset);
+ assertEquals("Failed when corrupting txn opcode at " + txOffset,
+ // Restore backup, corrupt a byte a few bytes into the txn
+ corruptByteInFile(logFile, txOffset+5);
+ assertEquals("Failed when corrupting txn data at " + (txOffset+5),
+ // Corrupt the log at every offset to make sure that validation itself
+ // never throws an exception, and that the calculated lengths are monotonically
+ // increasing
+ long prevNumValid = 0;
+ for (long offset = 0; offset < validLength; offset++) {
+ corruptByteInFile(logFile, offset);
+ EditLogValidation val = FSEditLogLoader.validateEditLog(logFile);
+ assertTrue(val.numTransactions >= prevNumValid);
+ prevNumValid = val.numTransactions;
+ * Corrupt the byte at the given offset in the given file,
+ * by subtracting 1 from it.
+ private void corruptByteInFile(File file, long offset)
+ RandomAccessFile raf = new RandomAccessFile(file, "rw");
+ raf.seek(offset);
+ int origByte = raf.read();
+ raf.writeByte(origByte - 1);
+ * Truncate the given file to the given length
+ private void truncateFile(File logFile, long newLength)
+ RandomAccessFile raf = new RandomAccessFile(logFile, "rw");
+ raf.setLength(newLength);
+ raf.close();
+ * Return the length of bytes in the given file after subtracting
+ * the trailer of 0xFF (OP_INVALID)s.
+ * This seeks to the end of the file and reads chunks backwards until
+ * it finds a non-0xFF byte.
+ * @throws IOException if the file cannot be read
+ private static long getNonTrailerLength(File f) throws IOException {
+ final int chunkSizeToRead = 256*1024;
+ byte buf[] = new byte[chunkSizeToRead];
+ FileChannel fc = fis.getChannel();
+ long size = fc.size();
+ long pos = size - (size % chunkSizeToRead);
+ while (pos >= 0) {
+ fc.position(pos);
+ int readLen = (int) Math.min(size - pos, chunkSizeToRead);
+ IOUtils.readFully(fis, buf, 0, readLen);
+ for (int i = readLen - 1; i >= 0; i--) {
+ if (buf[i] != FSEditLogOpCodes.OP_INVALID.getOpCode()) {
+ return pos + i + 1; // + 1 since we count this byte!
+ pos -= chunkSizeToRead;
@@ -1,91 +0,0 @@
-/**
- * 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 static org.junit.Assert.assertEquals;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.io.IOUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-public class TestFSImage {
- private static final String OUT_DIR = System.getProperty("test.build.data",
- "build/test/fsimage");
- private MiniDFSCluster miniDFSCluster = null;
- private static Configuration nnConf = new Configuration();
- private File current = new File(OUT_DIR);
- @Before
- public void setUpCluster() throws Exception {
- clearDirs();
- @After
- public void clusterShutdown() throws Exception {
- if (null != miniDFSCluster) {
- miniDFSCluster.shutdown();
- @Test
- public void testLoadFsEditsShouldReturnTrueWhenEditsNewExists()
- throws Exception {
- nnConf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, OUT_DIR + "/BNN1");
- NameNode.format(nnConf);
- miniDFSCluster = new MiniDFSCluster.Builder(nnConf).numDataNodes(1).build();
- FSImage image = miniDFSCluster.getNameNode().getFSImage();
- URI next = FSNamesystem
- .getNamespaceDirs(miniDFSCluster.getConfiguration(0)).iterator().next();
- File editsNew = new File(next.getRawPath() , "/current/edits.new");
- createEditsNew(editsNew, image);
- int loadFSEdits = image.loadFSEdits(image.getStorage().getStorageDir(0));
- assertEquals("The numEdits should not be zero.", 1, loadFSEdits);
- private void createEditsNew(File editsNew, FSImage image) throws Exception {
- FileOutputStream fileOutputStream = null;
- if (!editsNew.exists()) {
- editsNew.createNewFile();
- image.editLog.createEditLogFile(editsNew);
- IOUtils.closeStream(fileOutputStream);
- private void clearDirs() throws IOException {
- if (current.exists()) {
- FileUtil.fullyDelete(current);
-}
@@ -0,0 +1,471 @@
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
+import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.TransactionalLoadPlan;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.LogGroup;
+import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.LoadPlan;
+public class TestFSImageStorageInspector {
+ TestFSImageStorageInspector.class);
+ * Simple test with image, edits, and inprogress edits
+ public void testCurrentStorageInspector() throws IOException {
+ StorageDirectory mockDir = mockDirectory(
+ NameNodeDirType.IMAGE_AND_EDITS,
+ false,
+ "/foo/current/" + getImageFileName(123),
+ "/foo/current/" + getFinalizedEditsFileName(123, 456),
+ "/foo/current/" + getImageFileName(456),
+ "/foo/current/" + getInProgressEditsFileName(457));
+ inspector.inspectDirectory(mockDir);
+ mockLogValidation(inspector,
+ "/foo/current/" + getInProgressEditsFileName(457), 10);
+ assertEquals(2, inspector.foundEditLogs.size());
+ assertEquals(2, inspector.foundImages.size());
+ assertTrue(inspector.foundEditLogs.get(1).isInProgress());
+ assertEquals(456, latestImage.txId);
+ assertSame(mockDir, latestImage.sd);
+ assertTrue(inspector.isUpgradeFinalized());
+ LoadPlan plan = inspector.createLoadPlan();
+ LOG.info("Plan: " + plan);
+ assertEquals(new File("/foo/current/"+getImageFileName(456)),
+ plan.getImageFile());
+ assertArrayEquals(new File[] {
+ new File("/foo/current/" + getInProgressEditsFileName(457)) },
+ plan.getEditsFiles().toArray(new File[0]));
+ * Test that we check for gaps in txids when devising a load plan.
+ public void testPlanWithGaps() throws IOException {
+ "/foo/current/" + getFinalizedEditsFileName(457,900),
+ "/foo/current/" + getFinalizedEditsFileName(901,950),
+ "/foo/current/" + getFinalizedEditsFileName(952,1000)); // <-- missing edit 951!
+ inspector.createLoadPlan();
+ fail("Didn't throw IOE trying to load with gaps in edits");
+ assertTrue(ioe.getMessage().contains(
+ "would start at txid 951 but starts at txid 952"));
+ * Test the case where an in-progress log comes in the middle of a sequence
+ * of logs
+ public void testPlanWithInProgressInMiddle() throws IOException {
+ "/foo/current/" + getInProgressEditsFileName(901), // <-- inprogress in middle
+ "/foo/current/" + getFinalizedEditsFileName(952,1000));
+ "/foo/current/" + getInProgressEditsFileName(901), 51);
+ assertEquals(new File("/foo/current/" + getImageFileName(456)),
+ new File("/foo/current/" + getFinalizedEditsFileName(457,900)),
+ new File("/foo/current/" + getInProgressEditsFileName(901)),
+ new File("/foo/current/" + getFinalizedEditsFileName(952,1000)) },
+ * Test case for the usual case where no recovery of a log group is necessary
+ * (i.e all logs have the same start and end txids and finalized)
+ public void testLogGroupRecoveryNoop() throws IOException {
+ inspector.inspectDirectory(
+ mockDirectoryWithEditLogs("/foo1/current/"
+ + getFinalizedEditsFileName(123,456)));
+ mockDirectoryWithEditLogs("/foo2/current/"
+ mockDirectoryWithEditLogs("/foo3/current/"
+ LogGroup lg = inspector.logGroups.get(123L);
+ assertEquals(3, lg.logs.size());
+ lg.planRecovery();
+ assertFalse(lg.logs.get(0).isCorrupt());
+ assertFalse(lg.logs.get(1).isCorrupt());
+ assertFalse(lg.logs.get(2).isCorrupt());
+ * Test case where we have some in-progress and some finalized logs
+ * for a given txid.
+ public void testLogGroupRecoveryMixed() throws IOException {
+ + getInProgressEditsFileName(123)));
+ inspector.inspectDirectory(mockDirectory(
+ NameNodeDirType.IMAGE,
+ "/foo4/current/" + getImageFileName(122)));
+ FoundEditLog inProgressLog = lg.logs.get(2);
+ assertTrue(inProgressLog.isInProgress());
+ // Check that it was marked corrupt.
+ assertTrue(lg.logs.get(2).isCorrupt());
+ // Calling recover should move it aside
+ inProgressLog = spy(inProgressLog);
+ Mockito.doNothing().when(inProgressLog).moveAsideCorruptFile();
+ lg.logs.set(2, inProgressLog);
+ plan.doRecovery();
+ Mockito.verify(inProgressLog).moveAsideCorruptFile();
+ * Test case where we have finalized logs with different end txids
+ public void testLogGroupRecoveryInconsistentEndTxIds() throws IOException {
+ + getFinalizedEditsFileName(123,678)));
+ assertEquals(2, lg.logs.size());
+ fail("Didn't throw IOE on inconsistent end txids");
+ assertTrue(ioe.getMessage().contains("More than one ending txid"));
+ * Test case where we have only in-progress logs and need to synchronize
+ * based on valid length.
+ public void testLogGroupRecoveryInProgress() throws IOException {
+ String paths[] = new String[] {
+ "/foo1/current/" + getInProgressEditsFileName(123),
+ "/foo2/current/" + getInProgressEditsFileName(123),
+ "/foo3/current/" + getInProgressEditsFileName(123)
+ inspector.inspectDirectory(mockDirectoryWithEditLogs(paths[0]));
+ inspector.inspectDirectory(mockDirectoryWithEditLogs(paths[1]));
+ inspector.inspectDirectory(mockDirectoryWithEditLogs(paths[2]));
+ // Inject spies to return the valid counts we would like to see
+ mockLogValidation(inspector, paths[0], 2000);
+ mockLogValidation(inspector, paths[1], 2000);
+ mockLogValidation(inspector, paths[2], 1000);
+ // Check that the short one was marked corrupt
+ FoundEditLog badLog = lg.logs.get(2);
+ Mockito.doNothing().when(badLog).moveAsideCorruptFile();
+ Mockito.doNothing().when(lg.logs.get(0)).finalizeLog();
+ Mockito.doNothing().when(lg.logs.get(1)).finalizeLog();
+ lg.recover();
+ Mockito.verify(badLog).moveAsideCorruptFile();
+ Mockito.verify(lg.logs.get(0)).finalizeLog();
+ Mockito.verify(lg.logs.get(1)).finalizeLog();
+ * Mock out the log at the given path to return a specified number
+ * of transactions upon validation.
+ private void mockLogValidation(
+ String path, int numValidTransactions) throws IOException {
+ for (LogGroup lg : inspector.logGroups.values()) {
+ List<FoundEditLog> logs = lg.logs;
+ for (int i = 0; i < logs.size(); i++) {
+ FoundEditLog log = logs.get(i);
+ if (log.file.getPath().equals(path)) {
+ // mock out its validation
+ FoundEditLog spyLog = spy(log);
+ doReturn(new FSEditLogLoader.EditLogValidation(-1, numValidTransactions))
+ .when(spyLog).validateLog();
+ logs.set(i, spyLog);
+ fail("No log found to mock out at " + path);
+ * Test when edits and image are in separate directories.
+ public void testCurrentSplitEditsAndImage() throws IOException {
+ StorageDirectory mockImageDir = mockDirectory(
+ "/foo/current/" + getImageFileName(123));
+ StorageDirectory mockImageDir2 = mockDirectory(
+ "/foo2/current/" + getImageFileName(456));
+ StorageDirectory mockEditsDir = mockDirectory(
+ NameNodeDirType.EDITS,
+ "/foo3/current/" + getFinalizedEditsFileName(123, 456),
+ "/foo3/current/" + getInProgressEditsFileName(457));
+ inspector.inspectDirectory(mockImageDir);
+ inspector.inspectDirectory(mockEditsDir);
+ inspector.inspectDirectory(mockImageDir2);
+ "/foo3/current/" + getInProgressEditsFileName(457), 2);
+ // Check plan
+ TransactionalLoadPlan plan =
+ (TransactionalLoadPlan)inspector.createLoadPlan();
+ FoundFSImage pickedImage = plan.image;
+ assertEquals(456, pickedImage.txId);
+ assertSame(mockImageDir2, pickedImage.sd);
+ assertEquals(new File("/foo2/current/" + getImageFileName(456)),
+ new File("/foo3/current/" + getInProgressEditsFileName(457))
+ }, plan.getEditsFiles().toArray(new File[0]));
+ // Check log manifest
+ assertEquals("[[123,456]]", inspector.getEditLogManifest(123).toString());
+ assertEquals("[[123,456]]", inspector.getEditLogManifest(456).toString());
+ assertEquals("[]", inspector.getEditLogManifest(457).toString());
+ public void testLogManifest() throws IOException {
+ + getFinalizedEditsFileName(1,1),
+ "/foo1/current/"
+ + getFinalizedEditsFileName(2,200)));
+ + getInProgressEditsFileName(1),
+ "/foo2/current/"
+ + getFinalizedEditsFileName(201, 400)));
+ + getFinalizedEditsFileName(1, 1),
+ "/foo3/current/"
+ assertEquals("[[1,1], [2,200], [201,400]]",
+ inspector.getEditLogManifest(1).toString());
+ assertEquals("[[2,200], [201,400]]",
+ inspector.getEditLogManifest(2).toString());
+ inspector.getEditLogManifest(10).toString());
+ assertEquals("[[201,400]]",
+ inspector.getEditLogManifest(201).toString());
+ * Test case where an in-progress log is in an earlier name directory
+ * than a finalized log. Previously, getEditLogManifest wouldn't
+ * see this log.
+ public void testLogManifestInProgressComesFirst() throws IOException {
+ + getFinalizedEditsFileName(2622,2623),
+ + getFinalizedEditsFileName(2624,2625),
+ + getInProgressEditsFileName(2626)));
+ + getFinalizedEditsFileName(2626,2627),
+ + getFinalizedEditsFileName(2628,2629)));
+ assertEquals("[[2622,2623], [2624,2625], [2626,2627], [2628,2629]]",
+ inspector.getEditLogManifest(2621).toString());
+ private StorageDirectory mockDirectoryWithEditLogs(String... fileNames) {
+ return mockDirectory(NameNodeDirType.EDITS, false, fileNames);
+ * Make a mock storage directory that returns some set of file contents.
+ * @param type type of storage dir
+ * @param previousExists should we mock that the previous/ dir exists?
+ * @param fileNames the names of files contained in current/
+ static StorageDirectory mockDirectory(
+ StorageDirType type,
+ boolean previousExists,
+ String... fileNames) {
+ StorageDirectory sd = mock(StorageDirectory.class);
+ doReturn(type).when(sd).getStorageDirType();
+ // Version file should always exist
+ doReturn(FSImageTestUtil.mockFile(true)).when(sd).getVersionFile();
+ // Previous dir optionally exists
+ doReturn(FSImageTestUtil.mockFile(previousExists))
+ .when(sd).getPreviousDir();
+ // Return a mock 'current' directory which has the given paths
+ File[] files = new File[fileNames.length];
+ for (int i = 0; i < fileNames.length; i++) {
+ files[i] = new File(fileNames[i]);
+ File mockDir = Mockito.spy(new File("/dir/current"));
+ doReturn(files).when(mockDir).listFiles();
+ doReturn(mockDir).when(sd).getCurrentDir();
@@ -23,6 +23,7 @@ import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -33,6 +34,8 @@ import org.apache.hadoop.hdfs.protocol.FSLimitException;
import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
@@ -59,7 +62,7 @@ public class TestFsLimits {
private static class TestFSDirectory extends FSDirectory {
public TestFSDirectory() throws IOException {
- super(new FSImage(), getMockNamesystem(), conf);
+ super(new FSImage(conf), getMockNamesystem(), conf);
setReady(fsIsReady);
@@ -71,8 +74,12 @@ public class TestFsLimits {
@Before
- public void setUp() {
+ public void setUp() throws IOException {
conf = new Configuration();
+ conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+ fileAsURI(new File(MiniDFSCluster.getBaseDirectory(),
+ "namenode")).toString());
rootInode = new INodeDirectoryWithQuota(INodeDirectory.ROOT_NAME, perms, 0L, 0L);
inodes = new INode[]{ rootInode, null };
fs = null;
@@ -0,0 +1,155 @@
+import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
+import static org.apache.hadoop.test.GenericTestUtils.assertGlobEquals;
+ * Functional tests for NNStorageRetentionManager. This differs from
+ * {@link TestNNStorageRetentionManager} in that the other test suite
+ * is only unit/mock-based tests whereas this suite starts miniclusters,
+ * etc.
+public class TestNNStorageRetentionFunctional {
+ private static File TEST_ROOT_DIR =
+ new File(MiniDFSCluster.getBaseDirectory());
+ private static Log LOG = LogFactory.getLog(
+ TestNNStorageRetentionFunctional.class);
+ * Test case where two directories are configured as NAME_AND_EDITS
+ * and one of them fails to save storage. Since the edits and image
+ * failure states are decoupled, the failure of image saving should
+ * not prevent the purging of logs from that dir.
+ public void testPurgingWithNameEditsDirAfterFailure()
+ File sd0 = new File(TEST_ROOT_DIR, "nn0");
+ File sd1 = new File(TEST_ROOT_DIR, "nn1");
+ File cd0 = new File(sd0, "current");
+ File cd1 = new File(sd1, "current");
+ Joiner.on(",").join(sd0, sd1));
+ .numDataNodes(0)
+ .manageNameDfsDirs(false)
+ .format(true).build();
+ doSaveNamespace(nn);
+ LOG.info("After first save, images 0 and 2 should exist in both dirs");
+ assertGlobEquals(cd0, "fsimage_\\d*",
+ getImageFileName(0), getImageFileName(2));
+ assertGlobEquals(cd1, "fsimage_\\d*",
+ assertGlobEquals(cd0, "edits_.*",
+ getFinalizedEditsFileName(1, 2),
+ getInProgressEditsFileName(3));
+ assertGlobEquals(cd1, "edits_.*",
+ LOG.info("After second save, image 0 should be purged, " +
+ "and image 4 should exist in both.");
+ getImageFileName(2), getImageFileName(4));
+ getFinalizedEditsFileName(3, 4),
+ getInProgressEditsFileName(5));
+ LOG.info("Failing first storage dir by chmodding it");
+ sd0.setExecutable(false);
+ LOG.info("Restoring accessibility of first storage dir");
+ sd0.setExecutable(true);
+ LOG.info("nothing should have been purged in first storage dir");
+ LOG.info("fsimage_2 should be purged in second storage dir");
+ getImageFileName(4), getImageFileName(6));
+ getFinalizedEditsFileName(5, 6),
+ getInProgressEditsFileName(7));
+ LOG.info("On next save, we should purge logs from the failed dir," +
+ " but not images, since the image directory is in failed state.");
+ getImageFileName(6), getImageFileName(8));
+ getFinalizedEditsFileName(7, 8),
+ getInProgressEditsFileName(9));
+ private static void doSaveNamespace(NameNode nn) throws IOException {
+ LOG.info("Saving namespace...");
+ nn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+ nn.saveNamespace();
+ nn.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
@@ -0,0 +1,307 @@
+import org.junit.Assert;
+import org.mockito.ArgumentCaptor;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+public class TestNNStorageRetentionManager {
+ * Test the "easy case" where we have more images in the
+ * directory than we need to keep. Should purge the
+ * old ones.
+ public void testPurgeEasyCase() throws IOException {
+ TestCaseDescription tc = new TestCaseDescription();
+ tc.addRoot("/foo1", NameNodeDirType.IMAGE_AND_EDITS);
+ tc.addImage("/foo1/current/" + getImageFileName(100), true);
+ tc.addImage("/foo1/current/" + getImageFileName(200), true);
+ tc.addImage("/foo1/current/" + getImageFileName(300), false);
+ tc.addImage("/foo1/current/" + getImageFileName(400), false);
+ tc.addLog("/foo1/current/" + getFinalizedEditsFileName(101,200), true);
+ tc.addLog("/foo1/current/" + getFinalizedEditsFileName(201,300), true);
+ tc.addLog("/foo1/current/" + getFinalizedEditsFileName(301,400), false);
+ tc.addLog("/foo1/current/" + getInProgressEditsFileName(401), false);
+ // Test that other files don't get purged
+ tc.addLog("/foo1/current/VERSION", false);
+ runTest(tc);
+ * Same as above, but across multiple directories
+ public void testPurgeMultipleDirs() throws IOException {
+ tc.addRoot("/foo2", NameNodeDirType.IMAGE_AND_EDITS);
+ tc.addImage("/foo2/current/" + getImageFileName(200), true);
+ tc.addLog("/foo1/current/" + getFinalizedEditsFileName(101, 200), true);
+ tc.addLog("/foo1/current/" + getFinalizedEditsFileName(201, 300), true);
+ tc.addLog("/foo2/current/" + getFinalizedEditsFileName(201, 300), true);
+ tc.addLog("/foo1/current/" + getFinalizedEditsFileName(301, 400), false);
+ tc.addLog("/foo2/current/" + getFinalizedEditsFileName(301, 400), false);
+ * Test that if we have fewer fsimages than the configured
+ * retention, we don't purge any of them
+ public void testPurgeLessThanRetention() throws IOException {
+ tc.addImage("/foo1/current/" + getImageFileName(100), false);
+ tc.addLog("/foo1/current/" + getFinalizedEditsFileName(101,200), false);
+ tc.addLog("/foo1/current/" + getFinalizedEditsFileName(201,300), false);
+ * Check for edge case with no logs present at all.
+ public void testNoLogs() throws IOException {
+ * Check for edge case with no logs or images present at all.
+ public void testEmptyDir() throws IOException {
+ * Test that old in-progress logs are properly purged
+ public void testOldInProgress() throws IOException {
+ tc.addLog("/foo1/current/" + getInProgressEditsFileName(101), true);
+ public void testSeparateEditDirs() throws IOException {
+ tc.addRoot("/foo1", NameNodeDirType.IMAGE);
+ tc.addRoot("/foo2", NameNodeDirType.EDITS);
+ tc.addLog("/foo2/current/" + getFinalizedEditsFileName(101, 200), true);
+ tc.addLog("/foo2/current/" + getInProgressEditsFileName(401), false);
+ private void runTest(TestCaseDescription tc) throws IOException {
+ Configuration conf = new Configuration();
+ StoragePurger mockPurger =
+ Mockito.mock(NNStorageRetentionManager.StoragePurger.class);
+ ArgumentCaptor<FoundFSImage> imagesPurgedCaptor =
+ ArgumentCaptor.forClass(FoundFSImage.class);
+ ArgumentCaptor<FoundEditLog> logsPurgedCaptor =
+ ArgumentCaptor.forClass(FoundEditLog.class);
+ // Ask the manager to purge files we don't need any more
+ new NNStorageRetentionManager(conf,
+ tc.mockStorage(), tc.mockEditLog(), mockPurger)
+ .purgeOldStorage();
+ // Verify that it asked the purger to remove the correct files
+ Mockito.verify(mockPurger, Mockito.atLeast(0))
+ .purgeImage(imagesPurgedCaptor.capture());
+ .purgeLog(logsPurgedCaptor.capture());
+ // Check images
+ Set<String> purgedPaths = Sets.newHashSet();
+ for (FoundFSImage purged : imagesPurgedCaptor.getAllValues()) {
+ purgedPaths.add(purged.getFile().toString());
+ Assert.assertEquals(Joiner.on(",").join(tc.expectedPurgedImages),
+ Joiner.on(",").join(purgedPaths));
+ purgedPaths.clear();
+ for (FoundEditLog purged : logsPurgedCaptor.getAllValues()) {
+ Assert.assertEquals(Joiner.on(",").join(tc.expectedPurgedLogs),
+ private static class TestCaseDescription {
+ private Map<String, FakeRoot> dirRoots = Maps.newHashMap();
+ private Set<String> expectedPurgedLogs = Sets.newHashSet();
+ private Set<String> expectedPurgedImages = Sets.newHashSet();
+ private static class FakeRoot {
+ NameNodeDirType type;
+ List<String> files;
+ FakeRoot(NameNodeDirType type) {
+ this.type = type;
+ files = Lists.newArrayList();
+ StorageDirectory mockStorageDir() {
+ return TestFSImageStorageInspector.mockDirectory(
+ type, false,
+ files.toArray(new String[0]));
+ void addRoot(String root, NameNodeDirType dir) {
+ dirRoots.put(root, new FakeRoot(dir));
+ private void addFile(String path) {
+ for (Map.Entry<String, FakeRoot> entry : dirRoots.entrySet()) {
+ if (path.startsWith(entry.getKey())) {
+ entry.getValue().files.add(path);
+ void addLog(String path, boolean expectPurge) {
+ addFile(path);
+ if (expectPurge) {
+ expectedPurgedLogs.add(path);
+ void addImage(String path, boolean expectPurge) {
+ expectedPurgedImages.add(path);
+ NNStorage mockStorage() throws IOException {
+ List<StorageDirectory> sds = Lists.newArrayList();
+ for (FakeRoot root : dirRoots.values()) {
+ sds.add(root.mockStorageDir());
+ return mockStorageForDirs(sds.toArray(new StorageDirectory[0]));
+ public FSEditLog mockEditLog() {
+ final List<JournalManager> jms = Lists.newArrayList();
+ if (!root.type.isOfType(NameNodeDirType.EDITS)) continue;
+ FileJournalManager fjm = new FileJournalManager(
+ root.mockStorageDir());
+ jms.add(fjm);
+ FSEditLog mockLog = Mockito.mock(FSEditLog.class);
+ Mockito.doAnswer(new Answer<Void>() {
+ public Void answer(InvocationOnMock invocation) throws Throwable {
+ Object[] args = invocation.getArguments();
+ assert args.length == 2;
+ long txId = (Long) args[0];
+ StoragePurger purger = (StoragePurger) args[1];
+ for (JournalManager jm : jms) {
+ jm.purgeLogsOlderThan(txId, purger);
+ }).when(mockLog).purgeLogsOlderThan(
+ Mockito.anyLong(), (StoragePurger) Mockito.anyObject());
+ return mockLog;
+ private static NNStorage mockStorageForDirs(final StorageDirectory ... mockDirs)
+ NNStorage mockStorage = Mockito.mock(NNStorage.class);
+ FSImageStorageInspector inspector =
+ (FSImageStorageInspector) invocation.getArguments()[0];
+ for (StorageDirectory sd : mockDirs) {
+ }).when(mockStorage).inspectStorageDirs(
+ Mockito.<FSImageStorageInspector>anyObject());
+ return mockStorage;
@@ -28,6 +28,10 @@ import org.apache.hadoop.fs.FileSystem;
* This class tests various combinations of dfs.namenode.name.dir
@@ -46,13 +50,10 @@ public class TestNameEditsConfigs extends TestCase {
System.getProperty("test.build.data", "build/test/data"), "dfs/");
protected void setUp() throws java.lang.Exception {
- if(base_dir.exists())
- tearDown();
- protected void tearDown() throws java.lang.Exception {
- if (!FileUtil.fullyDelete(base_dir))
- throw new IOException("Cannot remove directory " + base_dir);
+ if(base_dir.exists()) {
+ if (!FileUtil.fullyDelete(base_dir))
+ throw new IOException("Cannot remove directory " + base_dir);
private void writeFile(FileSystem fileSys, Path name, int repl)
@@ -68,10 +69,22 @@ public class TestNameEditsConfigs extends TestCase {
void checkImageAndEditsFilesExistence(File dir,
- boolean imageMustExist,
- boolean editsMustExist) {
- assertTrue(imageMustExist == new File(dir, FILE_IMAGE).exists());
- assertTrue(editsMustExist == new File(dir, FILE_EDITS).exists());
+ boolean shouldHaveImages,
+ boolean shouldHaveEdits)
+ FSImageTransactionalStorageInspector ins = inspect(dir);
+ if (shouldHaveImages) {
+ assertTrue("Expect images in " + dir, ins.foundImages.size() > 0);
+ assertTrue("Expect no images in " + dir, ins.foundImages.isEmpty());
+ if (shouldHaveEdits) {
+ assertTrue("Expect edits in " + dir, ins.foundEditLogs.size() > 0);
+ assertTrue("Expect no edits in " + dir, ins.foundEditLogs.isEmpty());
private void checkFile(FileSystem fileSys, Path name, int repl)
@@ -110,9 +123,10 @@ public class TestNameEditsConfigs extends TestCase {
* do not read any stale image or edits.
* All along the test, we create and delete files at reach restart to make
* sure we are reading proper edits and image.
- public void testNameEditsConfigs() throws IOException {
+ public void testNameEditsConfigs() throws Exception {
Path file1 = new Path("TestNameEditsConfigs1");
Path file2 = new Path("TestNameEditsConfigs2");
Path file3 = new Path("TestNameEditsConfigs3");
@@ -120,12 +134,26 @@ public class TestNameEditsConfigs extends TestCase {
SecondaryNameNode secondary = null;
Configuration conf = null;
- File newNameDir = new File(base_dir, "name");
- File newEditsDir = new File(base_dir, "edits");
- File nameAndEdits = new File(base_dir, "name_and_edits");
- File checkpointNameDir = new File(base_dir, "secondname");
- File checkpointEditsDir = new File(base_dir, "secondedits");
- File checkpointNameAndEdits = new File(base_dir, "second_name_and_edits");
+ final File newNameDir = new File(base_dir, "name");
+ final File newEditsDir = new File(base_dir, "edits");
+ final File nameAndEdits = new File(base_dir, "name_and_edits");
+ final File checkpointNameDir = new File(base_dir, "secondname");
+ final File checkpointEditsDir = new File(base_dir, "secondedits");
+ final File checkpointNameAndEdits = new File(base_dir, "second_name_and_edits");
+ ImmutableList<File> allCurrentDirs = ImmutableList.of(
+ new File(nameAndEdits, "current"),
+ new File(newNameDir, "current"),
+ new File(newEditsDir, "current"),
+ new File(checkpointNameAndEdits, "current"),
+ new File(checkpointNameDir, "current"),
+ new File(checkpointEditsDir, "current"));
+ ImmutableList<File> imageCurrentDirs = ImmutableList.of(
+ new File(checkpointNameDir, "current"));
// Start namenode with same dfs.namenode.name.dir and dfs.namenode.edits.dir
@@ -191,23 +219,12 @@ public class TestNameEditsConfigs extends TestCase {
secondary.shutdown();
- checkImageAndEditsFilesExistence(nameAndEdits, true, true);
- checkImageAndEditsFilesExistence(newNameDir, true, false);
- checkImageAndEditsFilesExistence(newEditsDir, false, true);
- checkImageAndEditsFilesExistence(checkpointNameAndEdits, true, true);
- checkImageAndEditsFilesExistence(checkpointNameDir, true, false);
- checkImageAndEditsFilesExistence(checkpointEditsDir, false, true);
+ FSImageTestUtil.assertParallelFilesAreIdentical(allCurrentDirs,
+ ImmutableSet.of("VERSION"));
+ FSImageTestUtil.assertSameNewestImage(imageCurrentDirs);
// Now remove common directory both have and start namenode with
// separate name and edits dirs
- new File(nameAndEdits, FILE_EDITS).renameTo(
- new File(newNameDir, FILE_EDITS));
- new File(nameAndEdits, FILE_IMAGE).renameTo(
- new File(newEditsDir, FILE_IMAGE));
- new File(checkpointNameAndEdits, FILE_EDITS).renameTo(
- new File(checkpointNameDir, FILE_EDITS));
- new File(checkpointNameAndEdits, FILE_IMAGE).renameTo(
- new File(checkpointEditsDir, FILE_IMAGE));
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, newNameDir.getPath());
conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, newEditsDir.getPath());
@@ -237,7 +254,8 @@ public class TestNameEditsConfigs extends TestCase {
+ // No edit logs in new name dir
checkImageAndEditsFilesExistence(newNameDir, true, false);
checkImageAndEditsFilesExistence(newEditsDir, false, true);
checkImageAndEditsFilesExistence(checkpointNameDir, true, false);
@@ -281,12 +299,18 @@ public class TestNameEditsConfigs extends TestCase {
checkImageAndEditsFilesExistence(checkpointNameAndEdits, true, true);
+ private FSImageTransactionalStorageInspector inspect(File storageDir)
+ return FSImageTestUtil.inspectStorageDirectory(
+ new File(storageDir, "current"), NameNodeDirType.IMAGE_AND_EDITS);
* Test various configuration options of dfs.namenode.name.dir and dfs.namenode.edits.dir
* This test tries to simulate failure scenarios.
* 1. Start cluster with shared name and edits dir
* 2. Restart cluster by adding separate name and edits dirs
- * 3. Restart cluster by removing shared name and edits dir
+ * T3. Restart cluster by removing shared name and edits dir
* 4. Restart cluster with old shared name and edits dir, but only latest
* name dir. This should fail since we dont have latest edits dir
* 5. Restart cluster with old shared name and edits dir, but only latest
@@ -314,6 +338,10 @@ public class TestNameEditsConfigs extends TestCase {
.manageNameDfsDirs(false)
cluster.waitActive();
+ // Check that the dir has a VERSION file
+ assertTrue(new File(nameAndEdits, "current/VERSION").exists());
@@ -342,6 +370,12 @@ public class TestNameEditsConfigs extends TestCase {
+ // Check that the dirs have a VERSION file
+ assertTrue(new File(newNameDir, "current/VERSION").exists());
+ assertTrue(new File(newEditsDir, "current/VERSION").exists());
@@ -380,7 +414,7 @@ public class TestNameEditsConfigs extends TestCase {
fileSys.close();
// Add old shared directory for name and edits along with latest name
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, newNameDir.getPath() + "," +
@@ -401,7 +435,9 @@ public class TestNameEditsConfigs extends TestCase {
cluster = null;
- // Add old shared directory for name and edits along with latest edits
+ // Add old shared directory for name and edits along with latest edits.
+ // This is OK, since the latest edits will have segments leading all
+ // the way from the image in name_and_edits.
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameAndEdits.getPath());
conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, newEditsDir.getPath() +
@@ -413,11 +449,17 @@ public class TestNameEditsConfigs extends TestCase {
.format(false)
- assertTrue(false);
+ assertTrue(!fileSys.exists(file1));
+ checkFile(fileSys, file2, replication);
+ cleanupFile(fileSys, file2);
+ writeFile(fileSys, file3, replication);
+ checkFile(fileSys, file3, replication);
} catch (IOException e) { // expect to fail
System.out.println("cluster start failed due to missing latest name dir");
- cluster = null;
+ fileSys.close();
@@ -30,16 +30,12 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.util.PureJavaCrc32;
* A JUnit test for checking if restarting DFS preserves integrity.
@@ -84,6 +80,10 @@ public class TestParallelImageWrite extends TestCase {
+ // Force the NN to save its images on startup so long as
+ // there are any uncheckpointed txns
// Here we restart the MiniDFScluster without formatting namenode
cluster = new MiniDFSCluster.Builder(conf).format(false)
.numDataNodes(NUM_DATANODES).build();
@@ -102,16 +102,17 @@ public class TestParallelImageWrite extends TestCase {
assertEquals(dirstatus.getGroup() + "_XXX", newdirstatus.getGroup());
rootmtime = fs.getFileStatus(rootpath).getModificationTime();
- final long checkAfterRestart = checkImages(fsn, numNamenodeDirs);
+ final String checkAfterRestart = checkImages(fsn, numNamenodeDirs);
// Modify the system and then perform saveNamespace
files.cleanup(fs, dir);
files.createFiles(fs, dir);
fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
cluster.getNameNode().saveNamespace();
- final long checkAfterModify = checkImages(fsn, numNamenodeDirs);
- assertTrue("Modified namespace doesn't change fsimage contents",
- checkAfterRestart != checkAfterModify);
+ final String checkAfterModify = checkImages(fsn, numNamenodeDirs);
+ assertFalse("Modified namespace should change fsimage contents. " +
+ "was: " + checkAfterRestart + " now: " + checkAfterModify,
+ checkAfterRestart.equals(checkAfterModify));
fsn.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
@@ -124,42 +125,35 @@ public class TestParallelImageWrite extends TestCase {
* and non-empty, and there are the expected number of them.
* @param fsn - the FSNamesystem being checked.
* @param numImageDirs - the configured number of StorageDirectory of type IMAGE.
- * @return - the checksum of the FSImage files, which must all be the same.
+ * @return - the md5 hash of the most recent FSImage files, which must all be the same.
* @throws AssertionFailedError if image files are empty or different,
* if less than two StorageDirectory are provided, or if the
* actual number of StorageDirectory is less than configured.
- public static long checkImages(FSNamesystem fsn, int numImageDirs) throws Exception {
+ public static String checkImages(
+ FSNamesystem fsn, int numImageDirs)
NNStorage stg = fsn.getFSImage().getStorage();
//any failed StorageDirectory is removed from the storageDirs list
assertEquals("Some StorageDirectories failed Upgrade",
numImageDirs, stg.getNumStorageDirs(NameNodeDirType.IMAGE));
assertTrue("Not enough fsimage copies in MiniDFSCluster " +
"to test parallel write", numImageDirs > 1);
- //checksum the FSImage stored in each storageDir
- Iterator<StorageDirectory> iter = stg.dirIterator(NameNodeDirType.IMAGE);
- List<Long> checksums = new ArrayList<Long>();
- while (iter.hasNext()) {
- StorageDirectory sd = iter.next();
- File fsImage = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE);
- PureJavaCrc32 crc = new PureJavaCrc32();
- FileInputStream in = new FileInputStream(fsImage);
- byte[] buff = new byte[4096];
- int read = 0;
- while ((read = in.read(buff)) != -1) {
- crc.update(buff, 0, read);
- long val = crc.getValue();
- checksums.add(val);
- assertEquals(numImageDirs, checksums.size());
- long emptyCrc = crc.getValue();
- assertTrue("Empty fsimage file", checksums.get(0) != emptyCrc);
- for (int i = 1; i < numImageDirs; i++) {
- assertEquals(checksums.get(i - 1), checksums.get(i));
- return checksums.get(0);
+ // List of "current/" directory from each SD
+ List<File> dirs = FSImageTestUtil.getCurrentDirs(stg, NameNodeDirType.IMAGE);
+ // across directories, all files with same names should be identical hashes
+ FSImageTestUtil.assertSameNewestImage(dirs);
+ // Return the hash of the newest image file
+ StorageDirectory firstSd = stg.dirIterator(NameNodeDirType.IMAGE).next();
+ File latestImage = FSImageTestUtil.findLatestImageFile(firstSd);
+ String md5 = FSImageTestUtil.getImageFileMD5IgnoringTxId(latestImage);
+ System.err.println("md5 of " + latestImage + ": " + md5);
+ return md5;
@@ -18,10 +18,11 @@
import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.anyLong;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.spy;
@@ -31,6 +32,7 @@ import java.io.IOException;
@@ -42,7 +44,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.mockito.invocation.InvocationOnMock;
@@ -60,46 +62,47 @@ import org.mockito.stubbing.Answer;
public class TestSaveNamespace {
private static final Log LOG = LogFactory.getLog(TestSaveNamespace.class);
private static class FaultySaveImage implements Answer<Void> {
int count = 0;
- boolean exceptionType = true;
- // generate a RuntimeException
- public FaultySaveImage() {
- this.exceptionType = true;
+ boolean throwRTE = true;
// generate either a RuntimeException or IOException
- public FaultySaveImage(boolean etype) {
- this.exceptionType = etype;
+ public FaultySaveImage(boolean throwRTE) {
+ this.throwRTE = throwRTE;
public Void answer(InvocationOnMock invocation) throws Throwable {
Object[] args = invocation.getArguments();
- File f = (File)args[0];
+ StorageDirectory sd = (StorageDirectory)args[0];
if (count++ == 1) {
- LOG.info("Injecting fault for file: " + f);
- if (exceptionType) {
+ LOG.info("Injecting fault for sd: " + sd);
+ if (throwRTE) {
throw new RuntimeException("Injected fault: saveFSImage second time");
throw new IOException("Injected fault: saveFSImage second time");
- LOG.info("Not injecting fault for file: " + f);
+ LOG.info("Not injecting fault for sd: " + sd);
return (Void)invocation.callRealMethod();
private enum Fault {
- SAVE_FSIMAGE,
- MOVE_CURRENT,
- MOVE_LAST_CHECKPOINT
+ SAVE_SECOND_FSIMAGE_RTE,
+ SAVE_SECOND_FSIMAGE_IOE,
+ SAVE_ALL_FSIMAGES,
+ WRITE_STORAGE_ALL,
+ WRITE_STORAGE_ONE
- private void saveNamespaceWithInjectedFault(Fault fault) throws IOException {
+ private void saveNamespaceWithInjectedFault(Fault fault) throws Exception {
Configuration conf = getConf();
NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
DFSTestUtil.formatNameNode(conf);
@@ -108,46 +111,71 @@ public class TestSaveNamespace {
// Replace the FSImage with a spy
FSImage originalImage = fsn.dir.fsImage;
NNStorage storage = originalImage.getStorage();
- storage.close(); // unlock any directories that FSNamesystem's initialization may have locked
NNStorage spyStorage = spy(storage);
originalImage.storage = spyStorage;
FSImage spyImage = spy(originalImage);
fsn.dir.fsImage = spyImage;
- spyImage.getStorage().setStorageDirectories(FSNamesystem.getNamespaceDirs(conf),
+ boolean shouldFail = false; // should we expect the save operation to fail
// inject fault
switch(fault) {
- case SAVE_FSIMAGE:
+ case SAVE_SECOND_FSIMAGE_RTE:
// The spy throws a RuntimeException when writing to the second directory
- doAnswer(new FaultySaveImage()).
- when(spyImage).saveFSImage((File)anyObject());
+ doAnswer(new FaultySaveImage(true)).
+ when(spyImage).saveFSImage((StorageDirectory)anyObject(), anyLong());
+ shouldFail = false;
+ case SAVE_SECOND_FSIMAGE_IOE:
+ // The spy throws an IOException when writing to the second directory
+ doAnswer(new FaultySaveImage(false)).
+ case SAVE_ALL_FSIMAGES:
+ // The spy throws IOException in all directories
+ doThrow(new RuntimeException("Injected")).
+ shouldFail = true;
- case MOVE_CURRENT:
- // The spy throws a RuntimeException when calling moveCurrent()
- doThrow(new RuntimeException("Injected fault: moveCurrent")).
- when(spyStorage).moveCurrent((StorageDirectory)anyObject());
+ case WRITE_STORAGE_ALL:
+ // The spy throws an exception before writing any VERSION files
+ doThrow(new RuntimeException("Injected"))
+ .when(spyStorage).writeAll();
- case MOVE_LAST_CHECKPOINT:
- // The spy throws a RuntimeException when calling moveLastCheckpoint()
- doThrow(new RuntimeException("Injected fault: moveLastCheckpoint")).
- when(spyStorage).moveLastCheckpoint((StorageDirectory)anyObject());
+ case WRITE_STORAGE_ONE:
+ // The spy throws on exception on one particular storage directory
+ doAnswer(new FaultySaveImage(true))
+ .when(spyStorage).writeProperties((StorageDirectory)anyObject());
+ // TODO: unfortunately this fails -- should be improved.
+ // See HDFS-2173.
doAnEdit(fsn, 1);
- // Save namespace - this will fail because we inject a fault.
+ // Save namespace - this may fail, depending on fault injected
fsn.saveNamespace();
+ if (shouldFail) {
+ fail("Did not fail!");
} catch (Exception e) {
- LOG.info("Test caught expected exception", e);
+ if (! shouldFail) {
+ throw e;
+ LOG.info("Test caught expected exception", e);
+ fsn.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+ // Should still be able to perform edits
+ doAnEdit(fsn, 2);
// Now shut down and restart the namesystem
originalImage.close();
@@ -158,8 +186,9 @@ public class TestSaveNamespace {
// the namespace from the previous incarnation.
fsn = new FSNamesystem(conf);
- // Make sure the image loaded including our edit.
+ // Make sure the image loaded including our edits.
checkEditExists(fsn, 1);
+ checkEditExists(fsn, 2);
if (fsn != null) {
fsn.close();
@@ -185,35 +214,33 @@ public class TestSaveNamespace {
- // inject fault
- // The spy throws a IOException when writing to the second directory
- doAnswer(new FaultySaveImage(false)).
+ File rootDir = storage.getStorageDir(0).getRoot();
+ rootDir.setExecutable(false);
+ rootDir.setWritable(false);
+ rootDir.setReadable(false);
- // Save namespace - this injects a fault and marks one
- // directory as faulty.
+ // Save namespace - should mark the first storage dir as faulty
+ // since it's not traversable.
LOG.info("Doing the first savenamespace.");
- LOG.warn("First savenamespace sucessful.");
+ LOG.info("First savenamespace sucessful.");
assertTrue("Savenamespace should have marked one directory as bad." +
- " But found " + spyStorage.getRemovedStorageDirs().size() +
+ " But found " + storage.getRemovedStorageDirs().size() +
" bad directories.",
- spyStorage.getRemovedStorageDirs().size() == 1);
+ storage.getRemovedStorageDirs().size() == 1);
+ rootDir.setExecutable(true);
+ rootDir.setWritable(true);
+ rootDir.setReadable(true);
// The next call to savenamespace should try inserting the
// erroneous directory back to fs.name.dir. This command should
@@ -243,33 +270,53 @@ public class TestSaveNamespace {
LOG.info("Reloaded image is good.");
+ if (rootDir.exists()) {
+ fsn.close();
+ LOG.fatal("Failed to shut down", t);
- public void testCrashWhileSavingSecondImage() throws Exception {
- saveNamespaceWithInjectedFault(Fault.SAVE_FSIMAGE);
+ public void testRTEWhileSavingSecondImage() throws Exception {
+ saveNamespaceWithInjectedFault(Fault.SAVE_SECOND_FSIMAGE_RTE);
- public void testCrashWhileMoveCurrent() throws Exception {
- saveNamespaceWithInjectedFault(Fault.MOVE_CURRENT);
+ public void testIOEWhileSavingSecondImage() throws Exception {
+ saveNamespaceWithInjectedFault(Fault.SAVE_SECOND_FSIMAGE_IOE);
- public void testCrashWhileMoveLastCheckpoint() throws Exception {
- saveNamespaceWithInjectedFault(Fault.MOVE_LAST_CHECKPOINT);
+ public void testCrashInAllImageDirs() throws Exception {
+ saveNamespaceWithInjectedFault(Fault.SAVE_ALL_FSIMAGES);
+ public void testCrashWhenWritingVersionFiles() throws Exception {
+ saveNamespaceWithInjectedFault(Fault.WRITE_STORAGE_ALL);
+ public void testCrashWhenWritingVersionFileInOneDir() throws Exception {
+ saveNamespaceWithInjectedFault(Fault.WRITE_STORAGE_ONE);
* Test case where savenamespace fails in all directories
* and then the NN shuts down. Here we should recover from the
- * failed checkpoint by moving the directories back on next
- * NN start. This is a regression test for HDFS-1921.
+ * failed checkpoint since it only affected ".ckpt" files, not
+ * valid image files
public void testFailedSaveNamespace() throws Exception {
@@ -313,7 +360,8 @@ public class TestSaveNamespace {
FSNamesystem.getNamespaceEditsDirs(conf));
doThrow(new IOException("Injected fault: saveFSImage")).
+ when(spyImage).saveFSImage((StorageDirectory)anyObject(),
+ Mockito.anyLong());
@@ -360,16 +408,6 @@ public class TestSaveNamespace {
FSNamesystem fsn = new FSNamesystem(conf);
- // Replace the FSImage with a spy
- final FSImage originalImage = fsn.dir.fsImage;
- originalImage.getStorage().close();
- spyImage.getStorage().setStorageDirectories(
CheckpointSignature sig = fsn.rollEditLog();
@@ -382,7 +420,6 @@ public class TestSaveNamespace {
// Now shut down and restart the NN
fsn = null;
@@ -399,7 +436,45 @@ public class TestSaveNamespace {
+ public void testTxIdPersistence() throws Exception {
+ Configuration conf = getConf();
+ NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
+ DFSTestUtil.formatNameNode(conf);
+ FSNamesystem fsn = new FSNamesystem(conf);
+ // We have a BEGIN_LOG_SEGMENT txn to start
+ assertEquals(1, fsn.getEditLog().getLastWrittenTxId());
+ doAnEdit(fsn, 1);
+ assertEquals(2, fsn.getEditLog().getLastWrittenTxId());
+ fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+ fsn.saveNamespace();
+ // 2 more txns: END the first segment, BEGIN a new one
+ assertEquals(4, fsn.getEditLog().getLastWrittenTxId());
+ // Shut down and restart
+ fsn.getFSImage().close();
+ // 1 more txn to END that segment
+ assertEquals(5, fsn.getEditLog().getLastWrittenTxId());
+ fsn = null;
+ fsn = new FSNamesystem(conf);
+ // 1 more txn to start new segment on restart
+ assertEquals(6, fsn.getEditLog().getLastWrittenTxId());
+ if (fsn != null) {
private void doAnEdit(FSNamesystem fsn, int id) throws IOException {
// Make an edit
fsn.mkdirs(
@@ -106,9 +106,7 @@ public class TestSecurityTokenEditLog extends TestCase {
+ editLog.setOutputBufferCapacity(2048);
namesystem.getDelegationTokenSecretManager().startThreads();
@@ -129,25 +127,24 @@ public class TestSecurityTokenEditLog extends TestCase {
namesystem.getDelegationTokenSecretManager().stopThreads();
int numKeys = namesystem.getDelegationTokenSecretManager().getNumberOfKeys();
+ int expectedTransactions = NUM_THREADS * opsPerTrans * NUM_TRANSACTIONS + numKeys
+ + 2; // + 2 for BEGIN and END txns
+ for (StorageDirectory sd : fsimage.getStorage().dirIterable(NameNodeDirType.EDITS)) {
+ File editFile = NNStorage.getFinalizedEditsFile(sd, 1, 1 + expectedTransactions - 1);
- assertTrue("Verification for " + editFile + " failed. " +
- "Expected " + (NUM_THREADS * opsPerTrans * NUM_TRANSACTIONS + numKeys) + " transactions. "+
- "Found " + numEdits + " transactions.",
- numEdits == NUM_THREADS * opsPerTrans * NUM_TRANSACTIONS +numKeys);
+ new EditLogFileInputStream(editFile), 1);
+ assertEquals("Verification for " + editFile, expectedTransactions, numEdits);
@@ -21,16 +21,12 @@ import static org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption.I
-import java.io.RandomAccessFile;
import java.net.InetAddress;
-import java.util.Properties;
@@ -45,7 +41,6 @@ import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSClient;
@@ -56,7 +51,7 @@ import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.junit.Assert;
@@ -237,13 +232,11 @@ public class TestStartup extends TestCase {
if(sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
- img.getStorage();
- File imf = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE);
+ File imf = img.getStorage().getStorageFile(sd, NameNodeFile.IMAGE, 0);
LOG.info("--image file " + imf.getAbsolutePath() + "; len = " + imf.length() + "; expected = " + expectedImgSize);
assertEquals(expectedImgSize, imf.length());
} else if(sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
- File edf = NNStorage.getStorageFile(sd, NameNodeFile.EDITS);
+ File edf = img.getStorage().getStorageFile(sd, NameNodeFile.EDITS, 0);
LOG.info("-- edits file " + edf.getAbsolutePath() + "; len = " + edf.length() + "; expected = " + expectedEditsSize);
assertEquals(expectedEditsSize, edf.length());
@@ -348,8 +341,8 @@ public class TestStartup extends TestCase {
FSImage image = nn.getFSImage();
StorageDirectory sd = image.getStorage().getStorageDir(0); //only one
assertEquals(sd.getStorageDirType(), NameNodeDirType.IMAGE_AND_EDITS);
- File imf = image.getStorage().getStorageFile(sd, NameNodeFile.IMAGE);
- File edf = image.getStorage().getStorageFile(sd, NameNodeFile.EDITS);
+ File imf = image.getStorage().getStorageFile(sd, NameNodeFile.IMAGE, 0);
+ File edf = image.getStorage().getStorageFile(sd, NameNodeFile.EDITS, 0);
LOG.info("--image file " + imf.getAbsolutePath() + "; len = " + imf.length());
LOG.info("--edits file " + edf.getAbsolutePath() + "; len = " + edf.length());
@@ -430,70 +423,57 @@ public class TestStartup extends TestCase {
private void testImageChecksum(boolean compress) throws Exception {
- Configuration conf = new Configuration();
- FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
- File base_dir = new File(
- System.getProperty("test.build.data", "build/test/data"), "dfs/");
- conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
- new File(base_dir, "name").getPath());
- conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, false);
if (compress) {
conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY, true);
- DFSTestUtil.formatNameNode(conf);
- // create an image
- LOG.info("Create an fsimage");
- NameNode namenode = new NameNode(conf);
- namenode.getNamesystem().mkdirs("/test",
- new PermissionStatus("hairong", null, FsPermission.getDefault()), true);
- assertTrue(namenode.getFileInfo("/test").isDir());
- namenode.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
- namenode.saveNamespace();
- FSImage image = namenode.getFSImage();
- image.loadFSImage();
- File versionFile = image.getStorage().getStorageDir(0).getVersionFile();
- RandomAccessFile file = new RandomAccessFile(versionFile, "rws");
- FileInputStream in = null;
- FileOutputStream out = null;
- // read the property from version file
- in = new FileInputStream(file.getFD());
- file.seek(0);
- Properties props = new Properties();
- props.load(in);
- // get the MD5 property and change it
- String sMd5 = props.getProperty(NNStorage.MESSAGE_DIGEST_PROPERTY);
- MD5Hash md5 = new MD5Hash(sMd5);
- byte[] bytes = md5.getDigest();
- bytes[0] += 1;
- md5 = new MD5Hash(bytes);
- props.setProperty(NNStorage.MESSAGE_DIGEST_PROPERTY, md5.toString());
- // write the properties back to version file
- out = new FileOutputStream(file.getFD());
- props.store(out, null);
- file.setLength(out.getChannel().position());
- // now load the image again
- fail("Expect to get a checksumerror");
- assertTrue(e.getMessage().contains("is corrupt"));
+ fs.mkdirs(new Path("/test"));
+ // test/data/dfs/nameN/current/{fsimage,edits,...}
+ // Corrupt the md5 file to all 0s
+ File imageFile = new File(nameDir, "current/" + NNStorage.getImageFileName(0));
+ MD5FileUtils.saveMD5File(imageFile, new MD5Hash(new byte[16]));
+ fail("Should not have successfully started with corrupt image");
+ if (!ioe.getCause().getMessage().contains("is corrupt with MD5")) {
- IOUtils.closeStream(in);
- IOUtils.closeStream(out);
- namenode.stop();
- namenode.join();
@@ -17,6 +17,8 @@
@@ -43,7 +45,9 @@ public class TestStartupOptionUpgrade {
startOpt = StartupOption.UPGRADE;
startOpt.setClusterId(null);
- storage = new NNStorage(conf);
+ storage = new NNStorage(conf,
@After
@@ -136,4 +140,4 @@ public class TestStartupOptionUpgrade {
Assert.assertEquals("Clusterid should match with the existing one",
"currentcid", storage.getClusterID());
@@ -22,38 +22,38 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
-import java.math.BigInteger;
-import java.security.MessageDigest;
-import java.util.Random;
import java.util.Set;
+import static org.mockito.Matchers.anyByte;
import org.apache.hadoop.cli.CLITestCmdDFS;
import org.apache.hadoop.cli.util.CLICommandDFSAdmin;
import org.apache.hadoop.cli.util.CommandExecutor;
-import org.apache.hadoop.fs.FSDataOutputStream;
import com.google.common.collect.ImmutableSet;
@@ -71,20 +71,7 @@ public class TestStorageRestore {
static final int blockSize = 4096;
static final int fileSize = 8192;
private File path1, path2, path3;
- private MiniDFSCluster cluster;
- private void writeFile(FileSystem fileSys, Path name, int repl)
- FSDataOutputStream stm = fileSys.create(name, true,
- fileSys.getConf().getInt("io.file.buffer.size", 4096),
- (short)repl, (long)blockSize);
- byte[] buffer = new byte[fileSize];
- Random rand = new Random(seed);
- rand.nextBytes(buffer);
- stm.write(buffer);
- stm.close();
public void setUpNameDirs() throws Exception {
config = new HdfsConfiguration();
@@ -119,19 +106,9 @@ public class TestStorageRestore {
// set the restore feature on
config.setBoolean(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_RESTORE_KEY, true);
- * clean up
- public void cleanUpNameDirs() throws Exception {
- if (hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
- throw new IOException("Could not delete hdfs directory in tearDown '" + hdfsDir + "'");
- * invalidate storage by removing storage directories
+ * invalidate storage by removing the second and third storage directories
public void invalidateStorage(FSImage fi, Set<File> filesToInvalidate) throws IOException {
ArrayList<StorageDirectory> al = new ArrayList<StorageDirectory>(2);
@@ -145,6 +122,19 @@ public class TestStorageRestore {
// simulate an error
fi.getStorage().reportErrorsOnDirectories(al);
+ for (FSEditLog.JournalAndStream j : fi.getEditLog().getJournals()) {
+ if (j.getManager() instanceof FileJournalManager) {
+ FileJournalManager fm = (FileJournalManager)j.getManager();
+ if (fm.getStorageDirectory().getRoot().equals(path2)
+ || fm.getStorageDirectory().getRoot().equals(path3)) {
+ EditLogOutputStream mockStream = spy(j.getCurrentStream());
+ j.setCurrentStreamForTests(mockStream);
+ doThrow(new IOException("Injected fault: write")).
+ when(mockStream).write(Mockito.<FSEditLogOp>anyObject());
@@ -154,130 +144,14 @@ public class TestStorageRestore {
LOG.info("current storages and corresponding sizes:");
for(Iterator<StorageDirectory> it = fs.getStorage().dirIterator(); it.hasNext(); ) {
- if(sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
- LOG.info(" image file " + imf.getAbsolutePath() + "; len = " + imf.length());
- if(sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
- LOG.info(" edits file " + edf.getAbsolutePath() + "; len = " + edf.length());
- * This function returns a md5 hash of a file.
- * @param file input file
- * @return The md5 string
- public String getFileMD5(File file) throws Exception {
- String res = new String();
- MessageDigest mD = MessageDigest.getInstance("MD5");
- DataInputStream dis = new DataInputStream(new FileInputStream(file));
- while(true) {
- mD.update(dis.readByte());
- } catch (EOFException eof) {}
- BigInteger bigInt = new BigInteger(1, mD.digest());
- res = bigInt.toString(16);
- dis.close();
- return res;
- * read currentCheckpointTime directly from the file
- * @param currDir
- * @return the checkpoint time
- long readCheckpointTime(File currDir) throws IOException {
- File timeFile = new File(currDir, NameNodeFile.TIME.getName());
+ for (File f : curDir.listFiles()) {
+ LOG.info(" file " + f.getAbsolutePath() + "; len = " + f.length());
- * check if files exist/not exist
- public void checkFiles(boolean valid) throws IOException {
- //look at the valid storage
- File fsImg1 = new File(path1, Storage.STORAGE_DIR_CURRENT + "/" + NameNodeFile.IMAGE.getName());
- File fsImg2 = new File(path2, Storage.STORAGE_DIR_CURRENT + "/" + NameNodeFile.IMAGE.getName());
- File fsImg3 = new File(path3, Storage.STORAGE_DIR_CURRENT + "/" + NameNodeFile.IMAGE.getName());
- File fsEdits1 = new File(path1, Storage.STORAGE_DIR_CURRENT + "/" + NameNodeFile.EDITS.getName());
- File fsEdits2 = new File(path2, Storage.STORAGE_DIR_CURRENT + "/" + NameNodeFile.EDITS.getName());
- File fsEdits3 = new File(path3, Storage.STORAGE_DIR_CURRENT + "/" + NameNodeFile.EDITS.getName());
- long chkPt1 = readCheckpointTime(new File(path1, Storage.STORAGE_DIR_CURRENT));
- long chkPt2 = readCheckpointTime(new File(path2, Storage.STORAGE_DIR_CURRENT));
- long chkPt3 = readCheckpointTime(new File(path3, Storage.STORAGE_DIR_CURRENT));
- String md5_1 = null,md5_2 = null,md5_3 = null;
- md5_1 = getFileMD5(fsEdits1);
- md5_2 = getFileMD5(fsEdits2);
- md5_3 = getFileMD5(fsEdits3);
- } catch (Exception e) {
- System.err.println("md 5 calculation failed:" + e.getLocalizedMessage());
- this.printStorages(cluster.getNameNode().getFSImage());
- LOG.info("++++ image files = "+fsImg1.getAbsolutePath() + "," + fsImg2.getAbsolutePath() + ","+ fsImg3.getAbsolutePath());
- LOG.info("++++ edits files = "+fsEdits1.getAbsolutePath() + "," + fsEdits2.getAbsolutePath() + ","+ fsEdits3.getAbsolutePath());
- LOG.info("checkFiles compares lengths: img1=" + fsImg1.length() + ",img2=" + fsImg2.length() + ",img3=" + fsImg3.length());
- LOG.info("checkFiles compares lengths: edits1=" + fsEdits1.length() + ",edits2=" + fsEdits2.length() + ",edits3=" + fsEdits3.length());
- LOG.info("checkFiles compares chkPts: name1=" + chkPt1 + ",name2=" + chkPt2 + ",name3=" + chkPt3);
- LOG.info("checkFiles compares md5s: " + fsEdits1.getAbsolutePath() +
- "="+ md5_1 + "," + fsEdits2.getAbsolutePath() + "=" + md5_2 + "," +
- fsEdits3.getAbsolutePath() + "=" + md5_3);
- if(valid) {
- // should be the same
- assertTrue(fsImg1.length() == fsImg2.length());
- assertTrue(0 == fsImg3.length()); //shouldn't be created
- assertTrue(fsEdits1.length() == fsEdits2.length());
- assertTrue(fsEdits1.length() == fsEdits3.length());
- assertTrue(md5_1.equals(md5_2));
- assertTrue(md5_1.equals(md5_3));
- // checkpoint times
- assertTrue(chkPt1 == chkPt2);
- assertTrue(chkPt1 == chkPt3);
- // should be different
- //assertTrue(fsImg1.length() != fsImg2.length());
- //assertTrue(fsImg1.length() != fsImg3.length());
- assertTrue("edits1 = edits2", fsEdits1.length() != fsEdits2.length());
- assertTrue("edits1 = edits3", fsEdits1.length() != fsEdits3.length());
- assertTrue(!md5_1.equals(md5_2));
- assertTrue(!md5_1.equals(md5_3));
- assertTrue(chkPt1 > chkPt2);
- assertTrue(chkPt1 > chkPt3);
* test
* 1. create DFS cluster with 3 storage directories - 2 EDITS_IMAGE, 1 EDITS
@@ -293,7 +167,7 @@ public class TestStorageRestore {
public void testStorageRestore() throws Exception {
- int numDatanodes = 2;
+ int numDatanodes = 0;
cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes)
@@ -305,36 +179,88 @@ public class TestStorageRestore {
FileSystem fs = cluster.getFileSystem();
Path path = new Path("/", "test");
- writeFile(fs, path, 2);
+ assertTrue(fs.mkdirs(path));
- System.out.println("****testStorageRestore: file test written, invalidating storage...");
+ System.out.println("****testStorageRestore: dir 'test' created, invalidating storage...");
invalidateStorage(cluster.getNameNode().getFSImage(), ImmutableSet.of(path2, path3));
- //secondary.doCheckpoint(); // this will cause storages to be removed.
printStorages(cluster.getNameNode().getFSImage());
- System.out.println("****testStorageRestore: storage invalidated + doCheckpoint");
+ System.out.println("****testStorageRestore: storage invalidated");
path = new Path("/", "test1");
- System.out.println("****testStorageRestore: file test1 written");
- checkFiles(false); // SHOULD BE FALSE
+ System.out.println("****testStorageRestore: dir 'test1' created");
+ // We did another edit, so the still-active directory at 'path1'
+ // should now differ from the others
+ FSImageTestUtil.assertFileContentsDifferent(2,
+ new File(path1, "current/" + getInProgressEditsFileName(1)),
+ new File(path2, "current/" + getInProgressEditsFileName(1)),
+ new File(path3, "current/" + getInProgressEditsFileName(1)));
+ FSImageTestUtil.assertFileContentsSame(
System.out.println("****testStorageRestore: checkfiles(false) run");
secondary.doCheckpoint(); ///should enable storage..
- checkFiles(true);
- System.out.println("****testStorageRestore: second Checkpoint done and checkFiles(true) run");
+ // We should have a checkpoint through txid 4 in the two image dirs
+ // (txid=4 for BEGIN, mkdir, mkdir, END)
+ new File(path1, "current/" + getImageFileName(4)),
+ new File(path2, "current/" + getImageFileName(4)));
+ assertFalse("Should not have any image in an edits-only directory",
+ new File(path3, "current/" + getImageFileName(4)).exists());
+ // Should have finalized logs in the directory that didn't fail
+ assertTrue("Should have finalized logs in the directory that didn't fail",
+ new File(path1, "current/" + getFinalizedEditsFileName(1,4)).exists());
+ // Should not have finalized logs in the failed directories
+ assertFalse("Should not have finalized logs in the failed directories",
+ new File(path2, "current/" + getFinalizedEditsFileName(1,4)).exists());
+ new File(path3, "current/" + getFinalizedEditsFileName(1,4)).exists());
+ // The new log segment should be in all of the directories.
+ new File(path1, "current/" + getInProgressEditsFileName(5)),
+ new File(path2, "current/" + getInProgressEditsFileName(5)),
+ new File(path3, "current/" + getInProgressEditsFileName(5)));
+ String md5BeforeEdit = FSImageTestUtil.getFileMD5(
+ new File(path1, "current/" + getInProgressEditsFileName(5)));
- // verify that all the logs are active
- path = new Path("/", "test2");
- System.out.println("****testStorageRestore: wrote a file and checkFiles(true) run");
+ // The original image should still be the previously failed image
+ // directory after it got restored, since it's still useful for
+ // a recovery!
+ new File(path1, "current/" + getImageFileName(0)),
+ new File(path2, "current/" + getImageFileName(0)));
+ // Do another edit to verify that all the logs are active.
+ path = new Path("/", "test2");
+ // Logs should be changed by the edit.
+ String md5AfterEdit = FSImageTestUtil.getFileMD5(
+ assertFalse(md5BeforeEdit.equals(md5AfterEdit));
+ // And all logs should be changed.
+ // All logs should be finalized by clean shutdown
+ new File(path1, "current/" + getFinalizedEditsFileName(5,7)),
+ new File(path2, "current/" + getFinalizedEditsFileName(5,7)),
+ new File(path3, "current/" + getFinalizedEditsFileName(5,7)));
@@ -412,7 +338,7 @@ public class TestStorageRestore {
Path testPath = new Path("/", "test");
- writeFile(fs, testPath, 2);
+ assertTrue(fs.mkdirs(testPath));
printStorages(fsImage);
@@ -20,42 +20,80 @@ package org.apache.hadoop.hdfs.server.namenode;
-import java.io.FileNotFoundException;
public class TestTransferFsImage {
* Regression test for HDFS-1997. Test that, if an exception
- * occurs on the client side, it is properly reported as such
+ * occurs on the client side, it is properly reported as such,
+ * and reported to the associated NNStorage object.
public void testClientSideException() throws IOException {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(0).build();
+ List<File> localPath = Collections.<File>singletonList(
+ new File("/xxxxx-does-not-exist/blah"));
String fsName = NameNode.getHostPortString(
cluster.getNameNode().getHttpAddress());
- String id = "getimage=1";
+ String id = "getimage=1&txid=0";
- File[] localPath = new File[] {
- new File("/xxxxx-does-not-exist/blah")
- TransferFsImage.getFileClient(fsName, id, localPath, false);
+ TransferFsImage.getFileClient(fsName, id, localPath, mockStorage, false);
fail("Didn't get an exception!");
} catch (IOException ioe) {
- assertTrue("Expected FNFE, got: " + StringUtils.stringifyException(ioe),
- ioe instanceof FileNotFoundException);
+ Mockito.verify(mockStorage).reportErrorOnFile(localPath.get(0));
+ assertTrue(
+ "Unexpected exception: " + StringUtils.stringifyException(ioe),
+ ioe.getMessage().contains("Unable to download to any storage"));
+ * Similar to the above test, except that there are multiple local files
+ * and one of them can be saved.
+ public void testClientSideExceptionOnJustOneDir() throws IOException {
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+ List<File> localPaths = ImmutableList.of(
+ new File("/xxxxx-does-not-exist/blah"),
+ new File(TEST_DIR, "testfile")
+ );
+ String fsName = NameNode.getHostPortString(
+ cluster.getNameNode().getHttpAddress());
+ TransferFsImage.getFileClient(fsName, id, localPaths, mockStorage, false);
+ Mockito.verify(mockStorage).reportErrorOnFile(localPaths.get(0));
+ assertTrue("The valid local file should get saved properly",
+ localPaths.get(1).length() > 0);
@@ -20,8 +20,6 @@ package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
import java.util.Map;
import java.util.HashMap;
@@ -32,15 +30,11 @@ import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer;
import org.apache.hadoop.hdfs.tools.offlineEditsViewer.TokenizerFactory;
import org.apache.hadoop.hdfs.tools.offlineEditsViewer.EditsVisitorFactory;
-import org.apache.hadoop.hdfs.tools.offlineEditsViewer.EditsVisitor;
-import org.apache.hadoop.hdfs.tools.offlineEditsViewer.XmlEditsVisitor;
-import org.apache.hadoop.hdfs.tools.offlineEditsViewer.BinaryEditsVisitor;
import org.apache.hadoop.hdfs.server.namenode.OfflineEditsViewerHelper;
@@ -58,7 +52,7 @@ public class TestOfflineEditsViewer {
System.getProperty("test.build.data", "build/test/data");
private static String cacheDir =
- System.getProperty("test.cache.data", "build/test/data/cache");
+ System.getProperty("test.cache.data", "build/test/cache");
// to create edits and get edits filename
private static final OfflineEditsViewerHelper nnHelper
@@ -81,11 +75,13 @@ public class TestOfflineEditsViewer {
obsoleteOpCodes.put(FSEditLogOpCodes.OP_DATANODE_REMOVE, true);
obsoleteOpCodes.put(FSEditLogOpCodes.OP_SET_NS_QUOTA, true);
obsoleteOpCodes.put(FSEditLogOpCodes.OP_CLEAR_NS_QUOTA, true);
- // these are not written to files
- obsoleteOpCodes.put(FSEditLogOpCodes.OP_JSPOOL_START, true);
- obsoleteOpCodes.put(FSEditLogOpCodes.OP_CHECKPOINT_TIME, true);
+ public void setup() {
+ new File(cacheDir).mkdirs();
* Test the OfflineEditsViewer
@@ -1,428 +1,498 @@
<?xml version="1.0"?>
<EDITS>
- <EDITS_VERSION>-24</EDITS_VERSION>
+ <EDITS_VERSION>-38</EDITS_VERSION>
+ <RECORD>
+ <OPCODE>24</OPCODE>
+ <DATA>
+ <TRANSACTION_ID>1</TRANSACTION_ID>
+ </DATA>
+ <CHECKSUM>1504643968</CHECKSUM>
+ </RECORD>
<RECORD>
<OPCODE>21</OPCODE>
<DATA>
+ <TRANSACTION_ID>2</TRANSACTION_ID>
<KEY_ID>1</KEY_ID>
- <KEY_EXPIRY_DATE>1287183164658</KEY_EXPIRY_DATE>
+ <KEY_EXPIRY_DATE>1304751257518</KEY_EXPIRY_DATE>
<KEY_LENGTH>3</KEY_LENGTH>
- <KEY_BLOB>drEs</KEY_BLOB>
+ <KEY_BLOB>2FhO</KEY_BLOB>
</DATA>
+ <CHECKSUM>-174778556</CHECKSUM>
</RECORD>
+ <TRANSACTION_ID>3</TRANSACTION_ID>
<KEY_ID>2</KEY_ID>
- <KEY_EXPIRY_DATE>1287183164703</KEY_EXPIRY_DATE>
+ <KEY_EXPIRY_DATE>1304751257521</KEY_EXPIRY_DATE>
- <KEY_BLOB>1cGc</KEY_BLOB>
+ <KEY_BLOB>77-r</KEY_BLOB>
+ <CHECKSUM>1565957291</CHECKSUM>
<OPCODE>10</OPCODE>
+ <TRANSACTION_ID>4</TRANSACTION_ID>
<GENERATION_STAMP>1001</GENERATION_STAMP>
+ <CHECKSUM>1423210231</CHECKSUM>
<OPCODE>0</OPCODE>
+ <TRANSACTION_ID>5</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION>
- <MTIME>1286491964741</MTIME>
- <ATIME>1286491964741</ATIME>
+ <MTIME>1304060057562</MTIME>
+ <ATIME>1304060057562</ATIME>
<BLOCKSIZE>512</BLOCKSIZE>
<NUMBLOCKS>0</NUMBLOCKS>
<PERMISSION_STATUS>
- <USERNAME>steffl</USERNAME>
+ <USERNAME>todd</USERNAME>
<GROUPNAME>supergroup</GROUPNAME>
<FS_PERMISSIONS>420</FS_PERMISSIONS>
</PERMISSION_STATUS>
- <CLIENT_NAME>DFSClient_471171074</CLIENT_NAME>
+ <CLIENT_NAME>DFSClient_NONMAPREDUCE_-66857152_1</CLIENT_NAME>
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
+ <CHECKSUM>-1854451489</CHECKSUM>
<OPCODE>9</OPCODE>
+ <TRANSACTION_ID>6</TRANSACTION_ID>
- <MTIME>1286491964758</MTIME>
+ <MTIME>1304060057572</MTIME>
+ <CHECKSUM>617592855</CHECKSUM>
<OPCODE>1</OPCODE>
+ <TRANSACTION_ID>7</TRANSACTION_ID>
<LENGTH>3</LENGTH>
<SOURCE>/file_create</SOURCE>
<DESTINATION>/file_moved</DESTINATION>
- <TIMESTAMP>1286491964766</TIMESTAMP>
+ <TIMESTAMP>1304060057575</TIMESTAMP>
+ <CHECKSUM>367100554</CHECKSUM>
<OPCODE>2</OPCODE>
+ <TRANSACTION_ID>8</TRANSACTION_ID>
<LENGTH>2</LENGTH>
<PATH>/file_moved</PATH>
- <TIMESTAMP>1286491964775</TIMESTAMP>
+ <TIMESTAMP>1304060057577</TIMESTAMP>
+ <CHECKSUM>1048346698</CHECKSUM>
<OPCODE>3</OPCODE>
+ <TRANSACTION_ID>9</TRANSACTION_ID>
<PATH>/directory_mkdir</PATH>
- <TIMESTAMP>1286491964783</TIMESTAMP>
+ <TIMESTAMP>1304060057581</TIMESTAMP>
<ATIME>0</ATIME>
<FS_PERMISSIONS>493</FS_PERMISSIONS>
+ <CHECKSUM>1207240248</CHECKSUM>
+ <TRANSACTION_ID>10</TRANSACTION_ID>
<GENERATION_STAMP>1002</GENERATION_STAMP>
+ <CHECKSUM>85982431</CHECKSUM>
+ <TRANSACTION_ID>11</TRANSACTION_ID>
- <MTIME>1286491964796</MTIME>
- <ATIME>1286491964796</ATIME>
+ <MTIME>1304060057584</MTIME>
+ <ATIME>1304060057584</ATIME>
+ <CHECKSUM>1796314473</CHECKSUM>
+ <TRANSACTION_ID>12</TRANSACTION_ID>
- <MTIME>1286491964814</MTIME>
+ <MTIME>1304060057588</MTIME>
+ <CHECKSUM>1017626905</CHECKSUM>
<OPCODE>4</OPCODE>
+ <TRANSACTION_ID>13</TRANSACTION_ID>
+ <CHECKSUM>1842610087</CHECKSUM>
<OPCODE>7</OPCODE>
+ <TRANSACTION_ID>14</TRANSACTION_ID>
<FS_PERMISSIONS>511</FS_PERMISSIONS>
+ <CHECKSUM>605568911</CHECKSUM>
<OPCODE>8</OPCODE>
+ <TRANSACTION_ID>15</TRANSACTION_ID>
<USERNAME>newOwner</USERNAME>
<GROUPNAME/>
+ <CHECKSUM>-1411790340</CHECKSUM>
<OPCODE>13</OPCODE>
+ <TRANSACTION_ID>16</TRANSACTION_ID>
<MTIME>1285195527000</MTIME>
<ATIME>1285195527000</ATIME>
+ <CHECKSUM>1428793678</CHECKSUM>
<OPCODE>14</OPCODE>
+ <TRANSACTION_ID>17</TRANSACTION_ID>
<NS_QUOTA>1000</NS_QUOTA>
<DS_QUOTA>-1</DS_QUOTA>
+ <CHECKSUM>-1476130374</CHECKSUM>
<OPCODE>15</OPCODE>
+ <TRANSACTION_ID>18</TRANSACTION_ID>
- <TIMESTAMP>1286491964858</TIMESTAMP>
+ <TIMESTAMP>1304060057605</TIMESTAMP>
<RENAME_OPTIONS>AA</RENAME_OPTIONS>
+ <CHECKSUM>-1155144192</CHECKSUM>
+ <TRANSACTION_ID>19</TRANSACTION_ID>
<GENERATION_STAMP>1003</GENERATION_STAMP>
+ <CHECKSUM>1920677987</CHECKSUM>
+ <TRANSACTION_ID>20</TRANSACTION_ID>
<PATH>/file_concat_target</PATH>
- <MTIME>1286491964873</MTIME>
- <ATIME>1286491964873</ATIME>
+ <MTIME>1304060057613</MTIME>
+ <ATIME>1304060057613</ATIME>
+ <CHECKSUM>-428545606</CHECKSUM>
+ <TRANSACTION_ID>21</TRANSACTION_ID>
- <MTIME>1286491965024</MTIME>
+ <MTIME>1304060057694</MTIME>
<NUMBLOCKS>3</NUMBLOCKS>
<BLOCK>
- <BLOCK_ID>1096087107607101866</BLOCK_ID>
+ <BLOCK_ID>3459038074990663911</BLOCK_ID>
<BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
<BLOCK_GENERATION_STAMP>1003</BLOCK_GENERATION_STAMP>
</BLOCK>
- <BLOCK_ID>8798023959648425597</BLOCK_ID>
+ <BLOCK_ID>-5555244278278879146</BLOCK_ID>
- <BLOCK_ID>4060815343079109399</BLOCK_ID>
+ <BLOCK_ID>-6344128791846831740</BLOCK_ID>
+ <CHECKSUM>707995174</CHECKSUM>
+ <TRANSACTION_ID>22</TRANSACTION_ID>
<GENERATION_STAMP>1004</GENERATION_STAMP>
+ <CHECKSUM>-1500977009</CHECKSUM>
+ <TRANSACTION_ID>23</TRANSACTION_ID>
<PATH>/file_concat_0</PATH>
- <MTIME>1286491965035</MTIME>
- <ATIME>1286491965035</ATIME>
+ <MTIME>1304060057701</MTIME>
+ <ATIME>1304060057701</ATIME>
+ <CHECKSUM>-119850856</CHECKSUM>
+ <TRANSACTION_ID>24</TRANSACTION_ID>
- <MTIME>1286491965093</MTIME>
+ <MTIME>1304060057737</MTIME>
- <BLOCK_ID>85340326229460895</BLOCK_ID>
+ <BLOCK_ID>4671949296381030428</BLOCK_ID>
<BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
- <BLOCK_ID>4456960998526419279</BLOCK_ID>
+ <BLOCK_ID>-844362243522407159</BLOCK_ID>
- <BLOCK_ID>-6161739531018161735</BLOCK_ID>
+ <BLOCK_ID>3476886462779656950</BLOCK_ID>
+ <CHECKSUM>-766805874</CHECKSUM>
+ <TRANSACTION_ID>25</TRANSACTION_ID>
<GENERATION_STAMP>1005</GENERATION_STAMP>
+ <CHECKSUM>238426056</CHECKSUM>
+ <TRANSACTION_ID>26</TRANSACTION_ID>
<PATH>/file_concat_1</PATH>
- <MTIME>1286491965105</MTIME>
- <ATIME>1286491965105</ATIME>
+ <MTIME>1304060057742</MTIME>
+ <ATIME>1304060057742</ATIME>
+ <CHECKSUM>1156254705</CHECKSUM>
+ <TRANSACTION_ID>27</TRANSACTION_ID>
- <MTIME>1286491965148</MTIME>
+ <MTIME>1304060057764</MTIME>
- <BLOCK_ID>-3894328423940677915</BLOCK_ID>
+ <BLOCK_ID>-754893470864399741</BLOCK_ID>
<BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
- <BLOCK_ID>-2833847567910728858</BLOCK_ID>
+ <BLOCK_ID>1820875380010181049</BLOCK_ID>
- <BLOCK_ID>-3654781106237722465</BLOCK_ID>
+ <BLOCK_ID>8266387560744259971</BLOCK_ID>
+ <CHECKSUM>-654780301</CHECKSUM>
<OPCODE>16</OPCODE>
+ <TRANSACTION_ID>28</TRANSACTION_ID>
<LENGTH>4</LENGTH>
<CONCAT_TARGET>/file_concat_target</CONCAT_TARGET>
<CONCAT_SOURCE>/file_concat_0</CONCAT_SOURCE>
<CONCAT_SOURCE>/file_concat_1</CONCAT_SOURCE>
- <TIMESTAMP>1286491965157</TIMESTAMP>
+ <TIMESTAMP>1304060057767</TIMESTAMP>
+ <CHECKSUM>1273279541</CHECKSUM>
<OPCODE>17</OPCODE>
+ <TRANSACTION_ID>29</TRANSACTION_ID>
<SOURCE>/file_symlink</SOURCE>
<DESTINATION>/file_concat_target</DESTINATION>
- <MTIME>1286491965168</MTIME>
- <ATIME>1286491965168</ATIME>
+ <MTIME>1304060057770</MTIME>
+ <ATIME>1304060057770</ATIME>
+ <CHECKSUM>1385678569</CHECKSUM>
<OPCODE>18</OPCODE>
+ <TRANSACTION_ID>30</TRANSACTION_ID>
<T_VERSION>0</T_VERSION>
- <T_OWNER>steffl</T_OWNER>
+ <T_OWNER>todd</T_OWNER>
<T_RENEWER>JobTracker</T_RENEWER>
<T_REAL_USER/>
- <T_ISSUE_DATE>1286491965176</T_ISSUE_DATE>
- <T_MAX_DATE>1287096765176</T_MAX_DATE>
+ <T_ISSUE_DATE>1304060057773</T_ISSUE_DATE>
+ <T_MAX_DATE>1304664857773</T_MAX_DATE>
<T_SEQUENCE_NUMBER>1</T_SEQUENCE_NUMBER>
<T_MASTER_KEY_ID>2</T_MASTER_KEY_ID>
- <T_EXPIRY_TIME>1286578365176</T_EXPIRY_TIME>
+ <T_EXPIRY_TIME>1304146457773</T_EXPIRY_TIME>
+ <CHECKSUM>913145699</CHECKSUM>
<OPCODE>19</OPCODE>
+ <TRANSACTION_ID>31</TRANSACTION_ID>
- <T_EXPIRY_TIME>1286578365198</T_EXPIRY_TIME>
+ <T_EXPIRY_TIME>1304146457785</T_EXPIRY_TIME>
+ <CHECKSUM>-1772039941</CHECKSUM>
<OPCODE>20</OPCODE>
+ <TRANSACTION_ID>32</TRANSACTION_ID>
+ <CHECKSUM>1382094146</CHECKSUM>
+ <TRANSACTION_ID>33</TRANSACTION_ID>
<PATH>/reassign-lease-test</PATH>
@@ -438,14 +508,24 @@
<CLIENT_NAME>DFSClient_871171074</CLIENT_NAME>
+ <CHECKSUM>1975140107</CHECKSUM>
<OPCODE>22</OPCODE>
+ <TRANSACTION_ID>34</TRANSACTION_ID>
<CLIENT_NAME>HDFS_NameNode</CLIENT_NAME>
+ <OPCODE>23</OPCODE>
+ <TRANSACTION_ID>35</TRANSACTION_ID>
<OPCODE>-1</OPCODE>
@@ -28,7 +28,6 @@ import java.io.FileReader;
@@ -41,6 +40,7 @@ import org.apache.hadoop.fs.FileSystem;
@@ -128,11 +128,10 @@ public class TestOfflineImageViewer extends TestCase {
// Determine location of fsimage file
- URI [] files = cluster.getNameDirs(0).toArray(new URI[0]);
- orig = new File(files[0].getPath(), "current/fsimage");
- if (!orig.exists()) {
- fail("Didn't generate or can't find fsimage.");
+ orig = FSImageTestUtil.findLatestImageFile(
+ cluster.getNameNode().getFSImage().getStorage().getStorageDir(0));
+ if (orig == null) {
+ fail("Didn't generate or can't find fsimage");
if(cluster != null)
@@ -0,0 +1,132 @@
+import org.apache.hadoop.hdfs.DFSTestUtil;
+public class TestAtomicFileOutputStream {
+ private static final String TEST_STRING = "hello world";
+ private static final String TEST_STRING_2 = "goodbye world";
+ private static File BASE_DIR = new File(
+ System.getProperty("test.build.data", "build/test/data"));
+ private static File TEST_DIR = new File(BASE_DIR,
+ TestAtomicFileOutputStream.class.getName());
+ private static File DST_FILE = new File(TEST_DIR, "test.txt");
+ public void cleanupTestDir() throws IOException {
+ assertTrue(TEST_DIR.exists() || TEST_DIR.mkdirs());
+ FileUtil.deleteContents(TEST_DIR);
+ * Test case where there is no existing file
+ public void testWriteNewFile() throws IOException {
+ OutputStream fos = new AtomicFileOutputStream(DST_FILE);
+ assertFalse(DST_FILE.exists());
+ fos.write(TEST_STRING.getBytes());
+ fos.flush();
+ assertTrue(DST_FILE.exists());
+ String readBackData = DFSTestUtil.readFile(DST_FILE);
+ assertEquals(TEST_STRING, readBackData);
+ public void testOverwriteFile() throws IOException {
+ assertTrue("Creating empty dst file", DST_FILE.createNewFile());
+ assertTrue("Empty file still exists", DST_FILE.exists());
+ // Original contents still in place
+ assertEquals("", DFSTestUtil.readFile(DST_FILE));
+ // New contents replace original file
+ * Test case where the flush() fails at close time - make sure
+ * that we clean up after ourselves and don't touch any
+ * existing file at the destination
+ public void testFailToFlush() throws IOException {
+ // Create a file at destination
+ FileOutputStream fos = new FileOutputStream(DST_FILE);
+ fos.write(TEST_STRING_2.getBytes());
+ OutputStream failingStream = createFailingStream();
+ failingStream.write(TEST_STRING.getBytes());
+ failingStream.close();
+ fail("Close didn't throw exception");
+ // expected
+ // Should not have touched original file
+ assertEquals(TEST_STRING_2, DFSTestUtil.readFile(DST_FILE));
+ assertEquals("Temporary file should have been cleaned up",
+ DST_FILE.getName(), Joiner.on(",").join(TEST_DIR.list()));
+ * Create a stream that fails to flush at close time
+ private OutputStream createFailingStream() throws FileNotFoundException {
+ return new AtomicFileOutputStream(DST_FILE) {
+ public void flush() throws IOException {
+ throw new IOException("injected failure");
@@ -0,0 +1,111 @@
+import java.io.FileWriter;
+public class TestMD5FileUtils {
+ private static final File TEST_DIR_ROOT = new File(
+ private static final File TEST_DIR = new File(TEST_DIR_ROOT,
+ "TestMD5FileUtils");
+ private static final File TEST_FILE = new File(TEST_DIR,
+ "testMd5File.dat");
+ private static final int TEST_DATA_LEN = 128 * 1024; // 128KB test data
+ private static final byte[] TEST_DATA =
+ DFSTestUtil.generateSequentialBytes(0, TEST_DATA_LEN);
+ private static final MD5Hash TEST_MD5 = MD5Hash.digest(TEST_DATA);
+ public void setup() throws IOException {
+ FileUtil.fullyDelete(TEST_DIR);
+ assertTrue(TEST_DIR.mkdirs());
+ // Write a file out
+ FileOutputStream fos = new FileOutputStream(TEST_FILE);
+ fos.write(TEST_DATA);
+ public void testComputeMd5ForFile() throws Exception {
+ MD5Hash computedDigest = MD5FileUtils.computeMd5ForFile(TEST_FILE);
+ assertEquals(TEST_MD5, computedDigest);
+ public void testVerifyMD5FileGood() throws Exception {
+ MD5FileUtils.saveMD5File(TEST_FILE, TEST_MD5);
+ MD5FileUtils.verifySavedMD5(TEST_FILE, TEST_MD5);
+ * Test when .md5 file does not exist at all
+ @Test(expected=IOException.class)
+ public void testVerifyMD5FileMissing() throws Exception {
+ * Test when .md5 file exists but incorrect checksum
+ public void testVerifyMD5FileBadDigest() throws Exception {
+ MD5FileUtils.saveMD5File(TEST_FILE, MD5Hash.digest(new byte[0]));
+ fail("Did not throw");
+ // Expected
+ * Test when .md5 file exists but has a bad format
+ public void testVerifyMD5FileBadFormat() throws Exception {
+ FileWriter writer = new FileWriter(MD5FileUtils.getDigestFileForFile(TEST_FILE));
+ writer.write("this is not an md5 file");
+ writer.close();
@@ -17,16 +17,29 @@
package org.apache.hadoop.test;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
* Test provides some very generic helpers which might be used across the tests
public abstract class GenericTestUtils {
* Extracts the name of the method where the invocation has happened
* @return String name of the invoking method
@@ -34,4 +47,133 @@ public abstract class GenericTestUtils {
public static String getMethodName() {
return Thread.currentThread().getStackTrace()[2].getMethodName();
+ * Assert that a given file exists.
+ public static void assertExists(File f) {
+ Assert.assertTrue("File " + f + " should exist", f.exists());
+ * List all of the files in 'dir' that match the regex 'pattern'.
+ * Then check that this list is identical to 'expectedMatches'.
+ * @throws IOException if the dir is inaccessible
+ public static void assertGlobEquals(File dir, String pattern,
+ String ... expectedMatches) throws IOException {
+ Set<String> found = Sets.newTreeSet();
+ for (File f : FileUtil.listFiles(dir)) {
+ if (f.getName().matches(pattern)) {
+ found.add(f.getName());
+ Set<String> expectedSet = Sets.newTreeSet(
+ Arrays.asList(expectedMatches));
+ Assert.assertEquals("Bad files matching " + pattern + " in " + dir,
+ Joiner.on(",").join(found),
+ Joiner.on(",").join(expectedSet));
+ public static void assertExceptionContains(String string, Throwable t) {
+ String msg = t.getMessage();
+ Assert.assertTrue(
+ "Unexpected exception:" + StringUtils.stringifyException(t),
+ msg.contains(string));
+ public static void waitFor(Supplier<Boolean> check,
+ int checkEveryMillis, int waitForMillis)
+ throws TimeoutException, InterruptedException
+ long st = System.currentTimeMillis();
+ do {
+ boolean result = check.get();
+ if (result) {
+ Thread.sleep(checkEveryMillis);
+ } while (System.currentTimeMillis() - st < waitForMillis);
+ throw new TimeoutException("Timed out waiting for condition");
+ * Mockito answer helper that triggers one latch as soon as the
+ * method is called, then waits on another before continuing.
+ public static class DelayAnswer implements Answer<Object> {
+ private final Log LOG;
+ private final CountDownLatch fireLatch = new CountDownLatch(1);
+ private final CountDownLatch waitLatch = new CountDownLatch(1);
+ public DelayAnswer(Log log) {
+ this.LOG = log;
+ * Wait until the method is called.
+ public void waitForCall() throws InterruptedException {
+ fireLatch.await();
+ * Tell the method to proceed.
+ * This should only be called after waitForCall()
+ public void proceed() {
+ waitLatch.countDown();
+ public Object answer(InvocationOnMock invocation) throws Throwable {
+ LOG.info("DelayAnswer firing fireLatch");
+ fireLatch.countDown();
+ LOG.info("DelayAnswer waiting on waitLatch");
+ waitLatch.await();
+ LOG.info("DelayAnswer delay complete");
+ throw new IOException("Interrupted waiting on latch", ie);
+ return passThrough(invocation);
+ protected Object passThrough(InvocationOnMock invocation) throws Throwable {
+ return invocation.callRealMethod();
+ * An Answer implementation that simply forwards all calls through
+ * to a delegate.
+ * This is useful as the default Answer for a mock object, to create
+ * something like a spy on an RPC proxy. For example:
+ * <code>
+ * NamenodeProtocol origNNProxy = secondary.getNameNode();
+ * NamenodeProtocol spyNNProxy = Mockito.mock(NameNodeProtocol.class,
+ * new DelegateAnswer(origNNProxy);
+ * doThrow(...).when(spyNNProxy).getBlockLocations(...);
+ * ...
+ * </code>
+ public static class DelegateAnswer implements Answer<Object> {
+ private final Object delegate;
+ public DelegateAnswer(Object delegate) {
+ this.delegate = delegate;
+ return invocation.getMethod().invoke(
+ delegate, invocation.getArguments());