|
@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
|
|
@@ -345,7 +346,7 @@ public class FSEditLogLoader {
|
|
|
// 3. OP_ADD to open file for append (old append)
|
|
|
|
|
|
// See if the file already exists (persistBlocks call)
|
|
|
- INodesInPath iip = fsDir.getINodesInPath(path, true);
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(path, DirOp.WRITE);
|
|
|
INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path, true);
|
|
|
if (oldFile != null && addCloseOp.overwrite) {
|
|
|
// This is OP_ADD with overwrite
|
|
@@ -421,7 +422,7 @@ public class FSEditLogLoader {
|
|
|
" clientMachine " + addCloseOp.clientMachine);
|
|
|
}
|
|
|
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath(path, true);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(path, DirOp.READ);
|
|
|
final INodeFile file = INodeFile.valueOf(iip.getLastINode(), path);
|
|
|
|
|
|
// Update the salient file attributes.
|
|
@@ -457,7 +458,7 @@ public class FSEditLogLoader {
|
|
|
" clientMachine " + appendOp.clientMachine +
|
|
|
" newBlock " + appendOp.newBlock);
|
|
|
}
|
|
|
- INodesInPath iip = fsDir.getINodesInPath4Write(path);
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(path, DirOp.WRITE);
|
|
|
INodeFile file = INodeFile.valueOf(iip.getLastINode(), path);
|
|
|
if (!file.isUnderConstruction()) {
|
|
|
LocatedBlock lb = FSDirAppendOp.prepareFileForAppend(fsNamesys, iip,
|
|
@@ -481,7 +482,7 @@ public class FSEditLogLoader {
|
|
|
FSNamesystem.LOG.debug(op.opCode + ": " + path +
|
|
|
" numblocks : " + updateOp.blocks.length);
|
|
|
}
|
|
|
- INodesInPath iip = fsDir.getINodesInPath(path, true);
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(path, DirOp.READ);
|
|
|
INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
|
|
|
// Update in-memory data structures
|
|
|
updateBlocks(fsDir, updateOp, iip, oldFile);
|
|
@@ -507,7 +508,7 @@ public class FSEditLogLoader {
|
|
|
SetReplicationOp setReplicationOp = (SetReplicationOp)op;
|
|
|
String src = renameReservedPathsOnUpgrade(
|
|
|
setReplicationOp.path, logVersion);
|
|
|
- INodesInPath iip = fsDir.getINodesInPath4Write(src);
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(src, DirOp.WRITE);
|
|
|
short replication = fsNamesys.getBlockManager().adjustReplication(
|
|
|
setReplicationOp.replication);
|
|
|
FSDirAttrOp.unprotectedSetReplication(fsDir, iip, replication);
|
|
@@ -521,10 +522,10 @@ public class FSEditLogLoader {
|
|
|
srcs[i] =
|
|
|
renameReservedPathsOnUpgrade(concatDeleteOp.srcs[i], logVersion);
|
|
|
}
|
|
|
- INodesInPath targetIIP = fsDir.getINodesInPath4Write(trg);
|
|
|
+ INodesInPath targetIIP = fsDir.getINodesInPath(trg, DirOp.WRITE);
|
|
|
INodeFile[] srcFiles = new INodeFile[srcs.length];
|
|
|
for (int i = 0; i < srcs.length; i++) {
|
|
|
- INodesInPath srcIIP = fsDir.getINodesInPath4Write(srcs[i]);
|
|
|
+ INodesInPath srcIIP = fsDir.getINodesInPath(srcs[i], DirOp.WRITE);
|
|
|
srcFiles[i] = srcIIP.getLastINode().asFile();
|
|
|
}
|
|
|
FSDirConcatOp.unprotectedConcat(fsDir, targetIIP, srcFiles,
|
|
@@ -551,7 +552,7 @@ public class FSEditLogLoader {
|
|
|
DeleteOp deleteOp = (DeleteOp)op;
|
|
|
final String src = renameReservedPathsOnUpgrade(
|
|
|
deleteOp.path, logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath4Write(src, false);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(src, DirOp.WRITE_LINK);
|
|
|
FSDirDeleteOp.deleteForEditLog(fsDir, iip, deleteOp.timestamp);
|
|
|
|
|
|
if (toAddRetryCache) {
|
|
@@ -578,7 +579,7 @@ public class FSEditLogLoader {
|
|
|
SetPermissionsOp setPermissionsOp = (SetPermissionsOp)op;
|
|
|
final String src =
|
|
|
renameReservedPathsOnUpgrade(setPermissionsOp.src, logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath4Write(src);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(src, DirOp.WRITE);
|
|
|
FSDirAttrOp.unprotectedSetPermission(fsDir, iip,
|
|
|
setPermissionsOp.permissions);
|
|
|
break;
|
|
@@ -587,7 +588,7 @@ public class FSEditLogLoader {
|
|
|
SetOwnerOp setOwnerOp = (SetOwnerOp)op;
|
|
|
final String src = renameReservedPathsOnUpgrade(
|
|
|
setOwnerOp.src, logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath4Write(src);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(src, DirOp.WRITE);
|
|
|
FSDirAttrOp.unprotectedSetOwner(fsDir, iip,
|
|
|
setOwnerOp.username, setOwnerOp.groupname);
|
|
|
break;
|
|
@@ -596,7 +597,7 @@ public class FSEditLogLoader {
|
|
|
SetNSQuotaOp setNSQuotaOp = (SetNSQuotaOp)op;
|
|
|
final String src = renameReservedPathsOnUpgrade(
|
|
|
setNSQuotaOp.src, logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath4Write(src);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(src, DirOp.WRITE);
|
|
|
FSDirAttrOp.unprotectedSetQuota(fsDir, iip,
|
|
|
setNSQuotaOp.nsQuota, HdfsConstants.QUOTA_DONT_SET, null);
|
|
|
break;
|
|
@@ -605,7 +606,7 @@ public class FSEditLogLoader {
|
|
|
ClearNSQuotaOp clearNSQuotaOp = (ClearNSQuotaOp)op;
|
|
|
final String src = renameReservedPathsOnUpgrade(
|
|
|
clearNSQuotaOp.src, logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath4Write(src);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(src, DirOp.WRITE);
|
|
|
FSDirAttrOp.unprotectedSetQuota(fsDir, iip,
|
|
|
HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_DONT_SET, null);
|
|
|
break;
|
|
@@ -614,7 +615,7 @@ public class FSEditLogLoader {
|
|
|
SetQuotaOp setQuotaOp = (SetQuotaOp) op;
|
|
|
final String src = renameReservedPathsOnUpgrade(
|
|
|
setQuotaOp.src, logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath4Write(src);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(src, DirOp.WRITE);
|
|
|
FSDirAttrOp.unprotectedSetQuota(fsDir, iip,
|
|
|
setQuotaOp.nsQuota, setQuotaOp.dsQuota, null);
|
|
|
break;
|
|
@@ -624,7 +625,7 @@ public class FSEditLogLoader {
|
|
|
(FSEditLogOp.SetQuotaByStorageTypeOp) op;
|
|
|
final String src = renameReservedPathsOnUpgrade(
|
|
|
setQuotaByStorageTypeOp.src, logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath4Write(src);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(src, DirOp.WRITE);
|
|
|
FSDirAttrOp.unprotectedSetQuota(fsDir, iip,
|
|
|
HdfsConstants.QUOTA_DONT_SET, setQuotaByStorageTypeOp.dsQuota,
|
|
|
setQuotaByStorageTypeOp.type);
|
|
@@ -634,7 +635,7 @@ public class FSEditLogLoader {
|
|
|
TimesOp timesOp = (TimesOp)op;
|
|
|
final String src = renameReservedPathsOnUpgrade(
|
|
|
timesOp.path, logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath4Write(src);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(src, DirOp.WRITE);
|
|
|
FSDirAttrOp.unprotectedSetTimes(fsDir, iip,
|
|
|
timesOp.mtime, timesOp.atime, true);
|
|
|
break;
|
|
@@ -648,7 +649,7 @@ public class FSEditLogLoader {
|
|
|
lastInodeId);
|
|
|
final String path = renameReservedPathsOnUpgrade(symlinkOp.path,
|
|
|
logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath(path, false);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(path, DirOp.WRITE_LINK);
|
|
|
FSDirSymlinkOp.unprotectedAddSymlink(fsDir, iip.getExistingINodes(),
|
|
|
iip.getLastLocalName(), inodeId, symlinkOp.value, symlinkOp.mtime,
|
|
|
symlinkOp.atime, symlinkOp.permissionStatus);
|
|
@@ -708,7 +709,7 @@ public class FSEditLogLoader {
|
|
|
reassignLeaseOp.leaseHolder);
|
|
|
final String path =
|
|
|
renameReservedPathsOnUpgrade(reassignLeaseOp.path, logVersion);
|
|
|
- INodeFile pendingFile = fsDir.getINode(path).asFile();
|
|
|
+ INodeFile pendingFile = fsDir.getINode(path, DirOp.READ).asFile();
|
|
|
Preconditions.checkState(pendingFile.isUnderConstruction());
|
|
|
fsNamesys.reassignLeaseInternal(lease, reassignLeaseOp.newHolder,
|
|
|
pendingFile);
|
|
@@ -724,7 +725,7 @@ public class FSEditLogLoader {
|
|
|
final String snapshotRoot =
|
|
|
renameReservedPathsOnUpgrade(createSnapshotOp.snapshotRoot,
|
|
|
logVersion);
|
|
|
- INodesInPath iip = fsDir.getINodesInPath4Write(snapshotRoot);
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(snapshotRoot, DirOp.WRITE);
|
|
|
String path = fsNamesys.getSnapshotManager().createSnapshot(iip,
|
|
|
snapshotRoot, createSnapshotOp.snapshotName);
|
|
|
if (toAddRetryCache) {
|
|
@@ -740,7 +741,7 @@ public class FSEditLogLoader {
|
|
|
final String snapshotRoot =
|
|
|
renameReservedPathsOnUpgrade(deleteSnapshotOp.snapshotRoot,
|
|
|
logVersion);
|
|
|
- INodesInPath iip = fsDir.getINodesInPath4Write(snapshotRoot);
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(snapshotRoot, DirOp.WRITE);
|
|
|
fsNamesys.getSnapshotManager().deleteSnapshot(iip,
|
|
|
deleteSnapshotOp.snapshotName,
|
|
|
new INode.ReclaimContext(fsNamesys.dir.getBlockStoragePolicySuite(),
|
|
@@ -761,7 +762,7 @@ public class FSEditLogLoader {
|
|
|
final String snapshotRoot =
|
|
|
renameReservedPathsOnUpgrade(renameSnapshotOp.snapshotRoot,
|
|
|
logVersion);
|
|
|
- INodesInPath iip = fsDir.getINodesInPath4Write(snapshotRoot);
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(snapshotRoot, DirOp.WRITE);
|
|
|
fsNamesys.getSnapshotManager().renameSnapshot(iip,
|
|
|
snapshotRoot, renameSnapshotOp.snapshotOldName,
|
|
|
renameSnapshotOp.snapshotNewName);
|
|
@@ -886,13 +887,13 @@ public class FSEditLogLoader {
|
|
|
}
|
|
|
case OP_SET_ACL: {
|
|
|
SetAclOp setAclOp = (SetAclOp) op;
|
|
|
- FSDirAclOp.unprotectedSetAcl(fsDir, setAclOp.src, setAclOp.aclEntries,
|
|
|
- true);
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(setAclOp.src, DirOp.WRITE);
|
|
|
+ FSDirAclOp.unprotectedSetAcl(fsDir, iip, setAclOp.aclEntries, true);
|
|
|
break;
|
|
|
}
|
|
|
case OP_SET_XATTR: {
|
|
|
SetXAttrOp setXAttrOp = (SetXAttrOp) op;
|
|
|
- INodesInPath iip = fsDir.getINodesInPath4Write(setXAttrOp.src);
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(setXAttrOp.src, DirOp.WRITE);
|
|
|
FSDirXAttrOp.unprotectedSetXAttrs(fsDir, iip,
|
|
|
setXAttrOp.xAttrs,
|
|
|
EnumSet.of(XAttrSetFlag.CREATE,
|
|
@@ -914,7 +915,8 @@ public class FSEditLogLoader {
|
|
|
}
|
|
|
case OP_TRUNCATE: {
|
|
|
TruncateOp truncateOp = (TruncateOp) op;
|
|
|
- FSDirTruncateOp.unprotectedTruncate(fsNamesys, truncateOp.src,
|
|
|
+ INodesInPath iip = fsDir.getINodesInPath(truncateOp.src, DirOp.WRITE);
|
|
|
+ FSDirTruncateOp.unprotectedTruncate(fsNamesys, iip,
|
|
|
truncateOp.clientName, truncateOp.clientMachine,
|
|
|
truncateOp.newLength, truncateOp.timestamp, truncateOp.truncateBlock);
|
|
|
break;
|
|
@@ -923,7 +925,7 @@ public class FSEditLogLoader {
|
|
|
SetStoragePolicyOp setStoragePolicyOp = (SetStoragePolicyOp) op;
|
|
|
final String path = renameReservedPathsOnUpgrade(setStoragePolicyOp.path,
|
|
|
logVersion);
|
|
|
- final INodesInPath iip = fsDir.getINodesInPath4Write(path);
|
|
|
+ final INodesInPath iip = fsDir.getINodesInPath(path, DirOp.WRITE);
|
|
|
FSDirAttrOp.unprotectedSetStoragePolicy(
|
|
|
fsDir, fsNamesys.getBlockManager(), iip,
|
|
|
setStoragePolicyOp.policyId);
|