|
@@ -86,7 +86,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
|
-import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
@@ -97,6 +96,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
|
|
|
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.UnderReplicatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
|
|
@@ -207,9 +207,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
private PermissionStatus defaultPermission;
|
|
|
// FSNamesystemMetrics counter variables
|
|
|
@Metric private MutableCounterInt expiredHeartbeats;
|
|
|
- private long capacityTotal = 0L, capacityUsed = 0L, capacityRemaining = 0L;
|
|
|
- private long blockPoolUsed = 0L;
|
|
|
- private int totalLoad = 0;
|
|
|
|
|
|
// Scan interval is not configurable.
|
|
|
private static final long DELEGATION_TOKEN_REMOVER_SCAN_INTERVAL =
|
|
@@ -221,24 +218,17 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
//
|
|
|
public FSDirectory dir;
|
|
|
private BlockManager blockManager;
|
|
|
-
|
|
|
+ private DatanodeStatistics datanodeStatistics;
|
|
|
+
|
|
|
// Block pool ID used by this namenode
|
|
|
String blockPoolId;
|
|
|
|
|
|
- /**
|
|
|
- * Stores a subset of datanodeMap, containing nodes that are considered alive.
|
|
|
- * The HeartbeatMonitor periodically checks for out-dated entries,
|
|
|
- * and removes them from the list.
|
|
|
- */
|
|
|
- public ArrayList<DatanodeDescriptor> heartbeats = new ArrayList<DatanodeDescriptor>();
|
|
|
-
|
|
|
public LeaseManager leaseManager = new LeaseManager(this);
|
|
|
|
|
|
//
|
|
|
// Threaded object that checks to see if we have been
|
|
|
// getting heartbeats from all clients.
|
|
|
//
|
|
|
- Daemon hbthread = null; // HeartbeatMonitor thread
|
|
|
public Daemon lmthread = null; // LeaseMonitor thread
|
|
|
Daemon smmthread = null; // SafeModeMonitor thread
|
|
|
|
|
@@ -248,9 +238,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
private volatile boolean fsRunning = true;
|
|
|
long systemStart = 0;
|
|
|
|
|
|
- // heartbeatRecheckInterval is how often namenode checks for expired datanodes
|
|
|
- private long heartbeatRecheckInterval;
|
|
|
-
|
|
|
//resourceRecheckInterval is how often namenode checks for the disk space availability
|
|
|
private long resourceRecheckInterval;
|
|
|
|
|
@@ -303,6 +290,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
checkAvailableResources();
|
|
|
this.systemStart = now();
|
|
|
this.blockManager = new BlockManager(this, conf);
|
|
|
+ this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
|
|
|
this.fsLock = new ReentrantReadWriteLock(true); // fair locking
|
|
|
setConfigurationParameters(conf);
|
|
|
dtSecretManager = createDelegationTokenSecretManager(conf);
|
|
@@ -333,10 +321,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
void activate(Configuration conf) throws IOException {
|
|
|
setBlockTotal();
|
|
|
blockManager.activate(conf);
|
|
|
- this.hbthread = new Daemon(new HeartbeatMonitor());
|
|
|
this.lmthread = new Daemon(leaseManager.new Monitor());
|
|
|
-
|
|
|
- hbthread.start();
|
|
|
lmthread.start();
|
|
|
|
|
|
this.nnrmthread = new Daemon(new NameNodeResourceMonitor());
|
|
@@ -463,10 +448,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
DFSConfigKeys.DFS_NAMENODE_UPGRADE_PERMISSION_DEFAULT);
|
|
|
this.defaultPermission = PermissionStatus.createImmutable(
|
|
|
fsOwner.getShortUserName(), supergroup, new FsPermission(filePermission));
|
|
|
-
|
|
|
- this.heartbeatRecheckInterval = conf.getInt(
|
|
|
- DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
|
|
|
- DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT); // 5 minutes
|
|
|
|
|
|
this.serverDefaults = new FsServerDefaults(
|
|
|
conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE),
|
|
@@ -512,7 +493,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
fsRunning = false;
|
|
|
try {
|
|
|
if (blockManager != null) blockManager.close();
|
|
|
- if (hbthread != null) hbthread.interrupt();
|
|
|
if (smmthread != null) smmthread.interrupt();
|
|
|
if (dtSecretManager != null) dtSecretManager.stopThreads();
|
|
|
if (nnrmthread != null) nnrmthread.interrupt();
|
|
@@ -622,7 +602,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* Set permissions for an existing file.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public void setPermission(String src, FsPermission permission)
|
|
|
+ void setPermission(String src, FsPermission permission)
|
|
|
throws AccessControlException, FileNotFoundException, SafeModeException,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
HdfsFileStatus resultingStat = null;
|
|
@@ -651,7 +631,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* Set owner for an existing file.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public void setOwner(String src, String username, String group)
|
|
|
+ void setOwner(String src, String username, String group)
|
|
|
throws AccessControlException, FileNotFoundException, SafeModeException,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
HdfsFileStatus resultingStat = null;
|
|
@@ -818,12 +798,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
lastBlock, last.isComplete());
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /** Create a LocatedBlock. */
|
|
|
- public LocatedBlock createLocatedBlock(final Block b, final DatanodeInfo[] locations,
|
|
|
- final long offset, final boolean corrupt) throws IOException {
|
|
|
- return new LocatedBlock(getExtendedBlock(b), locations, offset, corrupt);
|
|
|
- }
|
|
|
|
|
|
|
|
|
/**
|
|
@@ -1018,7 +992,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
/**
|
|
|
* Create a symbolic link.
|
|
|
*/
|
|
|
- public void createSymlink(String target, String link,
|
|
|
+ void createSymlink(String target, String link,
|
|
|
PermissionStatus dirPerms, boolean createParent)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
HdfsFileStatus resultingStat = null;
|
|
@@ -1988,7 +1962,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* @see ClientProtocol#delete(String, boolean) for detailed descriptoin and
|
|
|
* description of exceptions
|
|
|
*/
|
|
|
- public boolean delete(String src, boolean recursive)
|
|
|
+ boolean delete(String src, boolean recursive)
|
|
|
throws AccessControlException, SafeModeException,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
@@ -2118,7 +2092,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
/**
|
|
|
* Create all the necessary directories
|
|
|
*/
|
|
|
- public boolean mkdirs(String src, PermissionStatus permissions,
|
|
|
+ boolean mkdirs(String src, PermissionStatus permissions,
|
|
|
boolean createParent) throws IOException, UnresolvedLinkException {
|
|
|
boolean status = false;
|
|
|
if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
@@ -2536,7 +2510,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* @throws UnresolvedLinkException if symbolic link is encountered
|
|
|
* @throws IOException if other I/O error occurred
|
|
|
*/
|
|
|
- public DirectoryListing getListing(String src, byte[] startAfter,
|
|
|
+ DirectoryListing getListing(String src, byte[] startAfter,
|
|
|
boolean needLocation)
|
|
|
throws AccessControlException, UnresolvedLinkException, IOException {
|
|
|
DirectoryListing dl;
|
|
@@ -2606,7 +2580,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* @see #registerDatanode(DatanodeRegistration)
|
|
|
* @return registration ID
|
|
|
*/
|
|
|
- public String getRegistrationID() {
|
|
|
+ String getRegistrationID() {
|
|
|
return Storage.getRegistrationID(dir.fsImage.getStorage());
|
|
|
}
|
|
|
|
|
@@ -2627,7 +2601,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
throws IOException {
|
|
|
readLock();
|
|
|
try {
|
|
|
- final int maxTransfer = blockManager.maxReplicationStreams - xmitsInProgress;
|
|
|
+ final int maxTransfer = blockManager.getMaxReplicationStreams()
|
|
|
+ - xmitsInProgress;
|
|
|
DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
|
|
|
nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed,
|
|
|
xceiverCount, maxTransfer, failedVolumes);
|
|
@@ -2655,35 +2630,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void updateStats(DatanodeDescriptor node, boolean isAdded) {
|
|
|
- //
|
|
|
- // The statistics are protected by the heartbeat lock
|
|
|
- // For decommissioning/decommissioned nodes, only used capacity
|
|
|
- // is counted.
|
|
|
- //
|
|
|
- assert(Thread.holdsLock(heartbeats));
|
|
|
- if (isAdded) {
|
|
|
- capacityUsed += node.getDfsUsed();
|
|
|
- blockPoolUsed += node.getBlockPoolUsed();
|
|
|
- totalLoad += node.getXceiverCount();
|
|
|
- if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
|
|
|
- capacityTotal += node.getCapacity();
|
|
|
- capacityRemaining += node.getRemaining();
|
|
|
- } else {
|
|
|
- capacityTotal += node.getDfsUsed();
|
|
|
- }
|
|
|
- } else {
|
|
|
- capacityUsed -= node.getDfsUsed();
|
|
|
- blockPoolUsed -= node.getBlockPoolUsed();
|
|
|
- totalLoad -= node.getXceiverCount();
|
|
|
- if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
|
|
|
- capacityTotal -= node.getCapacity();
|
|
|
- capacityRemaining -= node.getRemaining();
|
|
|
- } else {
|
|
|
- capacityTotal -= node.getDfsUsed();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Returns whether or not there were available resources at the last check of
|
|
@@ -2735,86 +2681,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
-
|
|
|
- /**
|
|
|
- * Periodically calls heartbeatCheck() and updateBlockKey()
|
|
|
- */
|
|
|
- class HeartbeatMonitor implements Runnable {
|
|
|
- private long lastHeartbeatCheck;
|
|
|
- private long lastBlockKeyUpdate;
|
|
|
- /**
|
|
|
- */
|
|
|
- public void run() {
|
|
|
- while (fsRunning) {
|
|
|
- try {
|
|
|
- long now = now();
|
|
|
- if (lastHeartbeatCheck + heartbeatRecheckInterval < now) {
|
|
|
- heartbeatCheck();
|
|
|
- lastHeartbeatCheck = now;
|
|
|
- }
|
|
|
- if (blockManager.isBlockTokenEnabled()
|
|
|
- && (lastBlockKeyUpdate + blockManager.getBlockKeyUpdateInterval() < now)) {
|
|
|
- blockManager.updateBlockKey();
|
|
|
- lastBlockKeyUpdate = now;
|
|
|
- }
|
|
|
- } catch (Exception e) {
|
|
|
- FSNamesystem.LOG.error("Exception while checking heartbeat", e);
|
|
|
- }
|
|
|
- try {
|
|
|
- Thread.sleep(5000); // 5 seconds
|
|
|
- } catch (InterruptedException ie) {
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- public void setNodeReplicationLimit(int limit) {
|
|
|
- blockManager.maxReplicationStreams = limit;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Remove a datanode descriptor.
|
|
|
- * @param nodeID datanode ID.
|
|
|
- * @throws UnregisteredNodeException
|
|
|
- */
|
|
|
- public void removeDatanode(final DatanodeID nodeID
|
|
|
- ) throws UnregisteredNodeException {
|
|
|
- writeLock();
|
|
|
- try {
|
|
|
- DatanodeDescriptor nodeInfo = getBlockManager().getDatanodeManager(
|
|
|
- ).getDatanode(nodeID);
|
|
|
- if (nodeInfo != null) {
|
|
|
- removeDatanode(nodeInfo);
|
|
|
- } else {
|
|
|
- NameNode.stateChangeLog.warn("BLOCK* NameSystem.removeDatanode: "
|
|
|
- + nodeID.getName() + " does not exist");
|
|
|
- }
|
|
|
- } finally {
|
|
|
- writeUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
- /**
|
|
|
- * Remove a datanode descriptor.
|
|
|
- * @param nodeInfo datanode descriptor.
|
|
|
- */
|
|
|
- public void removeDatanode(DatanodeDescriptor nodeInfo) {
|
|
|
- assert hasWriteLock();
|
|
|
- synchronized (heartbeats) {
|
|
|
- if (nodeInfo.isAlive) {
|
|
|
- updateStats(nodeInfo, false);
|
|
|
- heartbeats.remove(nodeInfo);
|
|
|
- nodeInfo.isAlive = false;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- blockManager.removeDatanode(nodeInfo);
|
|
|
-
|
|
|
- checkSafeMode();
|
|
|
- }
|
|
|
-
|
|
|
FSImage getFSImage() {
|
|
|
return dir.fsImage;
|
|
|
}
|
|
@@ -2822,61 +2689,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
FSEditLog getEditLog() {
|
|
|
return getFSImage().getEditLog();
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Check if there are any expired heartbeats, and if so,
|
|
|
- * whether any blocks have to be re-replicated.
|
|
|
- * While removing dead datanodes, make sure that only one datanode is marked
|
|
|
- * dead at a time within the synchronized section. Otherwise, a cascading
|
|
|
- * effect causes more datanodes to be declared dead.
|
|
|
- */
|
|
|
- void heartbeatCheck() {
|
|
|
- final DatanodeManager datanodeManager = getBlockManager().getDatanodeManager();
|
|
|
- // It's OK to check safe mode w/o taking the lock here, we re-check
|
|
|
- // for safe mode after taking the lock before removing a datanode.
|
|
|
- if (isInSafeMode()) {
|
|
|
- return;
|
|
|
- }
|
|
|
- boolean allAlive = false;
|
|
|
- while (!allAlive) {
|
|
|
- boolean foundDead = false;
|
|
|
- DatanodeID nodeID = null;
|
|
|
-
|
|
|
- // locate the first dead node.
|
|
|
- synchronized(heartbeats) {
|
|
|
- for (Iterator<DatanodeDescriptor> it = heartbeats.iterator();
|
|
|
- it.hasNext();) {
|
|
|
- DatanodeDescriptor nodeInfo = it.next();
|
|
|
- if (datanodeManager.isDatanodeDead(nodeInfo)) {
|
|
|
- expiredHeartbeats.incr();
|
|
|
- foundDead = true;
|
|
|
- nodeID = nodeInfo;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // acquire the fsnamesystem lock, and then remove the dead node.
|
|
|
- if (foundDead) {
|
|
|
- writeLock();
|
|
|
- if (isInSafeMode()) {
|
|
|
- return;
|
|
|
- }
|
|
|
- try {
|
|
|
- datanodeManager.removeDeadDatanode(nodeID);
|
|
|
- } finally {
|
|
|
- writeUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
- allAlive = !foundDead;
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* The given node is reporting all its blocks. Use this info to
|
|
|
* update the (machine-->blocklist) and (block-->machinelist) tables.
|
|
|
*/
|
|
|
- public void processReport(DatanodeID nodeID, String poolId,
|
|
|
+ void processReport(DatanodeID nodeID, String poolId,
|
|
|
BlockListAsLongs newReport) throws IOException {
|
|
|
long startTime, endTime;
|
|
|
|
|
@@ -3057,15 +2875,18 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return blockManager.getMissingBlocksCount();
|
|
|
}
|
|
|
|
|
|
+ /** Increment expired heartbeat counter. */
|
|
|
+ public void incrExpiredHeartbeats() {
|
|
|
+ expiredHeartbeats.incr();
|
|
|
+ }
|
|
|
+
|
|
|
+ /** @see ClientProtocol#getStats() */
|
|
|
long[] getStats() {
|
|
|
- synchronized(heartbeats) {
|
|
|
- return new long[] {this.capacityTotal, this.capacityUsed,
|
|
|
- this.capacityRemaining,
|
|
|
- getUnderReplicatedBlocks(),
|
|
|
- getCorruptReplicaBlocks(),
|
|
|
- getMissingBlocksCount(),
|
|
|
- getBlockPoolUsedSpace()};
|
|
|
- }
|
|
|
+ final long[] stats = datanodeStatistics.getStats();
|
|
|
+ stats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = getUnderReplicatedBlocks();
|
|
|
+ stats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = getCorruptReplicaBlocks();
|
|
|
+ stats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = getMissingBlocksCount();
|
|
|
+ return stats;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -3073,9 +2894,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
*/
|
|
|
@Override // FSNamesystemMBean
|
|
|
public long getCapacityTotal() {
|
|
|
- synchronized(heartbeats) {
|
|
|
- return capacityTotal;
|
|
|
- }
|
|
|
+ return datanodeStatistics.getCapacityTotal();
|
|
|
}
|
|
|
|
|
|
@Metric
|
|
@@ -3088,9 +2907,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
*/
|
|
|
@Override // FSNamesystemMBean
|
|
|
public long getCapacityUsed() {
|
|
|
- synchronized(heartbeats) {
|
|
|
- return capacityUsed;
|
|
|
- }
|
|
|
+ return datanodeStatistics.getCapacityUsed();
|
|
|
}
|
|
|
|
|
|
@Metric
|
|
@@ -3098,32 +2915,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return DFSUtil.roundBytesToGB(getCapacityUsed());
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Total used space by data nodes as percentage of total capacity
|
|
|
- */
|
|
|
- public float getCapacityUsedPercent() {
|
|
|
- synchronized(heartbeats){
|
|
|
- return DFSUtil.getPercentUsed(capacityUsed, capacityTotal);
|
|
|
- }
|
|
|
- }
|
|
|
- /**
|
|
|
- * Total used space by data nodes for non DFS purposes such
|
|
|
- * as storing temporary files on the local file system
|
|
|
- */
|
|
|
- public long getCapacityUsedNonDFS() {
|
|
|
- long nonDFSUsed = 0;
|
|
|
- synchronized(heartbeats){
|
|
|
- nonDFSUsed = capacityTotal - capacityRemaining - capacityUsed;
|
|
|
- }
|
|
|
- return nonDFSUsed < 0 ? 0 : nonDFSUsed;
|
|
|
- }
|
|
|
- /**
|
|
|
- * Total non-used raw bytes.
|
|
|
- */
|
|
|
+ @Override
|
|
|
public long getCapacityRemaining() {
|
|
|
- synchronized(heartbeats) {
|
|
|
- return capacityRemaining;
|
|
|
- }
|
|
|
+ return datanodeStatistics.getCapacityRemaining();
|
|
|
}
|
|
|
|
|
|
@Metric
|
|
@@ -3131,23 +2925,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return DFSUtil.roundBytesToGB(getCapacityRemaining());
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Total remaining space by data nodes as percentage of total capacity
|
|
|
- */
|
|
|
- public float getCapacityRemainingPercent() {
|
|
|
- synchronized(heartbeats){
|
|
|
- return DFSUtil.getPercentRemaining(capacityRemaining, capacityTotal);
|
|
|
- }
|
|
|
- }
|
|
|
/**
|
|
|
* Total number of connections.
|
|
|
*/
|
|
|
@Override // FSNamesystemMBean
|
|
|
@Metric
|
|
|
public int getTotalLoad() {
|
|
|
- synchronized (heartbeats) {
|
|
|
- return this.totalLoad;
|
|
|
- }
|
|
|
+ return datanodeStatistics.getXceiverCount();
|
|
|
}
|
|
|
|
|
|
int getNumberOfDatanodes(DatanodeReportType type) {
|
|
@@ -3757,8 +3541,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
return isInSafeMode();
|
|
|
}
|
|
|
-
|
|
|
- private void checkSafeMode() {
|
|
|
+
|
|
|
+ /** Check and trigger safe mode. */
|
|
|
+ public void checkSafeMode() {
|
|
|
// safeMode is volatile, and may be set to null at any time
|
|
|
SafeModeInfo safeMode = this.safeMode;
|
|
|
if (safeMode != null) {
|
|
@@ -4131,11 +3916,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return blockManager.getUnderReplicatedBlocksCount();
|
|
|
}
|
|
|
|
|
|
- /** Return number of under-replicated but not missing blocks */
|
|
|
- public long getUnderReplicatedNotMissingBlocks() {
|
|
|
- return blockManager.getUnderReplicatedNotMissingBlocks();
|
|
|
- }
|
|
|
-
|
|
|
/** Returns number of blocks with corrupt replicas */
|
|
|
@Metric({"CorruptBlocks", "Number of blocks with corrupt replicas"})
|
|
|
public long getCorruptReplicaBlocks() {
|
|
@@ -4207,14 +3987,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
/**
|
|
|
* Sets the generation stamp for this filesystem
|
|
|
*/
|
|
|
- public void setGenerationStamp(long stamp) {
|
|
|
+ void setGenerationStamp(long stamp) {
|
|
|
generationStamp.setStamp(stamp);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Gets the generation stamp for this filesystem
|
|
|
*/
|
|
|
- public long getGenerationStamp() {
|
|
|
+ long getGenerationStamp() {
|
|
|
return generationStamp.getStamp();
|
|
|
}
|
|
|
|
|
@@ -4854,31 +4634,27 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
|
|
|
@Override // NameNodeMXBean
|
|
|
public long getNonDfsUsedSpace() {
|
|
|
- return getCapacityUsedNonDFS();
|
|
|
+ return datanodeStatistics.getCapacityUsedNonDFS();
|
|
|
}
|
|
|
|
|
|
@Override // NameNodeMXBean
|
|
|
public float getPercentUsed() {
|
|
|
- return getCapacityUsedPercent();
|
|
|
+ return datanodeStatistics.getCapacityUsedPercent();
|
|
|
}
|
|
|
|
|
|
@Override // NameNodeMXBean
|
|
|
public long getBlockPoolUsedSpace() {
|
|
|
- synchronized(heartbeats) {
|
|
|
- return blockPoolUsed;
|
|
|
- }
|
|
|
+ return datanodeStatistics.getBlockPoolUsed();
|
|
|
}
|
|
|
|
|
|
@Override // NameNodeMXBean
|
|
|
public float getPercentBlockPoolUsed() {
|
|
|
- synchronized(heartbeats) {
|
|
|
- return DFSUtil.getPercentUsed(blockPoolUsed, capacityTotal);
|
|
|
- }
|
|
|
+ return datanodeStatistics.getPercentBlockPoolUsed();
|
|
|
}
|
|
|
|
|
|
@Override // NameNodeMXBean
|
|
|
public float getPercentRemaining() {
|
|
|
- return getCapacityRemainingPercent();
|
|
|
+ return datanodeStatistics.getCapacityRemainingPercent();
|
|
|
}
|
|
|
|
|
|
@Override // NameNodeMXBean
|