|
@@ -213,7 +213,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
|
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
|
|
@@ -228,6 +227,7 @@ import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion.Feature;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
|
|
@@ -983,8 +983,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (needToSave) {
|
|
|
fsImage.saveNamespace(this);
|
|
|
} else {
|
|
|
- updateStorageVersionForRollingUpgrade(fsImage.getLayoutVersion(),
|
|
|
- startOpt);
|
|
|
// No need to save, so mark the phase done.
|
|
|
StartupProgress prog = NameNode.getStartupProgress();
|
|
|
prog.beginPhase(Phase.SAVING_CHECKPOINT);
|
|
@@ -994,7 +992,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
// we shouldn't do it when coming up in standby state
|
|
|
if (!haEnabled || (haEnabled && startOpt == StartupOption.UPGRADE)
|
|
|
|| (haEnabled && startOpt == StartupOption.UPGRADEONLY)) {
|
|
|
- fsImage.openEditLogForWrite();
|
|
|
+ fsImage.openEditLogForWrite(getEffectiveLayoutVersion());
|
|
|
}
|
|
|
success = true;
|
|
|
} finally {
|
|
@@ -1006,18 +1004,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
imageLoadComplete();
|
|
|
}
|
|
|
|
|
|
- private void updateStorageVersionForRollingUpgrade(final long layoutVersion,
|
|
|
- StartupOption startOpt) throws IOException {
|
|
|
- boolean rollingStarted = RollingUpgradeStartupOption.STARTED
|
|
|
- .matches(startOpt) && layoutVersion > HdfsServerConstants
|
|
|
- .NAMENODE_LAYOUT_VERSION;
|
|
|
- boolean rollingRollback = RollingUpgradeStartupOption.ROLLBACK
|
|
|
- .matches(startOpt);
|
|
|
- if (rollingRollback || rollingStarted) {
|
|
|
- fsImage.updateStorageVersion();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
private void startSecretManager() {
|
|
|
if (dtSecretManager != null) {
|
|
|
try {
|
|
@@ -1135,7 +1121,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
nextTxId);
|
|
|
editLog.setNextTxId(nextTxId);
|
|
|
|
|
|
- getFSImage().editLog.openForWrite();
|
|
|
+ getFSImage().editLog.openForWrite(getEffectiveLayoutVersion());
|
|
|
}
|
|
|
|
|
|
// Enable quota checks.
|
|
@@ -1995,6 +1981,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
String clientName, String clientMachine,
|
|
|
long mtime)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
+ requireEffectiveLayoutVersionForFeature(Feature.TRUNCATE);
|
|
|
boolean ret;
|
|
|
try {
|
|
|
ret = truncateInt(src, newLength, clientName, clientMachine, mtime);
|
|
@@ -2555,7 +2542,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
if (writeToEditLog) {
|
|
|
- getEditLog().logAppendFile(src, file, newBlock, logRetryCache);
|
|
|
+ if (NameNodeLayoutVersion.supports(Feature.APPEND_NEW_BLOCK,
|
|
|
+ getEffectiveLayoutVersion())) {
|
|
|
+ getEditLog().logAppendFile(src, file, newBlock, logRetryCache);
|
|
|
+ } else {
|
|
|
+ getEditLog().logOpenFile(src, file, false, logRetryCache);
|
|
|
+ }
|
|
|
}
|
|
|
return ret;
|
|
|
}
|
|
@@ -2750,9 +2742,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
LastBlockWithStatus appendFile(String src, String holder,
|
|
|
String clientMachine, EnumSet<CreateFlag> flag, boolean logRetryCache)
|
|
|
throws IOException {
|
|
|
+ boolean newBlock = flag.contains(CreateFlag.NEW_BLOCK);
|
|
|
+ if (newBlock) {
|
|
|
+ requireEffectiveLayoutVersionForFeature(Feature.APPEND_NEW_BLOCK);
|
|
|
+ }
|
|
|
try {
|
|
|
- return appendFileInt(src, holder, clientMachine,
|
|
|
- flag.contains(CreateFlag.NEW_BLOCK), logRetryCache);
|
|
|
+ return appendFileInt(src, holder, clientMachine, newBlock, logRetryCache);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "append", src);
|
|
|
throw e;
|
|
@@ -3361,6 +3356,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
*/
|
|
|
void setQuota(String src, long nsQuota, long ssQuota, StorageType type)
|
|
|
throws IOException {
|
|
|
+ if (type != null) {
|
|
|
+ requireEffectiveLayoutVersionForFeature(Feature.QUOTA_BY_STORAGE_TYPE);
|
|
|
+ }
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
boolean success = false;
|
|
@@ -5236,7 +5234,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (Server.isRpcInvocation()) {
|
|
|
LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
|
|
|
}
|
|
|
- return getFSImage().rollEditLog();
|
|
|
+ return getFSImage().rollEditLog(getEffectiveLayoutVersion());
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -5252,7 +5250,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
LOG.info("Start checkpoint for " + backupNode.getAddress());
|
|
|
NamenodeCommand cmd = getFSImage().startCheckpoint(backupNode,
|
|
|
- activeNamenode);
|
|
|
+ activeNamenode, getEffectiveLayoutVersion());
|
|
|
getEditLog().logSync();
|
|
|
return cmd;
|
|
|
} finally {
|
|
@@ -6876,7 +6874,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
getEditLog().logStartRollingUpgrade(rollingUpgradeInfo.getStartTime());
|
|
|
if (haEnabled) {
|
|
|
// roll the edit log to make sure the standby NameNode can tail
|
|
|
- getFSImage().rollEditLog();
|
|
|
+ getFSImage().rollEditLog(getEffectiveLayoutVersion());
|
|
|
}
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -6979,6 +6977,60 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return rollingUpgradeInfo != null && !rollingUpgradeInfo.isFinalized();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Returns the layout version in effect. Under normal operation, this is the
|
|
|
+ * same as the software's current layout version, defined in
|
|
|
+ * {@link NameNodeLayoutVersion#CURRENT_LAYOUT_VERSION}. During a rolling
|
|
|
+ * upgrade, this can retain the layout version that was persisted to metadata
|
|
|
+ * prior to starting the rolling upgrade, back to a lower bound defined in
|
|
|
+ * {@link NameNodeLayoutVersion#MINIMUM_COMPATIBLE_LAYOUT_VERSION}. New
|
|
|
+ * fsimage files and edit log segments will continue to be written with this
|
|
|
+ * older layout version, so that the files are still readable by the old
|
|
|
+ * software version if the admin chooses to downgrade.
|
|
|
+ *
|
|
|
+ * @return layout version in effect
|
|
|
+ */
|
|
|
+ public int getEffectiveLayoutVersion() {
|
|
|
+ if (isRollingUpgrade()) {
|
|
|
+ int storageLV = fsImage.getStorage().getLayoutVersion();
|
|
|
+ if (storageLV >=
|
|
|
+ NameNodeLayoutVersion.MINIMUM_COMPATIBLE_LAYOUT_VERSION) {
|
|
|
+ // The prior layout version satisfies the minimum compatible layout
|
|
|
+ // version of the current software. Keep reporting the prior layout
|
|
|
+ // as the effective one. Downgrade is possible.
|
|
|
+ return storageLV;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // The current software cannot satisfy the layout version of the prior
|
|
|
+ // software. Proceed with using the current layout version.
|
|
|
+ return NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Performs a pre-condition check that the layout version in effect is
|
|
|
+ * sufficient to support the requested {@link Feature}. If not, then the
|
|
|
+ * method throws {@link HadoopIllegalArgumentException} to deny the operation.
|
|
|
+ * This exception class is registered as a terse exception, so it prevents
|
|
|
+ * verbose stack traces in the NameNode log. During a rolling upgrade, this
|
|
|
+ * method is used to restrict usage of new features. This prevents writing
|
|
|
+ * new edit log operations that would be unreadable by the old software
|
|
|
+ * version if the admin chooses to downgrade.
|
|
|
+ *
|
|
|
+ * @param f feature to check
|
|
|
+ * @throws HadoopIllegalArgumentException if the current layout version in
|
|
|
+ * effect is insufficient to support the feature
|
|
|
+ */
|
|
|
+ private void requireEffectiveLayoutVersionForFeature(Feature f)
|
|
|
+ throws HadoopIllegalArgumentException {
|
|
|
+ int lv = getEffectiveLayoutVersion();
|
|
|
+ if (!NameNodeLayoutVersion.supports(f, lv)) {
|
|
|
+ throw new HadoopIllegalArgumentException(String.format(
|
|
|
+ "Feature %s unsupported at NameNode layout version %d. If a " +
|
|
|
+ "rolling upgrade is in progress, then it must be finalized before " +
|
|
|
+ "using this feature.", f, lv));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
void checkRollingUpgrade(String action) throws RollingUpgradeException {
|
|
|
if (isRollingUpgrade()) {
|
|
|
throw new RollingUpgradeException("Failed to " + action
|
|
@@ -7002,7 +7054,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
getEditLog().logFinalizeRollingUpgrade(rollingUpgradeInfo.getFinalizeTime());
|
|
|
if (haEnabled) {
|
|
|
// roll the edit log to make sure the standby NameNode can tail
|
|
|
- getFSImage().rollEditLog();
|
|
|
+ getFSImage().rollEditLog(getEffectiveLayoutVersion());
|
|
|
}
|
|
|
getFSImage().updateStorageVersion();
|
|
|
getFSImage().renameCheckpoint(NameNodeFile.IMAGE_ROLLBACK,
|