|
@@ -526,7 +526,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
private volatile boolean startingActiveService = false;
|
|
|
|
|
|
private INodeId inodeId;
|
|
|
-
|
|
|
+
|
|
|
private final RetryCache retryCache;
|
|
|
|
|
|
private final NNConf nnConf;
|
|
@@ -1943,28 +1943,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @param srcs file that will be concatenated
|
|
|
* @throws IOException on error
|
|
|
*/
|
|
|
- void concat(String target, String [] srcs)
|
|
|
+ void concat(String target, String [] srcs, boolean logRetryCache)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
-
|
|
|
- // Either there is no previous request in progress or it has failed
|
|
|
if(FSNamesystem.LOG.isDebugEnabled()) {
|
|
|
FSNamesystem.LOG.debug("concat " + Arrays.toString(srcs) +
|
|
|
" to " + target);
|
|
|
}
|
|
|
-
|
|
|
- boolean success = false;
|
|
|
+
|
|
|
try {
|
|
|
- concatInt(target, srcs, cacheEntry != null);
|
|
|
- success = true;
|
|
|
+ concatInt(target, srcs, logRetryCache);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "concat", Arrays.toString(srcs), target, null);
|
|
|
throw e;
|
|
|
- } finally {
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2181,7 +2171,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* Create a symbolic link.
|
|
|
*/
|
|
|
void createSymlink(String target, String link,
|
|
|
- PermissionStatus dirPerms, boolean createParent)
|
|
|
+ PermissionStatus dirPerms, boolean createParent, boolean logRetryCache)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
if (!DFSUtil.isValidName(link)) {
|
|
|
throw new InvalidPathException("Invalid link name: " + link);
|
|
@@ -2189,19 +2179,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (FSDirectory.isReservedName(target)) {
|
|
|
throw new InvalidPathException("Invalid target name: " + target);
|
|
|
}
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
+
|
|
|
boolean success = false;
|
|
|
try {
|
|
|
- createSymlinkInt(target, link, dirPerms, createParent, cacheEntry != null);
|
|
|
+ createSymlinkInt(target, link, dirPerms, createParent, logRetryCache);
|
|
|
success = true;
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "createSymlink", link, target, null);
|
|
|
throw e;
|
|
|
- } finally {
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2489,26 +2474,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
HdfsFileStatus startFile(String src, PermissionStatus permissions,
|
|
|
String holder, String clientMachine, EnumSet<CreateFlag> flag,
|
|
|
boolean createParent, short replication, long blockSize,
|
|
|
- CryptoProtocolVersion[] supportedVersions)
|
|
|
+ CryptoProtocolVersion[] supportedVersions, boolean logRetryCache)
|
|
|
throws AccessControlException, SafeModeException,
|
|
|
FileAlreadyExistsException, UnresolvedLinkException,
|
|
|
FileNotFoundException, ParentNotDirectoryException, IOException {
|
|
|
+
|
|
|
HdfsFileStatus status = null;
|
|
|
- CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache,
|
|
|
- null);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return (HdfsFileStatus) cacheEntry.getPayload();
|
|
|
- }
|
|
|
-
|
|
|
try {
|
|
|
status = startFileInt(src, permissions, holder, clientMachine, flag,
|
|
|
createParent, replication, blockSize, supportedVersions,
|
|
|
- cacheEntry != null);
|
|
|
+ logRetryCache);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "create", src);
|
|
|
throw e;
|
|
|
- } finally {
|
|
|
- RetryCache.setState(cacheEntry, status != null, status);
|
|
|
}
|
|
|
return status;
|
|
|
}
|
|
@@ -3035,27 +3013,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
/**
|
|
|
* Append to an existing file in the namespace.
|
|
|
*/
|
|
|
- LocatedBlock appendFile(String src, String holder, String clientMachine)
|
|
|
- throws AccessControlException, SafeModeException,
|
|
|
- FileAlreadyExistsException, FileNotFoundException,
|
|
|
- ParentNotDirectoryException, IOException {
|
|
|
- LocatedBlock lb = null;
|
|
|
- CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache,
|
|
|
- null);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return (LocatedBlock) cacheEntry.getPayload();
|
|
|
- }
|
|
|
-
|
|
|
- boolean success = false;
|
|
|
+ LocatedBlock appendFile(
|
|
|
+ String src, String holder, String clientMachine, boolean logRetryCache)
|
|
|
+ throws IOException {
|
|
|
try {
|
|
|
- lb = appendFileInt(src, holder, clientMachine, cacheEntry != null);
|
|
|
- success = true;
|
|
|
- return lb;
|
|
|
+ return appendFileInt(src, holder, clientMachine, logRetryCache);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "append", src);
|
|
|
throw e;
|
|
|
- } finally {
|
|
|
- RetryCache.setState(cacheEntry, success, lb);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -3717,20 +3682,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @deprecated Use {@link #renameTo(String, String, Options.Rename...)} instead.
|
|
|
*/
|
|
|
@Deprecated
|
|
|
- boolean renameTo(String src, String dst)
|
|
|
+ boolean renameTo(String src, String dst, boolean logRetryCache)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return true; // Return previous response
|
|
|
- }
|
|
|
boolean ret = false;
|
|
|
try {
|
|
|
- ret = renameToInt(src, dst, cacheEntry != null);
|
|
|
+ ret = renameToInt(src, dst, logRetryCache);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "rename", src, dst, null);
|
|
|
throw e;
|
|
|
- } finally {
|
|
|
- RetryCache.setState(cacheEntry, ret);
|
|
|
}
|
|
|
return ret;
|
|
|
}
|
|
@@ -3775,7 +3734,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return status;
|
|
|
}
|
|
|
|
|
|
- /** @deprecated See {@link #renameTo(String, String)} */
|
|
|
+ /** @deprecated See {@link #renameTo(String, String, boolean)} */
|
|
|
@Deprecated
|
|
|
private boolean renameToInternal(FSPermissionChecker pc, String src,
|
|
|
String dst, boolean logRetryCache) throws IOException,
|
|
@@ -3808,7 +3767,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
|
|
|
/** Rename src to dst */
|
|
|
- void renameTo(final String srcArg, final String dstArg,
|
|
|
+ void renameTo(final String srcArg, final String dstArg, boolean logRetryCache,
|
|
|
Options.Rename... options) throws IOException, UnresolvedLinkException {
|
|
|
String src = srcArg;
|
|
|
String dst = dstArg;
|
|
@@ -3822,14 +3781,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
+
|
|
|
byte[][] srcComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
byte[][] dstComponents = FSDirectory.getPathComponentsForReservedPath(dst);
|
|
|
HdfsFileStatus resultingStat = null;
|
|
|
- boolean success = false;
|
|
|
writeLock();
|
|
|
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
|
|
try {
|
|
@@ -3837,13 +3792,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
checkNameNodeSafeMode("Cannot rename " + src);
|
|
|
src = dir.resolvePath(pc, src, srcComponents);
|
|
|
dst = dir.resolvePath(pc, dst, dstComponents);
|
|
|
- renameToInternal(pc, src, dst, cacheEntry != null,
|
|
|
- collectedBlocks, options);
|
|
|
+ renameToInternal(pc, src, dst, logRetryCache, collectedBlocks, options);
|
|
|
resultingStat = getAuditFileInfo(dst, false);
|
|
|
- success = true;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
if (!collectedBlocks.getToDeleteList().isEmpty()) {
|
|
@@ -3886,21 +3838,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @see ClientProtocol#delete(String, boolean) for detailed description and
|
|
|
* description of exceptions
|
|
|
*/
|
|
|
- boolean delete(String src, boolean recursive)
|
|
|
+ boolean delete(String src, boolean recursive, boolean logRetryCache)
|
|
|
throws AccessControlException, SafeModeException,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return true; // Return previous response
|
|
|
- }
|
|
|
+
|
|
|
boolean ret = false;
|
|
|
try {
|
|
|
- ret = deleteInt(src, recursive, cacheEntry != null);
|
|
|
+ ret = deleteInt(src, recursive, logRetryCache);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "delete", src);
|
|
|
throw e;
|
|
|
- } finally {
|
|
|
- RetryCache.setState(cacheEntry, ret);
|
|
|
}
|
|
|
return ret;
|
|
|
}
|
|
@@ -5478,12 +5425,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
void saveNamespace() throws AccessControlException, IOException {
|
|
|
checkOperation(OperationCategory.UNCHECKED);
|
|
|
checkSuperuserPrivilege();
|
|
|
-
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
- boolean success = false;
|
|
|
+
|
|
|
+
|
|
|
readLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.UNCHECKED);
|
|
@@ -5493,10 +5436,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
+ "in order to create namespace image.");
|
|
|
}
|
|
|
getFSImage().saveNamespace(this);
|
|
|
- success = true;
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
LOG.info("New namespace image has been created");
|
|
|
}
|
|
@@ -6327,24 +6268,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
NamenodeCommand startCheckpoint(NamenodeRegistration backupNode,
|
|
|
NamenodeRegistration activeNamenode) throws IOException {
|
|
|
checkOperation(OperationCategory.CHECKPOINT);
|
|
|
- CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache,
|
|
|
- null);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return (NamenodeCommand) cacheEntry.getPayload();
|
|
|
- }
|
|
|
writeLock();
|
|
|
- NamenodeCommand cmd = null;
|
|
|
try {
|
|
|
checkOperation(OperationCategory.CHECKPOINT);
|
|
|
checkNameNodeSafeMode("Checkpoint not started");
|
|
|
|
|
|
LOG.info("Start checkpoint for " + backupNode.getAddress());
|
|
|
- cmd = getFSImage().startCheckpoint(backupNode, activeNamenode);
|
|
|
+ NamenodeCommand cmd = getFSImage().startCheckpoint(backupNode,
|
|
|
+ activeNamenode);
|
|
|
getEditLog().logSync();
|
|
|
return cmd;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
- RetryCache.setState(cacheEntry, cmd != null, cmd);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -6362,22 +6297,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
void endCheckpoint(NamenodeRegistration registration,
|
|
|
CheckpointSignature sig) throws IOException {
|
|
|
checkOperation(OperationCategory.CHECKPOINT);
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
- boolean success = false;
|
|
|
readLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.CHECKPOINT);
|
|
|
-
|
|
|
checkNameNodeSafeMode("Checkpoint not ended");
|
|
|
LOG.info("End checkpoint for " + registration.getAddress());
|
|
|
getFSImage().endCheckpoint(sig);
|
|
|
- success = true;
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -6821,14 +6748,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @param newNodes datanodes in the pipeline
|
|
|
* @throws IOException if any error occurs
|
|
|
*/
|
|
|
- void updatePipeline(String clientName, ExtendedBlock oldBlock,
|
|
|
- ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
|
|
|
+ void updatePipeline(
|
|
|
+ String clientName, ExtendedBlock oldBlock, ExtendedBlock newBlock,
|
|
|
+ DatanodeID[] newNodes, String[] newStorageIDs, boolean logRetryCache)
|
|
|
throws IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
+
|
|
|
LOG.info("updatePipeline(" + oldBlock.getLocalBlock()
|
|
|
+ ", newGS=" + newBlock.getGenerationStamp()
|
|
|
+ ", newLength=" + newBlock.getNumBytes()
|
|
@@ -6837,18 +6762,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
+ ")");
|
|
|
waitForLoadingFSImage();
|
|
|
writeLock();
|
|
|
- boolean success = false;
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Pipeline not updated");
|
|
|
assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
|
|
|
+ oldBlock + " has different block identifier";
|
|
|
updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
|
|
|
- newStorageIDs, cacheEntry != null);
|
|
|
- success = true;
|
|
|
+ newStorageIDs, logRetryCache);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
LOG.info("updatePipeline(" + oldBlock.getLocalBlock() + " => "
|
|
@@ -7844,15 +7766,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @param snapshotRoot The directory path where the snapshot is taken
|
|
|
* @param snapshotName The name of the snapshot
|
|
|
*/
|
|
|
- String createSnapshot(String snapshotRoot, String snapshotName)
|
|
|
+ String createSnapshot(String snapshotRoot, String snapshotName,
|
|
|
+ boolean logRetryCache)
|
|
|
throws SafeModeException, IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
- CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache,
|
|
|
- null);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return (String) cacheEntry.getPayload();
|
|
|
- }
|
|
|
+
|
|
|
String snapshotPath = null;
|
|
|
writeLock();
|
|
|
try {
|
|
@@ -7878,11 +7797,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
} finally {
|
|
|
dir.writeUnlock();
|
|
|
}
|
|
|
- getEditLog().logCreateSnapshot(snapshotRoot, snapshotName,
|
|
|
- cacheEntry != null);
|
|
|
+ getEditLog().logCreateSnapshot(snapshotRoot, snapshotName, logRetryCache);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
- RetryCache.setState(cacheEntry, snapshotPath != null, snapshotPath);
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
|
|
@@ -7900,16 +7817,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @throws SafeModeException
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- void renameSnapshot(String path, String snapshotOldName,
|
|
|
- String snapshotNewName) throws SafeModeException, IOException {
|
|
|
+ void renameSnapshot(
|
|
|
+ String path, String snapshotOldName, String snapshotNewName,
|
|
|
+ boolean logRetryCache) throws IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
+
|
|
|
writeLock();
|
|
|
- boolean success = false;
|
|
|
+
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot rename snapshot for " + path);
|
|
@@ -7920,11 +7835,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
snapshotManager.renameSnapshot(path, snapshotOldName, snapshotNewName);
|
|
|
getEditLog().logRenameSnapshot(path, snapshotOldName, snapshotNewName,
|
|
|
- cacheEntry != null);
|
|
|
- success = true;
|
|
|
+ logRetryCache);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
+
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
|
|
@@ -8014,16 +7928,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @throws SafeModeException
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- void deleteSnapshot(String snapshotRoot, String snapshotName)
|
|
|
+ void deleteSnapshot(String snapshotRoot, String snapshotName,
|
|
|
+ boolean logRetryCache)
|
|
|
throws SafeModeException, IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = getPermissionChecker();
|
|
|
-
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
- boolean success = false;
|
|
|
+
|
|
|
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
|
|
writeLock();
|
|
|
try {
|
|
@@ -8043,12 +7953,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
dir.writeUnlock();
|
|
|
}
|
|
|
removedINodes.clear();
|
|
|
- getEditLog().logDeleteSnapshot(snapshotRoot, snapshotName,
|
|
|
- cacheEntry != null);
|
|
|
- success = true;
|
|
|
+ getEditLog().logDeleteSnapshot(snapshotRoot, snapshotName, logRetryCache);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
+
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
|
|
@@ -8246,20 +8154,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return new RollingUpgradeInfo(blockPoolId, false, startTime, finalizeTime);
|
|
|
}
|
|
|
|
|
|
- long addCacheDirective(CacheDirectiveInfo directive, EnumSet<CacheFlag> flags)
|
|
|
+ long addCacheDirective(CacheDirectiveInfo directive,
|
|
|
+ EnumSet<CacheFlag> flags, boolean logRetryCache)
|
|
|
throws IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = isPermissionEnabled ?
|
|
|
getPermissionChecker() : null;
|
|
|
- CacheEntryWithPayload cacheEntry =
|
|
|
- RetryCache.waitForCompletion(retryCache, null);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return (Long) cacheEntry.getPayload();
|
|
|
- }
|
|
|
- boolean success = false;
|
|
|
+
|
|
|
if (!flags.contains(CacheFlag.FORCE)) {
|
|
|
cacheManager.waitForRescanIfNeeded();
|
|
|
}
|
|
|
+ boolean success = false;
|
|
|
writeLock();
|
|
|
String effectiveDirectiveStr = null;
|
|
|
Long result = null;
|
|
@@ -8275,8 +8180,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
CacheDirectiveInfo effectiveDirective =
|
|
|
cacheManager.addDirective(directive, pc, flags);
|
|
|
- getEditLog().logAddCacheDirectiveInfo(effectiveDirective,
|
|
|
- cacheEntry != null);
|
|
|
+ getEditLog().logAddCacheDirectiveInfo(effectiveDirective, logRetryCache);
|
|
|
result = effectiveDirective.getId();
|
|
|
effectiveDirectiveStr = effectiveDirective.toString();
|
|
|
success = true;
|
|
@@ -8288,21 +8192,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (isAuditEnabled() && isExternalInvocation()) {
|
|
|
logAuditEvent(success, "addCacheDirective", effectiveDirectiveStr, null, null);
|
|
|
}
|
|
|
- RetryCache.setState(cacheEntry, success, result);
|
|
|
+
|
|
|
}
|
|
|
return result;
|
|
|
}
|
|
|
|
|
|
void modifyCacheDirective(CacheDirectiveInfo directive,
|
|
|
- EnumSet<CacheFlag> flags) throws IOException {
|
|
|
+ EnumSet<CacheFlag> flags, boolean logRetryCache) throws IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = isPermissionEnabled ?
|
|
|
getPermissionChecker() : null;
|
|
|
boolean success = false;
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return;
|
|
|
- }
|
|
|
+
|
|
|
if (!flags.contains(CacheFlag.FORCE)) {
|
|
|
cacheManager.waitForRescanIfNeeded();
|
|
|
}
|
|
@@ -8314,8 +8215,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
"Cannot add cache directive", safeMode);
|
|
|
}
|
|
|
cacheManager.modifyDirective(directive, pc, flags);
|
|
|
- getEditLog().logModifyCacheDirectiveInfo(directive,
|
|
|
- cacheEntry != null);
|
|
|
+ getEditLog().logModifyCacheDirectiveInfo(directive, logRetryCache);
|
|
|
success = true;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -8326,18 +8226,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
String idStr = "{id: " + directive.getId().toString() + "}";
|
|
|
logAuditEvent(success, "modifyCacheDirective", idStr, directive.toString(), null);
|
|
|
}
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void removeCacheDirective(Long id) throws IOException {
|
|
|
+ void removeCacheDirective(long id, boolean logRetryCache) throws IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = isPermissionEnabled ?
|
|
|
getPermissionChecker() : null;
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return;
|
|
|
- }
|
|
|
+
|
|
|
boolean success = false;
|
|
|
writeLock();
|
|
|
try {
|
|
@@ -8347,16 +8243,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
"Cannot remove cache directives", safeMode);
|
|
|
}
|
|
|
cacheManager.removeDirective(id, pc);
|
|
|
- getEditLog().logRemoveCacheDirectiveInfo(id, cacheEntry != null);
|
|
|
+ getEditLog().logRemoveCacheDirectiveInfo(id, logRetryCache);
|
|
|
success = true;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
if (isAuditEnabled() && isExternalInvocation()) {
|
|
|
- String idStr = "{id: " + id.toString() + "}";
|
|
|
+ String idStr = "{id: " + Long.toString(id) + "}";
|
|
|
logAuditEvent(success, "removeCacheDirective", idStr, null,
|
|
|
null);
|
|
|
}
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
getEditLog().logSync();
|
|
|
}
|
|
@@ -8385,14 +8280,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return results;
|
|
|
}
|
|
|
|
|
|
- public void addCachePool(CachePoolInfo req) throws IOException {
|
|
|
+ public void addCachePool(CachePoolInfo req, boolean logRetryCache)
|
|
|
+ throws IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc = isPermissionEnabled ?
|
|
|
getPermissionChecker() : null;
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
+
|
|
|
writeLock();
|
|
|
boolean success = false;
|
|
|
String poolInfoStr = null;
|
|
@@ -8407,27 +8300,24 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
CachePoolInfo info = cacheManager.addCachePool(req);
|
|
|
poolInfoStr = info.toString();
|
|
|
- getEditLog().logAddCachePool(info, cacheEntry != null);
|
|
|
+ getEditLog().logAddCachePool(info, logRetryCache);
|
|
|
success = true;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
if (isAuditEnabled() && isExternalInvocation()) {
|
|
|
logAuditEvent(success, "addCachePool", poolInfoStr, null, null);
|
|
|
}
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
|
|
|
getEditLog().logSync();
|
|
|
}
|
|
|
|
|
|
- public void modifyCachePool(CachePoolInfo req) throws IOException {
|
|
|
+ public void modifyCachePool(CachePoolInfo req, boolean logRetryCache)
|
|
|
+ throws IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc =
|
|
|
isPermissionEnabled ? getPermissionChecker() : null;
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
+
|
|
|
writeLock();
|
|
|
boolean success = false;
|
|
|
try {
|
|
@@ -8440,7 +8330,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
pc.checkSuperuserPrivilege();
|
|
|
}
|
|
|
cacheManager.modifyCachePool(req);
|
|
|
- getEditLog().logModifyCachePool(req, cacheEntry != null);
|
|
|
+ getEditLog().logModifyCachePool(req, logRetryCache);
|
|
|
success = true;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -8448,20 +8338,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
String poolNameStr = "{poolName: " + req.getPoolName() + "}";
|
|
|
logAuditEvent(success, "modifyCachePool", poolNameStr, req.toString(), null);
|
|
|
}
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
|
|
|
getEditLog().logSync();
|
|
|
}
|
|
|
|
|
|
- public void removeCachePool(String cachePoolName) throws IOException {
|
|
|
+ public void removeCachePool(String cachePoolName, boolean logRetryCache)
|
|
|
+ throws IOException {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
final FSPermissionChecker pc =
|
|
|
isPermissionEnabled ? getPermissionChecker() : null;
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
+
|
|
|
writeLock();
|
|
|
boolean success = false;
|
|
|
try {
|
|
@@ -8474,7 +8361,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
pc.checkSuperuserPrivilege();
|
|
|
}
|
|
|
cacheManager.removeCachePool(cachePoolName);
|
|
|
- getEditLog().logRemoveCachePool(cachePoolName, cacheEntry != null);
|
|
|
+ getEditLog().logRemoveCachePool(cachePoolName, logRetryCache);
|
|
|
success = true;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -8482,7 +8369,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
String poolNameStr = "{poolName: " + cachePoolName + "}";
|
|
|
logAuditEvent(success, "removeCachePool", poolNameStr, null, null);
|
|
|
}
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
|
|
|
getEditLog().logSync();
|
|
@@ -8675,15 +8561,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @throws UnresolvedLinkException if the path can't be resolved.
|
|
|
* @throws SafeModeException if the Namenode is in safe mode.
|
|
|
*/
|
|
|
- void createEncryptionZone(final String src, final String keyName)
|
|
|
+ void createEncryptionZone(final String src, final String keyName,
|
|
|
+ boolean logRetryCache)
|
|
|
throws IOException, UnresolvedLinkException,
|
|
|
SafeModeException, AccessControlException {
|
|
|
- final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
-
|
|
|
- boolean success = false;
|
|
|
try {
|
|
|
if (provider == null) {
|
|
|
throw new IOException(
|
|
@@ -8709,13 +8590,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
// If the provider supports pool for EDEKs, this will fill in the pool
|
|
|
generateEncryptedDataEncryptionKey(keyName);
|
|
|
createEncryptionZoneInt(src, metadata.getCipher(),
|
|
|
- keyName, cacheEntry != null);
|
|
|
- success = true;
|
|
|
+ keyName, logRetryCache);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "createEncryptionZone", src);
|
|
|
throw e;
|
|
|
- } finally {
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -8820,22 +8698,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @throws UnresolvedLinkException
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
|
|
|
+ void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
|
|
|
+ boolean logRetryCache)
|
|
|
throws AccessControlException, SafeModeException,
|
|
|
UnresolvedLinkException, IOException {
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
- boolean success = false;
|
|
|
try {
|
|
|
- setXAttrInt(src, xAttr, flag, cacheEntry != null);
|
|
|
- success = true;
|
|
|
+ setXAttrInt(src, xAttr, flag, logRetryCache);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "setXAttr", src);
|
|
|
throw e;
|
|
|
- } finally {
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -8985,20 +8856,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
* @throws UnresolvedLinkException
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- void removeXAttr(String src, XAttr xAttr) throws IOException {
|
|
|
- CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
|
|
- if (cacheEntry != null && cacheEntry.isSuccess()) {
|
|
|
- return; // Return previous response
|
|
|
- }
|
|
|
- boolean success = false;
|
|
|
+ void removeXAttr(String src, XAttr xAttr, boolean logRetryCache)
|
|
|
+ throws IOException {
|
|
|
try {
|
|
|
- removeXAttrInt(src, xAttr, cacheEntry != null);
|
|
|
- success = true;
|
|
|
+ removeXAttrInt(src, xAttr, logRetryCache);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "removeXAttr", src);
|
|
|
throw e;
|
|
|
- } finally {
|
|
|
- RetryCache.setState(cacheEntry, success);
|
|
|
}
|
|
|
}
|
|
|
|