|
@@ -210,6 +210,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState;
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
|
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
@@ -217,8 +218,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException;
|
|
|
|
-import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
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.NamenodeRole;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
|
|
@@ -322,6 +321,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
}
|
|
}
|
|
};
|
|
};
|
|
|
|
|
|
|
|
+ private final BlockIdManager blockIdManager;
|
|
|
|
+
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
public boolean isAuditEnabled() {
|
|
public boolean isAuditEnabled() {
|
|
return !isDefaultAuditLogger || auditLog.isInfoEnabled();
|
|
return !isDefaultAuditLogger || auditLog.isInfoEnabled();
|
|
@@ -481,34 +482,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
private final long minBlockSize; // minimum block size
|
|
private final long minBlockSize; // minimum block size
|
|
private final long maxBlocksPerFile; // maximum # of blocks per file
|
|
private final long maxBlocksPerFile; // maximum # of blocks per file
|
|
|
|
|
|
- /**
|
|
|
|
- * The global generation stamp for legacy blocks with randomly
|
|
|
|
- * generated block IDs.
|
|
|
|
- */
|
|
|
|
- private final GenerationStamp generationStampV1 = new GenerationStamp();
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * The global generation stamp for this file system.
|
|
|
|
- */
|
|
|
|
- private final GenerationStamp generationStampV2 = new GenerationStamp();
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * The value of the generation stamp when the first switch to sequential
|
|
|
|
- * block IDs was made. Blocks with generation stamps below this value
|
|
|
|
- * have randomly allocated block IDs. Blocks with generation stamps above
|
|
|
|
- * this value had sequentially allocated block IDs. Read from the fsImage
|
|
|
|
- * (or initialized as an offset from the V1 (legacy) generation stamp on
|
|
|
|
- * upgrade).
|
|
|
|
- */
|
|
|
|
- private long generationStampV1Limit =
|
|
|
|
- GenerationStamp.GRANDFATHER_GENERATION_STAMP;
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * The global block ID space for this file system.
|
|
|
|
- */
|
|
|
|
- @VisibleForTesting
|
|
|
|
- private final SequentialBlockIdGenerator blockIdGenerator;
|
|
|
|
-
|
|
|
|
// precision of access times.
|
|
// precision of access times.
|
|
private final long accessTimePrecision;
|
|
private final long accessTimePrecision;
|
|
|
|
|
|
@@ -640,11 +613,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
void clear() {
|
|
void clear() {
|
|
dir.reset();
|
|
dir.reset();
|
|
dtSecretManager.reset();
|
|
dtSecretManager.reset();
|
|
- generationStampV1.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
|
|
|
|
- generationStampV2.setCurrentValue(GenerationStamp.LAST_RESERVED_STAMP);
|
|
|
|
- blockIdGenerator.setCurrentValue(
|
|
|
|
- SequentialBlockIdGenerator.LAST_RESERVED_BLOCK_ID);
|
|
|
|
- generationStampV1Limit = GenerationStamp.GRANDFATHER_GENERATION_STAMP;
|
|
|
|
|
|
+ blockIdManager.clear();
|
|
leaseManager.removeAllLeases();
|
|
leaseManager.removeAllLeases();
|
|
inodeId.setCurrentValue(INodeId.LAST_RESERVED_ID);
|
|
inodeId.setCurrentValue(INodeId.LAST_RESERVED_ID);
|
|
snapshotManager.clearSnapshottableDirs();
|
|
snapshotManager.clearSnapshottableDirs();
|
|
@@ -794,7 +763,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
this.blockManager = new BlockManager(this, this, conf);
|
|
this.blockManager = new BlockManager(this, this, conf);
|
|
this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
|
|
this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
|
|
- this.blockIdGenerator = new SequentialBlockIdGenerator(this.blockManager);
|
|
|
|
|
|
+ this.blockIdManager = new BlockIdManager(blockManager);
|
|
|
|
|
|
this.isStoragePolicyEnabled =
|
|
this.isStoragePolicyEnabled =
|
|
conf.getBoolean(DFS_STORAGE_POLICY_ENABLED_KEY,
|
|
conf.getBoolean(DFS_STORAGE_POLICY_ENABLED_KEY,
|
|
@@ -1358,7 +1327,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
* @throws SafeModeException
|
|
* @throws SafeModeException
|
|
* Otherwise if NameNode is in SafeMode.
|
|
* Otherwise if NameNode is in SafeMode.
|
|
*/
|
|
*/
|
|
- private void checkNameNodeSafeMode(String errorMsg)
|
|
|
|
|
|
+ void checkNameNodeSafeMode(String errorMsg)
|
|
throws RetriableException, SafeModeException {
|
|
throws RetriableException, SafeModeException {
|
|
if (isInSafeMode()) {
|
|
if (isInSafeMode()) {
|
|
SafeModeException se = new SafeModeException(errorMsg, safeMode);
|
|
SafeModeException se = new SafeModeException(errorMsg, safeMode);
|
|
@@ -4598,7 +4567,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
// start recovery of the last block for this file
|
|
// start recovery of the last block for this file
|
|
- long blockRecoveryId = nextGenerationStamp(isLegacyBlock(uc));
|
|
|
|
|
|
+ long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc));
|
|
lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
|
|
lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
|
|
uc.initializeBlockRecovery(blockRecoveryId);
|
|
uc.initializeBlockRecovery(blockRecoveryId);
|
|
leaseManager.renewLease(lease);
|
|
leaseManager.renewLease(lease);
|
|
@@ -6734,91 +6703,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
return getBlockManager().getDatanodeManager().getNumStaleStorages();
|
|
return getBlockManager().getDatanodeManager().getNumStaleStorages();
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Sets the current generation stamp for legacy blocks
|
|
|
|
- */
|
|
|
|
- void setGenerationStampV1(long stamp) {
|
|
|
|
- generationStampV1.setCurrentValue(stamp);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Gets the current generation stamp for legacy blocks
|
|
|
|
- */
|
|
|
|
- long getGenerationStampV1() {
|
|
|
|
- return generationStampV1.getCurrentValue();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Gets the current generation stamp for this filesystem
|
|
|
|
- */
|
|
|
|
- void setGenerationStampV2(long stamp) {
|
|
|
|
- generationStampV2.setCurrentValue(stamp);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Gets the current generation stamp for this filesystem
|
|
|
|
- */
|
|
|
|
- long getGenerationStampV2() {
|
|
|
|
- return generationStampV2.getCurrentValue();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Upgrades the generation stamp for the filesystem
|
|
|
|
- * by reserving a sufficient range for all existing blocks.
|
|
|
|
- * Should be invoked only during the first upgrade to
|
|
|
|
- * sequential block IDs.
|
|
|
|
- */
|
|
|
|
- long upgradeGenerationStampToV2() {
|
|
|
|
- Preconditions.checkState(generationStampV2.getCurrentValue() ==
|
|
|
|
- GenerationStamp.LAST_RESERVED_STAMP);
|
|
|
|
-
|
|
|
|
- generationStampV2.skipTo(
|
|
|
|
- generationStampV1.getCurrentValue() +
|
|
|
|
- HdfsConstants.RESERVED_GENERATION_STAMPS_V1);
|
|
|
|
-
|
|
|
|
- generationStampV1Limit = generationStampV2.getCurrentValue();
|
|
|
|
- return generationStampV2.getCurrentValue();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Sets the generation stamp that delineates random and sequentially
|
|
|
|
- * allocated block IDs.
|
|
|
|
- * @param stamp set generation stamp limit to this value
|
|
|
|
- */
|
|
|
|
- void setGenerationStampV1Limit(long stamp) {
|
|
|
|
- Preconditions.checkState(generationStampV1Limit ==
|
|
|
|
- GenerationStamp.GRANDFATHER_GENERATION_STAMP);
|
|
|
|
- generationStampV1Limit = stamp;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Gets the value of the generation stamp that delineates sequential
|
|
|
|
- * and random block IDs.
|
|
|
|
- */
|
|
|
|
- long getGenerationStampAtblockIdSwitch() {
|
|
|
|
- return generationStampV1Limit;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @VisibleForTesting
|
|
|
|
- SequentialBlockIdGenerator getBlockIdGenerator() {
|
|
|
|
- return blockIdGenerator;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Sets the maximum allocated block ID for this filesystem. This is
|
|
|
|
- * the basis for allocating new block IDs.
|
|
|
|
- */
|
|
|
|
- void setLastAllocatedBlockId(long blockId) {
|
|
|
|
- blockIdGenerator.skipTo(blockId);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Gets the maximum sequentially allocated block ID for this filesystem
|
|
|
|
- */
|
|
|
|
- long getLastAllocatedBlockId() {
|
|
|
|
- return blockIdGenerator.getCurrentValue();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Increments, logs and then returns the stamp
|
|
* Increments, logs and then returns the stamp
|
|
*/
|
|
*/
|
|
@@ -6827,12 +6711,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
checkNameNodeSafeMode("Cannot get next generation stamp");
|
|
checkNameNodeSafeMode("Cannot get next generation stamp");
|
|
|
|
|
|
- long gs;
|
|
|
|
|
|
+ long gs = blockIdManager.nextGenerationStamp(legacyBlock);
|
|
if (legacyBlock) {
|
|
if (legacyBlock) {
|
|
- gs = getNextGenerationStampV1();
|
|
|
|
getEditLog().logGenerationStampV1(gs);
|
|
getEditLog().logGenerationStampV1(gs);
|
|
} else {
|
|
} else {
|
|
- gs = getNextGenerationStampV2();
|
|
|
|
getEditLog().logGenerationStampV2(gs);
|
|
getEditLog().logGenerationStampV2(gs);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -6840,47 +6722,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
return gs;
|
|
return gs;
|
|
}
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
|
- long getNextGenerationStampV1() throws IOException {
|
|
|
|
- long genStampV1 = generationStampV1.nextValue();
|
|
|
|
-
|
|
|
|
- if (genStampV1 >= generationStampV1Limit) {
|
|
|
|
- // We ran out of generation stamps for legacy blocks. In practice, it
|
|
|
|
- // is extremely unlikely as we reserved 1T v1 generation stamps. The
|
|
|
|
- // result is that we can no longer append to the legacy blocks that
|
|
|
|
- // were created before the upgrade to sequential block IDs.
|
|
|
|
- throw new OutOfV1GenerationStampsException();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- return genStampV1;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- @VisibleForTesting
|
|
|
|
- long getNextGenerationStampV2() {
|
|
|
|
- return generationStampV2.nextValue();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- long getGenerationStampV1Limit() {
|
|
|
|
- return generationStampV1Limit;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Determine whether the block ID was randomly generated (legacy) or
|
|
|
|
- * sequentially generated. The generation stamp value is used to
|
|
|
|
- * make the distinction.
|
|
|
|
- * @return true if the block ID was randomly generated, false otherwise.
|
|
|
|
- */
|
|
|
|
- boolean isLegacyBlock(Block block) {
|
|
|
|
- return block.getGenerationStamp() < getGenerationStampV1Limit();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Increments, logs and then returns the block ID
|
|
* Increments, logs and then returns the block ID
|
|
*/
|
|
*/
|
|
private long nextBlockId() throws IOException {
|
|
private long nextBlockId() throws IOException {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
checkNameNodeSafeMode("Cannot get next block ID");
|
|
checkNameNodeSafeMode("Cannot get next block ID");
|
|
- final long blockId = blockIdGenerator.nextValue();
|
|
|
|
|
|
+ final long blockId = blockIdManager.nextBlockId();
|
|
getEditLog().logAllocateBlockId(blockId);
|
|
getEditLog().logAllocateBlockId(blockId);
|
|
// NB: callers sync the log
|
|
// NB: callers sync the log
|
|
return blockId;
|
|
return blockId;
|
|
@@ -6995,8 +6843,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
checkUCBlock(block, clientName);
|
|
checkUCBlock(block, clientName);
|
|
|
|
|
|
// get a new generation stamp and an access token
|
|
// get a new generation stamp and an access token
|
|
- block.setGenerationStamp(
|
|
|
|
- nextGenerationStamp(isLegacyBlock(block.getLocalBlock())));
|
|
|
|
|
|
+ block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock())));
|
|
locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
|
|
locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
|
|
blockManager.setBlockToken(locatedBlock, AccessMode.WRITE);
|
|
blockManager.setBlockToken(locatedBlock, AccessMode.WRITE);
|
|
} finally {
|
|
} finally {
|
|
@@ -7868,6 +7715,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
public BlockManager getBlockManager() {
|
|
public BlockManager getBlockManager() {
|
|
return blockManager;
|
|
return blockManager;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ public BlockIdManager getBlockIdManager() {
|
|
|
|
+ return blockIdManager;
|
|
|
|
+ }
|
|
|
|
+
|
|
/** @return the FSDirectory. */
|
|
/** @return the FSDirectory. */
|
|
public FSDirectory getFSDirectory() {
|
|
public FSDirectory getFSDirectory() {
|
|
return dir;
|
|
return dir;
|
|
@@ -7935,11 +7787,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public boolean isGenStampInFuture(Block block) {
|
|
public boolean isGenStampInFuture(Block block) {
|
|
- if (isLegacyBlock(block)) {
|
|
|
|
- return block.getGenerationStamp() > getGenerationStampV1();
|
|
|
|
- } else {
|
|
|
|
- return block.getGenerationStamp() > getGenerationStampV2();
|
|
|
|
- }
|
|
|
|
|
|
+ return blockIdManager.isGenStampInFuture(block);
|
|
}
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|