|
@@ -195,7 +195,6 @@ 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.SecretManagerState;
|
|
|
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.BlockInfoStriped;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
@@ -236,6 +235,7 @@ import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.top.metrics.TopMetrics;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
|
|
@@ -316,8 +316,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
NameNodeMXBean {
|
|
|
public static final Log LOG = LogFactory.getLog(FSNamesystem.class);
|
|
|
|
|
|
- private final BlockIdManager blockIdManager;
|
|
|
-
|
|
|
boolean isAuditEnabled() {
|
|
|
return (!isDefaultAuditLogger || auditLog.isInfoEnabled())
|
|
|
&& !auditLoggers.isEmpty();
|
|
@@ -557,7 +555,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
void clear() {
|
|
|
dir.reset();
|
|
|
dtSecretManager.reset();
|
|
|
- blockIdManager.clear();
|
|
|
leaseManager.removeAllLeases();
|
|
|
snapshotManager.clearSnapshottableDirs();
|
|
|
cacheManager.clear();
|
|
@@ -570,8 +567,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
LeaseManager getLeaseManager() {
|
|
|
return leaseManager;
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
+
|
|
|
public boolean isHaEnabled() {
|
|
|
return haEnabled;
|
|
|
}
|
|
@@ -728,9 +724,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
// block manager needs the haEnabled initialized
|
|
|
- this.blockManager = new BlockManager(this, conf);
|
|
|
+ this.blockManager = new BlockManager(this, haEnabled, conf);
|
|
|
this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
|
|
|
- this.blockIdManager = new BlockIdManager(blockManager);
|
|
|
|
|
|
// Get the checksum type from config
|
|
|
String checksumTypeStr = conf.get(DFS_CHECKSUM_TYPE_KEY,
|
|
@@ -1253,8 +1248,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
getFSImage().editLog.close();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
+
|
|
|
public void checkOperation(OperationCategory op) throws StandbyException {
|
|
|
if (haContext != null) {
|
|
|
// null in some unit tests
|
|
@@ -1542,8 +1536,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
public boolean isRunning() {
|
|
|
return fsRunning;
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
+
|
|
|
public boolean isInStandbyState() {
|
|
|
if (haContext == null || haContext.getState() == null) {
|
|
|
// We're still starting up. In this case, if HA is
|
|
@@ -1555,6 +1548,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return HAServiceState.STANDBY == haContext.getState().getServiceState();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * return a list of blocks & their locations on <code>datanode</code> whose
|
|
|
+ * total size is <code>size</code>
|
|
|
+ *
|
|
|
+ * @param datanode on which blocks are located
|
|
|
+ * @param size total size of blocks
|
|
|
+ */
|
|
|
+ public BlocksWithLocations getBlocks(DatanodeID datanode, long size)
|
|
|
+ throws IOException {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
+ readLock();
|
|
|
+ try {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
+ return getBlockManager().getBlocksWithLocations(datanode, size);
|
|
|
+ } finally {
|
|
|
+ readUnlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Dump all metadata into specified file
|
|
|
*/
|
|
@@ -3041,7 +3053,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
// start recovery of the last block for this file
|
|
|
long blockRecoveryId = nextGenerationStamp(
|
|
|
- blockIdManager.isLegacyBlock(lastBlock));
|
|
|
+ blockManager.isLegacyBlock(lastBlock));
|
|
|
lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
|
|
|
if(copyOnTruncate) {
|
|
|
lastBlock.setGenerationStamp(blockRecoveryId);
|
|
@@ -4482,11 +4494,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* Increments, logs and then returns the stamp
|
|
|
*/
|
|
|
long nextGenerationStamp(boolean legacyBlock)
|
|
|
- throws IOException, SafeModeException {
|
|
|
+ throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
checkNameNodeSafeMode("Cannot get next generation stamp");
|
|
|
|
|
|
- long gs = blockIdManager.nextGenerationStamp(legacyBlock);
|
|
|
+ long gs = blockManager.nextGenerationStamp(legacyBlock);
|
|
|
if (legacyBlock) {
|
|
|
getEditLog().logGenerationStampV1(gs);
|
|
|
} else {
|
|
@@ -4504,8 +4516,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
private long nextBlockId(boolean isStriped) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
checkNameNodeSafeMode("Cannot get next block ID");
|
|
|
- final long blockId = isStriped ?
|
|
|
- blockIdManager.nextStripedBlockId() : blockIdManager.nextContiguousBlockId();
|
|
|
+ final long blockId = blockManager.nextBlockId(isStriped);
|
|
|
getEditLog().logAllocateBlockId(blockId);
|
|
|
// NB: callers sync the log
|
|
|
return blockId;
|
|
@@ -4632,7 +4643,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
final INodeFile file = checkUCBlock(block, clientName);
|
|
|
|
|
|
// get a new generation stamp and an access token
|
|
|
- block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock())));
|
|
|
+ block.setGenerationStamp(nextGenerationStamp(
|
|
|
+ blockManager.isLegacyBlock(block.getLocalBlock())));
|
|
|
|
|
|
locatedBlock = BlockManager.newLocatedBlock(
|
|
|
block, file.getLastBlock(), null, -1);
|
|
@@ -5472,10 +5484,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return blockManager;
|
|
|
}
|
|
|
|
|
|
- public BlockIdManager getBlockIdManager() {
|
|
|
- return blockIdManager;
|
|
|
- }
|
|
|
-
|
|
|
/** @return the FSDirectory. */
|
|
|
public FSDirectory getFSDirectory() {
|
|
|
return dir;
|
|
@@ -5611,11 +5619,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
throw it;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- @Override
|
|
|
- public boolean isGenStampInFuture(Block block) {
|
|
|
- return blockIdManager.isGenStampInFuture(block);
|
|
|
- }
|
|
|
|
|
|
@VisibleForTesting
|
|
|
public EditLogTailer getEditLogTailer() {
|