HDFS-1073. Redesign the NameNode's storage layout for image checkpoints and edit logs to introduce transaction IDs and be more robust. Contributed by Todd Lipcon and Ivan Kelly.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1152295 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Todd Lipcon 2011-07-29 16:28:45 +00:00
parent c58366acd1
commit 28e6a4e44a
99 changed files with 10074 additions and 4336 deletions

View File

@ -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
(todd)
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
(Ivan Kelly and Todd Lipcon via todd)
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
(Ivan Kelly and Todd Lipcon via todd)
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.
(todd)
HDFS-2048. Add upgrade tests and fix upgrade from 0.22 with corrupt image.
(todd)
HDFS-2027. Image inspector should return finalized logs before unfinalized
logs. (todd)
Release 0.22.0 - Unreleased
INCOMPATIBLE CHANGES

View File

@ -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

View File

@ -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"/>

View File

@ -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

View File

@ -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.
</li>
</ul>
<p>
@ -322,9 +322,9 @@
the maximum delay between two consecutive checkpoints
</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.
</li>
</ul>
<p>

View File

@ -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.
</description>
</property>
<property>
<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>
</property>

View File

@ -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";

View File

@ -85,7 +85,6 @@ private static void addDeprecatedKeys() {
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);

View File

@ -65,6 +65,9 @@ public enum SafeModeAction{ SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET; }
// 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:

View File

@ -78,7 +78,9 @@ public static enum Feature {
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;

View File

@ -181,6 +181,16 @@ public Iterator<StorageDirectory> dirIterator(StorageDirType dirType) {
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 void lock() throws IOException {
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 void unlock() throws IOException {
lock.channel().close();
lock = null;
}
@Override
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)
return true;
} 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 void unlockAll() throws IOException {
}
}
/**
* 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)
return true;
} 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();
}

View File

@ -25,6 +25,8 @@
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 String toString() {
.append(";nsid=").append(namespaceID).append(";c=").append(cTime);
return sb.toString();
}
public String toColonSeparatedString() {
return Joiner.on(":").join(
layoutVersion, namespaceID, cTime, clusterID);
}
}

View File

@ -19,29 +19,21 @@
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 @@
*/
@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()}
*/
BackupImage() {
super();
private boolean stopApplyingEditsOnNextRoll = false;
/**
* Construct a backup image.
* @param conf Configuration
* @throws IOException if storage cannot be initialised.
*/
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 @@ static enum JSpoolState {
* 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;
@ -122,283 +136,261 @@ void recoverCreateRead(Collection<URI> imageDirs,
}
}
/**
* Reset storage directories.
* <p>
* Unlock the storage.
* Rename <code>current</code> to <code>lastcheckpoint.tmp</code>
* and recreate empty <code>current</code>.
* @throws IOException
*/
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);
}
}
/**
* 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>.
* @throws IOException
*/
void loadCheckpoint(CheckpointSignature sig) throws IOException {
// load current image and journal if it is not in memory already
if(!editLog.isOpen())
editLog.open();
FSDirectory fsDir = getFSNamesystem().dir;
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));
} finally {
getFSDirectoryRootLock().writeUnlock();
}
loadFSEdits(sdEdits);
}
// set storage fields
storage.setStorageInfo(sig);
storage.setImageDigest(sig.imageDigest);
storage.setCheckpointTime(sig.checkpointTime);
}
/**
* Save meta-data into fsimage files.
* and create empty edits.
*/
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);
saveNamespace();
}
/**
* 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.
* 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()
*/
synchronized void journal(int length, byte[] data) throws IOException {
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:
break;
}
// write to files
editLog.logEdit(length, data);
editLog.logSync();
} finally {
backupInputStream.clear();
synchronized void journal(long firstTxId, int numTxns, byte[] data) throws IOException {
if (LOG.isTraceEnabled()) {
LOG.trace("Got journal, " +
"state = " + bnState +
"; firstTxId = " + firstTxId +
"; numTxns = " + numTxns);
}
}
private synchronized void waitSpoolEnd() {
while(jsState == JSpoolState.WAIT) {
try {
wait();
} catch (InterruptedException e) {}
}
// 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.
*/
synchronized void startJournalSpool(NamenodeRegistration nnReg)
throws IOException {
switch(jsState) {
case OFF:
break;
case INPROGRESS:
switch(bnState) {
case DROP_UNTIL_NEXT_ROLL:
return;
case WAIT:
waitSpoolEnd();
case IN_SYNC:
// update NameSpace in memory
applyEdits(firstTxId, numTxns, data);
break;
case JOURNAL_ONLY:
break;
default:
throw new AssertionError("Unhandled state: " + bnState);
}
// create journal spool directories
for (Iterator<StorageDirectory> it
= storage.dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
StorageDirectory sd = it.next();
File jsDir = getJSpoolDir(sd);
if (!jsDir.exists() && !jsDir.mkdirs()) {
throw new IOException("Mkdirs failed to create "
+ jsDir.getCanonicalPath());
}
// create edit file if missing
File eFile = storage.getEditFile(sd);
if(!eFile.exists()) {
editLog.createEditLogFile(eFile);
}
}
if(!editLog.isOpen())
editLog.open();
// 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;
// write to BN's local edit log.
logEditsLocally(firstTxId, numTxns, data);
}
synchronized void setCheckpointTime(int length, byte[] data)
throws IOException {
/**
* Write the batch of edits to the local copy of the edit logs.
*/
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();
}
/**
* Apply the batch of edits to the local namespace.
*/
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 {
// unpack new checkpoint time
if (LOG.isTraceEnabled()) {
LOG.debug("data:" + StringUtils.byteToHexString(data));
}
backupInputStream.setBytes(data);
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());
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);
}
lastAppliedTxId += numTxns;
getFSNamesystem().dir.updateCountForINodeWithQuota(); // inefficient!
} finally {
backupInputStream.clear();
}
}
/**
* 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).
* 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 {
Iterator<StorageDirectory> itEdits
= storage.dirIterator(NameNodeDirType.EDITS);
if(! itEdits.hasNext())
throw new IOException("Could not locate checkpoint directories");
StorageDirectory sdEdits = itEdits.next();
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);
DataInputStream in = new DataInputStream(bin);
FSEditLogLoader logLoader = new FSEditLogLoader(namesystem);
int logVersion = logLoader.readLogVersion(in);
Checksum checksum = null;
if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
checksum = FSEditLog.getChecksum();
in = new DataInputStream(new CheckedInputStream(bin, checksum));
}
numEdits += logLoader.loadEditRecords(logVersion, in, checksum, false);
Preconditions.checkState(bnState == BNState.JOURNAL_ONLY,
"bad state: %s", bnState);
// first time reached the end of spool
jsState = JSpoolState.WAIT;
numEdits += logLoader.loadEditRecords(logVersion,
in, checksum, true);
getFSNamesystem().dir.updateCountForINodeWithQuota();
edits.close();
while (!tryConvergeJournalSpool()) {
;
}
assert bnState == BNState.IN_SYNC;
}
private boolean tryConvergeJournalSpool() throws IOException {
Preconditions.checkState(bnState == BNState.JOURNAL_ONLY,
"bad state: %s", bnState);
// 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
FSImage.LOG.info("Edits file " + jSpoolFile.getCanonicalPath()
+ " of size " + jSpoolFile.length() + " edits # " + numEdits
+ " loaded in " + (now()-startTime)/1000 + " seconds.");
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();
try {
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;
} finally {
IOUtils.closeStream(stream);
}
// 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);
LOG.info("Successfully synced BackupNode with NameNode at txnid " +
lastAppliedTxId);
setState(BNState.IN_SYNC);
}
return true;
}
// write version file
resetVersion(false, storage.getImageDigest());
/**
* Transition edit log to a new state, logging as necessary.
*/
private synchronized void setState(BNState newState) {
if (LOG.isDebugEnabled()) {
LOG.debug("State transition " + bnState + " -> " + newState,
new Exception("trace"));
}
bnState = newState;
}
// wake up journal writer
synchronized(this) {
jsState = JSpoolState.OFF;
/**
* 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)
throws IOException {
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();
}
}
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.");
setState(BNState.JOURNAL_ONLY);
}
stopApplyingEditsOnNextRoll = false;
notifyAll();
}
}
// Rename lastcheckpoint.tmp to previous.checkpoint
for (Iterator<StorageDirectory> it = storage.dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next();
storage.moveLastCheckpoint(sd);
/**
* 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;
}
/**
* After {@link #freezeNamespaceAtNextRoll()} has been called, wait until
* the BN receives notification of the next log roll.
*/
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");
try {
wait();
} catch (InterruptedException ie) {
LOG.warn("Interrupted waiting for namespace to freeze", ie);
throw new IOException(ie);
}
}
LOG.info("BackupNode namespace frozen.");
}
/**
* Override close() so that we don't finalize edit logs.
*/
@Override
public synchronized void close() throws IOException {
editLog.abortCurrentLogSegment();
storage.close();
}
}

View File

@ -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;
}
@Override
public EditLogOutputStream startLogSegment(long txId) throws IOException {
EditLogBackupOutputStream stm = new EditLogBackupOutputStream(bnReg, nnReg);
stm.startLogSegment(txId);
return stm;
}
@Override
public void finalizeLogSegment(long firstTxId, long lastTxId)
throws IOException {
}
@Override
public void setOutputBufferCapacity(int size) {
}
@Override
public void purgeLogsOlderThan(long minTxIdToKeep, StoragePurger purger)
throws IOException {
}
public boolean matchesRegistration(NamenodeRegistration bnReg) {
return bnReg.getAddress().equals(this.bnReg.getAddress());
}
@Override
public EditLogInputStream getInProgressInputStream(long segmentStartsAtTxId) {
return null;
}
}

View File

@ -22,21 +22,20 @@
import java.net.SocketTimeoutException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
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 @@
* </ol>
*/
@InterfaceAudience.Private
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 @@ protected void setHttpServerAddress(Configuration conf){
@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();
}
@Override // NameNode
@ -179,6 +177,17 @@ public void stop() {
super.stop();
}
@Override
public long getProtocolVersion(String protocol, long clientVersion)
throws IOException {
if (protocol.equals(JournalProtocol.class.getName())) {
return JournalProtocol.versionID;
} else {
return super.getProtocolVersion(protocol, clientVersion);
}
}
/////////////////////////////////////////////////////
// NamenodeProtocol implementation for backup node.
/////////////////////////////////////////////////////
@ -205,34 +214,36 @@ public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
public void endCheckpoint(NamenodeRegistration registration,
CheckpointSignature sig) throws IOException {
throw new UnsupportedActionException("endCheckpoint");
}
}
@Override // NamenodeProtocol
/////////////////////////////////////////////////////
// BackupNodeProtocol implementation for backup node.
/////////////////////////////////////////////////////
@Override
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:
return;
case (int)JA_JOURNAL:
bnImage.journal(length, args);
return;
case (int)JA_JSPOOL_START:
bnImage.startJournalSpool(nnReg);
return;
case (int)JA_CHECKPOINT_TIME:
bnImage.setCheckpointTime(length, args);
setRegistration(); // keep registration up to date
return;
default:
throw new IOException("Unexpected journal action: " + jAction);
}
getBNImage().journal(firstTxId, numTxns, records);
}
@Override
public void startLogSegment(NamenodeRegistration registration, long txid)
throws IOException {
verifyRequest(registration);
getBNImage().namenodeStartedLogSegment(txid);
}
//////////////////////////////////////////////////////
BackupImage getBNImage() {
return (BackupImage)getFSImage();
}
boolean shouldCheckpointAtStartup() {
@ -241,9 +252,9 @@ boolean shouldCheckpointAtStartup() {
assert fsImage.getStorage().getNumStorageDirs() > 0;
return ! fsImage.getStorage().getStorageDir(0).getVersionFile().exists();
}
if(namesystem == null || namesystem.dir == null || getFSImage() == null)
return true;
return fsImage.getEditLog().getNumEditStreams() == 0;
// BN always checkpoints on startup in order to get in sync with namespace
return true;
}
private NamespaceInfo handshake(Configuration conf) throws IOException {
@ -287,14 +298,6 @@ void doCheckpoint() throws IOException {
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 @@ void setCheckpointState(CheckpointStates cs) {
*/
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());
} else {
nsInfo.validateStorage(storage);
}
setRegistration();
NamenodeRegistration nnReg = null;
while(!isStopRequested()) {
@ -338,23 +342,6 @@ else if(!nnReg.isRole(NamenodeRole.NAMENODE)) {
nnRpcAddress = nnReg.getAddress();
}
/**
* Reset node namespace state in memory and in storage directories.
* @throws IOException
*/
void resetNamespace() throws IOException {
((BackupImage)getFSImage()).reset();
}
/**
* Get size of the local journal (edit log).
* @return size of the current journal
* @throws IOException
*/
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 {

View File

@ -24,10 +24,11 @@
import org.apache.hadoop.classification.InterfaceAudience;
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 org.apache.hadoop.io.WritableUtils;
import com.google.common.collect.ComparisonChain;
/**
* A unique signature intended to identify checkpoint transactions.
*/
@ -35,41 +36,35 @@
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();
checkpointTime = fsImage.getStorage().getCheckpointTime();
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 String toString() {
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 @@ void validateStorageInfo(FSImage si) throws IOException {
// 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 boolean equals(Object o) {
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 int hashCode() {
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();
}
}

View File

@ -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.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collection;
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 static org.apache.hadoop.hdfs.server.common.Util.now;
import org.apache.hadoop.hdfs.server.namenode.FSImage.CheckpointStates;
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.CheckpointCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
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 @@ private NamenodeProtocol getNamenode(){
try {
initialize(conf);
} catch(IOException e) {
LOG.warn("Checkpointer got exception", e);
shutdown();
throw e;
}
@ -87,6 +89,7 @@ private NamenodeProtocol getNamenode(){
/**
* Initialize checkpoint.
*/
@SuppressWarnings("deprecation")
private void initialize(Configuration conf) throws IOException {
// Create connection to the namenode.
shouldRun = true;
@ -94,8 +97,9 @@ private void initialize(Configuration conf) throws IOException {
// 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 @@ private void initialize(Configuration conf) throws IOException {
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 @@ public void run() {
periodMSec *= 1000;
long lastCheckpointTime = 0;
if(!backupNode.shouldCheckpointAtStartup())
if (!backupNode.shouldCheckpointAtStartup()) {
lastCheckpointTime = now();
}
while(shouldRun) {
try {
long now = now();
@ -137,8 +141,8 @@ public void run() {
if(now >= lastCheckpointTime + periodMSec) {
shouldCheckpoint = true;
} else {
long size = getJournalSize();
if(size >= checkpointSize)
long txns = countUncheckpointedTxns();
if(txns >= checkpointTxnCount)
shouldCheckpoint = true;
}
if(shouldCheckpoint) {
@ -160,64 +164,24 @@ public void run() {
}
}
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()]);
assert files.length > 0 : "No checkpoint targets.";
TransferFsImage.getFileClient(nnHttpAddr, fileid, files, false);
LOG.info("Downloaded file " + files[0].getName() + " size " +
files[0].length() + " bytes.");
}
/**
* 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 @@ void doCheckpoint() throws IOException {
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);
// Make sure we're talking to the same NN!
sig.validateStorageInfo(bnImage);
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();
if (firstRemoteLog.getStartTxId() > lastApplied + 1) {
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);
}
BackupImage bnImage = getFSImage();
bnImage.getStorage().setBlockPoolID(backupNode.getBlockPoolId());
bnImage.getStorage().setClusterID(backupNode.getClusterId());
bnImage.loadCheckpoint(sig);
sig.validateStorageInfo(bnImage);
bnImage.saveCheckpoint();
long txid = bnImage.getLastAppliedTxId();
bnImage.saveFSImageInAllDirs(txid);
bnStorage.writeAll();
if(cpCmd.needToReturnImage())
uploadCheckpoint(sig);
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();
for (RemoteEditLog log : manifest.getLogs()) {
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);
}
}

View File

@ -42,10 +42,6 @@ private static class ByteBufferInputStream extends ByteArrayInputStream {
super(new byte[0]);
}
byte[] getData() {
return super.buf;
}
void setData(byte[] newBytes) {
super.buf = newBytes;
super.count = newBytes == null ? 0 : newBytes.length;

View File

@ -19,10 +19,11 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Arrays;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.ipc.RPC;
@ -39,7 +40,7 @@
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);
try {
this.backupNode =
(NamenodeProtocol) RPC.getProxy(NamenodeProtocol.class,
NamenodeProtocol.versionID, bnAddress, new HdfsConfiguration());
RPC.getProxy(JournalProtocol.class,
JournalProtocol.versionID, bnAddress, new HdfsConfiguration());
} catch(IOException e) {
Storage.LOG.error("Error connecting to: " + bnAddress, e);
throw 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 @@ public void close() throws IOException {
doubleBuf = null;
}
@Override
public void abort() throws IOException {
RPC.stopProxy(backupNode);
doubleBuf = null;
}
@Override // EditLogOutputStream
void setReadyToFlush() throws IOException {
doubleBuf.setReadyToFlush();
@ -116,11 +123,21 @@ void setReadyToFlush() throws IOException {
@Override // 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();
assert out.getLength() == 0 : "Output buffer is not empty";
backupNode.journal(nnRegistration,
firstTxToFlush, numReadyTxns, data);
}
}
@ -134,16 +151,6 @@ long length() throws IOException {
return 0;
}
private void send(int ja) throws IOException {
try {
int length = out.getLength();
out.write(FSEditLogOpCodes.OP_INVALID.getOpCode());
backupNode.journal(nnRegistration, ja, length, out.getData());
} finally {
out.reset();
}
}
/**
* Get backup node registration.
*/
@ -151,17 +158,7 @@ NamenodeRegistration getRegistration() {
return bnRegistration;
}
/**
* Verify that the backup node is alive.
*/
boolean isAlive() {
try {
send(NamenodeProtocol.JA_IS_ALIVE);
} catch(IOException ei) {
Storage.LOG.info(bnRegistration.getRole() + " "
+ bnRegistration.getAddress() + " is not alive. ", ei);
return false;
}
return true;
void startLogSegment(long txId) throws IOException {
backupNode.startLogSegment(nnRegistration, txId);
}
}

View File

@ -70,4 +70,10 @@ long length() throws IOException {
// file size + size of both buffers
return file.length();
}
@Override
public String toString() {
return getName();
}
}

View File

@ -25,6 +25,8 @@
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.hdfs.protocol.FSConstants;
import org.apache.hadoop.io.IOUtils;
@ -35,6 +37,8 @@
* 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 @@ void write(FSEditLogOp op) throws IOException {
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 @@ void create() throws IOException {
@Override
public void close() throws IOException {
if (fp == null) {
throw new IOException("Trying to use aborted output stream");
}
try {
// close should have been called after all pending transactions
// have been flushed & synced.
@ -130,6 +145,16 @@ public void close() throws IOException {
fc = null;
fp = null;
}
fp = null;
}
@Override
public void abort() throws IOException {
if (fp == null) {
return;
}
IOUtils.cleanup(LOG, fp);
fp = null;
}
/**
@ -148,6 +173,10 @@ void setReadyToFlush() throws IOException {
*/
@Override
protected void flushAndSync() throws IOException {
if (fp == null) {
throw new IOException("Trying to use aborted output stream");
}
preallocate(); // preallocate file if necessary
doubleBuf.flushTo(fp);
fc.force(false); // metadata updates not needed because of preallocation
@ -189,21 +218,19 @@ private void preallocate() throws IOException {
}
}
/**
* Operations like OP_JSPOOL_START and OP_CHECKPOINT_TIME should not be
* written into edits file.
*/
@Override
boolean isOperationSupported(byte op) {
return op < FSEditLogOpCodes.OP_JSPOOL_START.getOpCode() - 1;
}
/**
* Returns the file associated with this stream.
*/
File getFile() {
return file;
}
/**
* @return true if this stream is currently open.
*/
public boolean isOpen() {
return fp != null;
}
@VisibleForTesting
public void setFileChannelForTesting(FileChannel fc) {

View File

@ -18,9 +18,13 @@
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
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 void writeRaw(byte[] bytes, int offset, int length)
*/
abstract void create() throws IOException;
/** {@inheritDoc} */
/**
* 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 @@ public boolean shouldForceSync() {
return false;
}
boolean isOperationSupported(byte op) {
return true;
}
/**
* Return total time spent in {@link #flushAndSync()}
*/

View File

@ -20,6 +20,7 @@
import java.io.IOException;
import java.io.OutputStream;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.Writer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.IOUtils;
@ -35,20 +36,19 @@
*/
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);
}
void writeRaw(byte[] bytes, int offset, int length) throws IOException {
@ -71,10 +71,9 @@ void close() throws IOException {
void setReadyToFlush() {
assert isFlushed() : "previous data not flushed yet";
DataOutputBuffer tmp = bufReady;
TxnBuffer tmp = bufReady;
bufReady = bufCurrent;
bufCurrent = tmp;
writer = new FSEditLogOp.Writer(bufCurrent);
}
/**
@ -102,4 +101,50 @@ public int countBufferedBytes() {
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;
} else {
assert op.txid > firstTxId;
}
writer.writeOp(op);
numTxns++;
}
@Override
public DataOutputBuffer reset() {
super.reset();
firstTxId = FSConstants.INVALID_TXID;
numTxns = 0;
return this;
}
}
}

View File

@ -22,9 +22,7 @@
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
@ -157,26 +155,33 @@ private BlockManager getBlockManager() {
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;
try {
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;
} finally {
if (!success) {
fsImage.close();
}
}
writeLock();
try {

View File

@ -21,12 +21,12 @@
import java.io.BufferedInputStream;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.File;
import java.io.FileInputStream;
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.Arrays;
import java.util.zip.CheckedInputStream;
import java.util.zip.Checksum;
import org.apache.hadoop.fs.permission.PermissionStatus;
@ -37,6 +37,8 @@
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.common.Storage;
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 FSEditLogLoader(FSNamesystem fsNamesys) {
* 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)
throws IOException {
long startTime = now();
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;
try {
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;
}
int loadFSEdits(EditLogInputStream edits, boolean closeOnExit) throws IOException {
int loadFSEdits(EditLogInputStream edits, boolean closeOnExit,
long expectedStartingTxId)
throws IOException {
BufferedInputStream bin = new BufferedInputStream(edits);
DataInputStream in = new DataInputStream(bin);
int numEdits = 0;
int logVersion = 0;
try {
logVersion = readLogVersion(in);
Checksum checksum = null;
if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
checksum = FSEditLog.getChecksum();
in = new DataInputStream(new CheckedInputStream(bin, checksum));
}
numEdits = loadEditRecords(logVersion, in, checksum, false);
LogHeader header = LogHeader.read(in);
numEdits = loadEditRecords(
header.logVersion, in, header.checksum, false,
expectedStartingTxId);
} finally {
if(closeOnExit)
in.close();
}
if (logVersion != FSConstants.LAYOUT_VERSION) // other version
numEdits++; // save this image asap
return numEdits;
}
@SuppressWarnings("deprecation")
int loadEditRecords(int logVersion, DataInputStream in,
Checksum checksum, boolean closeOnExit) throws IOException {
Checksum checksum, boolean closeOnExit,
long expectedStartingTxId)
throws IOException {
FSDirectory fsDir = fsNamesys.dir;
int numEdits = 0;
@ -162,6 +131,8 @@ int loadEditRecords(int logVersion, DataInputStream in,
Arrays.fill(recentOpcodeOffsets, -1);
try {
long txId = expectedStartingTxId - 1;
try {
FSEditLogOp.Reader reader = new FSEditLogOp.Reader(in, logVersion,
checksum);
@ -169,6 +140,15 @@ int loadEditRecords(int logVersion, DataInputStream in,
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 @@ int loadEditRecords(int logVersion, DataInputStream in,
reassignLeaseOp.path, reassignLeaseOp.newHolder, pendingFile);
break;
}
case OP_START_LOG_SEGMENT:
case OP_END_LOG_SEGMENT: {
// no data in here currently.
numOpOther++;
break;
}
case OP_DATANODE_ADD:
case OP_DATANODE_REMOVE:
numOpOther++;
@ -494,6 +480,61 @@ private void check203UpgradeFailure(int logVersion, IOException ex)
}
}
/**
* 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);
try {
PositionTrackingInputStream tracker = new PositionTrackingInputStream(
new BufferedInputStream(fis));
DataInputStream dis = new DataInputStream(tracker);
LogHeader header;
try {
header = LogHeader.read(dis);
} catch (Throwable t) {
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;
try {
while (true) {
lastPos = tracker.getPos();
if (reader.readOp() == null) {
break;
}
numValid++;
}
} catch (Throwable t) {
// 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);
} finally {
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.
*/

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.util.zip.CheckedInputStream;
import java.util.zip.Checksum;
import java.util.EnumMap;
@ -29,10 +30,12 @@
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.FSConstants;
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;
import org.apache.hadoop.hdfs.server.common.Storage;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.*;
import org.apache.hadoop.security.token.delegation.DelegationKey;
@ -61,6 +64,8 @@
@InterfaceStability.Unstable
public abstract class FSEditLogOp {
final FSEditLogOpCodes opCode;
long txid;
@SuppressWarnings("deprecation")
private static ThreadLocal<EnumMap<FSEditLogOpCodes, FSEditLogOp>> opInstances =
@ -93,8 +98,10 @@ protected EnumMap<FSEditLogOpCodes, FSEditLogOp> initialValue() {
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 @@ protected EnumMap<FSEditLogOpCodes, FSEditLogOp> initialValue() {
*/
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,6 +1219,28 @@ void readFields(DataInputStream in, int logVersion)
this.key.readFields(in);
}
}
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 LogSegmentOp getInstance(FSEditLogOpCodes code) {
return (LogSegmentOp)opInstances.get().get(code);
}
public void readFields(DataInputStream in, int logVersion)
throws IOException {
// no data stored in these ops yet
}
@Override
void writeFields(DataOutputStream out) throws IOException {
// no data stored
}
}
static class InvalidOp extends FSEditLogOp {
private InvalidOp() {
@ -1228,54 +1262,6 @@ void readFields(DataInputStream in, int logVersion)
}
}
static class JSpoolStartOp extends FSEditLogOp {
private JSpoolStartOp() {
super(OP_JSPOOL_START);
}
static JSpoolStartOp getInstance() {
return (JSpoolStartOp)opInstances.get().get(OP_JSPOOL_START);
}
@Override
void writeFields(DataOutputStream out) throws IOException {
}
@Override
void readFields(DataInputStream in, int logVersion)
throws IOException {
}
}
static class CheckpointTimeOp extends FSEditLogOp {
long checkpointTime;
private CheckpointTimeOp() {
super(OP_CHECKPOINT_TIME);
}
CheckpointTimeOp setCheckpointTime(long time) {
this.checkpointTime = time;
return this;
}
static CheckpointTimeOp getInstance() {
return (CheckpointTimeOp)opInstances.get()
.get(OP_CHECKPOINT_TIME);
}
@Override
void writeFields(DataOutputStream out) throws IOException {
new LongWritable(checkpointTime).write(out);
}
@Override
void readFields(DataInputStream in, int logVersion)
throws IOException {
this.checkpointTime = readLong(in);
}
}
static private short readShort(DataInputStream in) throws IOException {
return Short.parseShort(FSImageSerialization.readString(in));
}
@ -1337,6 +1323,62 @@ private static long readLongWritable(DataInputStream in) throws IOException {
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 {
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;
try {
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 Writer(DataOutputBuffer out) {
public void writeOp(FSEditLogOp op) throws IOException {
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 static class Reader {
@SuppressWarnings("deprecation")
public Reader(DataInputStream in, int logVersion,
Checksum checksum) {
this.in = in;
if (checksum != null) {
this.in = new DataInputStream(
new CheckedInputStream(in, checksum));
} else {
this.in = in;
}
this.logVersion = logVersion;
this.checksum = checksum;
}
@ -1423,9 +1471,15 @@ public FSEditLogOp readOp() throws IOException {
if (op == null) {
throw new IOException("Read invalid opcode " + opCode);
}
if (LayoutVersion.supports(Feature.STORED_TXIDS, logVersion)) {
// 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 FSEditLogOp readOp() throws IOException {
* Validate a transaction's checksum
*/
private void validateChecksum(DataInputStream in,
Checksum checksum)
Checksum checksum,
long txid)
throws IOException {
if (checksum != null) {
int calculatedChecksum = (int)checksum.getValue();
@ -1441,7 +1496,7 @@ private void validateChecksum(DataInputStream in,
if (readChecksum != calculatedChecksum) {
throw new ChecksumException(
"Transaction is corrupt. Calculated checksum is " +
calculatedChecksum + " but read checksum " + readChecksum, -1);
calculatedChecksum + " but read checksum " + readChecksum, txid);
}
}
}

View File

@ -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;

View File

@ -44,6 +44,7 @@
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.io.Text;
@ -71,10 +72,8 @@ static class Loader {
/** 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 @@ static class Loader {
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 @@ MD5Hash getLoadedImageMd5() {
return imgDigest;
}
/**
* Return the namespace ID of the image that has been loaded.
* @throws IllegalStateException if load() has not yet been called.
*/
int getLoadedNamespaceID() {
long getLoadedImageTxId() {
checkLoaded();
return imgNamespaceID;
return imgTxId;
}
/**
@ -156,10 +142,14 @@ void load(File curFile)
* 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 @@ void load(File curFile)
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();
} else {
imgTxId = 0;
}
// read compression related info
FSImageCompression compression;
@ -234,7 +233,7 @@ private void updateRootAttr(INode root) {
private void loadLocalNameINodes(long numFiles, DataInputStream in)
throws IOException {
assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
imgVersion);
getLayoutVersion());
assert numFiles > 0;
// load root
@ -324,11 +323,12 @@ private void loadFullNameINodes(long numFiles,
* @return an inode
*/
private INode loadINode(DataInputStream in)
throws IOException {
throws IOException {
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 @@ private INode loadINode(DataInputStream in)
modificationTime, atime, nsQuota, dsQuota, blockSize);
}
private void loadDatanodes(DataInputStream in) throws IOException {
private void loadDatanodes(DataInputStream in)
throws IOException {
int imgVersion = getLayoutVersion();
if (imgVersion > -3) // pre datanode image version
return;
if (imgVersion <= -12) {
@ -412,6 +415,7 @@ private void loadDatanodes(DataInputStream in) throws IOException {
private void loadFilesUnderConstruction(DataInputStream in)
throws IOException {
FSDirectory fsDir = namesystem.dir;
int imgVersion = getLayoutVersion();
if (imgVersion > -13) // pre lease image version
return;
int size = in.readInt();
@ -437,7 +441,10 @@ private void loadFilesUnderConstruction(DataInputStream in)
}
}
private void loadSecretManagerState(DataInputStream in) throws IOException {
private void loadSecretManagerState(DataInputStream in)
throws IOException {
int imgVersion = getLayoutVersion();
if (!LayoutVersion.supports(Feature.DELEGATION_TOKEN, imgVersion)) {
//SecretManagerState is not available.
//This must not happen if security is turned on.
@ -446,8 +453,14 @@ private void loadSecretManagerState(DataInputStream in) throws IOException {
namesystem.loadSecretManagerState(in);
}
private int getLayoutVersion() {
return namesystem.getFSImage().getStorage().getLayoutVersion();
}
private long readNumFiles(DataInputStream in)
throws IOException {
int imgVersion = getLayoutVersion();
private long readNumFiles(DataInputStream in) throws IOException {
if (LayoutVersion.supports(Feature.NAMESPACE_QUOTA, imgVersion)) {
return in.readLong();
} else {
@ -526,6 +539,7 @@ MD5Hash getSavedDigest() {
}
void save(File newFile,
long txid,
FSNamesystem sourceNamesystem,
FSImageCompression compression)
throws IOException {
@ -542,9 +556,11 @@ void save(File newFile,
DataOutputStream out = new DataOutputStream(fos);
try {
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);

View File

@ -0,0 +1,272 @@
/**
* 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.DataInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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>();
@Override
void inspectDirectory(StorageDirectory sd) throws IOException {
// Was the file just formatted?
if (!sd.getVersionFile().exists()) {
hasOutOfDateStorageDirs = true;
return;
}
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)
hasOutOfDateStorageDirs = true;
// 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));
try {
timeStamp = in.readLong();
} finally {
in.close();
}
}
return timeStamp;
}
@Override
boolean isUpgradeFinalized() {
return isUpgradeFinalized;
}
@Override
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();
}
@Override
boolean needToSave() {
return hasOutOfDateStorageDirs ||
checkpointTimes.size() != 1 ||
latestNameCheckpointTime > latestEditsCheckpointTime;
}
private class PreTransactionalLoadPlan extends LoadPlan {
@Override
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);
}
} else {
//
// 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");
if (!ckptFile.renameTo(curFile)) {
throw new IOException("Unable to rename " + ckptFile +
" to " + curFile);
}
}
}
}
return needToSave;
}
@Override
File getImageFile() {
return NNStorage.getStorageFile(latestNameSD, NameNodeFile.IMAGE);
}
@Override
List<File> getEditsFiles() {
if (latestNameCheckpointTime > latestEditsCheckpointTime) {
// the image is already current, discard edits
LOG.debug(
"Name checkpoint time is newer than edits, not loading edits.");
return Collections.<File>emptyList();
}
return getEditsInStorageDir(latestEditsSD);
}
@Override
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;
}
}

View File

@ -0,0 +1,99 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.io.File;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
/**
* Interface responsible for inspecting a set of storage directories and devising
* a plan to load the namespace from them.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
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();
@Override
public String toString() {
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();
}
}
}

View File

@ -0,0 +1,688 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Set;
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.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
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.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
class FSImageTransactionalStorageInspector extends FSImageStorageInspector {
public static final Log LOG = LogFactory.getLog(
FSImageTransactionalStorageInspector.class);
private boolean needToSave = false;
private boolean isUpgradeFinalized = true;
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+)");
@Override
public void inspectDirectory(StorageDirectory sd) throws IOException {
// Was the directory just formatted?
if (!sd.getVersionFile().exists()) {
LOG.info("No version file in " + sd.getRoot());
needToSave |= true;
return;
}
File currentDir = sd.getCurrentDir();
File filesInStorage[];
try {
filesInStorage = FileUtil.listFiles(currentDir);
} catch (IOException ioe) {
LOG.warn("Unable to inspect storage directory " + currentDir,
ioe);
return;
}
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()) {
if (sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
try {
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
}
} else {
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.
try {
maxSeenTxId = Math.max(maxSeenTxId, NNStorage.readTransactionIdFile(sd));
} catch (IOException ioe) {
LOG.warn("Unable to determine the max transaction ID seen by " + sd, ioe);
}
List<FoundEditLog> editLogs = matchEditLogs(filesInStorage);
if (sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
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));
}
// set finalized flag
isUpgradeFinalized = isUpgradeFinalized && !sd.getPreviousDir().exists();
}
static List<FoundEditLog> matchEditLogs(File[] filesInStorage) {
List<FoundEditLog> ret = Lists.newArrayList();
for (File f : filesInStorage) {
String name = f.getName();
// Check for edits
Matcher editsMatch = EDITS_REGEX.matcher(name);
if (editsMatch.matches()) {
try {
long startTxId = Long.valueOf(editsMatch.group(1));
long endTxId = Long.valueOf(editsMatch.group(2));
ret.add(new FoundEditLog(f, startTxId, endTxId));
} catch (NumberFormatException nfe) {
LOG.error("Edits file " + f + " has improperly formatted " +
"transaction ID");
// skip
}
}
// Check for in-progress edits
Matcher inProgressEditsMatch = EDITS_INPROGRESS_REGEX.matcher(name);
if (inProgressEditsMatch.matches()) {
try {
long startTxId = Long.valueOf(inProgressEditsMatch.group(1));
ret.add(
new FoundEditLog(f, startTxId, FoundEditLog.UNKNOWN_END));
} catch (NumberFormatException nfe) {
LOG.error("In-progress edits file " + f + " has improperly " +
"formatted transaction ID");
// skip
}
}
}
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);
}
@Override
public boolean isUpgradeFinalized() {
return 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;
}
}
return ret;
}
public List<FoundFSImage> getFoundImages() {
return ImmutableList.copyOf(foundImages);
}
public List<FoundEditLog> getFoundEditLogs() {
return ImmutableList.copyOf(foundEditLogs);
}
@Override
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);
} else {
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 " +
"txid " + sinceTxId);
}
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;
} else {
// 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.";
} else {
msg += " but the most recent log file found starts with txid " +
lastLogGroupStartTxId;
}
throw new IOException(msg);
}
return new LogLoadPlan(recoveryLogs,
Lists.newArrayList(usefulGroups.values()));
}
@Override
public boolean needToSave() {
return 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
for (FoundEditLog log : logs) {
if (!log.isCorrupt()) {
return log;
}
}
// 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() {
for (FoundEditLog log : logs) {
if (!log.isInProgress()) {
return true;
}
}
return false;
}
/**
* @return the last txid included in the logs in this group
* @throws IllegalStateException if it is unknown -
* {@see #hasKnownLastTxId()}
*/
long getLastTxId() {
for (FoundEditLog log : logs) {
if (!log.isInProgress()) {
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;
} else {
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 {
for (FoundEditLog log : logs) {
if (log.isInProgress()) {
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.");
log.markCorrupt();
}
return;
}
long maxValidTxnCount = Long.MIN_VALUE;
for (FoundEditLog log : logs) {
long validTxnCount = log.validateLog().numTransactions;
LOG.warn(" Log " + log.getFile() +
" valid txns=" + validTxnCount +
" valid len=" + log.validateLog().validLength);
maxValidTxnCount = Math.max(maxValidTxnCount, validTxnCount);
}
for (FoundEditLog log : logs) {
long txns = log.validateLog().numTransactions;
if (txns < maxValidTxnCount) {
LOG.warn("Marking log at " + log.getFile() + " as corrupt since " +
"it is has only " + txns + " valid txns whereas another " +
"log has " + maxValidTxnCount);
log.markCorrupt();
} 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.
LOG.warn("Marking log at " + log.getFile() + " as corrupt since " +
"it has no transactions in it.");
log.markCorrupt();
}
}
}
/**
* 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 {
for (FoundEditLog log : logs) {
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;
}
File getFile() {
return file;
}
public long getTxId() {
return txId;
}
@Override
public String toString() {
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.startTxId = startTxId;
this.lastTxId = endTxId;
this.file = file;
}
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;
}
long getLastTxId() {
return lastTxId;
}
EditLogValidation validateLog() throws IOException {
if (cachedValidation == null) {
cachedValidation = FSEditLogLoader.validateEditLog(file);
}
return cachedValidation;
}
boolean isInProgress() {
return (lastTxId == UNKNOWN_END);
}
File getFile() {
return file;
}
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);
if (!success) {
throw new IOException(
"Couldn't rename corrupt log " + src + " to " + dst);
}
file = dst;
}
@Override
public String toString() {
return file.toString();
}
}
static class TransactionalLoadPlan extends LoadPlan {
final FoundFSImage image;
final LogLoadPlan logPlan;
public TransactionalLoadPlan(FoundFSImage image,
LogLoadPlan logPlan) {
super();
this.image = image;
this.logPlan = logPlan;
}
@Override
boolean doRecovery() throws IOException {
logPlan.doRecovery();
return false;
}
@Override
File getImageFile() {
return image.getFile();
}
@Override
List<File> getEditsFiles() {
return logPlan.getEditsFiles();
}
@Override
StorageDirectory getStorageDirectoryForProperties() {
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>();
for (FoundEditLog log : editLogs) {
ret.add(log.getFile());
}
return ret;
}
}
}

View File

@ -105,6 +105,7 @@
import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
@ -124,6 +125,7 @@
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
@ -323,8 +325,7 @@ private void initialize(Configuration conf, FSImage fsImage)
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 static Collection<URI> getStorageDirs(Configuration conf,
+ 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 @@ void saveNamespace() throws AccessControlException, IOException {
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.");
} finally {
readUnlock();
@ -4003,8 +4005,8 @@ String getSafeModeTip() {
}
}
long getEditLogSize() throws IOException {
return getEditLog().getEditLogSize();
public long getTransactionID() {
return getEditLog().getSyncTxId();
}
CheckpointSignature rollEditLog() throws IOException {
@ -4019,24 +4021,9 @@ CheckpointSignature rollEditLog() throws IOException {
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();
try {
if (isInSafeMode()) {
throw new SafeModeException("Image not rolled", safeMode);
}
LOG.info("Roll FSImage from " + Server.getRemoteAddress());
getFSImage().rollFSImage(sig, true);
} finally {
writeUnlock();
}
public RemoteEditLogManifest getEditLogManifest(long sinceTxId) throws IOException {
return getEditLog().getEditLogManifest(sinceTxId);
}
NamenodeCommand startCheckpoint(
@ -4516,31 +4503,29 @@ void saveFilesUnderConstruction(DataOutputStream out) throws IOException {
}
/**
* Register a name-node.
* <p>
* 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
* @throws IOException
* @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)
throws IOException {
void registerBackupNode(NamenodeRegistration bnReg,
NamenodeRegistration nnReg) throws IOException {
writeLock();
try {
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);
}
} finally {
writeUnlock();
}
@ -5080,4 +5065,6 @@ public BlockManager getBlockManager() {
void removeDecomNodeFromList(List<DatanodeDescriptor> nodeList) {
getBlockManager().getDatanodeManager().removeDecomNodeFromList(nodeList);
}
}

View File

@ -0,0 +1,114 @@
/**
* 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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.io.File;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundEditLog;
import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
/**
* 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) {
this.sd = sd;
}
@Override
public EditLogOutputStream startLogSegment(long txid) throws IOException {
File newInProgress = NNStorage.getInProgressEditsFile(sd, txid);
EditLogOutputStream stm = new EditLogFileOutputStream(newInProgress,
outputBufferCapacity);
stm.create();
return stm;
}
@Override
public void finalizeLogSegment(long firstTxId, long lastTxId)
throws IOException {
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);
}
}
@VisibleForTesting
public StorageDirectory getStorageDirectory() {
return sd;
}
@Override
public String toString() {
return "FileJournalManager for storage directory " + sd;
}
@Override
public void setOutputBufferCapacity(int size) {
this.outputBufferCapacity = size;
}
@Override
public void purgeLogsOlderThan(long minTxIdToKeep, StoragePurger purger)
throws IOException {
File[] files = FileUtil.listFiles(sd.getCurrentDir());
List<FoundEditLog> editLogs =
FSImageTransactionalStorageInspector.matchEditLogs(files);
for (FoundEditLog log : editLogs) {
if (log.getStartTxId() < minTxIdToKeep &&
log.getLastTxId() < minTxIdToKeep) {
purger.purgeLog(log);
}
}
}
@Override
public EditLogInputStream getInProgressInputStream(long segmentStartsAtTxId)
throws IOException {
File f = NNStorage.getInProgressEditsFile(sd, segmentStartsAtTxId);
return new EditLogFileInputStream(f);
}
}

View File

@ -20,6 +20,8 @@
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.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.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
import com.google.common.base.Preconditions;
/**
* 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();
try {
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 void doGet(final HttpServletRequest request,
+ request.getRemoteHost());
return;
}
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);
return;
}
UserGroupInformation.getCurrentUser().doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
if (ff.getImage()) {
response.setHeader(TransferFsImage.CONTENT_LENGTH,
String.valueOf(nnImage.getStorage()
.getFsImageName().length()));
// send fsImage
TransferFsImage.getFileServer(response.getOutputStream(),
nnImage.getStorage().getFsImageName(),
getThrottler(conf));
} else if (ff.getEdit()) {
response.setHeader(TransferFsImage.CONTENT_LENGTH,
String.valueOf(nnImage.getStorage()
.getFsEditName().length()));
// send edits
TransferFsImage.getFileServer(response.getOutputStream(),
nnImage.getStorage().getFsEditName(),
getThrottler(conf));
} 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>() {
@Override
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);
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(), imageFile,
getThrottler(conf));
} else if (parsedParams.isGetEdit()) {
long startTxId = parsedParams.getStartTxId();
long endTxId = parsedParams.getEndTxId();
File editFile = nnImage.getStorage()
.findFinalizedEditsFile(startTxId, endTxId);
setVerificationHeaders(response, editFile);
// send edits
TransferFsImage.getFileServer(response.getOutputStream(), editFile,
getThrottler(conf));
} else if (parsedParams.isPutImage()) {
final long txid = parsedParams.getTxId();
if (! currentlyDownloadingCheckpoints.add(txid)) {
throw new IOException(
"Another checkpointer is already in the process of uploading a" +
" checkpoint made at transaction ID " + txid);
}
try {
if (nnImage.getStorage().findImageFile(txid) != null) {
throw new IOException(
"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>() {
@Override
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();
} finally {
currentlyDownloadingCheckpoints.remove(txid);
}
nnImage.checkpointUploadDone();
}
return null;
}
@ -182,4 +227,148 @@ protected boolean isValidRequestor(String remoteUser, Configuration conf)
if(LOG.isDebugEnabled()) LOG.debug("isValidRequestor is rejecting: " + remoteUser);
return false;
}
/**
* Set headers for content length, and, if available, md5.
* @throws IOException
*/
private void setVerificationHeaders(HttpServletResponse response, File file)
throws IOException {
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,
StorageInfo remoteStorageInfo) {
return "getedit=1&" + START_TXID_PARAM + "=" + log.getStartTxId()
+ "&" + END_TXID_PARAM + "=" + log.getEndTxId()
+ "&" + STORAGEINFO_PARAM + "=" +
remoteStorageInfo.toColonSeparatedString();
}
static String getParamStringToPutImage(long txid,
InetSocketAddress imageListenAddress, NNStorage storage) {
return "putimage=1" +
"&" + TXID_PARAM + "=" + txid +
"&port=" + imageListenAddress.getPort() +
"&machine=" + imageListenAddress.getHostName()
+ "&" + STORAGEINFO_PARAM + "=" +
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;
txId = parseLongParam(request, TXID_PARAM);
} 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;
}
public long getTxId() {
Preconditions.checkState(isGetImage || isPutImage);
return txId;
}
public long getStartTxId() {
Preconditions.checkState(isGetEdit);
return startTxId;
}
public long getEndTxId() {
Preconditions.checkState(isGetEdit);
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)
throws IOException {
// 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);
}
}
}

View File

@ -0,0 +1,68 @@
/**
* 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;
/**
* 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;
/**
* @return an EditLogInputStream that reads from the same log that
* the edit log is currently writing. May return null if this journal
* manager does not support this operation.
*/
EditLogInputStream getInProgressInputStream(long segmentStartsAtTxId)
throws IOException;
}

View File

@ -17,22 +17,20 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.BufferedReader;
import java.io.Closeable;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.FileReader;
import java.io.IOException;
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.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Properties;
@ -51,13 +49,17 @@
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage;
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.io.MD5Hash;
import org.apache.hadoop.hdfs.util.AtomicFileOutputStream;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.DNS;
import com.google.common.base.Preconditions;
import com.google.common.annotations.VisibleForTesting;
/**
* NNStorage is responsible for management of the StorageDirectories used by
* the NameNode.
@ -66,17 +68,19 @@
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 boolean isOfType(StorageDirType type) {
}
}
/**
* 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.
* @throws IOException
*/
void errorOccurred(StorageDirectory sd) throws IOException;
/**
* A storage directory has been formatted.
* @param sd The storage directory being formatted.
* @throws IOException
*/
void formatOccurred(StorageDirectory sd) throws IOException;
/**
* A storage directory is now available use.
* @param sd The storage directory which has become available.
* @throws IOException
*/
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 @@ interface NNStorageListener {
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 @@ interface NNStorageListener {
= 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)
throws IOException {
super(NodeType.NAME_NODE);
storageDirs = new CopyOnWriteArrayList<StorageDirectory>();
this.listeners = new CopyOnWriteArrayList<NNStorageListener>();
this.blockpoolID = bpid;
setStorageDirectories(imageDirs, editsDirs);
}
@Override // Storage
@ -207,7 +183,6 @@ public boolean isPreUpgradableLayout(StorageDirectory sd) throws IOException {
@Override // Closeable
public void close() throws IOException {
listeners.clear();
unlockAll();
storageDirs.clear();
}
@ -232,10 +207,7 @@ boolean getRestoreFailedStorage() {
/**
* 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 @@ void attemptRestoreRemovedStorage() {
LOG.info("currently disabled dir " + root.getAbsolutePath() +
"; type="+sd.getStorageDirType()
+ ";canwrite="+root.canWrite());
try {
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);
}
} catch(IOException e) {
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 @@ List<StorageDirectory> getRemovedStorageDirs() {
}
/**
* 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 @@ List<StorageDirectory> getRemovedStorageDirs() {
* @param fsEditsDirs Locations to store edit logs.
* @throws IOException
*/
@VisibleForTesting
synchronized void setStorageDirectories(Collection<URI> fsNameDirs,
Collection<URI> fsEditsDirs)
throws IOException {
@ -411,110 +373,84 @@ Collection<URI> getDirectories(NameNodeDirType dirType)
}
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));
try {
timeStamp = in.readLong();
txid = Long.valueOf(br.readLine());
} finally {
in.close();
IOUtils.cleanup(LOG, br);
}
}
return timeStamp;
return txid;
}
/**
* Write last checkpoint time into a separate file.
*
* @param sd
* @throws IOException
*/
public void writeCheckpointTime(StorageDirectory sd) throws IOException {
if (checkpointTime < 0L)
return; // do not write negative time
File timeFile = getStorageFile(sd, NameNodeFile.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);
try {
out.writeLong(checkpointTime);
out.flush();
fos.getChannel().force(true);
fos.write(String.valueOf(txid).getBytes());
fos.write('\n');
} finally {
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.
* 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 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.
*/
public void setCheckpointTimeInStorage(long newCpT) {
checkpointTime = newCpT;
// Write new checkpoint time in all storage directories
for(Iterator<StorageDirectory> it =
dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next();
public void writeTransactionIdFileToStorage(long txid) {
// Write txid marker in all storage directories
for (StorageDirectory sd : storageDirs) {
try {
writeCheckpointTime(sd);
writeTransactionIdFile(sd, txid);
} catch(IOException e) {
// Close any edits stream associated with this dir and remove directory
LOG.warn("incrementCheckpointTime failed on "
+ sd.getRoot().getPath() + ";type="+sd.getStorageDirType());
try {
reportErrorsOnDirectory(sd);
} catch (IOException ioe) {
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 void setCheckpointTimeInStorage(long newCpT) {
*
* @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 File[] getFsImageNameCheckpoint() {
* 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;
for (Iterator<StorageDirectory> it =
dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
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 null;
}
/**
* @return The name of the first editlog file.
*/
public File getFsEditName() throws IOException {
for (Iterator<StorageDirectory> it
= dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
StorageDirectory sd = it.next();
if(sd.getRoot().canRead())
return getEditFile(sd);
}
return null;
}
/**
* @return The name of the first time file.
*/
public File getFsTimeName() {
StorageDirectory sd = null;
// NameNodeFile.TIME shoul be same on all directories
for (Iterator<StorageDirectory> it =
dirIterator(); it.hasNext();)
sd = it.next();
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
for (NNStorageListener listener : listeners) {
listener.formatOccurred(sd);
}
writeProperties(sd);
writeTransactionIdFile(sd, 0);
LOG.info("Storage directory " + sd.getRoot()
+ " has been successfully formatted.");
@ -597,7 +506,6 @@ public void format(String clusterId) throws IOException {
this.clusterID = clusterId;
this.blockpoolID = newBlockPoolID();
this.cTime = 0L;
this.setCheckpointTime(now());
for (Iterator<StorageDirectory> it =
dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next();
@ -624,50 +532,6 @@ private int newNamespaceID() {
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)
throws IOException {
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}
*
* @see org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory#getPreviousCheckpoint()
* @see org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory#getLastCheckpointTmp()
*/
protected void moveLastCheckpoint(StorageDirectory sd)
throws IOException {
File tmpCkptDir = sd.getLastCheckpointTmp();
File prevCkptDir = sd.getPreviousCheckpoint();
// remove previous.checkpoint
if (prevCkptDir.exists())
deleteDir(prevCkptDir);
// mv lastcheckpoint.tmp -> previous.checkpoint
if(tmpCkptDir.exists())
rename(tmpCkptDir, prevCkptDir);
}
@Override // Storage
protected void setFieldsFromProperties(
Properties props, StorageDirectory sd) throws IOException {
@ -689,26 +553,35 @@ protected void setFieldsFromProperties(
setDistributedUpgradeState(
sDUS == null? false : Boolean.parseBoolean(sDUS),
sDUV == null? getLayoutVersion() : Integer.parseInt(sDUV));
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) {
throw new InconsistentFSStateException(sd.getRoot(),
"file " + STORAGE_FILE_VERSION +
" has image MD5 digest when version is " + layoutVersion);
}
this.setCheckpointTime(readCheckpointTime(sd));
setDeprecatedPropertiesForUpgrade(props);
}
/**
* Write last checkpoint time and version file into the storage directory.
* 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);
}
}
/**
* 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 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 @@ protected void setPropertiesFromFields(Properties props,
props.setProperty("distributedUpgradeVersion",
Integer.toString(uVersion));
}
if (LayoutVersion.supports(Feature.FSIMAGE_CHECKSUM, layoutVersion)) {
// 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());
}
writeCheckpointTime(sd);
}
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());
}
@VisibleForTesting
public static String getCheckpointImageFileName(long txid) {
return String.format("%s_%019d",
NameNodeFile.IMAGE_NEW.getName(), txid);
}
@VisibleForTesting
public static String getImageFileName(long txid) {
return String.format("%s_%019d",
NameNodeFile.IMAGE.getName(), txid);
}
@VisibleForTesting
public static String getInProgressEditsFileName(long startTxId) {
return String.format("%s_%019d", NameNodeFile.EDITS_INPROGRESS.getName(),
startTxId);
}
static File getInProgressEditsFile(StorageDirectory sd, long startTxId) {
return new File(sd.getCurrentDir(), getInProgressEditsFileName(startTxId));
}
static File getFinalizedEditsFile(StorageDirectory sd,
long startTxId, long endTxId) {
return new File(sd.getCurrentDir(),
getFinalizedEditsFileName(startTxId, endTxId));
}
static File getImageFile(StorageDirectory sd, long txid) {
return new File(sd.getCurrentDir(),
getImageFileName(txid));
}
@VisibleForTesting
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)
throws IOException {
File ret = findFile(NameNodeDirType.EDITS,
getFinalizedEditsFileName(startTxId, endTxId));
if (ret == null) {
throw new IOException(
"No edits file for txid " + startTxId + "-" + endTxId + " exists!");
}
return ret;
}
/**
* 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,
getImageFileName(txid));
}
/**
* @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 null;
}
/**
* @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) {
ArrayList<File> list = new ArrayList<File>();
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));
}
return list;
}
@ -809,7 +741,9 @@ int getDistributedUpgradeVersion() {
* @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 @@ void initializeDistributedUpgrade() throws IOException {
+ 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.
*/
MD5Hash getImageDigest() {
return imageDigest;
}
/**
* 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 @@ void setDisablePreUpgradableLayoutCheck(boolean val) {
* @param sds A list of storage directories to mark as errored.
* @throws IOException
*/
void reportErrorsOnDirectories(List<StorageDirectory> sds) throws IOException {
void reportErrorsOnDirectories(List<StorageDirectory> sds) {
for (StorageDirectory sd : sds) {
reportErrorsOnDirectory(sd);
}
@ -904,17 +811,12 @@ void reportErrorsOnDirectories(List<StorageDirectory> sds) throws IOException {
* @param sd A storage directory to mark as errored.
* @throws IOException
*/
void reportErrorsOnDirectory(StorageDirectory sd)
throws IOException {
void reportErrorsOnDirectory(StorageDirectory sd) {
LOG.error("Error reported on storage directory " + sd);
String lsd = listStorageDirectories();
LOG.debug("current list of storage dirs:" + lsd);
for (NNStorageListener listener : listeners) {
listener.errorOccurred(sd);
}
LOG.warn("About to remove corresponding storage: "
+ sd.getRoot().getAbsolutePath());
try {
@ -927,8 +829,7 @@ void reportErrorsOnDirectory(StorageDirectory sd)
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 @@ void processStartupOptionsForUpgrade(StartupOption startOpt, int layoutVersion)
}
}
/**
* 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();
for (StorageDirectory sd : storageDirs) {
String dirPath = sd.getRoot().getAbsolutePath();
if (!dirPath.endsWith("/")) {
dirPath += "/";
}
if (absPath.startsWith(dirPath)) {
reportErrorsOnDirectory(sd);
return;
}
}
}
/**
* Generate new clusterID.
*
@ -1065,4 +989,67 @@ private void setBlockPoolID(File storage, String bpid)
public String getBlockPoolID() {
return blockpoolID;
}
/**
* Iterate over all current storage directories, inspecting them
* with the given inspector.
*/
void inspectStorageDirs(FSImageStorageInspector inspector)
throws IOException {
// 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();
inspector.inspectDirectory(sd);
}
}
/**
* 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()
throws IOException {
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();) {
StorageDirectory sd = it.next();
if (!sd.getVersionFile().exists()) {
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");
}
} else {
inspector = new FSImagePreTransactionalStorageInspector();
}
inspectStorageDirs(inspector);
return inspector;
}
}

View File

@ -0,0 +1,151 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundEditLog;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundFSImage;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import com.google.common.collect.Lists;
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 {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
storage.inspectStorageDirs(inspector);
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 {
@Override
public void purgeLog(FoundEditLog log) {
deleteOrWarn(log.getFile());
}
@Override
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);
}
}
}
}

View File

@ -78,6 +78,7 @@
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.NodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
@ -382,7 +383,7 @@ NamenodeRegistration setRegistration() {
nodeRegistration = new NamenodeRegistration(
getHostPortString(rpcAddress),
getHostPortString(getHttpAddress()),
getFSImage().getStorage(), getRole(), getFSImage().getStorage().getCheckpointTime());
getFSImage().getStorage(), getRole());
return nodeRegistration;
}
@ -647,8 +648,9 @@ public void errorReport(NamenodeRegistration registration,
public NamenodeRegistration register(NamenodeRegistration registration)
throws IOException {
verifyVersion(registration.getVersion());
namesystem.registerBackupNode(registration);
return setRegistration();
NamenodeRegistration myRegistration = setRegistration();
namesystem.registerBackupNode(registration, myRegistration);
return myRegistration;
}
@Override // NamenodeProtocol
@ -669,22 +671,6 @@ public void endCheckpoint(NamenodeRegistration registration,
namesystem.endCheckpoint(registration, sig);
}
@Override // NamenodeProtocol
public long journalSize(NamenodeRegistration registration)
throws IOException {
verifyRequest(registration);
return namesystem.getEditLogSize();
}
@Override // NamenodeProtocol
public void journal(NamenodeRegistration registration,
int jAction,
int length,
byte[] args) throws IOException {
// Active name-node cannot journal.
throw new UnsupportedActionException("journal");
}
@Override // ClientProtocol
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
throws IOException {
@ -1056,21 +1042,20 @@ public void refreshNodes() throws IOException {
namesystem.refreshNodes(new HdfsConfiguration());
}
@Deprecated // NamenodeProtocol
public long getEditLogSize() throws IOException {
return namesystem.getEditLogSize();
@Override // NamenodeProtocol
public long getTransactionID() {
return namesystem.getTransactionID();
}
@Deprecated
@Override // NamenodeProtocol
public CheckpointSignature rollEditLog() throws IOException {
return namesystem.rollEditLog();
}
@Deprecated
@Override // NamenodeProtocol
public void rollFsImage(CheckpointSignature sig) throws IOException {
namesystem.rollFSImage(sig);
@Override
public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
throws IOException {
return namesystem.getEditLogManifest(sinceTxId);
}
@Override // ClientProtocol
@ -1279,26 +1264,11 @@ public void verifyVersion(int version) throws IOException {
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 @@ private static boolean format(Configuration conf,
}
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);
return false;
}
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"

View File

@ -23,11 +23,19 @@
import java.net.URI;
import java.security.PrivilegedAction;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
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;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -43,10 +51,11 @@
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
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.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.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@ -60,6 +69,9 @@
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.StringUtils;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
/**********************************************************
* 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 String infoBindAddress;
private FSNamesystem namesystem;
private Collection<URI> checkpointDirs;
private Collection<URI> checkpointEditsDirs;
/** How often to checkpoint regardless of number of txns */
private long checkpointPeriod; // in seconds
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 String toString() {
+ "\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;
}
@VisibleForTesting
FSImage getFSImage() {
return checkpointImage;
}
@VisibleForTesting
void setFSImage(CheckpointStorage image) {
this.checkpointImage = image;
}
@VisibleForTesting
NamenodeProtocol getNameNode() {
return namenode;
}
@VisibleForTesting
void setNameNode(NamenodeProtocol namenode) {
this.namenode = namenode;
}
@VisibleForTesting
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 {
try {
NameNode.initializeGenericKeys(conf);
initialize(conf);
initialize(conf, commandLineOpts);
} catch(IOException e) {
shutdown();
LOG.fatal("Failed to start secondary namenode. ", e);
@ -143,8 +188,10 @@ public static InetSocketAddress getHttpAddress(Configuration conf) {
/**
* Initialize SecondaryNameNode.
* @param commandLineOpts
*/
private void initialize(final Configuration conf) throws IOException {
private void initialize(final Configuration conf,
CommandLineOpts commandLineOpts) throws IOException {
final InetSocketAddress infoSocAddr = getHttpAddress(conf);
infoBindAddress = infoSocAddr.getHostName();
UserGroupInformation.setConfiguration(conf);
@ -171,14 +218,19 @@ private void initialize(final Configuration conf) throws IOException {
"/tmp/hadoop/dfs/namesecondary");
checkpointEditsDirs = FSImage.getCheckpointEditsDirs(conf,
"/tmp/hadoop/dfs/namesecondary");
checkpointImage = new CheckpointStorage(conf);
checkpointImage.recoverCreate(checkpointDirs, checkpointEditsDirs);
checkpointImage = new CheckpointStorage(conf, checkpointDirs, checkpointEditsDirs);
checkpointImage.recoverCreate(commandLineOpts.shouldFormat());
// Initialize other scheduling parameters from the configuration
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 HttpServer run() throws IOException, InterruptedException {
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 HttpServer run() throws IOException, InterruptedException {
// 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 " +
"(" + checkpointPeriod/60 + " min)");
LOG.warn("Log Size Trigger :" + checkpointSize + " bytes " +
"(" + checkpointSize/1024 + " KB)");
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 Object run() {
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) {
try {
@ -307,8 +369,7 @@ public void doWork() {
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 void doWork() {
} 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);
e.printStackTrace();
@ -331,49 +391,53 @@ public void doWork() {
* @return true if a new image has been downloaded and needs to be loaded
* @throws IOException
*/
private boolean downloadCheckpointFiles(final CheckpointSignature sig
) throws IOException {
static boolean downloadCheckpointFiles(
final String nnHostPort,
final FSImage dstImage,
final CheckpointSignature sig,
final RemoteEditLogManifest manifest
) throws IOException {
// 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);
}
try {
Boolean b = UserGroupInformation.getCurrentUser().doAs(
new PrivilegedExceptionAction<Boolean>() {
@Override
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.");
} else {
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.");
MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage(
nnHostPort, sig.mostRecentCheckpointTxId, dstImage.getStorage(), true);
dstImage.saveDigestAndRenameCheckpointImage(
sig.mostRecentCheckpointTxId, downloadedHash);
}
// get edits file
fileid = "getedit=1";
list = getFSImage().getStorage().getFiles(
NameNodeFile.EDITS, NameNodeDirType.EDITS);
srcNames = list.toArray(new File[list.size()]);;
assert srcNames.length > 0 : "No checkpoint targets.";
TransferFsImage.getFileClient(fsName, fileid, srcNames, false);
LOG.info("Downloaded file " + srcNames[0].getName() + " size " +
srcNames[0].length() + " bytes.");
for (RemoteEditLog log : manifest.getLogs()) {
TransferFsImage.downloadEditsToStorage(
nnHostPort, log, dstImage.getStorage());
}
checkpointImage.checkpointUploadDone();
return Boolean.valueOf(downloadImage);
}
});
@ -387,18 +451,6 @@ InetSocketAddress getNameNodeAddress() {
return nameNodeAddr;
}
/**
* Copy the new fsimage into the NameNode
*/
private void putFSImage(CheckpointSignature sig) throws IOException {
String fileid = "putimage=1&port=" + imagePort +
"&machine=" + infoBindAddress +
"&token=" + sig.toString() +
"&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 @@ private String getInfoServer() throws IOException {
return configuredAddress;
}
}
/**
* Return the host:port of where this SecondaryNameNode is listening
* for image transfers
*/
private InetSocketAddress getImageListenAddress() {
return new InetSocketAddress(infoBindAddress, imagePort);
}
/**
* Create a new checkpoint
* @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();
// Make sure we're talking to the same NN!
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);
} else {
// 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 @@ boolean doCheckpoint() throws IOException {
"after creating edits.new");
}
boolean loadImage = downloadCheckpointFiles(sig); // Fetch fsimage and edits
doMerge(sig, loadImage); // Do the merge
RemoteEditLogManifest manifest =
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);
// error simulation code for junit test
if (ErrorSimulator.getErrorSimulation(1)) {
@ -458,91 +536,53 @@ boolean doCheckpoint() throws IOException {
"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.recoverCreate(checkpointDirs, checkpointEditsDirs);
checkpointImage.startCheckpoint();
}
/**
* Merge downloaded image and edits and write the new image into
* current storage directory.
*/
private void doMerge(CheckpointSignature sig, boolean loadImage)
throws IOException {
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;
private int processStartupCommand(CommandLineOpts opts) throws Exception {
if (opts.getCommand() == null) {
return 0;
}
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) {
printUsage(cmd);
return exitCode;
}
if (argv.length == 2 && !"force".equals(argv[i])) {
printUsage(cmd);
return exitCode;
}
}
exitCode = 0;
String cmd = opts.getCommand().toString().toLowerCase();
int exitCode = 0;
try {
if ("-checkpoint".equals(cmd)) {
long size = namenode.getEditLogSize();
if (size >= checkpointSize ||
argv.length == 2 && "force".equals(argv[i])) {
switch (opts.getCommand()) {
case CHECKPOINT:
long count = countUncheckpointedTxns();
if (count > checkpointTxnCount ||
opts.shouldForceCheckpoint()) {
doCheckpoint();
} else {
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)) {
long size = namenode.getEditLogSize();
System.out.println("EditLog size is " + size + " bytes");
} else {
exitCode = -1;
LOG.error(cmd.substring(1) + ": Unknown command");
printUsage("");
break;
case GETEDITSIZE:
long uncheckpointed = countUncheckpointedTxns();
System.out.println("NameNode has " + uncheckpointed +
" uncheckpointed transactions");
break;
default:
throw new AssertionError("bad command enum: " + opts.getCommand());
}
} catch (RemoteException e) {
//
// This is a error returned by hadoop server. Print
@ -551,41 +591,32 @@ private int processArgs(String[] argv) throws Exception {
try {
String[] content;
content = e.getLocalizedMessage().split("\n");
LOG.error(cmd.substring(1) + ": "
+ content[0]);
LOG.error(cmd + ": " + content[0]);
} catch (Exception ex) {
LOG.error(cmd.substring(1) + ": "
+ ex.getLocalizedMessage());
LOG.error(cmd + ": " + ex.getLocalizedMessage());
}
} catch (IOException e) {
//
// IO exception encountered locally.
//
exitCode = -1;
LOG.error(cmd.substring(1) + ": "
+ e.getLocalizedMessage());
LOG.error(cmd + ": " + e.getLocalizedMessage());
} finally {
// 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) {
if ("-geteditsize".equals(cmd)) {
System.err.println("Usage: java SecondaryNameNode"
+ " [-geteditsize]");
} else if ("-checkpoint".equals(cmd)) {
System.err.println("Usage: java SecondaryNameNode"
+ " [-checkpoint [force]]");
} else {
System.err.println("Usage: java SecondaryNameNode " +
"[-checkpoint [force]] " +
"[-geteditsize] ");
}
private long countUncheckpointedTxns() throws IOException {
long curTxId = namenode.getTransactionID();
long uncheckpointedTxns = curTxId -
checkpointImage.getStorage().getMostRecentCheckpointTxId();
assert uncheckpointedTxns >= 0;
return uncheckpointedTxns;
}
boolean shouldCheckpointBasedOnCount() throws IOException {
return countUncheckpointedTxns() >= checkpointTxnCount;
}
/**
@ -594,41 +625,151 @@ private void printUsage(String cmd) {
* @exception Exception if the filesystem does not exist.
*/
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();
try {
opts.parse(argv);
} catch (ParseException pe) {
LOG.error(pe.getMessage());
opts.usage();
return null;
}
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.
*
* @throws IOException
*/
void recoverCreate(Collection<URI> dataDirs,
Collection<URI> editsDirs) throws IOException {
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.attemptRestoreRemovedStorage();
storage.unlockAll();
for (Iterator<StorageDirectory> it =
storage.dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next();
@ -643,6 +784,13 @@ void recoverCreate(Collection<URI> dataDirs,
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();
}
StorageState curState;
try {
curState = sd.analyzeStorage(HdfsConstants.StartupOption.REGULAR, storage);
@ -655,6 +803,11 @@ void recoverCreate(Collection<URI> dataDirs,
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);
break;
default: // recovery is possible
sd.doRecover(curState);
@ -665,63 +818,41 @@ void recoverCreate(Collection<URI> dataDirs,
}
}
}
/**
* Prepare directories for a new checkpoint.
* <p>
* Rename <code>current</code> to <code>lastcheckpoint.tmp</code>
* and recreate <code>current</code>.
* @throws IOException
* 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();) {
StorageDirectory sd = it.next();
storage.moveCurrent(sd);
}
}
void endCheckpoint() throws IOException {
for (Iterator<StorageDirectory> it
= storage.dirIterator(); it.hasNext();) {
StorageDirectory sd = it.next();
storage.moveLastCheckpoint(sd);
}
}
/**
* Merge image and edits, and verify consistency with the signature.
*/
private void doMerge(CheckpointSignature sig, boolean loadImage)
throws IOException {
getEditLog().open();
StorageDirectory sdName = null;
StorageDirectory sdEdits = null;
Iterator<StorageDirectory> it = null;
if (loadImage) {
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);
if (it.hasNext())
sdEdits = it.next();
if (sdEdits == null)
throw new IOException("Could not locate checkpoint edits");
if (loadImage) {
// to avoid assert in loadFSImage()
this.getStorage().layoutVersion = -1;
getStorage();
loadFSImage(NNStorage.getStorageFile(sdName, NameNodeFile.IMAGE));
}
loadFSEdits(sdEdits);
storage.setClusterID(sig.getClusterID());
storage.setBlockPoolID(sig.getBlockpoolID());
sig.validateStorageInfo(this);
saveNamespace(false);
}
}
static void doMerge(
CheckpointSignature sig, RemoteEditLogManifest manifest,
boolean loadImage, FSImage dstImage) throws IOException {
NNStorage dstStorage = dstImage.getStorage();
dstStorage.setStorageInfo(sig);
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");
}
dstImage.reloadFromImageFile(file);
}
Checkpointer.rollForwardByApplyingLogs(manifest, dstImage);
dstImage.saveFSImageInAllDirs(dstImage.getLastAppliedTxId());
dstStorage.writeAll();
}
}

View File

@ -21,19 +21,22 @@
import java.net.*;
import java.security.DigestInputStream;
import java.security.MessageDigest;
import java.util.Iterator;
import java.util.Map;
import java.util.List;
import java.lang.Math;
import javax.servlet.http.HttpServletResponse;
import javax.servlet.http.HttpServletRequest;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
import org.apache.hadoop.hdfs.DFSUtil.ErrorSimulator;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.security.UserGroupInformation;
import com.google.common.collect.Lists;
/**
* This class provides fetching a specified file from the NameNode.
@ -41,88 +44,77 @@
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
* @throws IOException
*/
public TransferFsImage(Map<String,String[]> pmap,
HttpServletRequest request,
HttpServletResponse response
) throws IOException {
isGetImage = isGetEdit = isPutImage = false;
remoteport = 0;
machineName = null;
token = null;
public final static String MD5_HEADER = "X-MD5-Digest";
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]);
private static final Log LOG = LogFactory.getLog(TransferFsImage.class);
static MD5Hash downloadImageToStorage(
String fsName, long imageTxId, NNStorage dstStorage, boolean needDigest)
throws IOException {
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(
log.getStartTxId(), log.getEndTxId());
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);
return;
} else {
LOG.debug("Dest file: " + f);
}
}
int numGets = (isGetImage?1:0) + (isGetEdit?1:0);
if ((numGets > 1) || (numGets == 0) && !isPutImage) {
throw new IOException("Illegal parameters to TransferFsImage");
}
getFileClient(fsName, fileid, dstFiles, dstStorage, false);
LOG.info("Downloaded file " + dstFiles.get(0).getName() + " size " +
dstFiles.get(0).length() + " bytes.");
}
boolean getEdit() {
return isGetEdit;
}
boolean getImage() {
return isGetImage;
}
boolean putImage() {
return isPutImage;
}
CheckpointSignature getToken() {
return token;
}
/**
* 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 @@ static void getFileServer(OutputStream outstream, File localfile,
if (num <= 0) {
break;
}
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 @@ static void getFileServer(OutputStream outstream, File localfile,
/**
* 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)
throws IOException {
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 @@ static MD5Hash getFileClient(String fsName, String id, File[] localPath,
// 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) {
throw new IOException(
"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 @@ static MD5Hash getFileClient(String fsName, String id, File[] localPath,
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 @@ static MD5Hash getFileClient(String fsName, String id, File[] localPath,
digester = MD5Hash.getDigester();
stream = new DigestInputStream(stream, digester);
}
FileOutputStream[] output = null;
boolean finishedReceiving = false;
List<FileOutputStream> outputStreams = Lists.newArrayList();
try {
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) {
try {
if (f.exists()) {
LOG.warn("Overwriting existing file " + f
+ " with file downloaded from " + str);
}
outputStreams.add(new FileOutputStream(f));
} catch (IOException ioe) {
LOG.warn("Unable to download file " + f, ioe);
dstStorage.reportErrorOnFile(f);
}
}
if (outputStreams.isEmpty()) {
throw new IOException(
"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;
for (int i = 0; i < output.length; i++) {
output[i].write(buf, 0, num);
for (FileOutputStream fos : outputStreams) {
fos.write(buf, 0, num);
}
}
}
finishedReceiving = true;
} finally {
stream.close();
if (output != null) {
for (int i = 0; i < output.length; i++) {
if (output[i] != null) {
output[i].getChannel().force(true);
output[i].close();
}
}
for (FileOutputStream fos : outputStreams) {
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 @@ static MD5Hash getFileClient(String fsName, String id, File[] localPath,
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;
} else {
return null;
}
}
private static MD5Hash parseMD5Header(HttpURLConnection connection) {
String header = connection.getHeaderField(MD5_HEADER);
return (header != null) ? new MD5Hash(header) : null;
}
}

View File

@ -47,19 +47,16 @@
@InterfaceStability.Evolving
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 CheckpointSignature getSignature() {
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 boolean needToReturnImage() {
public void write(DataOutput out) throws IOException {
super.write(out);
cSig.write(out);
out.writeBoolean(isImageObsolete);
out.writeBoolean(needToReturnImage);
}
@ -112,7 +98,6 @@ public void readFields(DataInput in) throws IOException {
super.readFields(in);
cSig = new CheckpointSignature();
cSig.readFields(in);
isImageObsolete = in.readBoolean();
needToReturnImage = in.readBoolean();
}
}

View File

@ -0,0 +1,62 @@
/**
* 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.protocol;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSConfigKeys;
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)
@InterfaceAudience.Private
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 numTxns number of transactions
* @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;
}

View File

@ -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 BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
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.
* @throws IOException
* @deprecated
* See {@link org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode}
*/
@Deprecated
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 BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
@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)
* @throws IOException
* @deprecated
* See {@link org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode}
*/
@Deprecated
public void rollFsImage(CheckpointSignature sig)
throws IOException;
/**
* Request name-node version and storage information.
*
@ -177,31 +153,14 @@ public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
*/
public void endCheckpoint(NamenodeRegistration registration,
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.
* @throws IOException
* 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
*/
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
* @throws IOException
*/
public void journal(NamenodeRegistration registration,
int jAction,
int length,
byte[] records) throws IOException;
public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
throws IOException;
}

View File

@ -43,7 +43,6 @@ public class NamenodeRegistration extends StorageInfo
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 @@ public NamenodeRegistration() {
public NamenodeRegistration(String address,
String httpAddress,
StorageInfo storageInfo,
NamenodeRole role,
long checkpointTime) {
NamenodeRole role) {
super();
this.rpcAddress = address;
this.httpAddress = httpAddress;
this.setStorageInfo(storageInfo);
this.role = role;
this.checkpointTime = checkpointTime;
}
@Override // NodeRegistration
@ -96,13 +93,6 @@ public boolean isRole(NamenodeRole that) {
return role.equals(that);
}
/**
* Get the age of the image.
*/
public long getCheckpointTime() {
return checkpointTime;
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
@ -120,7 +110,6 @@ public void write(DataOutput out) throws IOException {
Text.writeString(out, httpAddress);
Text.writeString(out, role.name());
super.write(out);
out.writeLong(checkpointTime);
}
@Override // Writable
@ -129,6 +118,5 @@ public void readFields(DataInput in) throws IOException {
httpAddress = Text.readString(in);
role = NamenodeRole.valueOf(Text.readString(in));
super.readFields(in);
checkpointTime = in.readLong();
}
}

View File

@ -27,6 +27,7 @@
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.hdfs.DeprecatedUTF8;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
@ -98,4 +99,26 @@ public void readFields(DataInput in) throws IOException {
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() + ".");
}
}
}

View File

@ -0,0 +1,64 @@
/**
* 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.protocol;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.io.Writable;
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.startTxId = startTxId;
this.endTxId = endTxId;
}
public long getStartTxId() {
return startTxId;
}
public long getEndTxId() {
return endTxId;
}
@Override
public String toString() {
return "[" + startTxId + "," + endTxId + "]";
}
@Override
public void write(DataOutput out) throws IOException {
out.writeLong(startTxId);
out.writeLong(endTxId);
}
@Override
public void readFields(DataInput in) throws IOException {
startTxId = in.readLong();
endTxId = in.readLong();
}
}

View File

@ -0,0 +1,99 @@
/**
* 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.protocol;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.io.Writable;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
/**
* 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);
}
@Override
public String toString() {
return "[" + Joiner.on(", ").join(logs) + "]";
}
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(logs.size());
for (RemoteEditLog log : logs) {
log.write(out);
}
}
@Override
public void readFields(DataInput in) throws IOException {
int numLogs = in.readInt();
logs = Lists.newArrayList();
for (int i = 0; i < numLogs; i++) {
RemoteEditLog log = new RemoteEditLog();
log.readFields(in);
logs.add(log);
}
checkState();
}
}

View File

@ -31,6 +31,7 @@ public enum EditsElement {
EDITS_VERSION,
RECORD,
OPCODE,
TRANSACTION_ID,
DATA,
// elements in the data part of the editLog records
LENGTH,

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
import java.io.EOFException;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -40,7 +41,7 @@
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 @@ public boolean canLoadVersion(int version) {
return false;
}
/**
* 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 @@ private void visit_OP_CLOSE() throws IOException {
*/
private void visit_OP_ADD_or_OP_CLOSE(FSEditLogOpCodes editsOpCode)
throws IOException {
visitTxId();
IntToken opAddLength = v.visitInt(EditsElement.LENGTH);
// this happens if the edits is not properly ended (-1 op code),
@ -135,6 +145,8 @@ private void visit_OP_ADD_or_OP_CLOSE(FSEditLogOpCodes editsOpCode)
* Visit OP_RENAME_OLD
*/
private void visit_OP_RENAME_OLD() throws IOException {
visitTxId();
v.visitInt( EditsElement.LENGTH);
v.visitStringUTF8( EditsElement.SOURCE);
v.visitStringUTF8( EditsElement.DESTINATION);
@ -145,6 +157,8 @@ private void visit_OP_RENAME_OLD() throws IOException {
* Visit OP_DELETE
*/
private void visit_OP_DELETE() throws IOException {
visitTxId();
v.visitInt( EditsElement.LENGTH);
v.visitStringUTF8( EditsElement.PATH);
v.visitStringUTF8( EditsElement.TIMESTAMP);
@ -154,6 +168,8 @@ private void visit_OP_DELETE() throws IOException {
* Visit OP_MKDIR
*/
private void visit_OP_MKDIR() throws IOException {
visitTxId();
v.visitInt( EditsElement.LENGTH);
v.visitStringUTF8( EditsElement.PATH);
v.visitStringUTF8( EditsElement.TIMESTAMP);
@ -172,6 +188,8 @@ private void visit_OP_MKDIR() throws IOException {
* Visit OP_SET_REPLICATION
*/
private void visit_OP_SET_REPLICATION() throws IOException {
visitTxId();
v.visitStringUTF8(EditsElement.PATH);
v.visitStringUTF8(EditsElement.REPLICATION);
}
@ -180,6 +198,8 @@ private void visit_OP_SET_REPLICATION() throws IOException {
* Visit OP_SET_PERMISSIONS
*/
private void visit_OP_SET_PERMISSIONS() throws IOException {
visitTxId();
v.visitStringUTF8( EditsElement.PATH);
v.visitShort( EditsElement.FS_PERMISSIONS);
}
@ -188,6 +208,8 @@ private void visit_OP_SET_PERMISSIONS() throws IOException {
* Visit OP_SET_OWNER
*/
private void visit_OP_SET_OWNER() throws IOException {
visitTxId();
v.visitStringUTF8(EditsElement.PATH);
v.visitStringUTF8(EditsElement.USERNAME);
v.visitStringUTF8(EditsElement.GROUPNAME);
@ -197,6 +219,8 @@ private void visit_OP_SET_OWNER() throws IOException {
* Visit OP_SET_GENSTAMP
*/
private void visit_OP_SET_GENSTAMP() throws IOException {
visitTxId();
v.visitLong(EditsElement.GENERATION_STAMP);
}
@ -204,6 +228,8 @@ private void visit_OP_SET_GENSTAMP() throws IOException {
* Visit OP_TIMES
*/
private void visit_OP_TIMES() throws IOException {
visitTxId();
v.visitInt( EditsElement.LENGTH);
v.visitStringUTF8( EditsElement.PATH);
v.visitStringUTF8( EditsElement.MTIME);
@ -214,6 +240,8 @@ private void visit_OP_TIMES() throws IOException {
* Visit OP_SET_QUOTA
*/
private void visit_OP_SET_QUOTA() throws IOException {
visitTxId();
v.visitStringUTF8( EditsElement.PATH);
v.visitLong( EditsElement.NS_QUOTA);
v.visitLong( EditsElement.DS_QUOTA);
@ -223,6 +251,8 @@ private void visit_OP_SET_QUOTA() throws IOException {
* Visit OP_RENAME
*/
private void visit_OP_RENAME() throws IOException {
visitTxId();
v.visitInt( EditsElement.LENGTH);
v.visitStringUTF8( EditsElement.SOURCE);
v.visitStringUTF8( EditsElement.DESTINATION);
@ -234,6 +264,8 @@ private void visit_OP_RENAME() throws IOException {
* Visit OP_CONCAT_DELETE
*/
private void visit_OP_CONCAT_DELETE() throws IOException {
visitTxId();
IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
v.visitStringUTF8(EditsElement.CONCAT_TARGET);
// all except of CONCAT_TARGET and TIMESTAMP
@ -248,6 +280,8 @@ private void visit_OP_CONCAT_DELETE() throws IOException {
* Visit OP_SYMLINK
*/
private void visit_OP_SYMLINK() throws IOException {
visitTxId();
v.visitInt( EditsElement.LENGTH);
v.visitStringUTF8( EditsElement.SOURCE);
v.visitStringUTF8( EditsElement.DESTINATION);
@ -267,6 +301,8 @@ private void visit_OP_SYMLINK() throws IOException {
* Visit OP_GET_DELEGATION_TOKEN
*/
private void visit_OP_GET_DELEGATION_TOKEN() throws IOException {
visitTxId();
v.visitByte( EditsElement.T_VERSION);
v.visitStringText( EditsElement.T_OWNER);
v.visitStringText( EditsElement.T_RENEWER);
@ -283,6 +319,8 @@ private void visit_OP_GET_DELEGATION_TOKEN() throws IOException {
*/
private void visit_OP_RENEW_DELEGATION_TOKEN()
throws IOException {
visitTxId();
v.visitByte( EditsElement.T_VERSION);
v.visitStringText( EditsElement.T_OWNER);
v.visitStringText( EditsElement.T_RENEWER);
@ -299,6 +337,8 @@ private void visit_OP_RENEW_DELEGATION_TOKEN()
*/
private void visit_OP_CANCEL_DELEGATION_TOKEN()
throws IOException {
visitTxId();
v.visitByte( EditsElement.T_VERSION);
v.visitStringText( EditsElement.T_OWNER);
v.visitStringText( EditsElement.T_RENEWER);
@ -314,6 +354,8 @@ private void visit_OP_CANCEL_DELEGATION_TOKEN()
*/
private void visit_OP_UPDATE_MASTER_KEY()
throws IOException {
visitTxId();
v.visitVInt( EditsElement.KEY_ID);
v.visitVLong( EditsElement.KEY_EXPIRY_DATE);
VIntToken blobLengthToken = v.visitVInt(EditsElement.KEY_LENGTH);
@ -322,11 +364,29 @@ private void visit_OP_UPDATE_MASTER_KEY()
private void visit_OP_REASSIGN_LEASE()
throws IOException {
visitTxId();
v.visitStringUTF8(EditsElement.CLIENT_NAME);
v.visitStringUTF8(EditsElement.PATH);
v.visitStringUTF8(EditsElement.CLIENT_NAME);
}
/**
* Visit OP_BEGIN_LOG_SEGMENT
*/
private void visit_OP_BEGIN_LOG_SEGMENT()
throws IOException {
visitTxId();
}
/**
* Visit OP_END_LOG_SEGMENT
*/
private void visit_OP_END_LOG_SEGMENT()
throws IOException {
visitTxId();
}
private void visitOpCode(FSEditLogOpCodes editsOpCode)
throws IOException {
@ -391,6 +451,12 @@ private void visitOpCode(FSEditLogOpCodes editsOpCode)
case OP_REASSIGN_LEASE: // 22
visit_OP_REASSIGN_LEASE();
break;
case OP_END_LOG_SEGMENT: // 23
visit_OP_END_LOG_SEGMENT();
break;
case OP_START_LOG_SEGMENT: // 24
visit_OP_BEGIN_LOG_SEGMENT();
break;
default:
{
throw new IOException("Unknown op code " + editsOpCode);
@ -419,7 +485,17 @@ public void loadEdits() throws IOException {
do {
v.visitEnclosingElement(EditsElement.RECORD);
ByteToken opCodeToken = v.visitByte(EditsElement.OPCODE);
ByteToken opCodeToken;
try {
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);

View File

@ -17,18 +17,12 @@
*/
package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
import java.io.BufferedInputStream;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;

View File

@ -104,6 +104,10 @@ public void fromString(String s) throws IOException {
public void fromBinary(DataInputStream in) throws IOException {
value = in.readByte();
}
public void fromByte(byte b) {
value = b;
}
@Override
public String toString() {

View File

@ -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 @@ public void loadImage(DataInputStream in, ImageVisitor v,
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);

View File

@ -71,7 +71,8 @@ public enum ImageElement {
NUM_DELEGATION_TOKENS,
DELEGATION_TOKENS,
DELEGATION_TOKEN_IDENTIFIER,
DELEGATION_TOKEN_EXPIRY_TIME
DELEGATION_TOKEN_EXPIRY_TIME,
TRANSACTION_ID
}
/**

View File

@ -0,0 +1,94 @@
/**
* 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.util;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.FilterOutputStream;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.IOUtils;
/**
* 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();
}
@Override
public void close() throws IOException {
boolean triedToClose = false, success = false;
try {
flush();
((FileOutputStream)out).getChannel().force(true);
triedToClose = true;
super.close();
success = true;
} finally {
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);
}
}
} else {
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);
}
}
}
}
}

View File

@ -0,0 +1,153 @@
/**
* 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.util;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
import java.security.DigestInputStream;
import java.security.MessageDigest;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.util.StringUtils;
/**
* Static functions for dealing with files of the same format
* that the Unix "md5sum" utility writes.
*/
public abstract class MD5FileUtils {
private static final Log LOG = LogFactory.getLog(
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.
* @throws IOException
*/
public static void verifySavedMD5(File dataFile, MD5Hash expectedMD5)
throws IOException {
MD5Hash storedHash = readStoredMd5ForFile(dataFile);
// Check the hash itself
if (!expectedMD5.equals(storedHash)) {
throw new IOException(
"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()) {
return null;
}
BufferedReader reader =
new BufferedReader(new FileReader(md5File));
try {
md5Line = reader.readLine();
if (md5Line == null) { md5Line = ""; }
md5Line = md5Line.trim();
} catch (IOException ioe) {
throw new IOException("Error reading md5 file at " + md5File, ioe);
} finally {
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())) {
throw new IOException(
"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);
try {
MessageDigest digester = MD5Hash.getDigester();
DigestInputStream dis = new DigestInputStream(in, digester);
IOUtils.copyBytes(dis, new IOUtils.NullOutputStream(), 128*1024);
return new MD5Hash(digester.digest());
} finally {
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
* @throws IOException
*/
public static void saveMD5File(File dataFile, MD5Hash digest)
throws IOException {
File md5File = getDigestFileForFile(dataFile);
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);
}
}

View File

@ -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.

View File

@ -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 Builder nameNodePort(int val) {
this.nameNodePort = val;
return this;
}
/**
* Default: 0
*/
public Builder nameNodeHttpPort(int val) {
this.nameNodeHttpPort = val;
return this;
}
/**
* Default: 1
@ -247,6 +256,7 @@ private MiniDFSCluster(Builder builder) throws IOException {
builder.federation = true;
initMiniDFSCluster(builder.nameNodePort,
builder.nameNodeHttpPort,
builder.conf,
builder.numDataNodes,
builder.format,
@ -473,12 +483,13 @@ public MiniDFSCluster(int nameNodePort,
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,
Configuration conf,
int numDataNodes, boolean format, boolean manageNameDfsDirs,
boolean manageDataDfsDirs, StartupOption operation, String[] racks,
String[] hosts, long[] simulatedCapacities, String clusterId,
@ -526,7 +537,8 @@ private void initMiniDFSCluster(int nameNodePort, Configuration conf,
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);

View File

@ -18,14 +18,18 @@
package org.apache.hadoop.hdfs;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
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.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import com.google.common.collect.Lists;
/**
* This test ensures the appropriate response from the system when
@ -57,14 +61,17 @@ void log(String label, int numDirs) {
* 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(

View File

@ -22,6 +22,8 @@
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import junit.framework.TestCase;
@ -32,8 +34,11 @@
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.util.StringUtils;
import com.google.common.collect.Lists;
/**
* 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 @@ void log(String label, int numDirs) {
* 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());
}
break;
case DATA_NODE:
for (int i = 0; i < baseDirs.length; i++) {
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:
FSImageTestUtil.assertReasonableNameCurrentDir(curDir);
break;
case DATA_NODE:
assertEquals(
UpgradeUtilities.checksumContents(
nodeType, new File(baseDirs[i],"current")),
UpgradeUtilities.checksumMasterDataNodeContents());
UpgradeUtilities.checksumContents(nodeType, curDir),
UpgradeUtilities.checksumMasterDataNodeContents());
break;
}
break;
}
FSImageTestUtil.assertParallelFilesAreIdentical(
curDirs, Collections.<String>emptySet());
for (int i = 0; i < baseDirs.length; i++) {
assertFalse(new File(baseDirs[i],"previous").isDirectory());
}
@ -241,21 +246,17 @@ public void testRollback() throws Exception {
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);
UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
for (File f : baseDirs) {
FileUtil.fullyDelete(new File(f,"fsimage"));
}
deleteMatchingFiles(baseDirs, "fsimage_.*");
startNameNodeShouldFail(StartupOption.ROLLBACK,
"Image file is not found");
"No valid image files found");
UpgradeUtilities.createEmptyDirs(nameNodeDirs);
log("NameNode rollback with corrupt version file", numDirs);
@ -284,6 +285,16 @@ public void testRollback() throws Exception {
} // 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();

View File

@ -25,6 +25,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.NAME_NODE;
import static org.apache.hadoop.hdfs.server.common.HdfsConstants.NodeType.DATA_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 @@ void log(String label, int numDirs, int testCaseNum, boolean[] state) {
+ 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 @@ void log(String label, int numDirs, int testCaseNum, boolean[] state) {
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 @@ String[] createNameNodeStorageState(boolean[] state) throws Exception {
String[] createDataNodeStorageState(boolean[] state) throws Exception {
String[] baseDirs = conf.getStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
UpgradeUtilities.createEmptyDirs(baseDirs);
if (state[0]) // current
if (state[CURRENT_EXISTS]) // current
UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "current");
if (state[1]) // previous
if (state[PREVIOUS_EXISTS]) // previous
UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "previous");
if (state[2]) // previous.tmp
if (state[PREVIOUS_TMP_EXISTS]) // previous.tmp
UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "previous.tmp");
if (state[3]) // removed.tmp
if (state[REMOVED_TMP_EXISTS]) // removed.tmp
UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "removed.tmp");
if (state[4]) // lastcheckpoint.tmp
UpgradeUtilities.createDataNodeStorageDirs(baseDirs, "lastcheckpoint.tmp");
return baseDirs;
}
@ -189,19 +194,16 @@ String[] createBlockPoolStorageState(String bpid, boolean[] state) throws Except
// After copying the storage directories from master datanode, empty
// the block pool storage directories
String[] bpDirs = UpgradeUtilities.createEmptyBPDirs(baseDirs, bpid);
if (state[0]) // current
if (state[CURRENT_EXISTS]) // current
UpgradeUtilities.createBlockPoolStorageDirs(baseDirs, "current", bpid);
if (state[1]) // previous
if (state[PREVIOUS_EXISTS]) // previous
UpgradeUtilities.createBlockPoolStorageDirs(baseDirs, "previous", bpid);
if (state[2]) // previous.tmp
if (state[PREVIOUS_TMP_EXISTS]) // previous.tmp
UpgradeUtilities.createBlockPoolStorageDirs(baseDirs, "previous.tmp",
bpid);
if (state[3]) // removed.tmp
if (state[REMOVED_TMP_EXISTS]) // removed.tmp
UpgradeUtilities
.createBlockPoolStorageDirs(baseDirs, "removed.tmp", bpid);
if (state[4]) // lastcheckpoint.tmp
UpgradeUtilities.createBlockPoolStorageDirs(baseDirs,
"lastcheckpoint.tmp", bpid);
return bpDirs;
}
@ -220,9 +222,9 @@ void checkResultNameNode(String[] baseDirs,
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());
assertNotNull(FSImageTestUtil.findNewestImageFile(
baseDirs[i] + "/current"));
assertTrue(new File(baseDirs[i],"current/seen_txid").isFile());
}
}
if (previousShouldExist) {
@ -318,9 +320,9 @@ public void testNNStorageStates() throws Exception {
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 void testNNStorageStates() throws Exception {
// 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 void testDNStorageStates() throws Exception {
conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
for (int i = 0; i < NUM_DN_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("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);
} else {
@ -403,16 +405,16 @@ public void testBlockPoolStorageStates() throws Exception {
conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
for (int i = 0; i < NUM_DN_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("BLOCK_POOL recovery", numDirs, i, testCase);
createNameNodeStorageState(new boolean[] { true, true, false, false,
false });
cluster = createCluster(conf);
baseDirs = createBlockPoolStorageState(bpid, 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);
} else {

View File

@ -27,15 +27,20 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
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.apache.hadoop.util.StringUtils;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import com.google.common.base.Joiner;
import static org.junit.Assert.*;
/**
@ -45,6 +50,7 @@
*/
public class TestDFSUpgrade {
private static final int EXPECTED_TXID = 17;
private static final Log LOG = LogFactory.getLog(TestDFSUpgrade.class.getName());
private Configuration conf;
private int testCounter = 0;
@ -66,15 +72,22 @@ void log(String label, int numDirs) {
* its files with their original checksum. It is assumed that the
* server has recovered and upgraded.
*/
void checkNameNode(String[] baseDirs) throws IOException {
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());
void checkNameNode(String[] baseDirs, long imageTxId) throws IOException {
for (String baseDir : baseDirs) {
LOG.info("Checking namenode directory " + baseDir);
LOG.info("==== Contents ====:\n " +
Joiner.on(" \n").join(new File(baseDir, "current").list()));
LOG.info("==================");
File previous = new File(baseDirs[i], "previous");
assertTrue(new File(baseDir,"current").isDirectory());
assertTrue(new File(baseDir,"current/VERSION").isFile());
assertTrue(new File(baseDir,"current/"
+ getInProgressEditsFileName(imageTxId + 1)).isFile());
assertTrue(new File(baseDir,"current/"
+ getImageFileName(imageTxId)).isFile());
assertTrue(new File(baseDir,"current/seen_txid").isFile());
File previous = new File(baseDir, "previous");
assertTrue(previous.isDirectory());
assertEquals(UpgradeUtilities.checksumContents(NAME_NODE, previous),
UpgradeUtilities.checksumMasterNameNodeContents());
@ -200,7 +213,6 @@ public void testUpgrade() throws Exception {
StorageInfo storageInfo = null;
for (int numDirs = 1; numDirs <= 2; numDirs++) {
conf = new HdfsConfiguration();
conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
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 void testUpgrade() throws Exception {
log("Normal NameNode upgrade", numDirs);
UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
cluster = createCluster();
checkNameNode(nameNodeDirs);
checkNameNode(nameNodeDirs, EXPECTED_TXID);
if (numDirs > 1)
TestParallelImageWrite.checkImages(cluster.getNamesystem(), numDirs);
cluster.shutdown();
@ -277,25 +289,21 @@ public void testUpgrade() throws Exception {
UpgradeUtilities.createEmptyDirs(dataNodeDirs);
log("NameNode upgrade with no edits file", numDirs);
baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
for (File f : baseDirs) {
FileUtil.fullyDelete(new File(f,"edits"));
}
UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
deleteStorageFilesWithPrefix(nameNodeDirs, "edits_");
startNameNodeShouldFail(StartupOption.UPGRADE);
UpgradeUtilities.createEmptyDirs(nameNodeDirs);
log("NameNode upgrade with no image file", numDirs);
baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
for (File f : baseDirs) {
FileUtil.fullyDelete(new File(f,"fsimage"));
}
UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
deleteStorageFilesWithPrefix(nameNodeDirs, "fsimage_");
startNameNodeShouldFail(StartupOption.UPGRADE);
UpgradeUtilities.createEmptyDirs(nameNodeDirs);
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"));
}
startNameNodeShouldFail(StartupOption.UPGRADE);
UpgradeUtilities.createEmptyDirs(nameNodeDirs);
@ -338,7 +346,7 @@ public void testUpgrade() throws Exception {
log("Normal NameNode upgrade", numDirs);
UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
cluster = createCluster();
checkNameNode(nameNodeDirs);
checkNameNode(nameNodeDirs, EXPECTED_TXID);
TestParallelImageWrite.checkImages(cluster.getNamesystem(), numDirs);
cluster.shutdown();
UpgradeUtilities.createEmptyDirs(nameNodeDirs);
@ -369,6 +377,19 @@ public void testUpgrade4() throws Exception {
}
}
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

View File

@ -32,6 +32,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.commons.logging.Log;
@ -183,38 +184,6 @@ private void verifyFileSystem(DistributedFileSystem dfs) throws IOException {
}
}
public void testUpgradeFromRel14Image() throws IOException {
unpackStorage();
MiniDFSCluster cluster = null;
try {
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.");
try {
Thread.sleep(1000);
} catch (InterruptedException ignored) {}
}
verifyFileSystem(dfs);
} finally {
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 void testFailOnPreUpgradeImage() throws IOException {
}
}
/**
* 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);
upgradeAndVerify();
}
/**
* 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");
FSImageTestUtil.corruptVersionFile(
new File(baseDir, "name2/current/VERSION"),
"imageMD5Digest", "22222222222222222222222222222222");
// Upgrade should now fail
try {
upgradeAndVerify();
fail("Upgrade did not fail with bad MD5");
} catch (IOException ioe) {
String msg = StringUtils.stringifyException(ioe);
if (!msg.contains("is corrupt with MD5 checksum")) {
throw ioe;
}
}
}
private void upgradeAndVerify() throws IOException {
MiniDFSCluster cluster = null;
try {
Configuration conf = new HdfsConfiguration();
@ -287,8 +296,12 @@ public void testUpgradeFromRel22Image() throws IOException {
Thread.sleep(1000);
} catch (InterruptedException ignored) {}
}
verifyFileSystem(dfs);
} finally {
if (cluster != null) { cluster.shutdown(); }
}
}
}
}

View File

@ -27,7 +27,6 @@
import java.io.IOException;
import java.io.OutputStream;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.logging.Log;
@ -45,11 +44,10 @@
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.junit.Test;
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 void testRecoverFinalizedBlock() throws Throwable {
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 void testCompleteOtherLeaseHoldersFile() throws Throwable {
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());
@ -290,42 +289,5 @@ public void run() {
} finally {
cluster.shutdown();
}
}
/**
* 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();
try {
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();
}
}
}

View File

@ -25,8 +25,10 @@
import java.io.IOException;
import java.io.OutputStream;
import java.io.RandomAccessFile;
import java.net.URI;
import java.util.Arrays;
import java.util.Random;
import java.util.Collections;
import java.util.zip.CRC32;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -407,7 +409,9 @@ public static File[] createBlockPoolStorageDirs(String[] parents,
*/
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++) {

View File

@ -0,0 +1,39 @@
/**
* 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.common;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
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;
}
}

View File

@ -190,11 +190,9 @@ public static void main(String[] args)
}
}
FSImage fsImage = new FSImage(editsLogDir.getAbsoluteFile().toURI());
FileNameGenerator nameGenerator = new FileNameGenerator(BASE_PATH, 100);
FSEditLog editLog = fsImage.getEditLog();
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);

View File

@ -0,0 +1,375 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundEditLog;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundFSImage;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.IOUtils;
import org.mockito.Mockito;
import com.google.common.base.Joiner;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
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)
throws IOException {
File tmpFile = File.createTempFile("hadoop_imagefile_tmp", "fsimage");
tmpFile.deleteOnExit();
try {
Files.copy(imageFile, tmpFile);
RandomAccessFile raf = new RandomAccessFile(tmpFile, "rw");
try {
raf.seek(IMAGE_TXID_POS);
raf.writeLong(0);
} finally {
IOUtils.closeStream(raf);
}
return getFileMD5(tmpFile);
} finally {
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();
return sd;
}
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 {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(mockStorageDirectory(dir, dirType));
return inspector;
}
/**
* 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)
throws IOException {
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) {
FSImageTransactionalStorageInspector inspector =
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 dir : dirs) {
for (File f : dir.listFiles()) {
if (ignoredFileNames.contains(f.getName())) {
continue;
}
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);
} else {
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
{
Map<File, String> md5s = getFileMD5s(files);
if (Sets.newHashSet(md5s.values()).size() != expectedUniqueHashes) {
fail("Expected " + expectedUniqueHashes + " different hashes, got:\n " +
Joiner.on("\n ")
.withKeyValueSeparator("=")
.join(md5s));
}
}
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 ret;
}
/**
* @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 ret;
}
/**
* @return the fsimage file with the most recent transaction ID in the
* given storage directory.
*/
public static File findLatestImageFile(StorageDirectory sd)
throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(sd);
return inspector.getLatestImage().getFile();
}
/**
* @return the fsimage file with the most recent transaction ID in the
* given 'current/' directory.
*/
public static File findNewestImageFile(String currentDirPath) throws IOException {
StorageDirectory sd = FSImageTestUtil.mockStorageDirectory(
new File(currentDirPath), NameNodeDirType.IMAGE);
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(sd);
FoundFSImage latestImage = inspector.getLatestImage();
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)
throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(sd);
List<FoundEditLog> foundEditLogs = Lists.newArrayList(
inspector.getFoundEditLogs());
return Collections.max(foundEditLogs, new Comparator<FoundEditLog>() {
@Override
public int compare(FoundEditLog a, FoundEditLog b) {
return ComparisonChain.start()
.compare(a.getStartTxId(), b.getStartTxId())
.compare(a.getLastTxId(), b.getLastTxId())
.result();
}
});
}
/**
* 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)
throws IOException {
Properties props = new Properties();
FileInputStream fis = new FileInputStream(versionFile);
FileOutputStream out = null;
try {
props.load(fis);
IOUtils.closeStream(fis);
props.setProperty(key, value);
out = new FileOutputStream(versionFile);
props.store(out, null);
} finally {
IOUtils.cleanup(null, fis, out);
}
}
public static void assertReasonableNameCurrentDir(File curDir)
throws IOException {
assertTrue(curDir.isDirectory());
assertTrue(new File(curDir, "VERSION").isFile());
assertTrue(new File(curDir, "seen_txid").isFile());
File image = findNewestImageFile(curDir.toString());
assertNotNull(image);
}
}

View File

@ -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 String generateEdits() throws IOException {
*
* @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);
StorageDirectory sd = it.next();
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 void shutdownCluster() throws IOException {
* 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 Object run() throws IOException {
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();
}
}

View File

@ -19,9 +19,12 @@
import java.io.File;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
@ -29,14 +32,28 @@
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImage.CheckpointStates;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundEditLog;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import junit.framework.TestCase;
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 @@ protected void setUp() throws Exception {
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)
throws IOException {
TestCheckpoint.writeFile(fileSys, name, repl);
}
static void checkFile(FileSystem fileSys, Path name, int repl)
throws IOException {
TestCheckpoint.checkFile(fileSys, name, repl);
}
void cleanupFile(FileSystem fileSys, Path name)
throws IOException {
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;
do {
try {
LOG.info("Waiting checkpoint to complete...");
LOG.info("Waiting checkpoint to complete... " +
"checkpoint txid should increase above " + txid);
Thread.sleep(1000);
} 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);
}
void testCheckpoint(StartupOption op) throws IOException {
Path file1 = new Path("checkpoint.dat");
Path file2 = new Path("checkpoint2.dat");
/**
* 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();
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);
MiniDFSCluster cluster = null;
FileSystem fileSys = null;
BackupNode backup = null;
try {
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(numDatanodes).build();
.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 =
nn.getFSImage().getStorage().getMostRecentCheckpointTxId();
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(),
nn.getFSImage().getEditLog().getCurSegmentTxId());
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
backup = startBackupNode(conf, StartupOption.BACKUP, 1);
testBNInSync(cluster, backup, 4);
assertNotNull(backup.getNamesystem().getFileInfo("/edit-while-bn-down", false));
} finally {
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>() {
@Override
public Boolean get() {
LOG.info("Checking for " + src + " on BN");
try {
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)
throws Exception {
// 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(),
null));
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");
Configuration conf = new HdfsConfiguration();
conf.set(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, "0");
conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // disable block scanner
conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 1);
MiniDFSCluster cluster = null;
FileSystem fileSys = null;
BackupNode backup = null;
try {
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(0).build();
fileSys = cluster.getFileSystem();
//
// verify that 'format' really blew away all pre-existing files
@ -130,14 +259,14 @@ void testCheckpoint(StartupOption op) throws IOException {
//
// 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);
} catch(IOException e) {
LOG.error("Error in TestBackupNode:", e);
assertTrue(e.getLocalizedMessage(), false);
@ -146,32 +275,46 @@ void testCheckpoint(StartupOption op) throws IOException {
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/");
FSImageTestUtil.assertParallelFilesAreIdentical(
ImmutableList.of(bnCurDir, nnCurDir),
ImmutableSet.<String>of("VERSION"));
try {
//
// 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();
fileSys = cluster.getFileSystem();
// check that file1 still exists
checkFile(fileSys, file1, replication);
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);
//
// Take a checkpoint
//
backup = startBackupNode(conf, op, 1);
waitCheckpointDone(backup);
long txid = cluster.getNameNode().getTransactionID();
waitCheckpointDone(cluster, backup, txid);
for (int i = 0; i < 10; i++) {
fileSys.mkdirs(new Path("file_" + i));
}
txid = cluster.getNameNode().getTransactionID();
backup.doCheckpoint();
waitCheckpointDone(cluster, backup, txid);
txid = cluster.getNameNode().getTransactionID();
backup.doCheckpoint();
waitCheckpointDone(cluster, backup, txid);
} catch(IOException e) {
LOG.error("Error in TestBackupNode:", e);
assertTrue(e.getLocalizedMessage(), false);
@ -180,22 +323,22 @@ void testCheckpoint(StartupOption op) throws IOException {
if(fileSys != null) fileSys.close();
if(cluster != null) cluster.shutdown();
}
LOG.info("NameNode fsimage length = " + imageFileNN.length());
LOG.info("Backup Node fsimage length = " + imageFileBN.length());
assertTrue(imageFileNN.length() == imageFileBN.length());
FSImageTestUtil.assertParallelFilesAreIdentical(
ImmutableList.of(bnCurDir, nnCurDir),
ImmutableSet.<String>of("VERSION"));
try {
//
// 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();
fileSys = cluster.getFileSystem();
assertTrue(!fileSys.exists(file1));
// verify that file2 exists
checkFile(fileSys, file2, replication);
assertTrue(fileSys.exists(file2));
} catch(IOException e) {
LOG.error("Error in TestBackupNode:", e);
assertTrue(e.getLocalizedMessage(), false);
@ -204,52 +347,4 @@ void testCheckpoint(StartupOption op) throws IOException {
cluster.shutdown();
}
}
/**
* Test that only one backup node can register.
* @throws IOException
*/
public void testBackupRegistration() throws IOException {
Configuration conf1 = new HdfsConfiguration();
Configuration conf2 = null;
MiniDFSCluster cluster = null;
BackupNode backup1 = null;
BackupNode backup2 = null;
try {
// 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");
try {
backup2 = startBackupNode(conf2, StartupOption.BACKUP, 2);
backup2.stop();
backup2 = null;
assertTrue("Only one backup node should be able to start", false);
} catch(IOException e) {
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;
try {
backup2 = startBackupNode(conf2, StartupOption.BACKUP, 2);
} catch(IOException e) {
assertTrue("Backup node 2 should be able to start", false);
}
} catch(IOException e) {
LOG.error("Error in TestBackupNode:", e);
assertTrue(e.getLocalizedMessage(), false);
} finally {
if(backup1 != null) backup1.stop();
if(backup2 != null) backup2.stop();
if(cluster != null) cluster.shutdown();
}
}
}

View File

@ -17,10 +17,9 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.junit.Assert.*;
import junit.framework.Assert;
import java.io.*;
import java.net.URI;
import java.util.Collection;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DistributedFileSystem;
@ -28,9 +27,9 @@
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundEditLog;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
@ -47,20 +46,6 @@ public class TestCheckPointForSecurityTokens {
short replication = 3;
MiniDFSCluster cluster = null;
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)
throws IOException {
cluster.getNamesystem().cancelDelegationToken(token);
@ -95,10 +80,12 @@ public void testSaveNamespace() throws IOException {
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);
NameNode nn = cluster.getNameNode();
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 void testSaveNamespace() throws IOException {
} catch(Exception e) {
throw new IOException(e.getMessage());
}
// verify that the edits file is empty
for(URI uri : editsDirs) {
File ed = new File(uri.getPath());
Assert.assertTrue(new File(ed, "current/edits").length() == Integer.SIZE/Byte.SIZE);
// verify that the edits file is empty except for the START txn
for (StorageDirectory sd : nn.getFSImage().getStorage().dirIterable(null)) {
FoundEditLog log = FSImageTestUtil.findLatestEditsLog(sd);
assertTrue(log.isInProgress());
assertEquals("In-progress log " + log + " should only have START txn",
1, log.validateLog().numTransactions);
}
// restart cluster

View File

@ -50,7 +50,7 @@ private String getClusterId(Configuration config) throws IOException {
// 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);

View File

@ -20,6 +20,7 @@
import junit.framework.TestCase;
import java.io.*;
import java.net.URI;
import java.util.Collection;
import java.util.Iterator;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
@ -27,44 +28,75 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.*;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
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.common.HdfsConstants.NamenodeRole;
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;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.log4j.Level;
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);
static {
((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 void run() {
}
}
/**
* 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 {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = null;
try {
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());
} finally {
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
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = null;
FileSystem fileSys = null;
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
cluster.waitActive();
fileSys = cluster.getFileSystem();
final FSNamesystem namesystem = cluster.getNamesystem();
FSImage fsimage = namesystem.getFSImage();
final FSEditLog editLog = fsimage.getEditLog();
assertExistsInStorageDirs(
cluster, NameNodeDirType.EDITS,
NNStorage.getInProgressEditsFileName(1));
editLog.logSetReplication("fakefile", (short) 1);
editLog.logSync();
editLog.rollEditLog();
assertExistsInStorageDirs(
cluster, NameNodeDirType.EDITS,
NNStorage.getFinalizedEditsFileName(1,3));
assertExistsInStorageDirs(
cluster, NameNodeDirType.EDITS,
NNStorage.getInProgressEditsFileName(4));
editLog.logSetReplication("fakefile", (short) 2);
editLog.logSync();
editLog.close();
} finally {
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 @@ private void testEditLog(int initialSize) throws IOException {
FSEditLog editLog = fsimage.getEditLog();
// set small size of flush buffer
editLog.setBufferCapacity(initialSize);
editLog.close();
editLog.open();
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 @@ private void testEditLog(int initialSize) throws IOException {
}
}
editLog.close();
editLog.open();
// Roll another time to finalize edits_inprogress_3
fsimage.rollEditLog();
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);
for (Iterator<StorageDirectory> it =
fsimage.getStorage().dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
File editFile = NNStorage.getStorageFile(it.next(), NameNodeFile.EDITS);
FSEditLogLoader loader = new FSEditLogLoader(namesystem);
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);
}
} finally {
if(fileSys != null) fileSys.close();
if(cluster != null) cluster.shutdown();
try {
if(fileSys != null) fileSys.close();
if(cluster != null) cluster.shutdown();
} catch (Throwable t) {
LOG.error("Couldn't shut down cleanly", t);
}
}
}
@ -231,29 +371,29 @@ public void testSyncBatching() throws Exception {
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",
0, editLog.getSyncTxId());
1, editLog.getSyncTxId());
// Log an edit from thread B
doLogEdit(threadB, editLog, "thread-b 1");
assertEquals("logging edit without syncing should do not affect txid",
0, editLog.getSyncTxId());
1, editLog.getSyncTxId());
// 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",
2, editLog.getSyncTxId());
3, editLog.getSyncTxId());
//Should have incremented the batch count exactly once
assertCounter("TransactionsBatchedInSync", 1L,
@ -296,12 +436,12 @@ public void testBatchedSyncWithClosedLogs() throws Exception {
// Log an edit from thread A
doLogEdit(threadA, editLog, "thread-a 1");
assertEquals("logging edit without syncing should do not affect txid",
0, editLog.getSyncTxId());
1, editLog.getSyncTxId());
// 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 void testEditChecksum() throws Exception {
FSImage fsimage = namesystem.getFSImage();
final FSEditLog editLog = fsimage.getEditLog();
fileSys.mkdirs(new Path("/tmp"));
File editFile = editLog.getFsEditName();
StorageDirectory sd = fsimage.getStorage().dirIterator(NameNodeDirType.EDITS).next();
editLog.close();
cluster.shutdown();
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 void testEditChecksum() throws Exception {
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 {
Configuration conf = new HdfsConfiguration();
/**
* 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 {
MiniDFSCluster cluster = null;
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
cluster.waitActive();
final FSNamesystem fsn = cluster.getNamesystem();
// Set up spys
final FSImage originalImage = fsn.getFSImage();
NNStorage storage = originalImage.getStorage();
NNStorage spyStorage = spy(storage);
originalImage.storage = spyStorage;
final FSEditLog editLog = originalImage.getEditLog();
FSEditLog spyLog = spy(editLog);
FSImage spyImage = spy(originalImage);
fsn.dir.fsImage = spyImage;
spyImage.storage.setStorageDirectories(
FSNamesystem.getNamespaceDirs(conf),
FSNamesystem.getNamespaceEditsDirs(conf));
// Fail every attempt to open a new edit file
doThrow(new IOException("Injected fault: open")).
when(spyLog).addNewEditLogStream((File)anyObject());
Configuration conf = new HdfsConfiguration();
conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY,
CHECKPOINT_ON_STARTUP_MIN_TXNS);
try {
spyLog.close();
spyLog.open();
fail("open did not fail even when all directories failed!");
} catch(IOException ioe) {
LOG.info("Got expected exception", ioe);
LOG.info("\n===========================================\n" +
"Starting empty cluster");
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(NUM_DATA_NODES)
.format(true)
.build();
cluster.waitActive();
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);
// Directory layout looks like:
// test/data/dfs/nameN/current/{fsimage_N,edits_...}
File currentDir = new File(nameDir, "current");
// We should see the file as in-progress
File editsFile = new File(currentDir,
NNStorage.getInProgressEditsFileName(1));
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
LOG.info("\n===========================================\n" +
"Starting same cluster after simulated crash");
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(NUM_DATA_NODES)
.format(false)
.build();
cluster.waitActive();
// We should still have the files we wrote prior to the simulated crash
fs = cluster.getFileSystem();
for (int i = 0; i < numTransactions; i++) {
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;
} else {
// otherwise, it shouldn't have made a checkpoint
expectedTxId = 0;
}
imageFile = FSImageTestUtil.findNewestImageFile(
currentDir.getAbsolutePath());
assertNotNull("No image found in " + nameDir, imageFile);
assertEquals(NNStorage.getImageFileName(expectedTxId),
imageFile.getName());
// Started successfully. Shut it down and make sure it can restart.
cluster.shutdown();
cluster = null;
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(NUM_DATA_NODES)
.format(false)
.build();
cluster.waitActive();
} finally {
spyLog.close();
if (cluster != null) {
cluster.shutdown();
}
}
}
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 {
// start a cluster
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = null;
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(NUM_DATA_NODES).build();
cluster.shutdown();
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) {
break;
}
}
// Reset and try it with a working open
Mockito.reset(spyLog);
spyImage.storage.setStorageDirectories(
FSNamesystem.getNamespaceDirs(conf),
FSNamesystem.getNamespaceEditsDirs(conf));
spyLog.open();
// Close everything off
spyLog.close();
originalImage.close();
fsn.close();
try {
cluster = new MiniDFSCluster.Builder(conf)
.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);
}
cluster.shutdown();
}
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();
}
public long length() throws IOException {
return len;
}
public int read(byte[] b, int off, int len) throws IOException {
return input.read(b, off, len);
}
public void close() throws IOException {
input.close();
}
@Override // JournalStream
public String getName() {
return "AnonEditLogByteInputStream";
}
@Override // JournalStream
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);
try {
logDir.setWritable(false);
log.open();
fail("Did no throw exception on only having a bad dir");
} catch (IOException ioe) {
GenericTestUtils.assertExceptionContains(
"no journals successfully started", ioe);
} finally {
logDir.setWritable(true);
log.close();
}
}
}

View File

@ -26,40 +26,64 @@
import java.io.IOException;
import java.nio.channels.FileChannel;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DU;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
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 {
Configuration conf = new HdfsConfiguration();
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());
}
@Test
@ -86,4 +110,48 @@ public void testClose() throws IOException {
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();
}
/**
* Tests EditLogFileOutputStream doesn't throw NullPointerException on
* close/close sequence. See HDFS-2011.
*/
@Test
public void testEditLogFileOutputStreamCloseClose() throws IOException {
// close after a close should result in an IOE
EditLogFileOutputStream editLogStream =
new EditLogFileOutputStream(TEST_EDITS, 0);
editLogStream.close();
try {
editLogStream.close();
} catch (IOException ioe) {
String msg = StringUtils.stringifyException(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.
*/
@Test
public void testEditLogFileOutputStreamAbortAbort() throws IOException {
// abort after a close should just ignore
EditLogFileOutputStream editLogStream =
new EditLogFileOutputStream(TEST_EDITS, 0);
editLogStream.abort();
editLogStream.abort();
}
}

View File

@ -0,0 +1,176 @@
/**
* 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.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 java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.After;
import org.junit.Before;
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.
*/
@Before
public void setUpMiniCluster() throws IOException {
conf = new HdfsConfiguration();
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
cluster.waitActive();
fs = cluster.getFileSystem();
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();
cluster.shutdown();
}
@Test
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);
assertTrue(doAnEdit());
// A single journal failure should not result in a call to runtime.exit(...).
assertExitInvocations(0);
assertFalse(cluster.getNameNode().isInSafeMode());
}
@Test
public void testAllEditsDirsFailOnFlush() throws IOException {
assertTrue(doAnEdit());
// Invalidate both edits journals.
invalidateEditsDirAtIndex(0, true);
invalidateEditsDirAtIndex(1, true);
// Make sure runtime.exit(...) hasn't been called at all yet.
assertExitInvocations(0);
assertTrue(doAnEdit());
// The previous edit could not be synced to any persistent storage, should
// have halted the NN.
assertExitInvocations(1);
}
@Test
public void testSingleFailedEditsDirOnSetReadyToFlush() throws IOException {
assertTrue(doAnEdit());
// Invalidate one edits journal.
invalidateEditsDirAtIndex(0, false);
// Make sure runtime.exit(...) hasn't been called at all yet.
assertExitInvocations(0);
assertTrue(doAnEdit());
// A single journal failure should not result in a call to runtime.exit(...).
assertExitInvocations(0);
assertFalse(cluster.getNameNode().isInSafeMode());
}
/**
* 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();
} else {
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) {
FSImage fsimage = cluster.getNamesystem().getFSImage();
FSEditLog editLog = fsimage.getEditLog();
FSEditLog.JournalAndStream jas = editLog.getJournals().get(index);
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());
}
}

View File

@ -18,11 +18,12 @@
package org.apache.hadoop.hdfs.server.namenode;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import java.io.*;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
@ -40,8 +41,7 @@
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
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;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import static org.junit.Assert.*;
import org.junit.Test;
@ -55,6 +55,10 @@
* and namespace saving.
*/
public class TestEditLogRace {
static {
((Log4JLogger)FSEditLog.LOG).getLogger().setLevel(Level.ALL);
}
private static final Log LOG = LogFactory.getLog(TestEditLogRace.class);
private static final String NAME_DIR =
@ -181,27 +185,29 @@ public void testEditLogRolling() throws Exception {
FSImage fsimage = namesystem.getFSImage();
FSEditLog editLog = fsimage.getEditLog();
// set small size of flush buffer
editLog.setBufferCapacity(2048);
editLog.close();
editLog.open();
StorageDirectory sd = fsimage.getStorage().getStorageDir(0);
startTransactionWorkers(namesystem, caughtErr);
long previousLogTxId = 1;
for (int i = 0; i < NUM_ROLLS && caughtErr.get() == null; i++) {
try {
Thread.sleep(20);
} catch (InterruptedException e) {}
LOG.info("Starting roll " + i + ".");
editLog.rollEditLog();
LOG.info("Roll complete " + i + ".");
CheckpointSignature sig = namesystem.rollEditLog();
long nextLog = sig.curSegmentTxId;
String logFileName = NNStorage.getFinalizedEditsFileName(
previousLogTxId, nextLog - 1);
previousLogTxId += verifyEditLogs(namesystem, fsimage, logFileName, previousLogTxId);
verifyEditLogs(namesystem, fsimage);
LOG.info("Starting purge " + i + ".");
editLog.purgeEditLog();
LOG.info("Complete purge " + i + ".");
assertEquals(previousLogTxId, nextLog);
File expectedLog = NNStorage.getInProgressEditsFile(sd, previousLogTxId);
assertTrue("Expect " + expectedLog + " to exist", expectedLog.exists());
}
} finally {
stopTransactionWorkers();
@ -214,19 +220,32 @@ public void testEditLogRolling() throws Exception {
}
}
private void verifyEditLogs(FSNamesystem namesystem, FSImage fsimage)
private long verifyEditLogs(FSNamesystem namesystem, FSImage fsimage,
String logFileName, long startTxId)
throws IOException {
long numEdits = -1;
// 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.
for (Iterator<StorageDirectory> it =
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);
System.out.println("Verifying file: " + editFile);
int numEdits = new FSEditLogLoader(namesystem).loadFSEdits(
new EditLogFileInputStream(editFile));
System.out.println("Number of edits: " + numEdits);
FSEditLogLoader loader = new FSEditLogLoader(namesystem);
int numEditsThisLog = loader.loadFSEdits(new EditLogFileInputStream(editFile),
startTxId);
System.out.println("Number of edits: " + numEditsThisLog);
assertTrue(numEdits == -1 || numEditsThisLog == numEdits);
numEdits = numEditsThisLog;
}
assertTrue(numEdits != -1);
return numEdits;
}
/**
@ -249,11 +268,6 @@ public void testSaveNamespace() throws Exception {
FSImage fsimage = namesystem.getFSImage();
FSEditLog editLog = fsimage.getEditLog();
// set small size of flush buffer
editLog.setBufferCapacity(2048);
editLog.close();
editLog.open();
startTransactionWorkers(namesystem, caughtErr);
for (int i = 0; i < NUM_SAVE_IMAGE && caughtErr.get() == null; i++) {
@ -266,14 +280,28 @@ public void testSaveNamespace() throws Exception {
namesystem.enterSafeMode(false);
// Verify edit logs before the save
verifyEditLogs(namesystem, fsimage);
// 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
verifyEditLogs(namesystem, fsimage);
long savedImageTxId = fsimage.getStorage().getMostRecentCheckpointTxId();
// Verify that edit logs post save got finalized and aren't corrupt
verifyEditLogs(namesystem, fsimage,
NNStorage.getFinalizedEditsFileName(logStartTxId, savedImageTxId),
logStartTxId);
// 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 void testSaveImageWhileSyncInProgress() throws Exception {
FSImage fsimage = namesystem.getFSImage();
FSEditLog editLog = fsimage.getEditLog();
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());
jas.setCurrentStreamForTests(spyElos);
final AtomicReference<Throwable> deferredException =
new AtomicReference<Throwable>();
@ -393,7 +422,14 @@ public Void answer(InvocationOnMock invocation) throws Throwable {
doAnEditThread.join();
assertNull(deferredException.get());
verifyEditLogs(namesystem, fsimage);
// 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));
} finally {
LOG.info("Closing namesystem");
if(namesystem != null) namesystem.close();
@ -478,7 +514,14 @@ public Void answer(InvocationOnMock invocation) throws Throwable {
doAnEditThread.join();
assertNull(deferredException.get());
verifyEditLogs(namesystem, fsimage);
// 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));
} finally {
LOG.info("Closing namesystem");
if(namesystem != null) namesystem.close();

View File

@ -20,9 +20,14 @@
import static org.junit.Assert.*;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.channels.FileChannel;
import java.util.Map;
import java.util.SortedMap;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
@ -31,10 +36,25 @@
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.io.IOUtils;
import org.apache.log4j.Level;
import org.junit.Test;
import com.google.common.collect.Maps;
import com.google.common.io.Files;
public class TestFSEditLogLoader {
static {
((Log4JLogger)FSImage.LOG).getLogger().setLevel(Level.ALL);
}
private static final File TEST_DIR = new File(
System.getProperty("test.build.data","build/test/data"));
private static final int NUM_DATA_NODES = 0;
@Test
@ -50,14 +70,15 @@ public void testDisplayRecentEditLogOpCodes() throws IOException {
final FSNamesystem namesystem = cluster.getNamesystem();
FSImage fsimage = namesystem.getFSImage();
final FSEditLog editLog = fsimage.getEditLog();
for (int i = 0; i < 20; i++) {
fileSys.mkdirs(new Path("/tmp/tmp" + i));
}
File editFile = editLog.getFsEditName();
editLog.close();
StorageDirectory sd = fsimage.getStorage().dirIterator(NameNodeDirType.EDITS).next();
cluster.shutdown();
File editFile = FSImageTestUtil.findLatestEditsLog(sd).getFile();
assertTrue("Should exist: " + editFile, editFile.exists());
// Corrupt the edits file.
long fileLen = editFile.length();
RandomAccessFile rwf = new RandomAccessFile(editFile, "rw");
@ -124,4 +145,168 @@ public void testReplicationAdjusted() throws IOException {
}
}
}
/**
* Test that the valid number of transactions can be counted from a file.
* @throws IOException
*/
@Test
public void testCountValidTransactions() throws IOException {
File testDir = new File(TEST_DIR, "testCountValidTransactions");
File logFile = new File(testDir,
NNStorage.getInProgressEditsFileName(1));
// 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();
try {
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();
}
} finally {
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
Files.copy(logFileBak, logFile);
truncateFile(logFile, txOffset + 1);
validation = FSEditLogLoader.validateEditLog(logFile);
assertEquals("Failed when truncating to length " + (txOffset + 1),
txid - 1, validation.numTransactions);
assertEquals(txOffset, validation.validLength);
// Restore backup, corrupt the txn opcode
Files.copy(logFileBak, logFile);
corruptByteInFile(logFile, txOffset);
validation = FSEditLogLoader.validateEditLog(logFile);
assertEquals("Failed when corrupting txn opcode at " + txOffset,
txid - 1, validation.numTransactions);
assertEquals(txOffset, validation.validLength);
// Restore backup, corrupt a byte a few bytes into the txn
Files.copy(logFileBak, logFile);
corruptByteInFile(logFile, txOffset+5);
validation = FSEditLogLoader.validateEditLog(logFile);
assertEquals("Failed when corrupting txn data at " + (txOffset+5),
txid - 1, validation.numTransactions);
assertEquals(txOffset, validation.validLength);
}
// 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++) {
Files.copy(logFileBak, logFile);
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)
throws IOException {
RandomAccessFile raf = new RandomAccessFile(file, "rw");
try {
raf.seek(offset);
int origByte = raf.read();
raf.seek(offset);
raf.writeByte(origByte - 1);
} finally {
IOUtils.closeStream(raf);
}
}
/**
* Truncate the given file to the given length
*/
private void truncateFile(File logFile, long newLength)
throws IOException {
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;
FileInputStream fis = new FileInputStream(f);
try {
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;
}
return 0;
} finally {
fis.close();
}
}
}

View File

@ -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 java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.net.URI;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
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()) {
try {
editsNew.createNewFile();
image.editLog.createEditLogFile(editsNew);
} finally {
IOUtils.closeStream(fileOutputStream);
}
}
}
private void clearDirs() throws IOException {
if (current.exists()) {
FileUtil.fullyDelete(current);
}
}
}

View File

@ -0,0 +1,471 @@
/**
* 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.*;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import java.io.File;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirType;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundEditLog;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundFSImage;
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;
import org.junit.Test;
import org.mockito.Mockito;
public class TestFSImageStorageInspector {
private static final Log LOG = LogFactory.getLog(
TestFSImageStorageInspector.class);
/**
* Simple test with image, edits, and inprogress edits
*/
@Test
public void testCurrentStorageInspector() throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
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());
FoundFSImage latestImage = inspector.getLatestImage();
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.
*/
@Test
public void testPlanWithGaps() throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
StorageDirectory mockDir = mockDirectory(
NameNodeDirType.IMAGE_AND_EDITS,
false,
"/foo/current/" + getImageFileName(123),
"/foo/current/" + getImageFileName(456),
"/foo/current/" + getFinalizedEditsFileName(457,900),
"/foo/current/" + getFinalizedEditsFileName(901,950),
"/foo/current/" + getFinalizedEditsFileName(952,1000)); // <-- missing edit 951!
inspector.inspectDirectory(mockDir);
try {
inspector.createLoadPlan();
fail("Didn't throw IOE trying to load with gaps in edits");
} catch (IOException ioe) {
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
*/
@Test
public void testPlanWithInProgressInMiddle() throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
StorageDirectory mockDir = mockDirectory(
NameNodeDirType.IMAGE_AND_EDITS,
false,
"/foo/current/" + getImageFileName(123),
"/foo/current/" + getImageFileName(456),
"/foo/current/" + getFinalizedEditsFileName(457,900),
"/foo/current/" + getInProgressEditsFileName(901), // <-- inprogress in middle
"/foo/current/" + getFinalizedEditsFileName(952,1000));
inspector.inspectDirectory(mockDir);
mockLogValidation(inspector,
"/foo/current/" + getInProgressEditsFileName(901), 51);
LoadPlan plan = inspector.createLoadPlan();
LOG.info("Plan: " + plan);
assertEquals(new File("/foo/current/" + getImageFileName(456)),
plan.getImageFile());
assertArrayEquals(new File[] {
new File("/foo/current/" + getFinalizedEditsFileName(457,900)),
new File("/foo/current/" + getInProgressEditsFileName(901)),
new File("/foo/current/" + getFinalizedEditsFileName(952,1000)) },
plan.getEditsFiles().toArray(new File[0]));
}
/**
* 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)
*/
@Test
public void testLogGroupRecoveryNoop() throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo1/current/"
+ getFinalizedEditsFileName(123,456)));
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo2/current/"
+ getFinalizedEditsFileName(123,456)));
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo3/current/"
+ getFinalizedEditsFileName(123,456)));
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.
*/
@Test
public void testLogGroupRecoveryMixed() throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo1/current/"
+ getFinalizedEditsFileName(123,456)));
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo2/current/"
+ getFinalizedEditsFileName(123,456)));
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo3/current/"
+ getInProgressEditsFileName(123)));
inspector.inspectDirectory(mockDirectory(
NameNodeDirType.IMAGE,
false,
"/foo4/current/" + getImageFileName(122)));
LogGroup lg = inspector.logGroups.get(123L);
assertEquals(3, lg.logs.size());
FoundEditLog inProgressLog = lg.logs.get(2);
assertTrue(inProgressLog.isInProgress());
LoadPlan plan = inspector.createLoadPlan();
// Check that it was marked corrupt.
assertFalse(lg.logs.get(0).isCorrupt());
assertFalse(lg.logs.get(1).isCorrupt());
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
*/
@Test
public void testLogGroupRecoveryInconsistentEndTxIds() throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo1/current/"
+ getFinalizedEditsFileName(123,456)));
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo2/current/"
+ getFinalizedEditsFileName(123,678)));
LogGroup lg = inspector.logGroups.get(123L);
assertEquals(2, lg.logs.size());
try {
lg.planRecovery();
fail("Didn't throw IOE on inconsistent end txids");
} catch (IOException ioe) {
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.
*/
@Test
public void testLogGroupRecoveryInProgress() throws IOException {
String paths[] = new String[] {
"/foo1/current/" + getInProgressEditsFileName(123),
"/foo2/current/" + getInProgressEditsFileName(123),
"/foo3/current/" + getInProgressEditsFileName(123)
};
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
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);
LogGroup lg = inspector.logGroups.get(123L);
assertEquals(3, lg.logs.size());
lg.planRecovery();
// Check that the short one was marked corrupt
assertFalse(lg.logs.get(0).isCorrupt());
assertFalse(lg.logs.get(1).isCorrupt());
assertTrue(lg.logs.get(2).isCorrupt());
// Calling recover should move it aside
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(
FSImageTransactionalStorageInspector inspector,
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);
return;
}
}
}
fail("No log found to mock out at " + path);
}
/**
* Test when edits and image are in separate directories.
*/
@Test
public void testCurrentSplitEditsAndImage() throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
StorageDirectory mockImageDir = mockDirectory(
NameNodeDirType.IMAGE,
false,
"/foo/current/" + getImageFileName(123));
StorageDirectory mockImageDir2 = mockDirectory(
NameNodeDirType.IMAGE,
false,
"/foo2/current/" + getImageFileName(456));
StorageDirectory mockEditsDir = mockDirectory(
NameNodeDirType.EDITS,
false,
"/foo3/current/" + getFinalizedEditsFileName(123, 456),
"/foo3/current/" + getInProgressEditsFileName(457));
inspector.inspectDirectory(mockImageDir);
inspector.inspectDirectory(mockEditsDir);
inspector.inspectDirectory(mockImageDir2);
mockLogValidation(inspector,
"/foo3/current/" + getInProgressEditsFileName(457), 2);
assertEquals(2, inspector.foundEditLogs.size());
assertEquals(2, inspector.foundImages.size());
assertTrue(inspector.foundEditLogs.get(1).isInProgress());
assertTrue(inspector.isUpgradeFinalized());
// 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)),
plan.getImageFile());
assertArrayEquals(new File[] {
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());
}
@Test
public void testLogManifest() throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo1/current/"
+ getFinalizedEditsFileName(1,1),
"/foo1/current/"
+ getFinalizedEditsFileName(2,200)));
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo2/current/"
+ getInProgressEditsFileName(1),
"/foo2/current/"
+ getFinalizedEditsFileName(201, 400)));
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo3/current/"
+ getFinalizedEditsFileName(1, 1),
"/foo3/current/"
+ getFinalizedEditsFileName(2,200)));
assertEquals("[[1,1], [2,200], [201,400]]",
inspector.getEditLogManifest(1).toString());
assertEquals("[[2,200], [201,400]]",
inspector.getEditLogManifest(2).toString());
assertEquals("[[2,200], [201,400]]",
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.
*/
@Test
public void testLogManifestInProgressComesFirst() throws IOException {
FSImageTransactionalStorageInspector inspector =
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo1/current/"
+ getFinalizedEditsFileName(2622,2623),
"/foo1/current/"
+ getFinalizedEditsFileName(2624,2625),
"/foo1/current/"
+ getInProgressEditsFileName(2626)));
inspector.inspectDirectory(
mockDirectoryWithEditLogs("/foo2/current/"
+ getFinalizedEditsFileName(2622,2623),
"/foo2/current/"
+ getFinalizedEditsFileName(2624,2625),
"/foo2/current/"
+ getFinalizedEditsFileName(2626,2627),
"/foo2/current/"
+ 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();
return sd;
}
}

View File

@ -23,6 +23,7 @@
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.File;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
@ -33,6 +34,8 @@
import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
import org.junit.Before;
import org.junit.Test;
@ -59,7 +62,7 @@ static private FSNamesystem getMockNamesystem() {
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 <T extends INode> void verifyFsLimits(INode[] pathComponents,
}
@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;

View File

@ -0,0 +1,155 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.io.File;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.junit.Test;
import com.google.common.base.Joiner;
import static org.apache.hadoop.test.GenericTestUtils.assertGlobEquals;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
/**
* 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.
*/
@Test
public void testPurgingWithNameEditsDirAfterFailure()
throws IOException {
MiniDFSCluster cluster = null;
Configuration conf = new HdfsConfiguration();
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");
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
Joiner.on(",").join(sd0, sd1));
try {
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(0)
.manageNameDfsDirs(false)
.format(true).build();
NameNode nn = cluster.getNameNode();
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*",
getImageFileName(0), getImageFileName(2));
assertGlobEquals(cd0, "edits_.*",
getFinalizedEditsFileName(1, 2),
getInProgressEditsFileName(3));
assertGlobEquals(cd1, "edits_.*",
getFinalizedEditsFileName(1, 2),
getInProgressEditsFileName(3));
doSaveNamespace(nn);
LOG.info("After second save, image 0 should be purged, " +
"and image 4 should exist in both.");
assertGlobEquals(cd0, "fsimage_\\d*",
getImageFileName(2), getImageFileName(4));
assertGlobEquals(cd1, "fsimage_\\d*",
getImageFileName(2), getImageFileName(4));
assertGlobEquals(cd0, "edits_.*",
getFinalizedEditsFileName(3, 4),
getInProgressEditsFileName(5));
assertGlobEquals(cd1, "edits_.*",
getFinalizedEditsFileName(3, 4),
getInProgressEditsFileName(5));
LOG.info("Failing first storage dir by chmodding it");
sd0.setExecutable(false);
doSaveNamespace(nn);
LOG.info("Restoring accessibility of first storage dir");
sd0.setExecutable(true);
LOG.info("nothing should have been purged in first storage dir");
assertGlobEquals(cd0, "fsimage_\\d*",
getImageFileName(2), getImageFileName(4));
assertGlobEquals(cd0, "edits_.*",
getFinalizedEditsFileName(3, 4),
getInProgressEditsFileName(5));
LOG.info("fsimage_2 should be purged in second storage dir");
assertGlobEquals(cd1, "fsimage_\\d*",
getImageFileName(4), getImageFileName(6));
assertGlobEquals(cd1, "edits_.*",
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.");
doSaveNamespace(nn);
assertGlobEquals(cd1, "fsimage_\\d*",
getImageFileName(6), getImageFileName(8));
assertGlobEquals(cd1, "edits_.*",
getFinalizedEditsFileName(7, 8),
getInProgressEditsFileName(9));
assertGlobEquals(cd0, "fsimage_\\d*",
getImageFileName(2), getImageFileName(4));
assertGlobEquals(cd0, "edits_.*",
getInProgressEditsFileName(9));
} finally {
sd0.setExecutable(true);
LOG.info("Shutting down...");
if (cluster != null) {
cluster.shutdown();
}
}
}
private static void doSaveNamespace(NameNode nn) throws IOException {
LOG.info("Saving namespace...");
nn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
nn.saveNamespace();
nn.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
}
}

View File

@ -0,0 +1,307 @@
/**
* 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 java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundEditLog;
import org.apache.hadoop.hdfs.server.namenode.FSImageTransactionalStorageInspector.FoundFSImage;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
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.
*/
@Test
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
*/
@Test
public void testPurgeMultipleDirs() throws IOException {
TestCaseDescription tc = new TestCaseDescription();
tc.addRoot("/foo1", NameNodeDirType.IMAGE_AND_EDITS);
tc.addRoot("/foo2", NameNodeDirType.IMAGE_AND_EDITS);
tc.addImage("/foo1/current/" + getImageFileName(100), true);
tc.addImage("/foo1/current/" + getImageFileName(200), true);
tc.addImage("/foo2/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("/foo2/current/" + getFinalizedEditsFileName(201, 300), true);
tc.addLog("/foo1/current/" + getFinalizedEditsFileName(301, 400), false);
tc.addLog("/foo2/current/" + getFinalizedEditsFileName(301, 400), false);
tc.addLog("/foo1/current/" + getInProgressEditsFileName(401), false);
runTest(tc);
}
/**
* Test that if we have fewer fsimages than the configured
* retention, we don't purge any of them
*/
@Test
public void testPurgeLessThanRetention() throws IOException {
TestCaseDescription tc = new TestCaseDescription();
tc.addRoot("/foo1", NameNodeDirType.IMAGE_AND_EDITS);
tc.addImage("/foo1/current/" + getImageFileName(100), false);
tc.addLog("/foo1/current/" + getFinalizedEditsFileName(101,200), false);
tc.addLog("/foo1/current/" + getFinalizedEditsFileName(201,300), false);
tc.addLog("/foo1/current/" + getFinalizedEditsFileName(301,400), false);
tc.addLog("/foo1/current/" + getInProgressEditsFileName(401), false);
runTest(tc);
}
/**
* Check for edge case with no logs present at all.
*/
@Test
public void testNoLogs() 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);
runTest(tc);
}
/**
* Check for edge case with no logs or images present at all.
*/
@Test
public void testEmptyDir() throws IOException {
TestCaseDescription tc = new TestCaseDescription();
tc.addRoot("/foo1", NameNodeDirType.IMAGE_AND_EDITS);
runTest(tc);
}
/**
* Test that old in-progress logs are properly purged
*/
@Test
public void testOldInProgress() 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/" + getInProgressEditsFileName(101), true);
runTest(tc);
}
@Test
public void testSeparateEditDirs() throws IOException {
TestCaseDescription tc = new TestCaseDescription();
tc.addRoot("/foo1", NameNodeDirType.IMAGE);
tc.addRoot("/foo2", NameNodeDirType.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("/foo2/current/" + getFinalizedEditsFileName(101, 200), true);
tc.addLog("/foo2/current/" + getFinalizedEditsFileName(201, 300), true);
tc.addLog("/foo2/current/" + getFinalizedEditsFileName(301, 400), false);
tc.addLog("/foo2/current/" + getInProgressEditsFileName(401), false);
runTest(tc);
}
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());
Mockito.verify(mockPurger, Mockito.atLeast(0))
.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));
// Check images
purgedPaths.clear();
for (FoundEditLog purged : logsPurgedCaptor.getAllValues()) {
purgedPaths.add(purged.getFile().toString());
}
Assert.assertEquals(Joiner.on(",").join(tc.expectedPurgedLogs),
Joiner.on(",").join(purgedPaths));
}
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) {
addFile(path);
if (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();
for (FakeRoot root : dirRoots.values()) {
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>() {
@Override
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);
}
return null;
}
}).when(mockLog).purgeLogsOlderThan(
Mockito.anyLong(), (StoragePurger) Mockito.anyObject());
return mockLog;
}
}
private static NNStorage mockStorageForDirs(final StorageDirectory ... mockDirs)
throws IOException {
NNStorage mockStorage = Mockito.mock(NNStorage.class);
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
FSImageStorageInspector inspector =
(FSImageStorageInspector) invocation.getArguments()[0];
for (StorageDirectory sd : mockDirs) {
inspector.inspectDirectory(sd);
}
return null;
}
}).when(mockStorage).inspectStorageDirs(
Mockito.<FSImageStorageInspector>anyObject());
return mockStorage;
}
}

View File

@ -28,6 +28,10 @@
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
/**
* 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 @@ private void writeFile(FileSystem fileSys, Path name, int repl)
}
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)
throws IOException {
FSImageTransactionalStorageInspector ins = inspect(dir);
if (shouldHaveImages) {
assertTrue("Expect images in " + dir, ins.foundImages.size() > 0);
} else {
assertTrue("Expect no images in " + dir, ins.foundImages.isEmpty());
}
if (shouldHaveEdits) {
assertTrue("Expect edits in " + dir, ins.foundEditLogs.size() > 0);
} else {
assertTrue("Expect no edits in " + dir, ins.foundEditLogs.isEmpty());
}
}
private void checkFile(FileSystem fileSys, Path name, int repl)
@ -110,9 +123,10 @@ SecondaryNameNode startSecondaryNameNode(Configuration conf
* 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.
* @throws Exception
*/
@SuppressWarnings("deprecation")
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 void testNameEditsConfigs() throws IOException {
SecondaryNameNode secondary = null;
Configuration conf = null;
FileSystem fileSys = 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(nameAndEdits, "current"),
new File(newNameDir, "current"),
new File(checkpointNameAndEdits, "current"),
new File(checkpointNameDir, "current"));
// Start namenode with same dfs.namenode.name.dir and dfs.namenode.edits.dir
conf = new HdfsConfiguration();
@ -191,23 +219,12 @@ public void testNameEditsConfigs() throws IOException {
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 = new HdfsConfiguration();
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 void testNameEditsConfigs() throws IOException {
cluster.shutdown();
secondary.shutdown();
}
// No edit logs in new name dir
checkImageAndEditsFilesExistence(newNameDir, true, false);
checkImageAndEditsFilesExistence(newEditsDir, false, true);
checkImageAndEditsFilesExistence(checkpointNameDir, true, false);
@ -281,12 +299,18 @@ public void testNameEditsConfigs() throws IOException {
checkImageAndEditsFilesExistence(checkpointNameAndEdits, true, true);
}
private FSImageTransactionalStorageInspector inspect(File storageDir)
throws IOException {
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 void testNameEditsConfigsFailure() throws IOException {
.manageNameDfsDirs(false)
.build();
cluster.waitActive();
// Check that the dir has a VERSION file
assertTrue(new File(nameAndEdits, "current/VERSION").exists());
fileSys = cluster.getFileSystem();
try {
@ -342,6 +370,12 @@ public void testNameEditsConfigsFailure() throws IOException {
.manageNameDfsDirs(false)
.build();
cluster.waitActive();
// Check that the dirs have a VERSION file
assertTrue(new File(nameAndEdits, "current/VERSION").exists());
assertTrue(new File(newNameDir, "current/VERSION").exists());
assertTrue(new File(newEditsDir, "current/VERSION").exists());
fileSys = cluster.getFileSystem();
try {
@ -380,7 +414,7 @@ public void testNameEditsConfigsFailure() throws IOException {
fileSys.close();
cluster.shutdown();
}
// Add old shared directory for name and edits along with latest name
conf = new HdfsConfiguration();
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, newNameDir.getPath() + "," +
@ -401,7 +435,9 @@ public void testNameEditsConfigsFailure() throws IOException {
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 = new HdfsConfiguration();
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 void testNameEditsConfigsFailure() throws IOException {
.format(false)
.manageNameDfsDirs(false)
.build();
assertTrue(false);
assertTrue(!fileSys.exists(file1));
assertTrue(fileSys.exists(file2));
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");
} finally {
cluster = null;
fileSys.close();
cluster.shutdown();
}
}
}

View File

@ -30,16 +30,12 @@
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.apache.hadoop.util.PureJavaCrc32;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import java.util.Iterator;
import java.util.Collections;
import java.util.List;
import java.util.ArrayList;
import java.io.File;
import java.io.FileInputStream;
/**
* A JUnit test for checking if restarting DFS preserves integrity.
@ -84,6 +80,10 @@ public void testRestartDFS() throws Exception {
if (cluster != null) { cluster.shutdown(); }
}
try {
// Force the NN to save its images on startup so long as
// there are any uncheckpointed txns
conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 1);
// Here we restart the MiniDFScluster without formatting namenode
cluster = new MiniDFSCluster.Builder(conf).format(false)
.numDataNodes(NUM_DATANODES).build();
@ -102,16 +102,17 @@ public void testRestartDFS() throws Exception {
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);
files.cleanup(fs, dir);
} finally {
@ -124,42 +125,35 @@ public void testRestartDFS() throws Exception {
* 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)
throws Exception {
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());
PureJavaCrc32 crc = new PureJavaCrc32();
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.assertParallelFilesAreIdentical(
dirs, Collections.<String>emptySet());
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;
}
}

View File

@ -18,10 +18,11 @@
package org.apache.hadoop.hdfs.server.namenode;
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.junit.Assert.fail;
import static org.junit.Assert.*;
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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.permission.FsPermission;
@ -42,7 +44,7 @@
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
@ -60,46 +62,47 @@
* </ol>
*/
public class TestSaveNamespace {
static {
((Log4JLogger)FSImage.LOG).getLogger().setLevel(Level.ALL);
}
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");
} else {
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 @@ private void saveNamespaceWithInjectedFault(Fault fault) throws IOException {
// 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),
FSNamesystem.getNamespaceEditsDirs(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;
break;
case MOVE_CURRENT:
// The spy throws a RuntimeException when calling moveCurrent()
doThrow(new RuntimeException("Injected fault: moveCurrent")).
when(spyStorage).moveCurrent((StorageDirectory)anyObject());
case SAVE_SECOND_FSIMAGE_IOE:
// The spy throws an IOException when writing to the second directory
doAnswer(new FaultySaveImage(false)).
when(spyImage).saveFSImage((StorageDirectory)anyObject(), anyLong());
shouldFail = false;
break;
case MOVE_LAST_CHECKPOINT:
// The spy throws a RuntimeException when calling moveLastCheckpoint()
doThrow(new RuntimeException("Injected fault: moveLastCheckpoint")).
when(spyStorage).moveLastCheckpoint((StorageDirectory)anyObject());
case SAVE_ALL_FSIMAGES:
// The spy throws IOException in all directories
doThrow(new RuntimeException("Injected")).
when(spyImage).saveFSImage((StorageDirectory)anyObject(), anyLong());
shouldFail = true;
break;
case WRITE_STORAGE_ALL:
// The spy throws an exception before writing any VERSION files
doThrow(new RuntimeException("Injected"))
.when(spyStorage).writeAll();
shouldFail = true;
break;
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.
shouldFail = true;
break;
}
try {
doAnEdit(fsn, 1);
// Save namespace - this will fail because we inject a fault.
// Save namespace - this may fail, depending on fault injected
fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
try {
fsn.saveNamespace();
if (shouldFail) {
fail("Did not fail!");
}
} catch (Exception e) {
LOG.info("Test caught expected exception", e);
if (! shouldFail) {
throw e;
} else {
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 @@ private void saveNamespaceWithInjectedFault(Fault fault) throws IOException {
// 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);
} finally {
if (fsn != null) {
fsn.close();
@ -185,35 +214,33 @@ public void testReinsertnamedirsInSavenamespace() throws Exception {
// 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),
FSNamesystem.getNamespaceEditsDirs(conf));
// inject fault
// The spy throws a IOException when writing to the second directory
doAnswer(new FaultySaveImage(false)).
when(spyImage).saveFSImage((File)anyObject());
File rootDir = storage.getStorageDir(0).getRoot();
rootDir.setExecutable(false);
rootDir.setWritable(false);
rootDir.setReadable(false);
try {
doAnEdit(fsn, 1);
fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
// 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.");
fsn.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 void testReinsertnamedirsInSavenamespace() throws Exception {
checkEditExists(fsn, 1);
LOG.info("Reloaded image is good.");
} finally {
if (rootDir.exists()) {
rootDir.setExecutable(true);
rootDir.setWritable(true);
rootDir.setReadable(true);
}
if (fsn != null) {
fsn.close();
try {
fsn.close();
} catch (Throwable t) {
LOG.fatal("Failed to shut down", t);
}
}
}
}
@Test
public void testCrashWhileSavingSecondImage() throws Exception {
saveNamespaceWithInjectedFault(Fault.SAVE_FSIMAGE);
public void testRTEWhileSavingSecondImage() throws Exception {
saveNamespaceWithInjectedFault(Fault.SAVE_SECOND_FSIMAGE_RTE);
}
@Test
public void testCrashWhileMoveCurrent() throws Exception {
saveNamespaceWithInjectedFault(Fault.MOVE_CURRENT);
public void testIOEWhileSavingSecondImage() throws Exception {
saveNamespaceWithInjectedFault(Fault.SAVE_SECOND_FSIMAGE_IOE);
}
@Test
public void testCrashWhileMoveLastCheckpoint() throws Exception {
saveNamespaceWithInjectedFault(Fault.MOVE_LAST_CHECKPOINT);
public void testCrashInAllImageDirs() throws Exception {
saveNamespaceWithInjectedFault(Fault.SAVE_ALL_FSIMAGES);
}
@Test
public void testCrashWhenWritingVersionFiles() throws Exception {
saveNamespaceWithInjectedFault(Fault.WRITE_STORAGE_ALL);
}
@Test
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
*/
@Test
public void testFailedSaveNamespace() throws Exception {
@ -313,7 +360,8 @@ public void doTestFailedSaveNamespace(boolean restoreStorageAfterFailure)
FSNamesystem.getNamespaceEditsDirs(conf));
doThrow(new IOException("Injected fault: saveFSImage")).
when(spyImage).saveFSImage((File)anyObject());
when(spyImage).saveFSImage((StorageDirectory)anyObject(),
Mockito.anyLong());
try {
doAnEdit(fsn, 1);
@ -360,16 +408,6 @@ public void testSaveWhileEditsRolled() throws Exception {
DFSTestUtil.formatNameNode(conf);
FSNamesystem fsn = new FSNamesystem(conf);
// Replace the FSImage with a spy
final FSImage originalImage = fsn.dir.fsImage;
originalImage.getStorage().close();
FSImage spyImage = spy(originalImage);
spyImage.getStorage().setStorageDirectories(
FSNamesystem.getNamespaceDirs(conf),
FSNamesystem.getNamespaceEditsDirs(conf));
fsn.dir.fsImage = spyImage;
try {
doAnEdit(fsn, 1);
CheckpointSignature sig = fsn.rollEditLog();
@ -382,7 +420,6 @@ public void testSaveWhileEditsRolled() throws Exception {
fsn.saveNamespace();
// Now shut down and restart the NN
originalImage.close();
fsn.close();
fsn = null;
@ -399,7 +436,45 @@ public void testSaveWhileEditsRolled() throws Exception {
}
}
}
@Test
public void testTxIdPersistence() throws Exception {
Configuration conf = getConf();
NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
DFSTestUtil.formatNameNode(conf);
FSNamesystem fsn = new FSNamesystem(conf);
try {
// 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();
fsn.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());
} finally {
if (fsn != null) {
fsn.close();
}
}
}
private void doAnEdit(FSNamesystem fsn, int id) throws IOException {
// Make an edit
fsn.mkdirs(

View File

@ -106,9 +106,7 @@ public void testEditLog() throws IOException {
FSEditLog editLog = fsimage.getEditLog();
// set small size of flush buffer
editLog.setBufferCapacity(2048);
editLog.close();
editLog.open();
editLog.setOutputBufferCapacity(2048);
namesystem.getDelegationTokenSecretManager().startThreads();
// Create threads and make them run transactions concurrently.
@ -129,25 +127,24 @@ public void testEditLog() throws IOException {
}
editLog.close();
// 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);
namesystem.getDelegationTokenSecretManager().stopThreads();
int numKeys = namesystem.getDelegationTokenSecretManager().getNumberOfKeys();
for (Iterator<StorageDirectory> it =
fsimage.getStorage().dirIterator(NameNodeDirType.EDITS); it.hasNext();) {
File editFile = fsimage.getStorage().getStorageFile(it.next(), NameNodeFile.EDITS);
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);
System.out.println("Verifying file: " + editFile);
FSEditLogLoader loader = new FSEditLogLoader(namesystem);
int numEdits = loader.loadFSEdits(
new EditLogFileInputStream(editFile));
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);
}
} finally {
if(fileSys != null) fileSys.close();

View File

@ -21,16 +21,12 @@
import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.net.InetAddress;
import java.net.URI;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Properties;
import java.util.Random;
import junit.framework.TestCase;
@ -45,7 +41,6 @@
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
@ -56,7 +51,7 @@
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;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.util.StringUtils;
import org.junit.Assert;
@ -237,13 +232,11 @@ private void verifyDifferentDirs(FSImage img, long expectedImgSize, long expecte
sd = it.next();
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)) {
img.getStorage();
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());
} else {
@ -348,8 +341,8 @@ public void testSNNStartup() throws IOException{
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 void testImageChecksum() throws Exception {
}
private void testImageChecksum(boolean compress) throws Exception {
Configuration conf = new Configuration();
FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1: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);
MiniDFSCluster cluster = null;
Configuration conf = new HdfsConfiguration();
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;
try {
// read the property from version file
in = new FileInputStream(file.getFD());
file.seek(0);
Properties props = new Properties();
props.load(in);
LOG.info("\n===========================================\n" +
"Starting empty cluster");
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(0)
.format(true)
.build();
cluster.waitActive();
FileSystem fs = cluster.getFileSystem();
fs.mkdirs(new Path("/test"));
// Directory layout looks like:
// test/data/dfs/nameN/current/{fsimage,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("Shutting down cluster #1");
cluster.shutdown();
cluster = null;
// 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
file.seek(0);
out = new FileOutputStream(file.getFD());
props.store(out, null);
out.flush();
file.setLength(out.getChannel().position());
// now load the image again
image.loadFSImage();
fail("Expect to get a checksumerror");
} catch(IOException e) {
assertTrue(e.getMessage().contains("is corrupt"));
// Corrupt the md5 file to all 0s
File imageFile = new File(nameDir, "current/" + NNStorage.getImageFileName(0));
MD5FileUtils.saveMD5File(imageFile, new MD5Hash(new byte[16]));
// Try to start a new cluster
LOG.info("\n===========================================\n" +
"Starting same cluster after simulated crash");
try {
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(0)
.format(false)
.build();
fail("Should not have successfully started with corrupt image");
} catch (IOException ioe) {
if (!ioe.getCause().getMessage().contains("is corrupt with MD5")) {
throw ioe;
}
}
} finally {
IOUtils.closeStream(in);
IOUtils.closeStream(out);
namenode.stop();
namenode.join();
if (cluster != null) {
cluster.shutdown();
}
}
}

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.net.URI;
import java.util.Collections;
import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
@ -43,7 +45,9 @@ public void setUp() throws Exception {
conf = new HdfsConfiguration();
startOpt = StartupOption.UPGRADE;
startOpt.setClusterId(null);
storage = new NNStorage(conf);
storage = new NNStorage(conf,
Collections.<URI>emptyList(),
Collections.<URI>emptyList());
}
@After
@ -136,4 +140,4 @@ public void testStartupOptUpgradeFromFederationWithCID()
Assert.assertEquals("Clusterid should match with the existing one",
"currentcid", storage.getClusterID());
}
}
}

View File

@ -22,38 +22,38 @@
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.DataInputStream;
import java.io.EOFException;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.math.BigInteger;
import java.security.MessageDigest;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.Random;
import java.util.Set;
import static org.mockito.Matchers.anyByte;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.spy;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.cli.CLITestCmdDFS;
import org.apache.hadoop.cli.util.CLICommandDFSAdmin;
import org.apache.hadoop.cli.util.CommandExecutor;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.junit.After;
import org.apache.hadoop.io.Writable;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
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)
throws IOException {
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();
}
private MiniDFSCluster cluster;
@Before
public void setUpNameDirs() throws Exception {
config = new HdfsConfiguration();
@ -119,19 +106,9 @@ public void setUpNameDirs() throws Exception {
// set the restore feature on
config.setBoolean(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_RESTORE_KEY, true);
}
/**
* clean up
*/
@After
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 void invalidateStorage(FSImage fi, Set<File> filesToInvalidate) throws IO
}
// 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 void printStorages(FSImage fs) {
LOG.info("current storages and corresponding sizes:");
for(Iterator<StorageDirectory> it = fs.getStorage().dirIterator(); it.hasNext(); ) {
StorageDirectory sd = it.next();
if(sd.getStorageDirType().isOfType(NameNodeDirType.IMAGE)) {
File imf = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE);
LOG.info(" image file " + imf.getAbsolutePath() + "; len = " + imf.length());
}
if(sd.getStorageDirType().isOfType(NameNodeDirType.EDITS)) {
File edf = NNStorage.getStorageFile(sd, NameNodeFile.EDITS);
LOG.info(" edits file " + edf.getAbsolutePath() + "; len = " + edf.length());
File curDir = sd.getCurrentDir();
for (File f : curDir.listFiles()) {
LOG.info(" file " + f.getAbsolutePath() + "; len = " + f.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));
try {
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
* @throws IOException
*/
long readCheckpointTime(File currDir) throws IOException {
File timeFile = new File(currDir, NameNodeFile.TIME.getName());
long timeStamp = 0L;
if (timeFile.exists() && timeFile.canRead()) {
DataInputStream in = new DataInputStream(new FileInputStream(timeFile));
try {
timeStamp = in.readLong();
} finally {
in.close();
}
}
return timeStamp;
}
/**
* check if files exist/not exist
* @throws IOException
*/
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;
try {
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);
} else {
// 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));
// checkpoint times
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 void checkFiles(boolean valid) throws IOException {
@SuppressWarnings("deprecation")
@Test
public void testStorageRestore() throws Exception {
int numDatanodes = 2;
int numDatanodes = 0;
cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes)
.manageNameDfsDirs(false)
.build();
@ -305,36 +179,88 @@ public void testStorageRestore() throws Exception {
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");
writeFile(fs, path, 2);
System.out.println("****testStorageRestore: file test1 written");
checkFiles(false); // SHOULD BE FALSE
assertTrue(fs.mkdirs(path));
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(
new File(path2, "current/" + getInProgressEditsFileName(1)),
new File(path3, "current/" + getInProgressEditsFileName(1)));
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)
FSImageTestUtil.assertFileContentsSame(
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());
assertFalse("Should not have finalized logs in the failed directories",
new File(path3, "current/" + getFinalizedEditsFileName(1,4)).exists());
// verify that all the logs are active
// The new log segment should be in all of the directories.
FSImageTestUtil.assertFileContentsSame(
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)));
// The original image should still be the previously failed image
// directory after it got restored, since it's still useful for
// a recovery!
FSImageTestUtil.assertFileContentsSame(
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");
writeFile(fs, path, 2);
System.out.println("****testStorageRestore: wrote a file and checkFiles(true) run");
checkFiles(true);
assertTrue(fs.mkdirs(path));
// Logs should be changed by the edit.
String md5AfterEdit = FSImageTestUtil.getFileMD5(
new File(path1, "current/" + getInProgressEditsFileName(5)));
assertFalse(md5BeforeEdit.equals(md5AfterEdit));
// And all logs should be changed.
FSImageTestUtil.assertFileContentsSame(
new File(path1, "current/" + getInProgressEditsFileName(5)),
new File(path2, "current/" + getInProgressEditsFileName(5)),
new File(path3, "current/" + getInProgressEditsFileName(5)));
secondary.shutdown();
cluster.shutdown();
// All logs should be finalized by clean shutdown
FSImageTestUtil.assertFileContentsSame(
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 void testMultipleSecondaryCheckpoint() throws IOException {
FileSystem fs = cluster.getFileSystem();
Path testPath = new Path("/", "test");
writeFile(fs, testPath, 2);
assertTrue(fs.mkdirs(testPath));
printStorages(fsImage);

View File

@ -20,42 +20,80 @@
import static org.junit.Assert.*;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.util.StringUtils;
import org.junit.Test;
import org.mockito.Mockito;
import com.google.common.collect.ImmutableList;
public class TestTransferFsImage {
private static final File TEST_DIR = new File(
System.getProperty("test.build.data","build/test/data"));
/**
* 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.
*/
@Test
public void testClientSideException() throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(0).build();
NNStorage mockStorage = Mockito.mock(NNStorage.class);
List<File> localPath = Collections.<File>singletonList(
new File("/xxxxx-does-not-exist/blah"));
try {
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"));
} finally {
cluster.shutdown();
}
}
/**
* Similar to the above test, except that there are multiple local files
* and one of them can be saved.
*/
@Test
public void testClientSideExceptionOnJustOneDir() throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(0).build();
NNStorage mockStorage = Mockito.mock(NNStorage.class);
List<File> localPaths = ImmutableList.of(
new File("/xxxxx-does-not-exist/blah"),
new File(TEST_DIR, "testfile")
);
try {
String fsName = NameNode.getHostPortString(
cluster.getNameNode().getHttpAddress());
String id = "getimage=1&txid=0";
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);
} finally {
cluster.shutdown();
}

View File

@ -20,8 +20,6 @@
import java.io.IOException;
import java.io.File;
import java.io.FileInputStream;
import java.io.DataInputStream;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.HashMap;
@ -32,15 +30,11 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
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.DFSTestUtil;
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 @@ private static void initializeObsoleteOpCodes() {
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);
}
@Before
public void setup() {
new File(cacheDir).mkdirs();
}
/**
* Test the OfflineEditsViewer
*/

View File

@ -1,428 +1,498 @@
<?xml version="1.0"?>
<EDITS>
<EDITS_VERSION>-24</EDITS_VERSION>
<EDITS_VERSION>-38</EDITS_VERSION>
<RECORD>
<OPCODE>21</OPCODE>
<OPCODE>24</OPCODE>
<DATA>
<KEY_ID>1</KEY_ID>
<KEY_EXPIRY_DATE>1287183164658</KEY_EXPIRY_DATE>
<KEY_LENGTH>3</KEY_LENGTH>
<KEY_BLOB>drEs</KEY_BLOB>
<TRANSACTION_ID>1</TRANSACTION_ID>
</DATA>
<CHECKSUM>1504643968</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>21</OPCODE>
<DATA>
<KEY_ID>2</KEY_ID>
<KEY_EXPIRY_DATE>1287183164703</KEY_EXPIRY_DATE>
<TRANSACTION_ID>2</TRANSACTION_ID>
<KEY_ID>1</KEY_ID>
<KEY_EXPIRY_DATE>1304751257518</KEY_EXPIRY_DATE>
<KEY_LENGTH>3</KEY_LENGTH>
<KEY_BLOB>1cGc</KEY_BLOB>
<KEY_BLOB>2FhO</KEY_BLOB>
</DATA>
<CHECKSUM>-174778556</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>21</OPCODE>
<DATA>
<TRANSACTION_ID>3</TRANSACTION_ID>
<KEY_ID>2</KEY_ID>
<KEY_EXPIRY_DATE>1304751257521</KEY_EXPIRY_DATE>
<KEY_LENGTH>3</KEY_LENGTH>
<KEY_BLOB>77-r</KEY_BLOB>
</DATA>
<CHECKSUM>1565957291</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>10</OPCODE>
<DATA>
<TRANSACTION_ID>4</TRANSACTION_ID>
<GENERATION_STAMP>1001</GENERATION_STAMP>
</DATA>
<CHECKSUM>1423210231</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>0</OPCODE>
<DATA>
<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>
</DATA>
<CHECKSUM>-1854451489</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>9</OPCODE>
<DATA>
<TRANSACTION_ID>6</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION>
<MTIME>1286491964758</MTIME>
<ATIME>1286491964741</ATIME>
<MTIME>1304060057572</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>
</DATA>
<CHECKSUM>617592855</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>1</OPCODE>
<DATA>
<TRANSACTION_ID>7</TRANSACTION_ID>
<LENGTH>3</LENGTH>
<SOURCE>/file_create</SOURCE>
<DESTINATION>/file_moved</DESTINATION>
<TIMESTAMP>1286491964766</TIMESTAMP>
<TIMESTAMP>1304060057575</TIMESTAMP>
</DATA>
<CHECKSUM>367100554</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>2</OPCODE>
<DATA>
<TRANSACTION_ID>8</TRANSACTION_ID>
<LENGTH>2</LENGTH>
<PATH>/file_moved</PATH>
<TIMESTAMP>1286491964775</TIMESTAMP>
<TIMESTAMP>1304060057577</TIMESTAMP>
</DATA>
<CHECKSUM>1048346698</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>3</OPCODE>
<DATA>
<TRANSACTION_ID>9</TRANSACTION_ID>
<LENGTH>3</LENGTH>
<PATH>/directory_mkdir</PATH>
<TIMESTAMP>1286491964783</TIMESTAMP>
<TIMESTAMP>1304060057581</TIMESTAMP>
<ATIME>0</ATIME>
<PERMISSION_STATUS>
<USERNAME>steffl</USERNAME>
<USERNAME>todd</USERNAME>
<GROUPNAME>supergroup</GROUPNAME>
<FS_PERMISSIONS>493</FS_PERMISSIONS>
</PERMISSION_STATUS>
</DATA>
<CHECKSUM>1207240248</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>10</OPCODE>
<DATA>
<TRANSACTION_ID>10</TRANSACTION_ID>
<GENERATION_STAMP>1002</GENERATION_STAMP>
</DATA>
<CHECKSUM>85982431</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>0</OPCODE>
<DATA>
<TRANSACTION_ID>11</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION>
<MTIME>1286491964796</MTIME>
<ATIME>1286491964796</ATIME>
<MTIME>1304060057584</MTIME>
<ATIME>1304060057584</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>
</DATA>
<CHECKSUM>1796314473</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>9</OPCODE>
<DATA>
<TRANSACTION_ID>12</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION>
<MTIME>1286491964814</MTIME>
<ATIME>1286491964796</ATIME>
<MTIME>1304060057588</MTIME>
<ATIME>1304060057584</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>
</DATA>
<CHECKSUM>1017626905</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>4</OPCODE>
<DATA>
<TRANSACTION_ID>13</TRANSACTION_ID>
<PATH>/file_create</PATH>
<REPLICATION>1</REPLICATION>
</DATA>
<CHECKSUM>1842610087</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>7</OPCODE>
<DATA>
<TRANSACTION_ID>14</TRANSACTION_ID>
<PATH>/file_create</PATH>
<FS_PERMISSIONS>511</FS_PERMISSIONS>
</DATA>
<CHECKSUM>605568911</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>8</OPCODE>
<DATA>
<TRANSACTION_ID>15</TRANSACTION_ID>
<PATH>/file_create</PATH>
<USERNAME>newOwner</USERNAME>
<GROUPNAME/>
</DATA>
<CHECKSUM>-1411790340</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>13</OPCODE>
<DATA>
<TRANSACTION_ID>16</TRANSACTION_ID>
<LENGTH>3</LENGTH>
<PATH>/file_create</PATH>
<MTIME>1285195527000</MTIME>
<ATIME>1285195527000</ATIME>
</DATA>
<CHECKSUM>1428793678</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>14</OPCODE>
<DATA>
<TRANSACTION_ID>17</TRANSACTION_ID>
<PATH>/directory_mkdir</PATH>
<NS_QUOTA>1000</NS_QUOTA>
<DS_QUOTA>-1</DS_QUOTA>
</DATA>
<CHECKSUM>-1476130374</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>15</OPCODE>
<DATA>
<TRANSACTION_ID>18</TRANSACTION_ID>
<LENGTH>3</LENGTH>
<SOURCE>/file_create</SOURCE>
<DESTINATION>/file_moved</DESTINATION>
<TIMESTAMP>1286491964858</TIMESTAMP>
<TIMESTAMP>1304060057605</TIMESTAMP>
<RENAME_OPTIONS>AA</RENAME_OPTIONS>
</DATA>
<CHECKSUM>-1155144192</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>10</OPCODE>
<DATA>
<TRANSACTION_ID>19</TRANSACTION_ID>
<GENERATION_STAMP>1003</GENERATION_STAMP>
</DATA>
<CHECKSUM>1920677987</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>0</OPCODE>
<DATA>
<TRANSACTION_ID>20</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_concat_target</PATH>
<REPLICATION>1</REPLICATION>
<MTIME>1286491964873</MTIME>
<ATIME>1286491964873</ATIME>
<MTIME>1304060057613</MTIME>
<ATIME>1304060057613</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>
</DATA>
<CHECKSUM>-428545606</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>9</OPCODE>
<DATA>
<TRANSACTION_ID>21</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_concat_target</PATH>
<REPLICATION>1</REPLICATION>
<MTIME>1286491965024</MTIME>
<ATIME>1286491964873</ATIME>
<MTIME>1304060057694</MTIME>
<ATIME>1304060057613</ATIME>
<BLOCKSIZE>512</BLOCKSIZE>
<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>
<BLOCK_ID>8798023959648425597</BLOCK_ID>
<BLOCK_ID>-5555244278278879146</BLOCK_ID>
<BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
<BLOCK_GENERATION_STAMP>1003</BLOCK_GENERATION_STAMP>
</BLOCK>
<BLOCK>
<BLOCK_ID>4060815343079109399</BLOCK_ID>
<BLOCK_ID>-6344128791846831740</BLOCK_ID>
<BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
<BLOCK_GENERATION_STAMP>1003</BLOCK_GENERATION_STAMP>
</BLOCK>
<PERMISSION_STATUS>
<USERNAME>steffl</USERNAME>
<USERNAME>todd</USERNAME>
<GROUPNAME>supergroup</GROUPNAME>
<FS_PERMISSIONS>420</FS_PERMISSIONS>
</PERMISSION_STATUS>
</DATA>
<CHECKSUM>707995174</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>10</OPCODE>
<DATA>
<TRANSACTION_ID>22</TRANSACTION_ID>
<GENERATION_STAMP>1004</GENERATION_STAMP>
</DATA>
<CHECKSUM>-1500977009</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>0</OPCODE>
<DATA>
<TRANSACTION_ID>23</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_concat_0</PATH>
<REPLICATION>1</REPLICATION>
<MTIME>1286491965035</MTIME>
<ATIME>1286491965035</ATIME>
<MTIME>1304060057701</MTIME>
<ATIME>1304060057701</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>
</DATA>
<CHECKSUM>-119850856</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>9</OPCODE>
<DATA>
<TRANSACTION_ID>24</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_concat_0</PATH>
<REPLICATION>1</REPLICATION>
<MTIME>1286491965093</MTIME>
<ATIME>1286491965035</ATIME>
<MTIME>1304060057737</MTIME>
<ATIME>1304060057701</ATIME>
<BLOCKSIZE>512</BLOCKSIZE>
<NUMBLOCKS>3</NUMBLOCKS>
<BLOCK>
<BLOCK_ID>85340326229460895</BLOCK_ID>
<BLOCK_ID>4671949296381030428</BLOCK_ID>
<BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
<BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
</BLOCK>
<BLOCK>
<BLOCK_ID>4456960998526419279</BLOCK_ID>
<BLOCK_ID>-844362243522407159</BLOCK_ID>
<BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
<BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
</BLOCK>
<BLOCK>
<BLOCK_ID>-6161739531018161735</BLOCK_ID>
<BLOCK_ID>3476886462779656950</BLOCK_ID>
<BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
<BLOCK_GENERATION_STAMP>1004</BLOCK_GENERATION_STAMP>
</BLOCK>
<PERMISSION_STATUS>
<USERNAME>steffl</USERNAME>
<USERNAME>todd</USERNAME>
<GROUPNAME>supergroup</GROUPNAME>
<FS_PERMISSIONS>420</FS_PERMISSIONS>
</PERMISSION_STATUS>
</DATA>
<CHECKSUM>-766805874</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>10</OPCODE>
<DATA>
<TRANSACTION_ID>25</TRANSACTION_ID>
<GENERATION_STAMP>1005</GENERATION_STAMP>
</DATA>
<CHECKSUM>238426056</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>0</OPCODE>
<DATA>
<TRANSACTION_ID>26</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_concat_1</PATH>
<REPLICATION>1</REPLICATION>
<MTIME>1286491965105</MTIME>
<ATIME>1286491965105</ATIME>
<MTIME>1304060057742</MTIME>
<ATIME>1304060057742</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>
</DATA>
<CHECKSUM>1156254705</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>9</OPCODE>
<DATA>
<TRANSACTION_ID>27</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/file_concat_1</PATH>
<REPLICATION>1</REPLICATION>
<MTIME>1286491965148</MTIME>
<ATIME>1286491965105</ATIME>
<MTIME>1304060057764</MTIME>
<ATIME>1304060057742</ATIME>
<BLOCKSIZE>512</BLOCKSIZE>
<NUMBLOCKS>3</NUMBLOCKS>
<BLOCK>
<BLOCK_ID>-3894328423940677915</BLOCK_ID>
<BLOCK_ID>-754893470864399741</BLOCK_ID>
<BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
<BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
</BLOCK>
<BLOCK>
<BLOCK_ID>-2833847567910728858</BLOCK_ID>
<BLOCK_ID>1820875380010181049</BLOCK_ID>
<BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
<BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
</BLOCK>
<BLOCK>
<BLOCK_ID>-3654781106237722465</BLOCK_ID>
<BLOCK_ID>8266387560744259971</BLOCK_ID>
<BLOCK_NUM_BYTES>512</BLOCK_NUM_BYTES>
<BLOCK_GENERATION_STAMP>1005</BLOCK_GENERATION_STAMP>
</BLOCK>
<PERMISSION_STATUS>
<USERNAME>steffl</USERNAME>
<USERNAME>todd</USERNAME>
<GROUPNAME>supergroup</GROUPNAME>
<FS_PERMISSIONS>420</FS_PERMISSIONS>
</PERMISSION_STATUS>
</DATA>
<CHECKSUM>-654780301</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>16</OPCODE>
<DATA>
<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>
</DATA>
<CHECKSUM>1273279541</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>17</OPCODE>
<DATA>
<TRANSACTION_ID>29</TRANSACTION_ID>
<LENGTH>4</LENGTH>
<SOURCE>/file_symlink</SOURCE>
<DESTINATION>/file_concat_target</DESTINATION>
<MTIME>1286491965168</MTIME>
<ATIME>1286491965168</ATIME>
<MTIME>1304060057770</MTIME>
<ATIME>1304060057770</ATIME>
<PERMISSION_STATUS>
<USERNAME>steffl</USERNAME>
<USERNAME>todd</USERNAME>
<GROUPNAME>supergroup</GROUPNAME>
<FS_PERMISSIONS>511</FS_PERMISSIONS>
</PERMISSION_STATUS>
</DATA>
<CHECKSUM>1385678569</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>18</OPCODE>
<DATA>
<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>
</DATA>
<CHECKSUM>913145699</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>19</OPCODE>
<DATA>
<TRANSACTION_ID>31</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>1286578365198</T_EXPIRY_TIME>
<T_EXPIRY_TIME>1304146457785</T_EXPIRY_TIME>
</DATA>
<CHECKSUM>-1772039941</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>20</OPCODE>
<DATA>
<TRANSACTION_ID>32</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>
</DATA>
<CHECKSUM>1382094146</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>0</OPCODE>
<DATA>
<TRANSACTION_ID>33</TRANSACTION_ID>
<LENGTH>5</LENGTH>
<PATH>/reassign-lease-test</PATH>
<REPLICATION>1</REPLICATION>
@ -438,14 +508,24 @@
<CLIENT_NAME>DFSClient_871171074</CLIENT_NAME>
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
</DATA>
<CHECKSUM>1975140107</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>22</OPCODE>
<DATA>
<TRANSACTION_ID>34</TRANSACTION_ID>
<CLIENT_NAME>DFSClient_871171074</CLIENT_NAME>
<PATH>/reassign-lease-test</PATH>
<CLIENT_NAME>HDFS_NameNode</CLIENT_NAME>
</DATA>
<CHECKSUM>1975140107</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>23</OPCODE>
<DATA>
<TRANSACTION_ID>35</TRANSACTION_ID>
</DATA>
<CHECKSUM>1975140107</CHECKSUM>
</RECORD>
<RECORD>
<OPCODE>-1</OPCODE>

View File

@ -28,7 +28,6 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.net.URI;
import java.util.HashMap;
import java.util.Set;
@ -41,6 +40,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
/**
@ -128,11 +128,10 @@ private File initFsimage() throws IOException {
cluster.getNameNode().saveNamespace();
// 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");
}
} finally {
if(cluster != null)

View File

@ -0,0 +1,132 @@
/**
* 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.util;
import static org.junit.Assert.*;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.aspectj.util.FileUtil;
import org.junit.Before;
import org.junit.Test;
import com.google.common.base.Joiner;
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");
@Before
public void cleanupTestDir() throws IOException {
assertTrue(TEST_DIR.exists() || TEST_DIR.mkdirs());
FileUtil.deleteContents(TEST_DIR);
}
/**
* Test case where there is no existing file
*/
@Test
public void testWriteNewFile() throws IOException {
OutputStream fos = new AtomicFileOutputStream(DST_FILE);
assertFalse(DST_FILE.exists());
fos.write(TEST_STRING.getBytes());
fos.flush();
assertFalse(DST_FILE.exists());
fos.close();
assertTrue(DST_FILE.exists());
String readBackData = DFSTestUtil.readFile(DST_FILE);
assertEquals(TEST_STRING, readBackData);
}
/**
* Test case where there is no existing file
*/
@Test
public void testOverwriteFile() throws IOException {
assertTrue("Creating empty dst file", DST_FILE.createNewFile());
OutputStream fos = new AtomicFileOutputStream(DST_FILE);
assertTrue("Empty file still exists", DST_FILE.exists());
fos.write(TEST_STRING.getBytes());
fos.flush();
// Original contents still in place
assertEquals("", DFSTestUtil.readFile(DST_FILE));
fos.close();
// New contents replace original file
String readBackData = DFSTestUtil.readFile(DST_FILE);
assertEquals(TEST_STRING, readBackData);
}
/**
* 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
*/
@Test
public void testFailToFlush() throws IOException {
// Create a file at destination
FileOutputStream fos = new FileOutputStream(DST_FILE);
fos.write(TEST_STRING_2.getBytes());
fos.close();
OutputStream failingStream = createFailingStream();
failingStream.write(TEST_STRING.getBytes());
try {
failingStream.close();
fail("Close didn't throw exception");
} catch (IOException ioe) {
// 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) {
@Override
public void flush() throws IOException {
throw new IOException("injected failure");
}
};
}
}

View File

@ -0,0 +1,111 @@
/**
* 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.util;
import java.io.File;
import java.io.FileOutputStream;
import java.io.FileWriter;
import java.io.IOException;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.MD5Hash;
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.*;
public class TestMD5FileUtils {
private static final File TEST_DIR_ROOT = new File(
System.getProperty("test.build.data","build/test/data"));
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);
@Before
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);
fos.close();
}
@Test
public void testComputeMd5ForFile() throws Exception {
MD5Hash computedDigest = MD5FileUtils.computeMd5ForFile(TEST_FILE);
assertEquals(TEST_MD5, computedDigest);
}
@Test
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 {
MD5FileUtils.verifySavedMD5(TEST_FILE, TEST_MD5);
}
/**
* Test when .md5 file exists but incorrect checksum
*/
@Test
public void testVerifyMD5FileBadDigest() throws Exception {
MD5FileUtils.saveMD5File(TEST_FILE, MD5Hash.digest(new byte[0]));
try {
MD5FileUtils.verifySavedMD5(TEST_FILE, TEST_MD5);
fail("Did not throw");
} catch (IOException ioe) {
// Expected
}
}
/**
* Test when .md5 file exists but has a bad format
*/
@Test
public void testVerifyMD5FileBadFormat() throws Exception {
FileWriter writer = new FileWriter(MD5FileUtils.getDigestFileForFile(TEST_FILE));
try {
writer.write("this is not an md5 file");
} finally {
writer.close();
}
try {
MD5FileUtils.verifySavedMD5(TEST_FILE, TEST_MD5);
fail("Did not throw");
} catch (IOException ioe) {
// expected
}
}
}

View File

@ -17,16 +17,29 @@
*/
package org.apache.hadoop.test;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.util.StringUtils;
import org.junit.Assert;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import com.google.common.base.Joiner;
import com.google.common.base.Supplier;
import com.google.common.collect.Sets;
/**
* 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) {
return;
}
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();
try {
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 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;
}
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
return invocation.getMethod().invoke(
delegate, invocation.getArguments());
}
}
}