|
@@ -2439,14 +2439,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
try {
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.FS);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot create symlink " + link);
|
|
|
auditStat = FSDirSymlinkOp.createSymlinkInt(this, target, link,
|
|
|
dirPerms, createParent, logRetryCache);
|
|
|
} finally {
|
|
|
- writeUnlock(operationName,
|
|
|
+ writeUnlock(FSNamesystemLockMode.FS, operationName,
|
|
|
getLockReportInfoSupplier(link, target, auditStat));
|
|
|
}
|
|
|
} catch (AccessControlException e) {
|
|
@@ -2796,7 +2796,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.FS);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot create file" + src);
|
|
@@ -2858,7 +2858,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
dir.writeUnlock();
|
|
|
}
|
|
|
} finally {
|
|
|
- writeUnlock("create", getLockReportInfoSupplier(src, null, stat));
|
|
|
+ writeUnlock(FSNamesystemLockMode.FS, "create",
|
|
|
+ getLockReportInfoSupplier(src, null, stat));
|
|
|
// There might be transactions logged while trying to recover the lease.
|
|
|
// They need to be sync'ed even when an exception was thrown.
|
|
|
if (!skipSync) {
|
|
@@ -2892,7 +2893,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot recover the lease of " + src);
|
|
@@ -2912,7 +2913,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
skipSync = true;
|
|
|
throw se;
|
|
|
} finally {
|
|
|
- writeUnlock("recoverLease");
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, operationName);
|
|
|
// There might be transactions logged while trying to recover the lease.
|
|
|
// They need to be sync'ed even when an exception was thrown.
|
|
|
if (!skipSync) {
|
|
@@ -3032,7 +3033,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot append to file" + srcArg);
|
|
@@ -3042,7 +3043,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
skipSync = true;
|
|
|
throw se;
|
|
|
} finally {
|
|
|
- writeUnlock(operationName, getLockReportInfoSupplier(srcArg));
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, operationName, getLockReportInfoSupplier(srcArg));
|
|
|
// There might be transactions logged while trying to recover the lease
|
|
|
// They need to be sync'ed even when an exception was thrown.
|
|
|
if (!skipSync) {
|
|
@@ -3089,13 +3090,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
- readLock();
|
|
|
+ readLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
r = FSDirWriteFileOp.validateAddBlock(this, pc, src, fileId, clientName,
|
|
|
previous, onRetryBlock);
|
|
|
} finally {
|
|
|
- readUnlock(operationName);
|
|
|
+ readUnlock(FSNamesystemLockMode.GLOBAL, operationName);
|
|
|
}
|
|
|
|
|
|
if (r == null) {
|
|
@@ -3108,14 +3109,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
blockManager, src, excludedNodes, favoredNodes, flags, r);
|
|
|
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
LocatedBlock lb;
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
lb = FSDirWriteFileOp.storeAllocatedBlock(
|
|
|
this, src, fileId, clientName, previous, targets);
|
|
|
} finally {
|
|
|
- writeUnlock(operationName);
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, operationName);
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
return lb;
|
|
@@ -3141,7 +3142,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
- readLock();
|
|
|
+ readLock(FSNamesystemLockMode.FS);
|
|
|
try {
|
|
|
// Changing this operation category to WRITE instead of making getAdditionalDatanode as a
|
|
|
// read method is aim to let Active NameNode to handle this RPC, because Active NameNode
|
|
@@ -3166,7 +3167,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
"src=%s, fileId=%d, blk=%s, clientName=%s, clientMachine=%s",
|
|
|
src, fileId, blk, clientName, clientMachine));
|
|
|
} finally {
|
|
|
- readUnlock("getAdditionalDatanode");
|
|
|
+ readUnlock(FSNamesystemLockMode.FS, operationName);
|
|
|
}
|
|
|
|
|
|
if (clientnode == null) {
|
|
@@ -3193,7 +3194,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
|
|
@@ -3201,7 +3202,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
NameNode.stateChangeLog.debug(
|
|
|
"BLOCK* NameSystem.abandonBlock: {} is removed from pendingCreates", b);
|
|
|
} finally {
|
|
|
- writeUnlock("abandonBlock");
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, operationName);
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
}
|
|
@@ -3216,7 +3217,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
throws LeaseExpiredException, FileNotFoundException {
|
|
|
String src = iip.getPath();
|
|
|
INode inode = iip.getLastINode();
|
|
|
- assert hasReadLock();
|
|
|
+ assert hasReadLock(FSNamesystemLockMode.FS);
|
|
|
if (inode == null) {
|
|
|
throw new FileNotFoundException("File does not exist: "
|
|
|
+ leaseExceptionString(src, fileId, holder));
|
|
@@ -3260,14 +3261,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot complete file " + src);
|
|
|
success = FSDirWriteFileOp.completeFile(this, pc, src, holder, last,
|
|
|
fileId);
|
|
|
} finally {
|
|
|
- writeUnlock("completeFile");
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, operationName);
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
if (success) {
|
|
@@ -3282,6 +3283,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @param blockType is the file under striping or contiguous layout?
|
|
|
*/
|
|
|
Block createNewBlock(BlockType blockType) throws IOException {
|
|
|
+ // nextBlockId and nextGenerationStamp need to write edit log, so it needs FSLock.
|
|
|
assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
|
|
|
Block b = new Block(nextBlockId(blockType), 0, 0);
|
|
|
// Increment the generation stamp for every new block.
|
|
@@ -3295,7 +3297,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* all blocks, otherwise check only penultimate block.
|
|
|
*/
|
|
|
boolean checkFileProgress(String src, INodeFile v, boolean checkall) {
|
|
|
- assert hasReadLock();
|
|
|
+ assert hasReadLock(FSNamesystemLockMode.GLOBAL);
|
|
|
if (checkall) {
|
|
|
return checkBlocksComplete(src, true, v.getBlocks());
|
|
|
} else {
|
|
@@ -3341,14 +3343,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
try {
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.FS);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot rename " + src);
|
|
|
ret = FSDirRenameOp.renameToInt(dir, pc, src, dst, logRetryCache);
|
|
|
} finally {
|
|
|
FileStatus status = ret != null ? ret.auditStat : null;
|
|
|
- writeUnlock(operationName,
|
|
|
+ writeUnlock(FSNamesystemLockMode.FS, operationName,
|
|
|
getLockReportInfoSupplier(src, dst, status));
|
|
|
}
|
|
|
} catch (AccessControlException e) {
|
|
@@ -3373,7 +3375,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
try {
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot rename " + src);
|
|
@@ -3381,7 +3383,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
options);
|
|
|
} finally {
|
|
|
FileStatus status = res != null ? res.auditStat : null;
|
|
|
- writeUnlock(operationName,
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, operationName,
|
|
|
getLockReportInfoSupplier(src, dst, status));
|
|
|
}
|
|
|
} catch (AccessControlException e) {
|
|
@@ -3416,7 +3418,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
boolean ret = false;
|
|
|
try {
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot delete " + src);
|
|
@@ -3424,7 +3426,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
this, pc, src, recursive, logRetryCache);
|
|
|
ret = toRemovedBlocks != null;
|
|
|
} finally {
|
|
|
- writeUnlock(operationName, getLockReportInfoSupplier(src));
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, operationName, getLockReportInfoSupplier(src));
|
|
|
}
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, operationName, src);
|
|
@@ -3454,7 +3456,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
void removeLeasesAndINodes(List<Long> removedUCFiles,
|
|
|
List<INode> removedINodes,
|
|
|
final boolean acquireINodeMapLock) {
|
|
|
- assert hasWriteLock();
|
|
|
+ assert hasWriteLock(FSNamesystemLockMode.FS);
|
|
|
for(long i : removedUCFiles) {
|
|
|
leaseManager.removeLease(i);
|
|
|
}
|
|
@@ -3559,14 +3561,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
try {
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.FS);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot create directory " + src);
|
|
|
auditStat = FSDirMkdirOp.mkdirs(this, pc, src, permissions,
|
|
|
createParent);
|
|
|
} finally {
|
|
|
- writeUnlock(operationName,
|
|
|
+ writeUnlock(FSNamesystemLockMode.FS, operationName,
|
|
|
getLockReportInfoSupplier(src, null, auditStat));
|
|
|
}
|
|
|
} catch (AccessControlException e) {
|
|
@@ -3707,7 +3709,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
FSPermissionChecker.setOperationType(operationName);
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot fsync file " + src);
|
|
@@ -3720,7 +3722,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, false);
|
|
|
} finally {
|
|
|
- writeUnlock("fsync");
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, operationName);
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
}
|
|
@@ -3743,7 +3745,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
String recoveryLeaseHolder) throws IOException {
|
|
|
LOG.info("Recovering " + lease + ", src=" + src);
|
|
|
assert !isInSafeMode();
|
|
|
- assert hasWriteLock();
|
|
|
+ // finalizeINodeFileUnderConstruction needs global write lock.
|
|
|
+ assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
|
|
|
|
|
|
final INodeFile pendingFile = iip.getLastINode().asFile();
|
|
|
int nrBlocks = pendingFile.numBlocks();
|
|
@@ -3905,7 +3908,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
private Lease reassignLease(Lease lease, String src, String newHolder,
|
|
|
INodeFile pendingFile) {
|
|
|
- assert hasWriteLock();
|
|
|
+ assert hasWriteLock(FSNamesystemLockMode.FS);
|
|
|
if(newHolder == null)
|
|
|
return lease;
|
|
|
// The following transaction is not synced. Make sure it's sync'ed later.
|
|
@@ -3914,7 +3917,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
Lease reassignLeaseInternal(Lease lease, String newHolder, INodeFile pendingFile) {
|
|
|
- assert hasWriteLock();
|
|
|
+ assert hasWriteLock(FSNamesystemLockMode.FS);
|
|
|
pendingFile.getFileUnderConstructionFeature().setClientName(newHolder);
|
|
|
return leaseManager.reassignLease(lease, pendingFile, newHolder);
|
|
|
}
|
|
@@ -5888,6 +5891,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
*/
|
|
|
long nextGenerationStamp(boolean legacyBlock)
|
|
|
throws IOException {
|
|
|
+ // TODO: Use FSLock to make nextGenerationStamp thread safe.
|
|
|
assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
|
|
|
checkNameNodeSafeMode("Cannot get next generation stamp");
|
|
|
|
|
@@ -5907,7 +5911,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @param blockType is the file under striping or contiguous layout?
|
|
|
*/
|
|
|
private long nextBlockId(BlockType blockType) throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
+ assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
|
|
|
checkNameNodeSafeMode("Cannot get next block ID");
|
|
|
final long blockId = blockManager.nextBlockId(blockType);
|
|
|
getEditLog().logAllocateBlockId(blockId);
|
|
@@ -5956,7 +5960,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
private INodeFile checkUCBlock(ExtendedBlock block,
|
|
|
String clientName) throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
+ assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
|
|
|
checkNameNodeSafeMode("Cannot get a new generation stamp and an "
|
|
|
+ "access token for block " + block);
|
|
|
|
|
@@ -6030,7 +6034,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
String clientName) throws IOException {
|
|
|
final LocatedBlock locatedBlock;
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
|
|
@@ -6064,7 +6068,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
blockManager.setBlockToken(locatedBlock,
|
|
|
BlockTokenIdentifier.AccessMode.WRITE);
|
|
|
} finally {
|
|
|
- writeUnlock("bumpBlockGenerationStamp");
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, "bumpBlockGenerationStamp");
|
|
|
}
|
|
|
// Ensure we record the new generation stamp
|
|
|
getEditLog().logSync();
|
|
@@ -6093,7 +6097,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
+ ", newNodes=" + Arrays.asList(newNodes)
|
|
|
+ ", client=" + clientName
|
|
|
+ ")");
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.GLOBAL);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Pipeline not updated");
|
|
@@ -6102,7 +6106,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
|
|
|
newStorageIDs, logRetryCache);
|
|
|
} finally {
|
|
|
- writeUnlock("updatePipeline");
|
|
|
+ writeUnlock(FSNamesystemLockMode.GLOBAL, "updatePipeline");
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
LOG.info("updatePipeline(" + oldBlock.getLocalBlock() + " => "
|
|
@@ -6113,7 +6117,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs,
|
|
|
boolean logRetryCache)
|
|
|
throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
+ assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
|
|
|
// check the vadility of the block and lease holder name
|
|
|
final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
|
|
|
final String src = pendingFile.getFullPathName();
|
|
@@ -6409,7 +6413,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
long expiryTime;
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
try {
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.FS);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
|
|
@@ -6426,7 +6430,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
getEditLog().logRenewDelegationToken(id, expiryTime);
|
|
|
tokenId = id.toStringStable();
|
|
|
} finally {
|
|
|
- writeUnlock(operationName, getLockReportInfoSupplier(tokenId));
|
|
|
+ writeUnlock(FSNamesystemLockMode.FS, operationName, getLockReportInfoSupplier(tokenId));
|
|
|
}
|
|
|
} catch (AccessControlException ace) {
|
|
|
final DelegationTokenIdentifier id = DFSUtil.decodeDelegationToken(token);
|
|
@@ -6450,7 +6454,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
String tokenId = null;
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
try {
|
|
|
- writeLock();
|
|
|
+ writeLock(FSNamesystemLockMode.FS);
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot cancel delegation token");
|
|
@@ -6460,7 +6464,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
getEditLog().logCancelDelegationToken(id);
|
|
|
tokenId = id.toStringStable();
|
|
|
} finally {
|
|
|
- writeUnlock(operationName, getLockReportInfoSupplier(tokenId));
|
|
|
+ writeUnlock(FSNamesystemLockMode.FS, operationName, getLockReportInfoSupplier(tokenId));
|
|
|
}
|
|
|
} catch (AccessControlException ace) {
|
|
|
final DelegationTokenIdentifier id = DFSUtil.decodeDelegationToken(token);
|
|
@@ -6535,7 +6539,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
private void logReassignLease(String leaseHolder, String src,
|
|
|
String newHolder) {
|
|
|
- assert hasWriteLock();
|
|
|
+ assert hasWriteLock(FSNamesystemLockMode.FS);
|
|
|
getEditLog().logReassignLease(leaseHolder, src, newHolder);
|
|
|
}
|
|
|
|