HDFS-8432. Introduce a minimum compatible layout version to allow downgrade in more rolling upgrade use cases. Contributed by Chris Nauroth.

(cherry picked from commit 71de367c5e)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
This commit is contained in:
cnauroth 2015-06-06 09:43:47 -07:00
parent 1d1841d40f
commit f99744b7a0
16 changed files with 301 additions and 72 deletions

View File

@ -263,6 +263,9 @@ Release 2.8.0 - UNRELEASED
HDFS-8535. Clarify that dfs usage in dfsadmin -report output includes all
block replicas. (Eddy Xu via wang)
HDFS-8432. Introduce a minimum compatible layout version to allow downgrade
in more rolling upgrade use cases. (cnauroth)
OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

View File

@ -162,14 +162,22 @@ public class LayoutVersion {
public static class FeatureInfo {
private final int lv;
private final int ancestorLV;
private final Integer minCompatLV;
private final String description;
private final boolean reserved;
private final LayoutFeature[] specialFeatures;
public FeatureInfo(final int lv, final int ancestorLV, final String description,
boolean reserved, LayoutFeature... specialFeatures) {
this(lv, ancestorLV, null, description, reserved, specialFeatures);
}
public FeatureInfo(final int lv, final int ancestorLV, Integer minCompatLV,
final String description, boolean reserved,
LayoutFeature... specialFeatures) {
this.lv = lv;
this.ancestorLV = ancestorLV;
this.minCompatLV = minCompatLV;
this.description = description;
this.reserved = reserved;
this.specialFeatures = specialFeatures;
@ -191,7 +199,20 @@ public class LayoutVersion {
return ancestorLV;
}
/**
/**
* Accessor method for feature minimum compatible layout version. If the
* feature does not define a minimum compatible layout version, then this
* method returns the feature's own layout version. This would indicate
* that the feature cannot provide compatibility with any prior layout
* version.
*
* @return int minimum compatible LV value
*/
public int getMinimumCompatibleLayoutVersion() {
return minCompatLV != null ? minCompatLV : lv;
}
/**
* Accessor method for feature description
* @return String feature description
*/
@ -220,8 +241,23 @@ public class LayoutVersion {
LayoutFeature[] features) {
// Go through all the enum constants and build a map of
// LayoutVersion <-> Set of all supported features in that LayoutVersion
SortedSet<LayoutFeature> existingFeatures = new TreeSet<LayoutFeature>(
new LayoutFeatureComparator());
for (SortedSet<LayoutFeature> s : map.values()) {
existingFeatures.addAll(s);
}
LayoutFeature prevF = existingFeatures.isEmpty() ? null :
existingFeatures.first();
for (LayoutFeature f : features) {
final FeatureInfo info = f.getInfo();
int minCompatLV = info.getMinimumCompatibleLayoutVersion();
if (prevF != null &&
minCompatLV > prevF.getInfo().getMinimumCompatibleLayoutVersion()) {
throw new AssertionError(String.format(
"Features must be listed in order of minimum compatible layout " +
"version. Check features %s and %s.", prevF, f));
}
prevF = f;
SortedSet<LayoutFeature> ancestorSet = map.get(info.getAncestorLayoutVersion());
if (ancestorSet == null) {
// Empty set
@ -282,6 +318,18 @@ public class LayoutVersion {
return getLastNonReservedFeature(features).getInfo().getLayoutVersion();
}
/**
* Gets the minimum compatible layout version.
*
* @param features all features to check
* @return minimum compatible layout version
*/
public static int getMinimumCompatibleLayoutVersion(
LayoutFeature[] features) {
return getLastNonReservedFeature(features).getInfo()
.getMinimumCompatibleLayoutVersion();
}
static LayoutFeature getLastNonReservedFeature(LayoutFeature[] features) {
for (int i = features.length -1; i >= 0; i--) {
final FeatureInfo info = features[i].getInfo();
@ -292,4 +340,3 @@ public class LayoutVersion {
throw new AssertionError("All layout versions are reserved.");
}
}

View File

@ -366,7 +366,8 @@ public class BackupImage extends FSImage {
}
}
editLog.setNextTxId(txid);
editLog.startLogSegment(txid, false);
editLog.startLogSegment(txid, false,
namesystem.getEffectiveLayoutVersion());
if (bnState == BNState.DROP_UNTIL_NEXT_ROLL) {
setState(BNState.JOURNAL_ONLY);
}

View File

@ -257,7 +257,9 @@ class Checkpointer extends Daemon {
backupNode.namesystem.setBlockTotal();
}
bnImage.saveFSImageInAllDirs(backupNode.getNamesystem(), txid);
bnStorage.writeAll();
if (!backupNode.namesystem.isRollingUpgrade()) {
bnStorage.writeAll();
}
} finally {
backupNode.namesystem.writeUnlock();
}

View File

@ -302,7 +302,7 @@ public class FSEditLog implements LogsPurgeable {
* Initialize the output stream for logging, opening the first
* log segment.
*/
synchronized void openForWrite() throws IOException {
synchronized void openForWrite(int layoutVersion) throws IOException {
Preconditions.checkState(state == State.BETWEEN_LOG_SEGMENTS,
"Bad state: %s", state);
@ -319,7 +319,7 @@ public class FSEditLog implements LogsPurgeable {
throw new IllegalStateException(error);
}
startLogSegment(segmentTxId, true);
startLogSegment(segmentTxId, true, layoutVersion);
assert state == State.IN_SEGMENT : "Bad state: " + state;
}
@ -1198,12 +1198,12 @@ public class FSEditLog implements LogsPurgeable {
* @return the transaction id of the BEGIN_LOG_SEGMENT transaction
* in the new log.
*/
synchronized long rollEditLog() throws IOException {
synchronized long rollEditLog(int layoutVersion) throws IOException {
LOG.info("Rolling edit logs");
endCurrentLogSegment(true);
long nextTxId = getLastWrittenTxId() + 1;
startLogSegment(nextTxId, true);
startLogSegment(nextTxId, true, layoutVersion);
assert curSegmentTxId == nextTxId;
return nextTxId;
@ -1214,7 +1214,7 @@ public class FSEditLog implements LogsPurgeable {
* Transitions from BETWEEN_LOG_SEGMENTS state to IN_LOG_SEGMENT state.
*/
synchronized void startLogSegment(final long segmentTxId,
boolean writeHeaderTxn) throws IOException {
boolean writeHeaderTxn, int layoutVersion) throws IOException {
LOG.info("Starting log segment at " + segmentTxId);
Preconditions.checkArgument(segmentTxId > 0,
"Bad txid: %s", segmentTxId);
@ -1234,8 +1234,7 @@ public class FSEditLog implements LogsPurgeable {
storage.attemptRestoreRemovedStorage();
try {
editLogStream = journalSet.startLogSegment(segmentTxId,
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
editLogStream = journalSet.startLogSegment(segmentTxId, layoutVersion);
} catch (IOException ex) {
throw new IOException("Unable to start log segment " +
segmentTxId + ": too few journals successfully started.", ex);

View File

@ -572,9 +572,9 @@ public class FSImage implements Closeable {
return editLog;
}
void openEditLogForWrite() throws IOException {
void openEditLogForWrite(int layoutVersion) throws IOException {
assert editLog != null : "editLog must be initialized";
editLog.openForWrite();
editLog.openForWrite(layoutVersion);
storage.writeTransactionIdFileToStorage(editLog.getCurSegmentTxId());
}
@ -1107,10 +1107,13 @@ public class FSImage implements Closeable {
try {
try {
saveFSImageInAllDirs(source, nnf, imageTxId, canceler);
storage.writeAll();
if (!source.isRollingUpgrade()) {
storage.writeAll();
}
} finally {
if (editLogWasOpen) {
editLog.startLogSegment(imageTxId + 1, true);
editLog.startLogSegment(imageTxId + 1, true,
source.getEffectiveLayoutVersion());
// Take this opportunity to note the current transaction.
// Even if the namespace save was cancelled, this marker
// is only used to determine what transaction ID is required
@ -1295,8 +1298,8 @@ public class FSImage implements Closeable {
}
}
CheckpointSignature rollEditLog() throws IOException {
getEditLog().rollEditLog();
CheckpointSignature rollEditLog(int layoutVersion) throws IOException {
getEditLog().rollEditLog(layoutVersion);
// Record this log segment ID in all of the storage directories, so
// we won't miss this log segment on a restart if the edits directories
// go missing.
@ -1321,7 +1324,8 @@ public class FSImage implements Closeable {
* @throws IOException
*/
NamenodeCommand startCheckpoint(NamenodeRegistration bnReg, // backup node
NamenodeRegistration nnReg) // active name-node
NamenodeRegistration nnReg,
int layoutVersion) // active name-node
throws IOException {
LOG.info("Start checkpoint at txid " + getEditLog().getLastWrittenTxId());
String msg = null;
@ -1350,7 +1354,7 @@ public class FSImage implements Closeable {
if(storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0)
// do not return image if there are no image directories
needToReturnImg = false;
CheckpointSignature sig = rollEditLog();
CheckpointSignature sig = rollEditLog(layoutVersion);
return new CheckpointCommand(sig, needToReturnImg);
}

View File

@ -465,7 +465,8 @@ public final class FSImageFormatProtobuf {
FileSummary.Builder b = FileSummary.newBuilder()
.setOndiskVersion(FSImageUtil.FILE_VERSION)
.setLayoutVersion(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
.setLayoutVersion(
context.getSourceNamesystem().getEffectiveLayoutVersion());
codec = compression.getImageCodec();
if (codec != null) {

View File

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

View File

@ -1159,7 +1159,8 @@ public class NameNode implements NameNodeStatusMXBean {
LOG.trace("copying op: " + op);
}
if (!segmentOpen) {
newSharedEditLog.startLogSegment(op.txid, false);
newSharedEditLog.startLogSegment(op.txid, false,
fsns.getEffectiveLayoutVersion());
segmentOpen = true;
}

View File

@ -35,6 +35,8 @@ public class NameNodeLayoutVersion {
public static final int CURRENT_LAYOUT_VERSION
= LayoutVersion.getCurrentLayoutVersion(Feature.values());
public static final int MINIMUM_COMPATIBLE_LAYOUT_VERSION
= LayoutVersion.getMinimumCompatibleLayoutVersion(Feature.values());
static {
LayoutVersion.updateMap(FEATURES, LayoutVersion.Feature.values());
@ -60,43 +62,59 @@ public class NameNodeLayoutVersion {
* its immediate predecessor, use the constructor where a specific ancestor
* can be passed.
* </li>
* <li>Specify a minimum compatible layout version. The minimum compatible
* layout version is the earliest prior version to which a downgrade is
* possible after initiating rolling upgrade. If the feature cannot satisfy
* compatibility with any prior version, then set its minimum compatible
* lqyout version to itself to indicate that downgrade is impossible.
* Satisfying compatibility might require adding logic to the new feature to
* reject operations or handle them differently while rolling upgrade is in
* progress. In general, it's possible to satisfy compatiblity for downgrade
* if the new feature just involves adding new edit log ops. Deeper
* structural changes, such as changing the way we place files in the metadata
* directories, might be incompatible. Feature implementations should strive
* for compatibility, because it's in the best interest of our users to
* support downgrade.
* </ul>
*/
public static enum Feature implements LayoutFeature {
ROLLING_UPGRADE(-55, -53, "Support rolling upgrade", false),
EDITLOG_LENGTH(-56, "Add length field to every edit log op"),
XATTRS(-57, "Extended attributes"),
CREATE_OVERWRITE(-58, "Use single editlog record for " +
ROLLING_UPGRADE(-55, -53, -55, "Support rolling upgrade", false),
EDITLOG_LENGTH(-56, -56, "Add length field to every edit log op"),
XATTRS(-57, -57, "Extended attributes"),
CREATE_OVERWRITE(-58, -58, "Use single editlog record for " +
"creating file with overwrite"),
XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"),
BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
TRUNCATE(-61, "Truncate"),
APPEND_NEW_BLOCK(-62, "Support appending to new block"),
QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types");
XATTRS_NAMESPACE_EXT(-59, -59, "Increase number of xattr namespaces"),
BLOCK_STORAGE_POLICY(-60, -60, "Block Storage policy"),
TRUNCATE(-61, -61, "Truncate"),
APPEND_NEW_BLOCK(-62, -61, "Support appending to new block"),
QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types");
private final FeatureInfo info;
/**
* Feature that is added at layout version {@code lv} - 1.
* @param lv new layout version with the addition of this feature
* @param minCompatLV minimium compatible layout version
* @param description description of the feature
*/
Feature(final int lv, final String description) {
this(lv, lv + 1, description, false);
Feature(final int lv, int minCompatLV, final String description) {
this(lv, lv + 1, minCompatLV, description, false);
}
/**
* NameNode feature that is added at layout version {@code ancestoryLV}.
* @param lv new layout version with the addition of this feature
* @param ancestorLV layout version from which the new lv is derived from.
* @param minCompatLV minimum compatible layout version
* @param description description of the feature
* @param reserved true when this is a layout version reserved for previous
* versions
* @param features set of features that are to be enabled for this version
*/
Feature(final int lv, final int ancestorLV, final String description,
boolean reserved, Feature... features) {
info = new FeatureInfo(lv, ancestorLV, description, reserved, features);
Feature(final int lv, final int ancestorLV, int minCompatLV,
final String description, boolean reserved, Feature... features) {
info = new FeatureInfo(lv, ancestorLV, minCompatLV, description, reserved,
features);
}
@Override

View File

@ -1084,6 +1084,8 @@ public class SecondaryNameNode implements Runnable,
Checkpointer.rollForwardByApplyingLogs(manifest, dstImage, dstNamesystem);
// The following has the side effect of purging old fsimages/edit logs.
dstImage.saveFSImageInAllDirs(dstNamesystem, dstImage.getLastAppliedTxId());
dstStorage.writeAll();
if (!dstNamesystem.isRollingUpgrade()) {
dstStorage.writeAll();
}
}
}

View File

@ -20,7 +20,13 @@ package org.apache.hadoop.hdfs.protocol;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.*;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.SortedSet;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
@ -103,7 +109,100 @@ public class TestLayoutVersion {
assertEquals(LAST_COMMON_FEATURE.getInfo().getLayoutVersion() - 1,
first.getInfo().getLayoutVersion());
}
/**
* Tests expected values for minimum compatible layout version in NameNode
* features. TRUNCATE, APPEND_NEW_BLOCK and QUOTA_BY_STORAGE_TYPE are all
* features that launched in the same release. TRUNCATE was added first, so
* we expect all 3 features to have a minimum compatible layout version equal
* to TRUNCATE's layout version. All features older than that existed prior
* to the concept of a minimum compatible layout version, so for each one, the
* minimum compatible layout version must be equal to itself.
*/
@Test
public void testNameNodeFeatureMinimumCompatibleLayoutVersions() {
int baseLV = NameNodeLayoutVersion.Feature.TRUNCATE.getInfo()
.getLayoutVersion();
EnumSet<NameNodeLayoutVersion.Feature> compatibleFeatures = EnumSet.of(
NameNodeLayoutVersion.Feature.TRUNCATE,
NameNodeLayoutVersion.Feature.APPEND_NEW_BLOCK,
NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE);
for (LayoutFeature f : compatibleFeatures) {
assertEquals(String.format("Expected minimum compatible layout version " +
"%d for feature %s.", baseLV, f), baseLV,
f.getInfo().getMinimumCompatibleLayoutVersion());
}
List<LayoutFeature> features = new ArrayList<>();
features.addAll(EnumSet.allOf(LayoutVersion.Feature.class));
features.addAll(EnumSet.allOf(NameNodeLayoutVersion.Feature.class));
for (LayoutFeature f : features) {
if (!compatibleFeatures.contains(f)) {
assertEquals(String.format("Expected feature %s to have minimum " +
"compatible layout version set to itself.", f),
f.getInfo().getLayoutVersion(),
f.getInfo().getMinimumCompatibleLayoutVersion());
}
}
}
/**
* Tests that NameNode features are listed in order of minimum compatible
* layout version. It would be inconsistent to have features listed out of
* order with respect to minimum compatible layout version, because it would
* imply going back in time to change compatibility logic in a software release
* that had already shipped.
*/
@Test
public void testNameNodeFeatureMinimumCompatibleLayoutVersionAscending() {
LayoutFeature prevF = null;
for (LayoutFeature f : EnumSet.allOf(NameNodeLayoutVersion.Feature.class)) {
if (prevF != null) {
assertTrue(String.format("Features %s and %s not listed in order of " +
"minimum compatible layout version.", prevF, f),
f.getInfo().getMinimumCompatibleLayoutVersion() <=
prevF.getInfo().getMinimumCompatibleLayoutVersion());
} else {
prevF = f;
}
}
}
/**
* Tests that attempting to add a new NameNode feature out of order with
* respect to minimum compatible layout version will fail fast.
*/
@Test(expected=AssertionError.class)
public void testNameNodeFeatureMinimumCompatibleLayoutVersionOutOfOrder() {
FeatureInfo ancestorF = LayoutVersion.Feature.RESERVED_REL2_4_0.getInfo();
LayoutFeature f = mock(LayoutFeature.class);
when(f.getInfo()).thenReturn(new FeatureInfo(
ancestorF.getLayoutVersion() - 1, ancestorF.getLayoutVersion(),
ancestorF.getMinimumCompatibleLayoutVersion() + 1, "Invalid feature.",
false));
Map<Integer, SortedSet<LayoutFeature>> features = new HashMap<>();
LayoutVersion.updateMap(features, LayoutVersion.Feature.values());
LayoutVersion.updateMap(features, new LayoutFeature[] { f });
}
/**
* Asserts the current minimum compatible layout version of the software, if a
* release were created from the codebase right now. This test is meant to
* make developers stop and reconsider if they introduce a change that requires
* a new minimum compatible layout version. This would make downgrade
* impossible.
*/
@Test
public void testCurrentMinimumCompatibleLayoutVersion() {
int expectedMinCompatLV = NameNodeLayoutVersion.Feature.TRUNCATE.getInfo()
.getLayoutVersion();
int actualMinCompatLV = LayoutVersion.getMinimumCompatibleLayoutVersion(
NameNodeLayoutVersion.Feature.values());
assertEquals("The minimum compatible layout version has changed. " +
"Downgrade to prior versions is no longer possible. Please either " +
"restore compatibility, or if the incompatibility is intentional, " +
"then update this assertion.", expectedMinCompatLV, actualMinCompatLV);
}
/**
* Given feature {@code f}, ensures the layout version of that feature
* supports all the features supported by it's ancestor.

View File

@ -203,7 +203,7 @@ public class CreateEditsLog {
FileNameGenerator nameGenerator = new FileNameGenerator(BASE_PATH, 100);
FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
editLog.openForWrite();
editLog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
addFiles(editLog, numFiles, replication, numBlocksPerFile, startingBlockId,
blockSize, nameGenerator);
editLog.logSync();

View File

@ -213,7 +213,7 @@ public abstract class FSImageTestUtil {
long firstTxId, long newInodeId) throws IOException {
FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
editLog.setNextTxId(firstTxId);
editLog.openForWrite();
editLog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
PermissionStatus perms = PermissionStatus.createImmutable("fakeuser", "fakegroup",
FsPermission.createImmutable((short)0755));

View File

@ -297,7 +297,7 @@ public class TestEditLog {
editLog.logSetReplication("fakefile", (short) 1);
editLog.logSync();
editLog.rollEditLog();
editLog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
assertExistsInStorageDirs(
cluster, NameNodeDirType.EDITS,
@ -370,7 +370,7 @@ public class TestEditLog {
// Roll log so new output buffer size takes effect
// we should now be writing to edits_inprogress_3
fsimage.rollEditLog();
fsimage.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
// Remember the current lastInodeId and will reset it back to test
// loading editlog segments.The transactions in the following allocate new
@ -401,7 +401,7 @@ public class TestEditLog {
trans.run();
// Roll another time to finalize edits_inprogress_3
fsimage.rollEditLog();
fsimage.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
long expectedTxns = ((NUM_THREADS+1) * 2 * NUM_TRANSACTIONS) + 2; // +2 for start/end txns
@ -940,7 +940,7 @@ public class TestEditLog {
FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir);
try {
FileUtil.setWritable(logDir, false);
log.openForWrite();
log.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
fail("Did no throw exception on only having a bad dir");
} catch (IOException ioe) {
GenericTestUtils.assertExceptionContains(
@ -965,7 +965,7 @@ public class TestEditLog {
new byte[500]);
try {
log.openForWrite();
log.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
NameNodeMetrics mockMetrics = Mockito.mock(NameNodeMetrics.class);
log.setMetricsForTests(mockMetrics);
@ -1139,7 +1139,7 @@ public class TestEditLog {
// logGenerationStamp is used, simply because it doesn't
// require complex arguments.
editlog.initJournalsForWrite();
editlog.openForWrite();
editlog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
for (int i = 2; i < TXNS_PER_ROLL; i++) {
editlog.logGenerationStampV2((long) 0);
}
@ -1151,7 +1151,7 @@ public class TestEditLog {
// the specified journal is aborted. It will be brought
// back into rotation automatically by rollEditLog
for (int i = 0; i < numrolls; i++) {
editlog.rollEditLog();
editlog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
editlog.logGenerationStampV2((long) i);
editlog.logSync();
@ -1483,7 +1483,7 @@ public class TestEditLog {
cluster, NameNodeDirType.EDITS,
NNStorage.getInProgressEditsFileName((i * 3) + 1));
editLog.logSync();
editLog.rollEditLog();
editLog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
assertExistsInStorageDirs(
cluster, NameNodeDirType.EDITS,
NNStorage.getFinalizedEditsFileName((i * 3) + 1, (i * 3) + 3));

View File

@ -285,7 +285,7 @@ public class TestFSEditLogLoader {
// FSEditLog#endCurrentLogSegment. For testing purposes, we
// disable that here.
doNothing().when(spyLog).endCurrentLogSegment(true);
spyLog.openForWrite();
spyLog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
assertTrue("should exist: " + inProgressFile, inProgressFile.exists());
for (int i = 0; i < numTx; i++) {