|
@@ -149,6 +149,7 @@ import org.apache.hadoop.hdfs.server.common.Storage;
|
|
|
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.NameNode.OperationCategory;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.PendingDataNodeMessages.BlockReceivedDeleteMessage;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.PendingDataNodeMessages.BlockReportMessage;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.PendingDataNodeMessages.CommitBlockSynchronizationMessage;
|
|
@@ -170,6 +171,7 @@ import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
+import org.apache.hadoop.ipc.StandbyException;
|
|
|
import org.apache.hadoop.metrics2.annotation.Metric;
|
|
|
import org.apache.hadoop.metrics2.annotation.Metrics;
|
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
@@ -563,6 +565,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
dir.fsImage.editLog.close();
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ void checkOperation(OperationCategory op) throws StandbyException {
|
|
|
+ haContext.checkOperation(op);
|
|
|
+ }
|
|
|
+
|
|
|
public static Collection<URI> getNamespaceDirs(Configuration conf) {
|
|
|
return getStorageDirs(conf, DFS_NAMENODE_NAME_DIR_KEY);
|
|
|
}
|
|
@@ -793,7 +800,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
return serverDefaults.getBlockSize();
|
|
|
}
|
|
|
|
|
|
- FsServerDefaults getServerDefaults() {
|
|
|
+ FsServerDefaults getServerDefaults() throws StandbyException {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
return serverDefaults;
|
|
|
}
|
|
|
|
|
@@ -820,6 +828,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot set permission for " + src, safeMode);
|
|
|
}
|
|
@@ -849,6 +859,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot set owner for " + src, safeMode);
|
|
|
}
|
|
@@ -939,13 +951,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
} else { // second attempt is with write lock
|
|
|
writeLock(); // writelock is needed to set accesstime
|
|
|
}
|
|
|
+ try {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
|
|
|
- // if the namenode is in safemode, then do not update access time
|
|
|
- if (isInSafeMode()) {
|
|
|
- doAccessTime = false;
|
|
|
- }
|
|
|
+ // if the namenode is in safemode, then do not update access time
|
|
|
+ if (isInSafeMode()) {
|
|
|
+ doAccessTime = false;
|
|
|
+ }
|
|
|
|
|
|
- try {
|
|
|
long now = now();
|
|
|
INodeFile inode = dir.getFileINode(src);
|
|
|
if (inode == null) {
|
|
@@ -1013,6 +1026,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot concat " + target, safeMode);
|
|
|
}
|
|
@@ -1144,6 +1158,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
// Write access is required to set access and modification times
|
|
|
if (isPermissionEnabled) {
|
|
|
checkPathAccess(src, FsAction.WRITE);
|
|
@@ -1174,6 +1190,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (!createParent) {
|
|
|
verifyParentDir(link);
|
|
|
}
|
|
@@ -1243,6 +1261,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
final boolean isFile;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot set replication for " + src, safeMode);
|
|
|
}
|
|
@@ -1273,6 +1293,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
if (isPermissionEnabled) {
|
|
|
checkTraverse(filename);
|
|
|
}
|
|
@@ -1315,6 +1336,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
FileNotFoundException, ParentNotDirectoryException, IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
startFileInternal(src, permissions, holder, clientMachine, flag,
|
|
|
createParent, replication, blockSize);
|
|
|
} finally {
|
|
@@ -1495,6 +1518,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException(
|
|
|
"Cannot recover the lease of " + src, safeMode);
|
|
@@ -1614,6 +1639,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
LocatedBlock lb = null;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
lb = startFileInternal(src, null, holder, clientMachine,
|
|
|
EnumSet.of(CreateFlag.APPEND),
|
|
|
false, blockManager.maxReplication, 0);
|
|
@@ -1678,6 +1705,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot add block to " + src, safeMode);
|
|
|
}
|
|
@@ -1711,6 +1740,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// Allocate a new block and record it in the INode.
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot add block to " + src, safeMode);
|
|
|
}
|
|
@@ -1757,6 +1787,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
final List<DatanodeDescriptor> chosen;
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
//check safe mode
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot add datanode; src=" + src
|
|
@@ -1798,6 +1829,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
//
|
|
|
// Remove the block from the pending creates list
|
|
|
//
|
|
@@ -1873,6 +1905,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
boolean success = false;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
success = completeFileInternal(src, holder,
|
|
|
ExtendedBlock.getLocalBlock(last));
|
|
|
} finally {
|
|
@@ -2012,6 +2046,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
status = renameToInternal(src, dst);
|
|
|
if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
|
|
|
resultingStat = dir.getFileInfo(dst, false);
|
|
@@ -2067,6 +2103,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
renameToInternal(src, dst, options);
|
|
|
if (auditLog.isInfoEnabled() && isExternalInvocation()) {
|
|
|
resultingStat = dir.getFileInfo(dst, false);
|
|
@@ -2145,6 +2183,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot delete " + src, safeMode);
|
|
|
}
|
|
@@ -2222,11 +2261,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*
|
|
|
* @return object containing information regarding the file
|
|
|
* or null if file not found
|
|
|
+ * @throws StandbyException
|
|
|
*/
|
|
|
HdfsFileStatus getFileInfo(String src, boolean resolveLink)
|
|
|
- throws AccessControlException, UnresolvedLinkException {
|
|
|
+ throws AccessControlException, UnresolvedLinkException,
|
|
|
+ StandbyException {
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
+
|
|
|
if (!DFSUtil.isValidName(src)) {
|
|
|
throw new InvalidPathException("Invalid file name: " + src);
|
|
|
}
|
|
@@ -2250,6 +2293,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
status = mkdirsInternal(src, permissions, createParent);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -2304,9 +2349,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
ContentSummary getContentSummary(String src) throws AccessControlException,
|
|
|
- FileNotFoundException, UnresolvedLinkException {
|
|
|
+ FileNotFoundException, UnresolvedLinkException, StandbyException {
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
+
|
|
|
if (isPermissionEnabled) {
|
|
|
checkPermission(src, false, null, null, null, FsAction.READ_EXECUTE);
|
|
|
}
|
|
@@ -2325,6 +2372,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot set quota on " + path, safeMode);
|
|
|
}
|
|
@@ -2349,6 +2397,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
+ src + " for " + clientName);
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot fsync file " + src, safeMode);
|
|
|
}
|
|
@@ -2558,6 +2607,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
String src = "";
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+ if (haContext.getState().equals(NameNode.STANDBY_STATE)) {
|
|
|
+ // TODO(HA) we'll never get here, since we check for WRITE operation above!
|
|
|
+ if (isGenStampInFuture(newgenerationstamp)) {
|
|
|
+ LOG.info("Required GS=" + newgenerationstamp
|
|
|
+ + ", Queuing commitBlockSynchronization message");
|
|
|
+ getPendingDataNodeMessages().queueMessage(
|
|
|
+ new PendingDataNodeMessages.CommitBlockSynchronizationMessage(
|
|
|
+ lastblock, newgenerationstamp, newlength, closeFile, deleteblock,
|
|
|
+ newtargets, newgenerationstamp));
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException(
|
|
|
"Cannot commitBlockSynchronization while in safe mode",
|
|
@@ -2658,6 +2721,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
void renewLease(String holder) throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot renew lease for " + holder, safeMode);
|
|
|
}
|
|
@@ -2685,6 +2750,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
DirectoryListing dl;
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
+
|
|
|
if (isPermissionEnabled) {
|
|
|
if (dir.isDir(src)) {
|
|
|
checkPathAccess(src, FsAction.READ_EXECUTE);
|
|
@@ -3699,6 +3766,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.CHECKPOINT);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Checkpoint not started", safeMode);
|
|
|
}
|
|
@@ -3715,6 +3784,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
CheckpointSignature sig) throws IOException {
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.CHECKPOINT);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Checkpoint not ended", safeMode);
|
|
|
}
|
|
@@ -3976,6 +4047,28 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
return pendingFile;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Client is reporting some bad block locations.
|
|
|
+ */
|
|
|
+ void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
|
|
|
+ writeLock();
|
|
|
+ try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
+ NameNode.stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
|
|
|
+ for (int i = 0; i < blocks.length; i++) {
|
|
|
+ ExtendedBlock blk = blocks[i].getBlock();
|
|
|
+ DatanodeInfo[] nodes = blocks[i].getLocations();
|
|
|
+ for (int j = 0; j < nodes.length; j++) {
|
|
|
+ DatanodeInfo dn = nodes[j];
|
|
|
+ blockManager.findAndMarkBlockAsCorrupt(blk, dn);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ writeUnlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Get a new generation stamp together with an access token for
|
|
|
* a block under construction
|
|
@@ -3993,6 +4086,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
LocatedBlock locatedBlock;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
// check vadility of parameters
|
|
|
checkUCBlock(block, clientName);
|
|
|
|
|
@@ -4022,6 +4117,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Pipeline not updated", safeMode);
|
|
|
}
|
|
@@ -4222,6 +4319,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
+
|
|
|
if (!isPopulatingReplQueues()) {
|
|
|
throw new IOException("Cannot run listCorruptFileBlocks because " +
|
|
|
"replication queues have not been initialized.");
|
|
@@ -4314,6 +4413,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
Token<DelegationTokenIdentifier> token;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot issue delegation token", safeMode);
|
|
|
}
|
|
@@ -4358,6 +4459,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
long expiryTime;
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot renew delegation token", safeMode);
|
|
|
}
|
|
@@ -4388,6 +4491,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot cancel delegation token", safeMode);
|
|
|
}
|
|
@@ -4727,4 +4832,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
public EditLogTailer getEditLogTailer() {
|
|
|
return editLogTailer;
|
|
|
}
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ void setFsLockForTests(ReentrantReadWriteLock lock) {
|
|
|
+ this.fsLock = lock;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ ReentrantReadWriteLock getFsLockForTests() {
|
|
|
+ return fsLock;
|
|
|
+ }
|
|
|
}
|