|
@@ -209,6 +209,7 @@ import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.ipc.RetryCache;
|
|
|
import org.apache.hadoop.ipc.RetryCache.CacheEntry;
|
|
|
import org.apache.hadoop.ipc.RetryCache.CacheEntryWithPayload;
|
|
|
+import org.apache.hadoop.ipc.RetriableException;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.ipc.StandbyException;
|
|
|
import org.apache.hadoop.metrics2.annotation.Metric;
|
|
@@ -1050,6 +1051,26 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * @throws RetriableException
|
|
|
+ * If 1) The NameNode is in SafeMode, 2) HA is enabled, and 3)
|
|
|
+ * NameNode is in active state
|
|
|
+ * @throws SafeModeException
|
|
|
+ * Otherwise if NameNode is in SafeMode.
|
|
|
+ */
|
|
|
+ private void checkNameNodeSafeMode(String errorMsg)
|
|
|
+ throws RetriableException, SafeModeException {
|
|
|
+ if (isInSafeMode()) {
|
|
|
+ SafeModeException se = new SafeModeException(errorMsg, safeMode);
|
|
|
+ if (haEnabled && haContext != null
|
|
|
+ && haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
|
|
|
+ throw new RetriableException(se);
|
|
|
+ } else {
|
|
|
+ throw se;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
public static Collection<URI> getNamespaceDirs(Configuration conf) {
|
|
|
return getStorageDirs(conf, DFS_NAMENODE_NAME_DIR_KEY);
|
|
|
}
|
|
@@ -1351,9 +1372,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot set permission for " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot set permission for " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
checkOwner(pc, src);
|
|
|
dir.setPermission(src, permission);
|
|
@@ -1390,9 +1409,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot set owner for " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot set owner for " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
checkOwner(pc, src);
|
|
|
if (!pc.isSuperUser()) {
|
|
@@ -1472,8 +1489,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
for (LocatedBlock b : ret.getLocatedBlocks()) {
|
|
|
// if safemode & no block locations yet then throw safemodeException
|
|
|
if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
|
|
|
- throw new SafeModeException("Zero blocklocations for " + src,
|
|
|
- safeMode);
|
|
|
+ SafeModeException se = new SafeModeException(
|
|
|
+ "Zero blocklocations for " + src, safeMode);
|
|
|
+ if (haEnabled && haContext != null &&
|
|
|
+ haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
|
|
|
+ throw new RetriableException(se);
|
|
|
+ } else {
|
|
|
+ throw se;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -1614,9 +1637,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot concat " + target, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot concat " + target);
|
|
|
concatInternal(pc, target, srcs, logRetryCache);
|
|
|
resultingStat = getAuditFileInfo(target, false);
|
|
|
} finally {
|
|
@@ -1764,9 +1785,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot set times " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot set times " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
|
|
|
// Write access is required to set access and modification times
|
|
@@ -1829,9 +1848,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot create symlink " + link, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot create symlink " + link);
|
|
|
link = FSDirectory.resolvePath(link, pathComponents, dir);
|
|
|
if (!createParent) {
|
|
|
verifyParentDir(link);
|
|
@@ -1889,9 +1906,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot set replication for " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot set replication for " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
if (isPermissionEnabled) {
|
|
|
checkPathAccess(pc, src, FsAction.WRITE);
|
|
@@ -2021,9 +2036,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot create file" + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot create file" + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
startFileInternal(pc, src, permissions, holder, clientMachine, create,
|
|
|
overwrite, createParent, replication, blockSize, logRetryCache);
|
|
@@ -2242,10 +2255,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException(
|
|
|
- "Cannot recover the lease of " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot recover the lease of " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
final INodeFile inode = INodeFile.valueOf(dir.getINode(src), src);
|
|
|
if (!inode.isUnderConstruction()) {
|
|
@@ -2396,9 +2406,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot append to file" + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot append to file" + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
lb = appendFileInternal(pc, src, holder, clientMachine, logRetryCache);
|
|
|
} catch (StandbyException se) {
|
|
@@ -2548,9 +2556,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkBlock(previous);
|
|
|
onRetryBlock[0] = null;
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot add block to " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot add block to " + src);
|
|
|
|
|
|
// have we exceeded the configured limit of fs objects.
|
|
|
checkFsObjectLimit();
|
|
@@ -2659,10 +2665,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.READ);
|
|
|
//check safe mode
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot add datanode; src=" + src
|
|
|
- + ", blk=" + blk, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot add datanode; src=" + src + ", blk=" + blk);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
|
|
|
//check lease
|
|
@@ -2707,10 +2710,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot abandon block " + b +
|
|
|
- " for fle" + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot abandon block " + b + " for fle" + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
|
|
|
//
|
|
@@ -2793,9 +2793,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot complete file " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot complete file " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
success = completeFileInternal(src, holder,
|
|
|
ExtendedBlock.getLocalBlock(last), fileId);
|
|
@@ -2971,9 +2969,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot rename " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot rename " + src);
|
|
|
src = FSDirectory.resolvePath(src, srcComponents, dir);
|
|
|
dst = FSDirectory.resolvePath(dst, dstComponents, dir);
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -3044,9 +3040,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot rename " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot rename " + src);
|
|
|
src = FSDirectory.resolvePath(src, srcComponents, dir);
|
|
|
dst = FSDirectory.resolvePath(dst, dstComponents, dir);
|
|
|
renameToInternal(pc, src, dst, cacheEntry != null, options);
|
|
@@ -3152,9 +3146,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot delete " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot delete " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
if (!recursive && dir.isNonEmptyDirectory(src)) {
|
|
|
throw new IOException(src + " is non empty");
|
|
@@ -3373,9 +3365,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot create directory " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot create directory " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
status = mkdirsInternal(pc, src, permissions, createParent);
|
|
|
if (status) {
|
|
@@ -3475,9 +3465,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot set quota on " + path, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot set quota on " + path);
|
|
|
dir.setQuota(path, nsQuota, dsQuota);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -3500,9 +3488,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot fsync file " + src, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot fsync file " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
INodeFileUnderConstruction pendingFile = checkLease(src, clientName);
|
|
|
if (lastBlockLength > 0) {
|
|
@@ -3727,11 +3713,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// If a DN tries to commit to the standby, the recovery will
|
|
|
// fail, and the next retry will succeed on the new NN.
|
|
|
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException(
|
|
|
- "Cannot commitBlockSynchronization while in safe mode",
|
|
|
- safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode(
|
|
|
+ "Cannot commitBlockSynchronization while in safe mode");
|
|
|
final BlockInfo storedBlock = getStoredBlock(
|
|
|
ExtendedBlock.getLocalBlock(lastblock));
|
|
|
if (storedBlock == null) {
|
|
@@ -3877,9 +3860,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot renew lease for " + holder, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot renew lease for " + holder);
|
|
|
leaseManager.renewLease(holder);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -4262,8 +4243,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.UNCHECKED);
|
|
|
if (!isInSafeMode()) {
|
|
|
- throw new IOException("Safe mode should be turned ON " +
|
|
|
- "in order to create namespace image.");
|
|
|
+ throw new IOException("Safe mode should be turned ON "
|
|
|
+ + "in order to create namespace image.");
|
|
|
}
|
|
|
getFSImage().saveNamespace(this);
|
|
|
success = true;
|
|
@@ -4340,7 +4321,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* replicas, and calculates the ratio of safe blocks to the total number
|
|
|
* of blocks in the system, which is the size of blocks in
|
|
|
* {@link FSNamesystem#blockManager}. When the ratio reaches the
|
|
|
- * {@link #threshold} it starts the {@link SafeModeMonitor} daemon in order
|
|
|
+ * {@link #threshold} it starts the SafeModeMonitor daemon in order
|
|
|
* to monitor whether the safe mode {@link #extension} is passed.
|
|
|
* Then it leaves safe mode and destroys itself.
|
|
|
* <p>
|
|
@@ -4348,10 +4329,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* not tracked because the name node is not intended to leave safe mode
|
|
|
* automatically in the case.
|
|
|
*
|
|
|
- * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction)
|
|
|
- * @see SafeModeMonitor
|
|
|
+ * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
|
|
|
*/
|
|
|
- class SafeModeInfo {
|
|
|
+ public class SafeModeInfo {
|
|
|
// configuration fields
|
|
|
/** Safe mode threshold condition %.*/
|
|
|
private double threshold;
|
|
@@ -5093,9 +5073,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.JOURNAL);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Log not rolled", safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Log not rolled");
|
|
|
LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
|
|
|
return getFSImage().rollEditLog();
|
|
|
} finally {
|
|
@@ -5116,9 +5094,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.CHECKPOINT);
|
|
|
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Checkpoint not started", safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Checkpoint not started");
|
|
|
LOG.info("Start checkpoint for " + backupNode.getAddress());
|
|
|
cmd = getFSImage().startCheckpoint(backupNode, activeNamenode);
|
|
|
getEditLog().logSync();
|
|
@@ -5152,9 +5128,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.CHECKPOINT);
|
|
|
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Checkpoint not ended", safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Checkpoint not ended");
|
|
|
LOG.info("End checkpoint for " + registration.getAddress());
|
|
|
getFSImage().endCheckpoint(sig);
|
|
|
success = true;
|
|
@@ -5506,10 +5480,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
long nextGenerationStamp(boolean legacyBlock)
|
|
|
throws IOException, SafeModeException {
|
|
|
assert hasWriteLock();
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException(
|
|
|
- "Cannot get next generation stamp", safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot get next generation stamp");
|
|
|
|
|
|
long gs;
|
|
|
if (legacyBlock) {
|
|
@@ -5562,12 +5533,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
/**
|
|
|
* Increments, logs and then returns the block ID
|
|
|
*/
|
|
|
- private long nextBlockId() throws SafeModeException {
|
|
|
+ private long nextBlockId() throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException(
|
|
|
- "Cannot get next block ID", safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot get next block ID");
|
|
|
final long blockId = blockIdGenerator.nextValue();
|
|
|
getEditLog().logAllocateBlockId(blockId);
|
|
|
// NB: callers sync the log
|
|
@@ -5577,10 +5545,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
private INodeFileUnderConstruction checkUCBlock(ExtendedBlock block,
|
|
|
String clientName) throws IOException {
|
|
|
assert hasWriteLock();
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot get a new generation stamp and an " +
|
|
|
- "access token for block " + block, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot get a new generation stamp and an "
|
|
|
+ + "access token for block " + block);
|
|
|
|
|
|
// check stored block state
|
|
|
BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
|
|
@@ -5693,9 +5659,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
boolean success = false;
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Pipeline not updated", safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Pipeline not updated");
|
|
|
assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
|
|
|
+ oldBlock + " has different block identifier";
|
|
|
updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
|
|
@@ -5955,9 +5919,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot issue delegation token", safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot issue delegation token");
|
|
|
if (!isAllowedDelegationTokenOp()) {
|
|
|
throw new IOException(
|
|
|
"Delegation Token can be issued only with kerberos or web authentication");
|
|
@@ -6002,9 +5964,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot renew delegation token", safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot renew delegation token");
|
|
|
if (!isAllowedDelegationTokenOp()) {
|
|
|
throw new IOException(
|
|
|
"Delegation Token can be renewed only with kerberos or web authentication");
|
|
@@ -6035,9 +5995,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot cancel delegation token", safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot cancel delegation token");
|
|
|
String canceller = getRemoteUser().getUserName();
|
|
|
DelegationTokenIdentifier id = dtSecretManager
|
|
|
.cancelToken(token, canceller);
|
|
@@ -6558,10 +6516,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot allow snapshot for " + path,
|
|
|
- safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot allow snapshot for " + path);
|
|
|
checkSuperuserPrivilege();
|
|
|
|
|
|
dir.writeLock();
|
|
@@ -6586,10 +6541,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot disallow snapshot for " + path,
|
|
|
- safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot disallow snapshot for " + path);
|
|
|
checkSuperuserPrivilege();
|
|
|
|
|
|
dir.writeLock();
|
|
@@ -6627,10 +6579,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
String snapshotPath = null;
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot create snapshot for "
|
|
|
- + snapshotRoot, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot create snapshot for " + snapshotRoot);
|
|
|
if (isPermissionEnabled) {
|
|
|
checkOwner(pc, snapshotRoot);
|
|
|
}
|
|
@@ -6679,10 +6628,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
boolean success = false;
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException("Cannot rename snapshot for " + path,
|
|
|
- safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot rename snapshot for " + path);
|
|
|
if (isPermissionEnabled) {
|
|
|
checkOwner(pc, path);
|
|
|
}
|
|
@@ -6797,10 +6743,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- if (isInSafeMode()) {
|
|
|
- throw new SafeModeException(
|
|
|
- "Cannot delete snapshot for " + snapshotRoot, safeMode);
|
|
|
- }
|
|
|
+ checkNameNodeSafeMode("Cannot delete snapshot for " + snapshotRoot);
|
|
|
if (isPermissionEnabled) {
|
|
|
checkOwner(pc, snapshotRoot);
|
|
|
}
|