|
@@ -34,6 +34,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT;
|
|
@@ -603,7 +604,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
this.maxFsObjects = conf.getLong(DFS_NAMENODE_MAX_OBJECTS_KEY,
|
|
|
DFS_NAMENODE_MAX_OBJECTS_DEFAULT);
|
|
|
|
|
|
- this.accessTimePrecision = conf.getLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 0);
|
|
|
+ this.accessTimePrecision = conf.getLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY,
|
|
|
+ DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT);
|
|
|
this.supportAppends = conf.getBoolean(DFS_SUPPORT_APPEND_KEY, DFS_SUPPORT_APPEND_DEFAULT);
|
|
|
LOG.info("Append Enabled: " + supportAppends);
|
|
|
|
|
@@ -1137,8 +1139,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*/
|
|
|
void metaSave(String filename) throws IOException {
|
|
|
checkSuperuserPrivilege();
|
|
|
+ checkOperation(OperationCategory.UNCHECKED);
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.UNCHECKED);
|
|
|
File file = new File(System.getProperty("hadoop.log.dir"), filename);
|
|
|
PrintWriter out = new PrintWriter(new BufferedWriter(
|
|
|
new OutputStreamWriter(new FileOutputStream(file, true), Charsets.UTF_8)));
|
|
@@ -1212,6 +1216,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -1249,6 +1254,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -1359,13 +1365,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws FileNotFoundException, UnresolvedLinkException, IOException {
|
|
|
|
|
|
for (int attempt = 0; attempt < 2; attempt++) {
|
|
|
- if (attempt == 0) { // first attempt is with readlock
|
|
|
+ boolean isReadOp = (attempt == 0);
|
|
|
+ if (isReadOp) { // first attempt is with readlock
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
readLock();
|
|
|
} else { // second attempt is with write lock
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock(); // writelock is needed to set accesstime
|
|
|
}
|
|
|
try {
|
|
|
- checkOperation(OperationCategory.READ);
|
|
|
+ if (isReadOp) {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
+ } else {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+ }
|
|
|
|
|
|
// if the namenode is in safemode, then do not update access time
|
|
|
if (isInSafeMode()) {
|
|
@@ -1380,7 +1393,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
if (now <= inode.getAccessTime() + getAccessTimePrecision()) {
|
|
|
// if we have to set access time but we only have the readlock, then
|
|
|
// restart this entire operation with the writeLock.
|
|
|
- if (attempt == 0) {
|
|
|
+ if (isReadOp) {
|
|
|
continue;
|
|
|
}
|
|
|
}
|
|
@@ -1392,7 +1405,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
return blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
|
|
|
inode.isUnderConstruction(), offset, length, needBlockToken);
|
|
|
} finally {
|
|
|
- if (attempt == 0) {
|
|
|
+ if (isReadOp) {
|
|
|
readUnlock();
|
|
|
} else {
|
|
|
writeUnlock();
|
|
@@ -1448,6 +1461,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -1595,6 +1609,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -1636,6 +1651,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -1711,6 +1727,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
blockManager.verifyReplication(src, replication, null);
|
|
|
final boolean isFile;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -1741,6 +1758,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
long getPreferredBlockSize(String filename)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
readLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.READ);
|
|
@@ -1803,6 +1821,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
boolean skipSync = false;
|
|
|
final HdfsFileStatus stat;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -1995,6 +2014,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws IOException {
|
|
|
boolean skipSync = false;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -2132,6 +2152,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
LocatedBlock lb = null;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -2198,8 +2219,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
// Part I. Analyze the state of the file with respect to the input data.
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
|
final INode[] inodes = analyzeFileState(
|
|
|
src, fileId, clientName, previous, onRetryBlock).getINodes();
|
|
@@ -2226,8 +2249,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// Allocate a new block, add it to the INode and the BlocksMap.
|
|
|
Block newBlock = null;
|
|
|
long offset;
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
// Run the full analysis again, since things could have changed
|
|
|
// while chooseTarget() was executing.
|
|
|
LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
@@ -2379,9 +2404,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
final DatanodeDescriptor clientnode;
|
|
|
final long preferredblocksize;
|
|
|
final List<DatanodeDescriptor> chosen;
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
readLock();
|
|
|
try {
|
|
|
- checkOperation(OperationCategory.WRITE);
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
//check safe mode
|
|
|
if (isInSafeMode()) {
|
|
|
throw new SafeModeException("Cannot add datanode; src=" + src
|
|
@@ -2421,6 +2447,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
boolean abandonBlock(ExtendedBlock b, String src, String holder)
|
|
|
throws LeaseExpiredException, FileNotFoundException,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -2498,6 +2525,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws SafeModeException, UnresolvedLinkException, IOException {
|
|
|
checkBlock(last);
|
|
|
boolean success = false;
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -2669,6 +2697,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
" to " + dst);
|
|
|
}
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -2725,6 +2754,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
+ src + " to " + dst);
|
|
|
}
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -2811,6 +2841,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
IOException {
|
|
|
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -2939,6 +2970,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
StandbyException, IOException {
|
|
|
HdfsFileStatus stat = null;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
readLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.READ);
|
|
@@ -2981,6 +3013,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
|
|
|
}
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -3041,6 +3074,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
FileNotFoundException, UnresolvedLinkException, StandbyException {
|
|
|
FSPermissionChecker pc = new FSPermissionChecker(fsOwnerShortUserName,
|
|
|
supergroup);
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
readLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.READ);
|
|
@@ -3061,6 +3095,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
void setQuota(String path, long nsQuota, long dsQuota)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
checkSuperuserPrivilege();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -3084,6 +3119,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
void fsync(String src, String clientName, long lastBlockLength)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -3293,6 +3329,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
String[] newtargetstorages)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
String src = "";
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -3397,6 +3434,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* Renew the lease(s) held by the given client
|
|
|
*/
|
|
|
void renewLease(String holder) throws IOException {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -3438,6 +3476,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throws AccessControlException, UnresolvedLinkException, IOException {
|
|
|
DirectoryListing dl;
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
readLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.READ);
|
|
@@ -3734,10 +3773,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
DatanodeInfo[] datanodeReport(final DatanodeReportType type
|
|
|
- ) throws AccessControlException {
|
|
|
+ ) throws AccessControlException, StandbyException {
|
|
|
checkSuperuserPrivilege();
|
|
|
+ checkOperation(OperationCategory.UNCHECKED);
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.UNCHECKED);
|
|
|
final DatanodeManager dm = getBlockManager().getDatanodeManager();
|
|
|
final List<DatanodeDescriptor> results = dm.getDatanodeListForReport(type);
|
|
|
|
|
@@ -3761,8 +3802,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*/
|
|
|
void saveNamespace() throws AccessControlException, IOException {
|
|
|
checkSuperuserPrivilege();
|
|
|
+ checkOperation(OperationCategory.UNCHECKED);
|
|
|
readLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.UNCHECKED);
|
|
|
if (!isInSafeMode()) {
|
|
|
throw new IOException("Safe mode should be turned ON " +
|
|
|
"in order to create namespace image.");
|
|
@@ -3780,10 +3823,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*
|
|
|
* @throws AccessControlException if superuser privilege is violated.
|
|
|
*/
|
|
|
- boolean restoreFailedStorage(String arg) throws AccessControlException {
|
|
|
+ boolean restoreFailedStorage(String arg) throws AccessControlException,
|
|
|
+ StandbyException {
|
|
|
checkSuperuserPrivilege();
|
|
|
+ checkOperation(OperationCategory.UNCHECKED);
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.UNCHECKED);
|
|
|
|
|
|
// if it is disabled - enable it and vice versa.
|
|
|
if(arg.equals("check"))
|
|
@@ -3804,6 +3850,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
void finalizeUpgrade() throws IOException {
|
|
|
checkSuperuserPrivilege();
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -4543,6 +4590,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
CheckpointSignature rollEditLog() throws IOException {
|
|
|
checkSuperuserPrivilege();
|
|
|
+ checkOperation(OperationCategory.JOURNAL);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.JOURNAL);
|
|
@@ -4560,6 +4608,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
NamenodeRegistration bnReg, // backup node
|
|
|
NamenodeRegistration nnReg) // active name-node
|
|
|
throws IOException {
|
|
|
+ checkOperation(OperationCategory.CHECKPOINT);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.CHECKPOINT);
|
|
@@ -4578,6 +4627,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
void endCheckpoint(NamenodeRegistration registration,
|
|
|
CheckpointSignature sig) throws IOException {
|
|
|
+ checkOperation(OperationCategory.CHECKPOINT);
|
|
|
readLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.CHECKPOINT);
|
|
@@ -4866,6 +4916,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* Client is reporting some bad block locations.
|
|
|
*/
|
|
|
void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -4900,6 +4951,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
LocatedBlock updateBlockForPipeline(ExtendedBlock block,
|
|
|
String clientName) throws IOException {
|
|
|
LocatedBlock locatedBlock;
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -4931,6 +4983,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
void updatePipeline(String clientName, ExtendedBlock oldBlock,
|
|
|
ExtendedBlock newBlock, DatanodeID[] newNodes)
|
|
|
throws IOException {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -5058,8 +5111,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*/
|
|
|
void releaseBackupNode(NamenodeRegistration registration)
|
|
|
throws IOException {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
if(getFSImage().getStorage().getNamespaceID()
|
|
|
!= registration.getNamespaceID())
|
|
|
throw new IOException("Incompatible namespaceIDs: "
|
|
@@ -5098,6 +5153,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
|
|
|
String[] cookieTab) throws IOException {
|
|
|
checkSuperuserPrivilege();
|
|
|
+ checkOperation(OperationCategory.READ);
|
|
|
readLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.READ);
|
|
@@ -5190,6 +5246,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
|
|
|
throws IOException {
|
|
|
Token<DelegationTokenIdentifier> token;
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -5236,6 +5293,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
long renewDelegationToken(Token<DelegationTokenIdentifier> token)
|
|
|
throws InvalidToken, IOException {
|
|
|
long expiryTime;
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -5268,6 +5326,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*/
|
|
|
void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
|
|
|
throws IOException {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|