|
@@ -37,8 +37,11 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
|
+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.Storage;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
|
|
@@ -103,13 +106,18 @@ public class FSEditLogLoader {
|
|
|
this.lastAppliedTxId = lastAppliedTxId;
|
|
|
}
|
|
|
|
|
|
+ long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
|
|
|
+ throws IOException {
|
|
|
+ return loadFSEdits(edits, expectedStartingTxId, null, null);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Load an edit log, and apply the changes to the in-memory structure
|
|
|
* This is where we apply edits that we've been writing to disk all
|
|
|
* along.
|
|
|
*/
|
|
|
long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId,
|
|
|
- MetaRecoveryContext recovery) throws IOException {
|
|
|
+ StartupOption startOpt, MetaRecoveryContext recovery) throws IOException {
|
|
|
StartupProgress prog = NameNode.getStartupProgress();
|
|
|
Step step = createStartupProgressStep(edits);
|
|
|
prog.beginStep(Phase.LOADING_EDITS, step);
|
|
@@ -117,8 +125,8 @@ public class FSEditLogLoader {
|
|
|
try {
|
|
|
long startTime = now();
|
|
|
FSImage.LOG.info("Start loading edits file " + edits.getName());
|
|
|
- long numEdits = loadEditRecords(edits, false,
|
|
|
- expectedStartingTxId, recovery);
|
|
|
+ long numEdits = loadEditRecords(edits, false, expectedStartingTxId,
|
|
|
+ startOpt, recovery);
|
|
|
FSImage.LOG.info("Edits file " + edits.getName()
|
|
|
+ " of size " + edits.length() + " edits # " + numEdits
|
|
|
+ " loaded in " + (now()-startTime)/1000 + " seconds");
|
|
@@ -131,8 +139,8 @@ public class FSEditLogLoader {
|
|
|
}
|
|
|
|
|
|
long loadEditRecords(EditLogInputStream in, boolean closeOnExit,
|
|
|
- long expectedStartingTxId, MetaRecoveryContext recovery)
|
|
|
- throws IOException {
|
|
|
+ long expectedStartingTxId, StartupOption startOpt,
|
|
|
+ MetaRecoveryContext recovery) throws IOException {
|
|
|
FSDirectory fsDir = fsNamesys.dir;
|
|
|
|
|
|
EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts =
|
|
@@ -204,7 +212,8 @@ public class FSEditLogLoader {
|
|
|
}
|
|
|
}
|
|
|
try {
|
|
|
- long inodeId = applyEditLogOp(op, fsDir, in.getVersion(), lastInodeId);
|
|
|
+ long inodeId = applyEditLogOp(op, fsDir, startOpt,
|
|
|
+ in.getVersion(), lastInodeId);
|
|
|
if (lastInodeId < inodeId) {
|
|
|
lastInodeId = inodeId;
|
|
|
}
|
|
@@ -212,6 +221,10 @@ public class FSEditLogLoader {
|
|
|
throw e;
|
|
|
} catch (Throwable e) {
|
|
|
LOG.error("Encountered exception on operation " + op, e);
|
|
|
+ if (recovery == null) {
|
|
|
+ throw e instanceof IOException? (IOException)e: new IOException(e);
|
|
|
+ }
|
|
|
+
|
|
|
MetaRecoveryContext.editLogLoaderPrompt("Failed to " +
|
|
|
"apply edit log operation " + op + ": error " +
|
|
|
e.getMessage(), recovery, "applying edits");
|
|
@@ -288,7 +301,7 @@ public class FSEditLogLoader {
|
|
|
|
|
|
@SuppressWarnings("deprecation")
|
|
|
private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
|
|
|
- int logVersion, long lastInodeId) throws IOException {
|
|
|
+ StartupOption startOpt, int logVersion, long lastInodeId) throws IOException {
|
|
|
long inodeId = INodeId.GRANDFATHER_INODE_ID;
|
|
|
if (LOG.isTraceEnabled()) {
|
|
|
LOG.trace("replaying edit log: " + op);
|
|
@@ -656,7 +669,18 @@ public class FSEditLogLoader {
|
|
|
break;
|
|
|
}
|
|
|
case OP_UPGRADE_MARKER: {
|
|
|
- throw new UpgradeMarkerException();
|
|
|
+ if (startOpt == StartupOption.ROLLINGUPGRADE) {
|
|
|
+ if (startOpt.getRollingUpgradeStartupOption()
|
|
|
+ == RollingUpgradeStartupOption.ROLLBACK) {
|
|
|
+ throw new UpgradeMarkerException();
|
|
|
+ } else if (startOpt.getRollingUpgradeStartupOption()
|
|
|
+ == RollingUpgradeStartupOption.DOWNGRADE) {
|
|
|
+ //ignore upgrade marker
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw new RollingUpgradeException(
|
|
|
+ "Unexpected upgrade marker in edit log: op=" + op);
|
|
|
}
|
|
|
case OP_ADD_CACHE_DIRECTIVE: {
|
|
|
AddCacheDirectiveInfoOp addOp = (AddCacheDirectiveInfoOp) op;
|