|
@@ -104,13 +104,13 @@ import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
|
|
|
import org.apache.hadoop.hdfs.server.common.Util;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
|
|
|
-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.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;
|
|
@@ -146,8 +146,8 @@ import org.mortbay.util.ajax.JSON;
|
|
|
***************************************************/
|
|
|
@InterfaceAudience.Private
|
|
|
@Metrics(context="dfs")
|
|
|
-public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
- FSClusterStats, NameNodeMXBean {
|
|
|
+public class FSNamesystem implements RwLock, FSConstants, FSClusterStats,
|
|
|
+ FSNamesystemMBean, NameNodeMXBean {
|
|
|
static final Log LOG = LogFactory.getLog(FSNamesystem.class);
|
|
|
|
|
|
private static final ThreadLocal<StringBuilder> auditBuffer =
|
|
@@ -211,20 +211,16 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
//
|
|
|
// Stores the correct file name hierarchy
|
|
|
//
|
|
|
- public FSDirectory dir;
|
|
|
+ FSDirectory dir;
|
|
|
private BlockManager blockManager;
|
|
|
private DatanodeStatistics datanodeStatistics;
|
|
|
|
|
|
// Block pool ID used by this namenode
|
|
|
- String blockPoolId;
|
|
|
+ private String blockPoolId;
|
|
|
|
|
|
- public LeaseManager leaseManager = new LeaseManager(this);
|
|
|
+ LeaseManager leaseManager = new LeaseManager(this);
|
|
|
|
|
|
- //
|
|
|
- // Threaded object that checks to see if we have been
|
|
|
- // getting heartbeats from all clients.
|
|
|
- //
|
|
|
- public Daemon lmthread = null; // LeaseMonitor thread
|
|
|
+ Daemon lmthread = null; // LeaseMonitor thread
|
|
|
Daemon smmthread = null; // SafeModeMonitor thread
|
|
|
|
|
|
Daemon nnrmthread = null; // NamenodeResourceMonitor thread
|
|
@@ -330,7 +326,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return getStorageDirs(conf, DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
|
|
|
}
|
|
|
|
|
|
- public static Collection<URI> getStorageDirs(Configuration conf,
|
|
|
+ private static Collection<URI> getStorageDirs(Configuration conf,
|
|
|
String propertyName) {
|
|
|
Collection<String> dirNames = conf.getTrimmedStringCollection(propertyName);
|
|
|
StartupOption startOpt = NameNode.getStartupOption(conf);
|
|
@@ -364,31 +360,31 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return getStorageDirs(conf, DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY);
|
|
|
}
|
|
|
|
|
|
- // utility methods to acquire and release read lock and write lock
|
|
|
+ @Override
|
|
|
public void readLock() {
|
|
|
this.fsLock.readLock().lock();
|
|
|
}
|
|
|
-
|
|
|
+ @Override
|
|
|
public void readUnlock() {
|
|
|
this.fsLock.readLock().unlock();
|
|
|
}
|
|
|
-
|
|
|
+ @Override
|
|
|
public void writeLock() {
|
|
|
this.fsLock.writeLock().lock();
|
|
|
}
|
|
|
-
|
|
|
+ @Override
|
|
|
public void writeUnlock() {
|
|
|
this.fsLock.writeLock().unlock();
|
|
|
}
|
|
|
-
|
|
|
+ @Override
|
|
|
public boolean hasWriteLock() {
|
|
|
return this.fsLock.isWriteLockedByCurrentThread();
|
|
|
}
|
|
|
-
|
|
|
- boolean hasReadLock() {
|
|
|
+ @Override
|
|
|
+ public boolean hasReadLock() {
|
|
|
return this.fsLock.getReadHoldCount() > 0;
|
|
|
}
|
|
|
-
|
|
|
+ @Override
|
|
|
public boolean hasReadOrWriteLock() {
|
|
|
return hasReadLock() || hasWriteLock();
|
|
|
}
|
|
@@ -473,7 +469,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
try {
|
|
|
return new NamespaceInfo(dir.fsImage.getStorage().getNamespaceID(),
|
|
|
getClusterId(), getBlockPoolId(),
|
|
|
- dir.fsImage.getStorage().getCTime(), getDistributedUpgradeVersion());
|
|
|
+ dir.fsImage.getStorage().getCTime(),
|
|
|
+ upgradeManager.getUpgradeVersion());
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
@@ -484,7 +481,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* Causes heartbeat and lease daemons to stop; waits briefly for
|
|
|
* them to finish, but a short timeout returns control back to caller.
|
|
|
*/
|
|
|
- public void close() {
|
|
|
+ void close() {
|
|
|
fsRunning = false;
|
|
|
try {
|
|
|
if (blockManager != null) blockManager.close();
|
|
@@ -562,30 +559,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return accessTimePrecision > 0;
|
|
|
}
|
|
|
|
|
|
- /////////////////////////////////////////////////////////
|
|
|
- //
|
|
|
- // These methods are called by secondary namenodes
|
|
|
- //
|
|
|
- /////////////////////////////////////////////////////////
|
|
|
- /**
|
|
|
- * 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
|
|
|
- */
|
|
|
- BlocksWithLocations getBlocks(DatanodeID datanode, long size)
|
|
|
- throws IOException {
|
|
|
- readLock();
|
|
|
- try {
|
|
|
- checkSuperuserPrivilege();
|
|
|
- return blockManager.getBlocksWithLocations(datanode, size);
|
|
|
- } finally {
|
|
|
- readUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
/////////////////////////////////////////////////////////
|
|
|
//
|
|
|
// These methods are called by HadoopFS clients
|
|
@@ -765,7 +738,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* @param srcs
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public void concat(String target, String [] srcs)
|
|
|
+ void concat(String target, String [] srcs)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
if(FSNamesystem.LOG.isDebugEnabled()) {
|
|
|
FSNamesystem.LOG.debug("concat " + Arrays.toString(srcs) +
|
|
@@ -813,7 +786,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
/** See {@link #concat(String, String[])} */
|
|
|
- public void concatInternal(String target, String [] srcs)
|
|
|
+ private void concatInternal(String target, String [] srcs)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
assert hasWriteLock();
|
|
|
|
|
@@ -1429,7 +1402,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* are replicated. Will return an empty 2-elt array if we want the
|
|
|
* client to "try again later".
|
|
|
*/
|
|
|
- public LocatedBlock getAdditionalBlock(String src,
|
|
|
+ LocatedBlock getAdditionalBlock(String src,
|
|
|
String clientName,
|
|
|
ExtendedBlock previous,
|
|
|
HashMap<Node, Node> excludedNodes
|
|
@@ -1632,7 +1605,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* (e.g if not all blocks have reached minimum replication yet)
|
|
|
* @throws IOException on error (eg lease mismatch, file not open, file deleted)
|
|
|
*/
|
|
|
- public boolean completeFile(String src, String holder, ExtendedBlock last)
|
|
|
+ boolean completeFile(String src, String holder, ExtendedBlock last)
|
|
|
throws SafeModeException, UnresolvedLinkException, IOException {
|
|
|
checkBlock(last);
|
|
|
boolean success = false;
|
|
@@ -2258,7 +2231,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- Lease reassignLease(Lease lease, String src, String newHolder,
|
|
|
+ private Lease reassignLease(Lease lease, String src, String newHolder,
|
|
|
INodeFileUnderConstruction pendingFile) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
if(newHolder == null)
|
|
@@ -2274,6 +2247,22 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return leaseManager.reassignLease(lease, src, newHolder);
|
|
|
}
|
|
|
|
|
|
+ /** Update disk space consumed. */
|
|
|
+ public void updateDiskSpaceConsumed(final INodeFileUnderConstruction fileINode,
|
|
|
+ final Block commitBlock) throws IOException {
|
|
|
+ assert hasWriteLock();
|
|
|
+
|
|
|
+ // Adjust disk space consumption if required
|
|
|
+ final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();
|
|
|
+ if (diff > 0) {
|
|
|
+ try {
|
|
|
+ String path = leaseManager.findPath(fileINode);
|
|
|
+ dir.updateSpaceConsumed(path, 0, -diff * fileINode.getReplication());
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Unexpected exception while updating disk space.", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
private void finalizeINodeFileUnderConstruction(String src,
|
|
|
INodeFileUnderConstruction pendingFile)
|
|
@@ -2473,8 +2462,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
*
|
|
|
* @see org.apache.hadoop.hdfs.server.datanode.DataNode
|
|
|
*/
|
|
|
- public void registerDatanode(DatanodeRegistration nodeReg)
|
|
|
- throws IOException {
|
|
|
+ void registerDatanode(DatanodeRegistration nodeReg) throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
getBlockManager().getDatanodeManager().registerDatanode(nodeReg);
|
|
@@ -2505,7 +2493,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* @return an array of datanode commands
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
|
|
|
+ DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
|
|
|
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
|
|
|
int xceiverCount, int xmitsInProgress, int failedVolumes)
|
|
|
throws IOException {
|
|
@@ -2521,7 +2509,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
//check distributed upgrade
|
|
|
- DatanodeCommand cmd = getDistributedUpgradeCommand();
|
|
|
+ DatanodeCommand cmd = upgradeManager.getBroadcastCommand();
|
|
|
if (cmd != null) {
|
|
|
return new DatanodeCommand[] {cmd};
|
|
|
}
|
|
@@ -2737,30 +2725,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public Date getStartTime() {
|
|
|
+ Date getStartTime() {
|
|
|
return new Date(systemStart);
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Rereads the config to get hosts and exclude list file names.
|
|
|
- * Rereads the files to update the hosts and exclude lists. It
|
|
|
- * checks if any of the hosts have changed states:
|
|
|
- * 1. Added to hosts --> no further work needed here.
|
|
|
- * 2. Removed from hosts --> mark AdminState as decommissioned.
|
|
|
- * 3. Added to exclude --> start decommission.
|
|
|
- * 4. Removed from exclude --> stop decommission.
|
|
|
- */
|
|
|
- public void refreshNodes(Configuration conf) throws IOException {
|
|
|
- checkSuperuserPrivilege();
|
|
|
- getBlockManager().getDatanodeManager().refreshHostsReader(conf);
|
|
|
- writeLock();
|
|
|
- try {
|
|
|
- getBlockManager().getDatanodeManager().refreshDatanodes();
|
|
|
- } finally {
|
|
|
- writeUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
void finalizeUpgrade() throws IOException {
|
|
|
checkSuperuserPrivilege();
|
|
|
getFSImage().finalizeUpgrade();
|
|
@@ -2908,7 +2876,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
// verify whether a distributed upgrade needs to be started
|
|
|
boolean needUpgrade = false;
|
|
|
try {
|
|
|
- needUpgrade = startDistributedUpgradeIfNeeded();
|
|
|
+ needUpgrade = upgradeManager.startUpgrade();
|
|
|
} catch(IOException e) {
|
|
|
FSNamesystem.LOG.error("IOException in startDistributedUpgradeIfNeeded", e);
|
|
|
}
|
|
@@ -3101,10 +3069,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
leaveMsg = "Safe mode will be turned off automatically";
|
|
|
}
|
|
|
if(isManual()) {
|
|
|
- if(getDistributedUpgradeState())
|
|
|
+ if(upgradeManager.getUpgradeState())
|
|
|
return leaveMsg + " upon completion of " +
|
|
|
"the distributed upgrade: upgrade progress = " +
|
|
|
- getDistributedUpgradeStatus() + "%";
|
|
|
+ upgradeManager.getUpgradeStatus() + "%";
|
|
|
leaveMsg = "Use \"hdfs dfsadmin -safemode leave\" to turn safe mode off";
|
|
|
}
|
|
|
|
|
@@ -3306,7 +3274,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
/**
|
|
|
* Set the total number of blocks in the system.
|
|
|
*/
|
|
|
- void setBlockTotal() {
|
|
|
+ private void setBlockTotal() {
|
|
|
// safeMode is volatile, and may be set to null at any time
|
|
|
SafeModeInfo safeMode = this.safeMode;
|
|
|
if (safeMode == null)
|
|
@@ -3327,7 +3295,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* Get the total number of COMPLETE blocks in the system.
|
|
|
* For safe mode only complete blocks are counted.
|
|
|
*/
|
|
|
- long getCompleteBlocksTotal() {
|
|
|
+ private long getCompleteBlocksTotal() {
|
|
|
// Calculate number of blocks under construction
|
|
|
long numUCBlocks = 0;
|
|
|
readLock();
|
|
@@ -3398,7 +3366,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
NameNode.stateChangeLog.info("STATE* Safe mode is already OFF.");
|
|
|
return;
|
|
|
}
|
|
|
- if(getDistributedUpgradeState())
|
|
|
+ if(upgradeManager.getUpgradeState())
|
|
|
throw new SafeModeException("Distributed upgrade is in progress",
|
|
|
safeMode);
|
|
|
safeMode.leave(checkForUpgrades);
|
|
@@ -3487,26 +3455,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return upgradeManager.processUpgradeCommand(comm);
|
|
|
}
|
|
|
|
|
|
- int getDistributedUpgradeVersion() {
|
|
|
- return upgradeManager.getUpgradeVersion();
|
|
|
- }
|
|
|
-
|
|
|
- UpgradeCommand getDistributedUpgradeCommand() throws IOException {
|
|
|
- return upgradeManager.getBroadcastCommand();
|
|
|
- }
|
|
|
-
|
|
|
- boolean getDistributedUpgradeState() {
|
|
|
- return upgradeManager.getUpgradeState();
|
|
|
- }
|
|
|
-
|
|
|
- short getDistributedUpgradeStatus() {
|
|
|
- return upgradeManager.getUpgradeStatus();
|
|
|
- }
|
|
|
-
|
|
|
- boolean startDistributedUpgradeIfNeeded() throws IOException {
|
|
|
- return upgradeManager.startUpgrade();
|
|
|
- }
|
|
|
-
|
|
|
PermissionStatus createFsOwnerPermissions(FsPermission permission) {
|
|
|
return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
|
|
|
}
|
|
@@ -3536,7 +3484,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return checkPermission(path, false, null, null, null, null);
|
|
|
}
|
|
|
|
|
|
- private void checkSuperuserPrivilege() throws AccessControlException {
|
|
|
+ /** Check if the user has superuser privilege. */
|
|
|
+ public void checkSuperuserPrivilege() throws AccessControlException {
|
|
|
if (isPermissionEnabled) {
|
|
|
FSPermissionChecker.checkSuperuserPrivilege(fsOwner, supergroup);
|
|
|
}
|
|
@@ -3644,7 +3593,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* Register the FSNamesystem MBean using the name
|
|
|
* "hadoop:service=NameNode,name=FSNamesystemState"
|
|
|
*/
|
|
|
- void registerMBean() {
|
|
|
+ private void registerMBean() {
|
|
|
// We can only implement one MXBean interface, so we keep the old one.
|
|
|
try {
|
|
|
StandardMBean bean = new StandardMBean(this, FSNamesystemMBean.class);
|
|
@@ -3805,7 +3754,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
/** @see updatePipeline(String, ExtendedBlock, ExtendedBlock, DatanodeID[]) */
|
|
|
- void updatePipelineInternal(String clientName, ExtendedBlock oldBlock,
|
|
|
+ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock,
|
|
|
ExtendedBlock newBlock, DatanodeID[] newNodes)
|
|
|
throws IOException {
|
|
|
assert hasWriteLock();
|
|
@@ -4043,7 +3992,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* Returns the DelegationTokenSecretManager instance in the namesystem.
|
|
|
* @return delegation token secret manager object
|
|
|
*/
|
|
|
- public DelegationTokenSecretManager getDelegationTokenSecretManager() {
|
|
|
+ DelegationTokenSecretManager getDelegationTokenSecretManager() {
|
|
|
return dtSecretManager;
|
|
|
}
|
|
|
|
|
@@ -4096,7 +4045,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* @throws InvalidToken
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
|
|
|
+ long renewDelegationToken(Token<DelegationTokenIdentifier> token)
|
|
|
throws InvalidToken, IOException {
|
|
|
long expiryTime;
|
|
|
writeLock();
|
|
@@ -4127,7 +4076,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* @param token
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
|
|
|
+ void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
|
|
|
throws IOException {
|
|
|
writeLock();
|
|
|
try {
|