|
@@ -113,7 +113,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
|
|
|
-import org.apache.hadoop.hdfs.util.RwLock;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
@@ -149,7 +148,7 @@ import org.mortbay.util.ajax.JSON;
|
|
|
***************************************************/
|
|
|
@InterfaceAudience.Private
|
|
|
@Metrics(context="dfs")
|
|
|
-public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
+public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
FSNamesystemMBean, NameNodeMXBean {
|
|
|
static final Log LOG = LogFactory.getLog(FSNamesystem.class);
|
|
|
|
|
@@ -517,7 +516,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** Is this name system running? */
|
|
|
+ @Override
|
|
|
public boolean isRunning() {
|
|
|
return fsRunning;
|
|
|
}
|
|
@@ -1200,13 +1199,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
leaseManager.addLease(cons.getClientName(), src);
|
|
|
|
|
|
// convert last block to under-construction
|
|
|
- LocatedBlock lb =
|
|
|
- blockManager.convertLastBlockToUnderConstruction(cons);
|
|
|
-
|
|
|
- if (lb != null) {
|
|
|
- blockManager.setBlockToken(lb, AccessMode.WRITE);
|
|
|
- }
|
|
|
- return lb;
|
|
|
+ return blockManager.convertLastBlockToUnderConstruction(cons);
|
|
|
} else {
|
|
|
// Now we can add the name to the filesystem. This file has no
|
|
|
// blocks associated with it.
|
|
@@ -1443,8 +1436,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
INodeFileUnderConstruction pendingFile = checkLease(src, clientName);
|
|
|
|
|
|
// commit the last block and complete it if it has minimum replicas
|
|
|
- blockManager.commitOrCompleteLastBlock(pendingFile, ExtendedBlock
|
|
|
- .getLocalBlock(previous));
|
|
|
+ commitOrCompleteLastBlock(pendingFile, ExtendedBlock.getLocalBlock(previous));
|
|
|
|
|
|
//
|
|
|
// If we fail this, bad things happen!
|
|
@@ -1643,7 +1635,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
|
|
|
INodeFileUnderConstruction pendingFile = checkLease(src, holder);
|
|
|
// commit the last block and complete it if it has minimum replicas
|
|
|
- blockManager.commitOrCompleteLastBlock(pendingFile, last);
|
|
|
+ commitOrCompleteLastBlock(pendingFile, last);
|
|
|
|
|
|
if (!checkFileProgress(pendingFile, true)) {
|
|
|
return false;
|
|
@@ -2256,10 +2248,12 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
return leaseManager.reassignLease(lease, src, newHolder);
|
|
|
}
|
|
|
|
|
|
- /** Update disk space consumed. */
|
|
|
- public void updateDiskSpaceConsumed(final INodeFileUnderConstruction fileINode,
|
|
|
+ private void commitOrCompleteLastBlock(final INodeFileUnderConstruction fileINode,
|
|
|
final Block commitBlock) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
+ if (!blockManager.commitOrCompleteLastBlock(fileINode, commitBlock)) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
|
|
|
// Adjust disk space consumption if required
|
|
|
final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();
|
|
@@ -2366,7 +2360,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
src = leaseManager.findPath(pendingFile);
|
|
|
if (closeFile) {
|
|
|
// commit the last block and complete it if it has minimum replicas
|
|
|
- blockManager.commitOrCompleteLastBlock(pendingFile, storedBlock);
|
|
|
+ commitOrCompleteLastBlock(pendingFile, storedBlock);
|
|
|
|
|
|
//remove lease, close file
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile);
|
|
@@ -2806,7 +2800,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
*
|
|
|
* @param conf configuration
|
|
|
*/
|
|
|
- SafeModeInfo(Configuration conf) {
|
|
|
+ private SafeModeInfo(Configuration conf) {
|
|
|
this.threshold = conf.getFloat(DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_KEY,
|
|
|
DFS_NAMENODE_SAFEMODE_THRESHOLD_PCT_DEFAULT);
|
|
|
this.datanodeThreshold = conf.getInt(
|
|
@@ -2850,7 +2844,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
* Check if safe mode is on.
|
|
|
* @return true if in safe mode
|
|
|
*/
|
|
|
- synchronized boolean isOn() {
|
|
|
+ private synchronized boolean isOn() {
|
|
|
try {
|
|
|
assert isConsistent() : " SafeMode: Inconsistent filesystem state: "
|
|
|
+ "Total num of blocks, active blocks, or "
|
|
@@ -2864,14 +2858,14 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
/**
|
|
|
* Check if we are populating replication queues.
|
|
|
*/
|
|
|
- synchronized boolean isPopulatingReplQueues() {
|
|
|
+ private synchronized boolean isPopulatingReplQueues() {
|
|
|
return initializedReplQueues;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Enter safe mode.
|
|
|
*/
|
|
|
- void enter() {
|
|
|
+ private void enter() {
|
|
|
this.reached = 0;
|
|
|
}
|
|
|
|
|
@@ -2881,7 +2875,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
* Switch to manual safe mode if distributed upgrade is required.<br>
|
|
|
* Check for invalid, under- & over-replicated blocks in the end of startup.
|
|
|
*/
|
|
|
- synchronized void leave(boolean checkForUpgrades) {
|
|
|
+ private synchronized void leave(boolean checkForUpgrades) {
|
|
|
if(checkForUpgrades) {
|
|
|
// verify whether a distributed upgrade needs to be started
|
|
|
boolean needUpgrade = false;
|
|
@@ -2921,7 +2915,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
/**
|
|
|
* Initialize replication queues.
|
|
|
*/
|
|
|
- synchronized void initializeReplQueues() {
|
|
|
+ private synchronized void initializeReplQueues() {
|
|
|
LOG.info("initializing replication queues");
|
|
|
if (isPopulatingReplQueues()) {
|
|
|
LOG.warn("Replication queues already initialized.");
|
|
@@ -2939,7 +2933,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
* Check whether we have reached the threshold for
|
|
|
* initializing replication queues.
|
|
|
*/
|
|
|
- synchronized boolean canInitializeReplQueues() {
|
|
|
+ private synchronized boolean canInitializeReplQueues() {
|
|
|
return blockSafe >= blockReplQueueThreshold;
|
|
|
}
|
|
|
|
|
@@ -2949,7 +2943,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
* the extension time have passed.
|
|
|
* @return true if can leave or false otherwise.
|
|
|
*/
|
|
|
- synchronized boolean canLeave() {
|
|
|
+ private synchronized boolean canLeave() {
|
|
|
if (reached == 0)
|
|
|
return false;
|
|
|
if (now() - reached < extension) {
|
|
@@ -2963,7 +2957,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
* There is no need to enter safe mode
|
|
|
* if DFS is empty or {@link #threshold} == 0
|
|
|
*/
|
|
|
- boolean needEnter() {
|
|
|
+ private boolean needEnter() {
|
|
|
return (threshold != 0 && blockSafe < blockThreshold) ||
|
|
|
(getNumLiveDataNodes() < datanodeThreshold) ||
|
|
|
(!nameNodeHasResourcesAvailable());
|
|
@@ -3007,7 +3001,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
/**
|
|
|
* Set total number of blocks.
|
|
|
*/
|
|
|
- synchronized void setBlockTotal(int total) {
|
|
|
+ private synchronized void setBlockTotal(int total) {
|
|
|
this.blockTotal = total;
|
|
|
this.blockThreshold = (int) (blockTotal * threshold);
|
|
|
this.blockReplQueueThreshold =
|
|
@@ -3020,7 +3014,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
* reached minimal replication.
|
|
|
* @param replication current replication
|
|
|
*/
|
|
|
- synchronized void incrementSafeBlockCount(short replication) {
|
|
|
+ private synchronized void incrementSafeBlockCount(short replication) {
|
|
|
if ((int)replication == safeReplication)
|
|
|
this.blockSafe++;
|
|
|
checkMode();
|
|
@@ -3031,7 +3025,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
* fallen below minimal replication.
|
|
|
* @param replication current replication
|
|
|
*/
|
|
|
- synchronized void decrementSafeBlockCount(short replication) {
|
|
|
+ private synchronized void decrementSafeBlockCount(short replication) {
|
|
|
if (replication == safeReplication-1)
|
|
|
this.blockSafe--;
|
|
|
checkMode();
|
|
@@ -3041,28 +3035,28 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
* Check if safe mode was entered manually or automatically (at startup, or
|
|
|
* when disk space is low).
|
|
|
*/
|
|
|
- boolean isManual() {
|
|
|
+ private boolean isManual() {
|
|
|
return extension == Integer.MAX_VALUE && !resourcesLow;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Set manual safe mode.
|
|
|
*/
|
|
|
- synchronized void setManual() {
|
|
|
+ private synchronized void setManual() {
|
|
|
extension = Integer.MAX_VALUE;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Check if safe mode was entered due to resources being low.
|
|
|
*/
|
|
|
- boolean areResourcesLow() {
|
|
|
+ private boolean areResourcesLow() {
|
|
|
return resourcesLow;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Set that resources are low for this instance of safe mode.
|
|
|
*/
|
|
|
- void setResourcesLow() {
|
|
|
+ private void setResourcesLow() {
|
|
|
resourcesLow = true;
|
|
|
}
|
|
|
|
|
@@ -3139,9 +3133,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
lastStatusReport = curTime;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Returns printable state of the class.
|
|
|
- */
|
|
|
+ @Override
|
|
|
public String toString() {
|
|
|
String resText = "Current safe blocks = "
|
|
|
+ blockSafe
|
|
@@ -3156,7 +3148,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
* Checks consistency of the class state.
|
|
|
* This is costly and currently called only in assert.
|
|
|
*/
|
|
|
- boolean isConsistent() throws IOException {
|
|
|
+ private boolean isConsistent() throws IOException {
|
|
|
if (blockTotal == -1 && blockSafe == -1) {
|
|
|
return true; // manual safe mode
|
|
|
}
|
|
@@ -3215,7 +3207,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
return isInSafeMode();
|
|
|
}
|
|
|
|
|
|
- /** Check and trigger safe mode. */
|
|
|
+ @Override
|
|
|
public void checkSafeMode() {
|
|
|
// safeMode is volatile, and may be set to null at any time
|
|
|
SafeModeInfo safeMode = this.safeMode;
|
|
@@ -3224,10 +3216,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Check whether the name node is in safe mode.
|
|
|
- * @return true if safe mode is ON, false otherwise
|
|
|
- */
|
|
|
+ @Override
|
|
|
public boolean isInSafeMode() {
|
|
|
// safeMode is volatile, and may be set to null at any time
|
|
|
SafeModeInfo safeMode = this.safeMode;
|
|
@@ -3235,10 +3224,8 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
return false;
|
|
|
return safeMode.isOn();
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Check whether the name node is in startup mode.
|
|
|
- */
|
|
|
+
|
|
|
+ @Override
|
|
|
public boolean isInStartupSafeMode() {
|
|
|
// safeMode is volatile, and may be set to null at any time
|
|
|
SafeModeInfo safeMode = this.safeMode;
|
|
@@ -3247,9 +3234,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
return !safeMode.isManual() && safeMode.isOn();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Check whether replication queues are populated.
|
|
|
- */
|
|
|
+ @Override
|
|
|
public boolean isPopulatingReplQueues() {
|
|
|
// safeMode is volatile, and may be set to null at any time
|
|
|
SafeModeInfo safeMode = this.safeMode;
|
|
@@ -3258,10 +3243,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
return safeMode.isPopulatingReplQueues();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Increment number of blocks that reached minimal replication.
|
|
|
- * @param replication current replication
|
|
|
- */
|
|
|
+ @Override
|
|
|
public void incrementSafeBlockCount(int replication) {
|
|
|
// safeMode is volatile, and may be set to null at any time
|
|
|
SafeModeInfo safeMode = this.safeMode;
|
|
@@ -3270,9 +3252,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
safeMode.incrementSafeBlockCount((short)replication);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Decrement number of blocks that reached minimal replication.
|
|
|
- */
|
|
|
+ @Override
|
|
|
public void decrementSafeBlockCount(Block b) {
|
|
|
// safeMode is volatile, and may be set to null at any time
|
|
|
SafeModeInfo safeMode = this.safeMode;
|
|
@@ -3397,10 +3377,6 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public long getTransactionID() {
|
|
|
- return getEditLog().getSyncTxId();
|
|
|
- }
|
|
|
-
|
|
|
CheckpointSignature rollEditLog() throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
@@ -3494,7 +3470,7 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
return checkPermission(path, false, null, null, null, null);
|
|
|
}
|
|
|
|
|
|
- /** Check if the user has superuser privilege. */
|
|
|
+ @Override
|
|
|
public void checkSuperuserPrivilege() throws AccessControlException {
|
|
|
if (isPermissionEnabled) {
|
|
|
FSPermissionChecker.checkSuperuserPrivilege(fsOwner, supergroup);
|
|
@@ -3916,10 +3892,6 @@ public class FSNamesystem implements RwLock, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public int numCorruptReplicas(Block blk) {
|
|
|
- return blockManager.numCorruptReplicas(blk);
|
|
|
- }
|
|
|
-
|
|
|
static class CorruptFileBlockInfo {
|
|
|
String path;
|
|
|
Block block;
|