|
@@ -543,18 +543,14 @@ public class FSImage implements Closeable {
|
|
|
private boolean loadFSImage(FSNamesystem target, StartupOption startOpt,
|
|
|
MetaRecoveryContext recovery)
|
|
|
throws IOException {
|
|
|
- final NameNodeFile nnf;
|
|
|
- if (startOpt == StartupOption.ROLLINGUPGRADE
|
|
|
- && startOpt.getRollingUpgradeStartupOption()
|
|
|
- == RollingUpgradeStartupOption.ROLLBACK) {
|
|
|
- nnf = NameNodeFile.IMAGE_ROLLBACK;
|
|
|
- } else {
|
|
|
- nnf = NameNodeFile.IMAGE;
|
|
|
- }
|
|
|
+ final boolean rollingRollback = startOpt == StartupOption.ROLLINGUPGRADE
|
|
|
+ && startOpt.getRollingUpgradeStartupOption() ==
|
|
|
+ RollingUpgradeStartupOption.ROLLBACK;
|
|
|
+ final NameNodeFile nnf = rollingRollback ? NameNodeFile.IMAGE_ROLLBACK
|
|
|
+ : NameNodeFile.IMAGE;
|
|
|
final FSImageStorageInspector inspector = storage.readAndInspectDirs(nnf);
|
|
|
|
|
|
isUpgradeFinalized = inspector.isUpgradeFinalized();
|
|
|
-
|
|
|
List<FSImageFile> imageFiles = inspector.getLatestImages();
|
|
|
|
|
|
StartupProgress prog = NameNode.getStartupProgress();
|
|
@@ -573,7 +569,17 @@ public class FSImage implements Closeable {
|
|
|
// If we're open for write, we're either non-HA or we're the active NN, so
|
|
|
// we better be able to load all the edits. If we're the standby NN, it's
|
|
|
// OK to not be able to read all of edits right now.
|
|
|
- long toAtLeastTxId = editLog.isOpenForWrite() ? inspector.getMaxSeenTxId() : 0;
|
|
|
+ // In the meanwhile, for HA upgrade, we will still write editlog thus need
|
|
|
+ // this toAtLeastTxId to be set to the max-seen txid
|
|
|
+ // For rollback in rolling upgrade, we need to set the toAtLeastTxId to
|
|
|
+ // the txid right before the upgrade marker.
|
|
|
+ long toAtLeastTxId = editLog.isOpenForWrite() ? inspector
|
|
|
+ .getMaxSeenTxId() : 0;
|
|
|
+ if (rollingRollback) {
|
|
|
+ // note that the first image in imageFiles is the special checkpoint
|
|
|
+ // for the rolling upgrade
|
|
|
+ toAtLeastTxId = imageFiles.get(0).getCheckpointTxId() + 2;
|
|
|
+ }
|
|
|
editStreams = editLog.selectInputStreams(
|
|
|
imageFiles.get(0).getCheckpointTxId() + 1,
|
|
|
toAtLeastTxId, recovery, false);
|
|
@@ -581,8 +587,7 @@ public class FSImage implements Closeable {
|
|
|
editStreams = FSImagePreTransactionalStorageInspector
|
|
|
.getEditLogStreams(storage);
|
|
|
}
|
|
|
- int maxOpSize = conf.getInt(DFSConfigKeys.
|
|
|
- DFS_NAMENODE_MAX_OP_SIZE_KEY,
|
|
|
+ int maxOpSize = conf.getInt(DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_KEY,
|
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT);
|
|
|
for (EditLogInputStream elis : editStreams) {
|
|
|
elis.setMaxOpSize(maxOpSize);
|
|
@@ -613,13 +618,34 @@ public class FSImage implements Closeable {
|
|
|
throw new IOException("Failed to load an FSImage file!");
|
|
|
}
|
|
|
prog.endPhase(Phase.LOADING_FSIMAGE);
|
|
|
- long txnsAdvanced = loadEdits(editStreams, target, startOpt, recovery);
|
|
|
- needToSave |= needsResaveBasedOnStaleCheckpoint(imageFile.getFile(),
|
|
|
- txnsAdvanced);
|
|
|
+ long txnsAdvanced = 0;
|
|
|
+
|
|
|
+ loadEdits(editStreams, target, startOpt, recovery);
|
|
|
+ if (rollingRollback) {
|
|
|
+ // Trigger the rollback for rolling upgrade.
|
|
|
+ // Here lastAppliedTxId == (markerTxId - 1), and we should decrease 1 from
|
|
|
+ // lastAppliedTxId for the start-segment transaction.
|
|
|
+ rollingRollback(lastAppliedTxId--, imageFiles.get(0).getCheckpointTxId());
|
|
|
+ needToSave = false;
|
|
|
+ } else {
|
|
|
+ needToSave |= needsResaveBasedOnStaleCheckpoint(imageFile.getFile(),
|
|
|
+ txnsAdvanced);
|
|
|
+ }
|
|
|
editLog.setNextTxId(lastAppliedTxId + 1);
|
|
|
return needToSave;
|
|
|
}
|
|
|
|
|
|
+ /** rollback for rolling upgrade. */
|
|
|
+ private void rollingRollback(long discardSegmentTxId, long ckptId)
|
|
|
+ throws IOException {
|
|
|
+ // discard discard unnecessary editlog segments starting from the given id
|
|
|
+ this.editLog.discardSegments(discardSegmentTxId);
|
|
|
+ // rename the special checkpoint
|
|
|
+ renameCheckpoint(ckptId, NameNodeFile.IMAGE_ROLLBACK, NameNodeFile.IMAGE);
|
|
|
+ // purge all the checkpoints after the marker
|
|
|
+ archivalManager.purgeCheckpoinsAfter(NameNodeFile.IMAGE, ckptId);
|
|
|
+ }
|
|
|
+
|
|
|
void loadFSImageFile(FSNamesystem target, MetaRecoveryContext recovery,
|
|
|
FSImageFile imageFile) throws IOException {
|
|
|
LOG.debug("Planning to load image :\n" + imageFile);
|
|
@@ -707,7 +733,7 @@ public class FSImage implements Closeable {
|
|
|
|
|
|
private long loadEdits(Iterable<EditLogInputStream> editStreams,
|
|
|
FSNamesystem target, StartupOption startOpt, MetaRecoveryContext recovery)
|
|
|
- throws IOException {
|
|
|
+ throws IOException {
|
|
|
LOG.debug("About to load edits:\n " + Joiner.on("\n ").join(editStreams));
|
|
|
StartupProgress prog = NameNode.getStartupProgress();
|
|
|
prog.beginPhase(Phase.LOADING_EDITS);
|
|
@@ -727,15 +753,19 @@ public class FSImage implements Closeable {
|
|
|
// have been successfully applied before the error.
|
|
|
lastAppliedTxId = loader.getLastAppliedTxId();
|
|
|
}
|
|
|
+ boolean rollingRollback = startOpt == StartupOption.ROLLINGUPGRADE &&
|
|
|
+ startOpt.getRollingUpgradeStartupOption() ==
|
|
|
+ RollingUpgradeStartupOption.ROLLBACK;
|
|
|
// If we are in recovery mode, we may have skipped over some txids.
|
|
|
- if (editIn.getLastTxId() != HdfsConstants.INVALID_TXID) {
|
|
|
+ if (editIn.getLastTxId() != HdfsConstants.INVALID_TXID
|
|
|
+ && !rollingRollback) {
|
|
|
lastAppliedTxId = editIn.getLastTxId();
|
|
|
}
|
|
|
}
|
|
|
} finally {
|
|
|
FSEditLog.closeAllStreams(editStreams);
|
|
|
// update the counts
|
|
|
- updateCountForQuota(target.dir.rootDir);
|
|
|
+ updateCountForQuota(target.dir.rootDir);
|
|
|
}
|
|
|
prog.endPhase(Phase.LOADING_EDITS);
|
|
|
return lastAppliedTxId - prevLastAppliedTxId;
|
|
@@ -836,11 +866,11 @@ public class FSImage implements Closeable {
|
|
|
/**
|
|
|
* Save the contents of the FS image to the file.
|
|
|
*/
|
|
|
- void saveFSImage(SaveNamespaceContext context, StorageDirectory sd)
|
|
|
- throws IOException {
|
|
|
+ void saveFSImage(SaveNamespaceContext context, StorageDirectory sd,
|
|
|
+ NameNodeFile dstType) throws IOException {
|
|
|
long txid = context.getTxId();
|
|
|
File newFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid);
|
|
|
- File dstFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, txid);
|
|
|
+ File dstFile = NNStorage.getStorageFile(sd, dstType, txid);
|
|
|
|
|
|
FSImageFormatProtobuf.Saver saver = new FSImageFormatProtobuf.Saver(context);
|
|
|
FSImageCompression compression = FSImageCompression.createCompression(conf);
|
|
@@ -864,16 +894,19 @@ public class FSImage implements Closeable {
|
|
|
private class FSImageSaver implements Runnable {
|
|
|
private final SaveNamespaceContext context;
|
|
|
private StorageDirectory sd;
|
|
|
+ private final NameNodeFile nnf;
|
|
|
|
|
|
- public FSImageSaver(SaveNamespaceContext context, StorageDirectory sd) {
|
|
|
+ public FSImageSaver(SaveNamespaceContext context, StorageDirectory sd,
|
|
|
+ NameNodeFile nnf) {
|
|
|
this.context = context;
|
|
|
this.sd = sd;
|
|
|
+ this.nnf = nnf;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void run() {
|
|
|
try {
|
|
|
- saveFSImage(context, sd);
|
|
|
+ saveFSImage(context, sd, nnf);
|
|
|
} catch (SaveNamespaceCancelledException snce) {
|
|
|
LOG.info("Cancelled image saving for " + sd.getRoot() +
|
|
|
": " + snce.getMessage());
|
|
@@ -971,7 +1004,7 @@ public class FSImage implements Closeable {
|
|
|
for (Iterator<StorageDirectory> it
|
|
|
= storage.dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
|
|
|
StorageDirectory sd = it.next();
|
|
|
- FSImageSaver saver = new FSImageSaver(ctx, sd);
|
|
|
+ FSImageSaver saver = new FSImageSaver(ctx, sd, nnf);
|
|
|
Thread saveThread = new Thread(saver, saver.toString());
|
|
|
saveThreads.add(saveThread);
|
|
|
saveThread.start();
|