|
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.HAUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
|
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
|
@@ -542,9 +543,16 @@ public class FSImage implements Closeable {
|
|
|
private boolean loadFSImage(FSNamesystem target, StartupOption startOpt,
|
|
|
MetaRecoveryContext recovery)
|
|
|
throws IOException {
|
|
|
- FSImageStorageInspector inspector = storage.readAndInspectDirs();
|
|
|
- FSImageFile imageFile = null;
|
|
|
-
|
|
|
+ final NameNodeFile nnf;
|
|
|
+ if (startOpt == StartupOption.ROLLINGUPGRADE
|
|
|
+ && startOpt.getRollingUpgradeStartupOption()
|
|
|
+ == RollingUpgradeStartupOption.ROLLBACK) {
|
|
|
+ nnf = NameNodeFile.IMAGE_ROLLBACK;
|
|
|
+ } else {
|
|
|
+ nnf = NameNodeFile.IMAGE;
|
|
|
+ }
|
|
|
+ final FSImageStorageInspector inspector = storage.readAndInspectDirs(nnf);
|
|
|
+
|
|
|
isUpgradeFinalized = inspector.isUpgradeFinalized();
|
|
|
|
|
|
List<FSImageFile> imageFiles = inspector.getLatestImages();
|
|
@@ -587,6 +595,7 @@ public class FSImage implements Closeable {
|
|
|
LOG.info("No edit log streams selected.");
|
|
|
}
|
|
|
|
|
|
+ FSImageFile imageFile = null;
|
|
|
for (int i = 0; i < imageFiles.size(); i++) {
|
|
|
try {
|
|
|
imageFile = imageFiles.get(i);
|
|
@@ -901,7 +910,7 @@ public class FSImage implements Closeable {
|
|
|
*/
|
|
|
public synchronized void saveNamespace(FSNamesystem source)
|
|
|
throws IOException {
|
|
|
- saveNamespace(source, null);
|
|
|
+ saveNamespace(source, NameNodeFile.IMAGE, null);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -910,7 +919,7 @@ public class FSImage implements Closeable {
|
|
|
* @param canceler
|
|
|
*/
|
|
|
public synchronized void saveNamespace(FSNamesystem source,
|
|
|
- Canceler canceler) throws IOException {
|
|
|
+ NameNodeFile nnf, Canceler canceler) throws IOException {
|
|
|
assert editLog != null : "editLog must be initialized";
|
|
|
LOG.info("Save namespace ...");
|
|
|
storage.attemptRestoreRemovedStorage();
|
|
@@ -922,7 +931,7 @@ public class FSImage implements Closeable {
|
|
|
}
|
|
|
long imageTxId = getLastAppliedOrWrittenTxId();
|
|
|
try {
|
|
|
- saveFSImageInAllDirs(source, imageTxId, canceler);
|
|
|
+ saveFSImageInAllDirs(source, nnf, imageTxId, canceler);
|
|
|
storage.writeAll();
|
|
|
} finally {
|
|
|
if (editLogWasOpen) {
|
|
@@ -941,12 +950,11 @@ public class FSImage implements Closeable {
|
|
|
*/
|
|
|
protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid)
|
|
|
throws IOException {
|
|
|
- saveFSImageInAllDirs(source, txid, null);
|
|
|
+ saveFSImageInAllDirs(source, NameNodeFile.IMAGE, txid, null);
|
|
|
}
|
|
|
|
|
|
- protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid,
|
|
|
- Canceler canceler)
|
|
|
- throws IOException {
|
|
|
+ private synchronized void saveFSImageInAllDirs(FSNamesystem source,
|
|
|
+ NameNodeFile nnf, long txid, Canceler canceler) throws IOException {
|
|
|
StartupProgress prog = NameNode.getStartupProgress();
|
|
|
prog.beginPhase(Phase.SAVING_CHECKPOINT);
|
|
|
if (storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0) {
|
|
@@ -983,11 +991,11 @@ public class FSImage implements Closeable {
|
|
|
assert false : "should have thrown above!";
|
|
|
}
|
|
|
|
|
|
- renameCheckpoint(txid);
|
|
|
+ renameCheckpoint(txid, NameNodeFile.IMAGE_NEW, nnf);
|
|
|
|
|
|
// Since we now have a new checkpoint, we can clean up some
|
|
|
// old edit logs and checkpoints.
|
|
|
- purgeOldStorage();
|
|
|
+ purgeOldStorage(nnf);
|
|
|
} finally {
|
|
|
// Notify any threads waiting on the checkpoint to be canceled
|
|
|
// that it is complete.
|
|
@@ -1001,23 +1009,35 @@ public class FSImage implements Closeable {
|
|
|
* Purge any files in the storage directories that are no longer
|
|
|
* necessary.
|
|
|
*/
|
|
|
- public void purgeOldStorage() {
|
|
|
+ void purgeOldStorage(NameNodeFile nnf) {
|
|
|
try {
|
|
|
- archivalManager.purgeOldStorage();
|
|
|
+ archivalManager.purgeOldStorage(nnf);
|
|
|
} catch (Exception e) {
|
|
|
- LOG.warn("Unable to purge old storage", e);
|
|
|
+ LOG.warn("Unable to purge old storage " + nnf.getName(), e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Purge all the checkpoints with the name style.
|
|
|
+ */
|
|
|
+ void purgeCheckpoints(NameNodeFile nnf) {
|
|
|
+ try {
|
|
|
+ archivalManager.purgeCheckpoints(nnf);
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.warn("Unable to purge checkpoints with name " + nnf.getName(), e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Renames new image
|
|
|
*/
|
|
|
- private void renameCheckpoint(long txid) throws IOException {
|
|
|
+ private void renameCheckpoint(long txid, NameNodeFile fromNnf,
|
|
|
+ NameNodeFile toNnf) throws IOException {
|
|
|
ArrayList<StorageDirectory> al = null;
|
|
|
|
|
|
for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.IMAGE)) {
|
|
|
try {
|
|
|
- renameCheckpointInDir(sd, txid);
|
|
|
+ renameImageFileInDir(sd, fromNnf, toNnf, txid);
|
|
|
} catch (IOException ioe) {
|
|
|
LOG.warn("Unable to rename checkpoint in " + sd, ioe);
|
|
|
if (al == null) {
|
|
@@ -1046,21 +1066,20 @@ public class FSImage implements Closeable {
|
|
|
storage.reportErrorsOnDirectories(al);
|
|
|
}
|
|
|
|
|
|
-
|
|
|
- private void renameCheckpointInDir(StorageDirectory sd, long txid)
|
|
|
- throws IOException {
|
|
|
- File ckpt = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid);
|
|
|
- File curFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, txid);
|
|
|
+ private void renameImageFileInDir(StorageDirectory sd,
|
|
|
+ NameNodeFile fromNnf, NameNodeFile toNnf, long txid) throws IOException {
|
|
|
+ final File fromFile = NNStorage.getStorageFile(sd, fromNnf, txid);
|
|
|
+ final File toFile = NNStorage.getStorageFile(sd, toNnf, txid);
|
|
|
// renameTo fails on Windows if the destination file
|
|
|
// already exists.
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("renaming " + ckpt.getAbsolutePath()
|
|
|
- + " to " + curFile.getAbsolutePath());
|
|
|
+ LOG.debug("renaming " + fromFile.getAbsolutePath()
|
|
|
+ + " to " + toFile.getAbsolutePath());
|
|
|
}
|
|
|
- if (!ckpt.renameTo(curFile)) {
|
|
|
- if (!curFile.delete() || !ckpt.renameTo(curFile)) {
|
|
|
- throw new IOException("renaming " + ckpt.getAbsolutePath() + " to " +
|
|
|
- curFile.getAbsolutePath() + " FAILED");
|
|
|
+ if (!fromFile.renameTo(toFile)) {
|
|
|
+ if (!toFile.delete() || !fromFile.renameTo(toFile)) {
|
|
|
+ throw new IOException("renaming " + fromFile.getAbsolutePath() + " to " +
|
|
|
+ toFile.getAbsolutePath() + " FAILED");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -1161,7 +1180,7 @@ public class FSImage implements Closeable {
|
|
|
CheckpointFaultInjector.getInstance().afterMD5Rename();
|
|
|
|
|
|
// Rename image from tmp file
|
|
|
- renameCheckpoint(txid);
|
|
|
+ renameCheckpoint(txid, NameNodeFile.IMAGE_NEW, NameNodeFile.IMAGE);
|
|
|
// So long as this is the newest image available,
|
|
|
// advertise it as such to other checkpointers
|
|
|
// from now on
|