|
@@ -17,20 +17,9 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
-import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
|
|
|
|
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_ENCRYPTION_ZONE;
|
|
|
|
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
|
|
|
|
-import static org.apache.hadoop.util.Time.now;
|
|
|
|
-
|
|
|
|
-import java.io.Closeable;
|
|
|
|
-import java.io.FileNotFoundException;
|
|
|
|
-import java.io.IOException;
|
|
|
|
-import java.util.ArrayList;
|
|
|
|
-import java.util.Arrays;
|
|
|
|
-import java.util.EnumSet;
|
|
|
|
-import java.util.List;
|
|
|
|
-import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
-
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
|
+import com.google.common.collect.Lists;
|
|
import com.google.protobuf.InvalidProtocolBufferException;
|
|
import com.google.protobuf.InvalidProtocolBufferException;
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
@@ -41,7 +30,6 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
import org.apache.hadoop.fs.FileEncryptionInfo;
|
|
import org.apache.hadoop.fs.FileEncryptionInfo;
|
|
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
-import org.apache.hadoop.fs.PathIsNotDirectoryException;
|
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
import org.apache.hadoop.fs.XAttr;
|
|
import org.apache.hadoop.fs.XAttr;
|
|
import org.apache.hadoop.fs.XAttrSetFlag;
|
|
import org.apache.hadoop.fs.XAttrSetFlag;
|
|
@@ -54,8 +42,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.XAttrHelper;
|
|
import org.apache.hadoop.hdfs.XAttrHelper;
|
|
import org.apache.hadoop.hdfs.protocol.AclException;
|
|
import org.apache.hadoop.hdfs.protocol.AclException;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|
|
|
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
|
import org.apache.hadoop.hdfs.protocol.EncryptionZone;
|
|
import org.apache.hadoop.hdfs.protocol.EncryptionZone;
|
|
import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
|
|
import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
|
|
import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
|
|
import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
|
|
@@ -68,22 +54,35 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
|
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
|
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
|
|
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
import org.apache.hadoop.hdfs.util.ByteArray;
|
|
import org.apache.hadoop.hdfs.util.ByteArray;
|
|
import org.apache.hadoop.hdfs.util.ChunkedArrayList;
|
|
import org.apache.hadoop.hdfs.util.ChunkedArrayList;
|
|
-
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
|
-import com.google.common.collect.Lists;
|
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
+import java.io.Closeable;
|
|
|
|
+import java.io.FileNotFoundException;
|
|
|
|
+import java.io.IOException;
|
|
|
|
+import java.util.ArrayList;
|
|
|
|
+import java.util.Arrays;
|
|
|
|
+import java.util.EnumSet;
|
|
|
|
+import java.util.List;
|
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
+
|
|
|
|
+import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT;
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
|
|
|
|
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_ENCRYPTION_ZONE;
|
|
|
|
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
|
|
|
|
+import static org.apache.hadoop.util.Time.now;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Both FSDirectory and FSNamesystem manage the state of the namespace.
|
|
* Both FSDirectory and FSNamesystem manage the state of the namespace.
|
|
* FSDirectory is a pure in-memory data structure, all of whose operations
|
|
* FSDirectory is a pure in-memory data structure, all of whose operations
|
|
@@ -145,6 +144,12 @@ public class FSDirectory implements Closeable {
|
|
private final boolean aclsEnabled;
|
|
private final boolean aclsEnabled;
|
|
private final boolean xattrsEnabled;
|
|
private final boolean xattrsEnabled;
|
|
private final int xattrMaxSize;
|
|
private final int xattrMaxSize;
|
|
|
|
+
|
|
|
|
+ // precision of access times.
|
|
|
|
+ private final long accessTimePrecision;
|
|
|
|
+ // whether setStoragePolicy is allowed.
|
|
|
|
+ private final boolean storagePolicyEnabled;
|
|
|
|
+
|
|
private final String fsOwnerShortUserName;
|
|
private final String fsOwnerShortUserName;
|
|
private final String supergroup;
|
|
private final String supergroup;
|
|
private final INodeId inodeId;
|
|
private final INodeId inodeId;
|
|
@@ -222,6 +227,15 @@ public class FSDirectory implements Closeable {
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY);
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY);
|
|
final String unlimited = xattrMaxSize == 0 ? " (unlimited)" : "";
|
|
final String unlimited = xattrMaxSize == 0 ? " (unlimited)" : "";
|
|
LOG.info("Maximum size of an xattr: " + xattrMaxSize + unlimited);
|
|
LOG.info("Maximum size of an xattr: " + xattrMaxSize + unlimited);
|
|
|
|
+
|
|
|
|
+ this.accessTimePrecision = conf.getLong(
|
|
|
|
+ DFS_NAMENODE_ACCESSTIME_PRECISION_KEY,
|
|
|
|
+ DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT);
|
|
|
|
+
|
|
|
|
+ this.storagePolicyEnabled =
|
|
|
|
+ conf.getBoolean(DFS_STORAGE_POLICY_ENABLED_KEY,
|
|
|
|
+ DFS_STORAGE_POLICY_ENABLED_DEFAULT);
|
|
|
|
+
|
|
int configuredLimit = conf.getInt(
|
|
int configuredLimit = conf.getInt(
|
|
DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
|
|
DFSConfigKeys.DFS_LIST_LIMIT, DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT);
|
|
this.lsLimit = configuredLimit>0 ?
|
|
this.lsLimit = configuredLimit>0 ?
|
|
@@ -287,6 +301,13 @@ public class FSDirectory implements Closeable {
|
|
return xattrsEnabled;
|
|
return xattrsEnabled;
|
|
}
|
|
}
|
|
int getXattrMaxSize() { return xattrMaxSize; }
|
|
int getXattrMaxSize() { return xattrMaxSize; }
|
|
|
|
+ boolean isStoragePolicyEnabled() {
|
|
|
|
+ return storagePolicyEnabled;
|
|
|
|
+ }
|
|
|
|
+ boolean isAccessTimeSupported() {
|
|
|
|
+ return accessTimePrecision > 0;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
|
|
int getLsLimit() {
|
|
int getLsLimit() {
|
|
return lsLimit;
|
|
return lsLimit;
|
|
@@ -523,172 +544,6 @@ public class FSDirectory implements Closeable {
|
|
return resolvePath(path, pathComponents, this);
|
|
return resolvePath(path, pathComponents, this);
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Set file replication
|
|
|
|
- *
|
|
|
|
- * @param src file name
|
|
|
|
- * @param replication new replication
|
|
|
|
- * @param blockRepls block replications - output parameter
|
|
|
|
- * @return array of file blocks
|
|
|
|
- * @throws QuotaExceededException
|
|
|
|
- * @throws SnapshotAccessControlException
|
|
|
|
- */
|
|
|
|
- Block[] setReplication(String src, short replication, short[] blockRepls)
|
|
|
|
- throws QuotaExceededException, UnresolvedLinkException,
|
|
|
|
- SnapshotAccessControlException {
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- return unprotectedSetReplication(src, replication, blockRepls);
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- Block[] unprotectedSetReplication(String src, short replication,
|
|
|
|
- short[] blockRepls) throws QuotaExceededException,
|
|
|
|
- UnresolvedLinkException, SnapshotAccessControlException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
-
|
|
|
|
- final INodesInPath iip = getINodesInPath4Write(src, true);
|
|
|
|
- final INode inode = iip.getLastINode();
|
|
|
|
- if (inode == null || !inode.isFile()) {
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
- INodeFile file = inode.asFile();
|
|
|
|
- final short oldBR = file.getBlockReplication();
|
|
|
|
-
|
|
|
|
- // before setFileReplication, check for increasing block replication.
|
|
|
|
- // if replication > oldBR, then newBR == replication.
|
|
|
|
- // if replication < oldBR, we don't know newBR yet.
|
|
|
|
- if (replication > oldBR) {
|
|
|
|
- long dsDelta = (replication - oldBR)*(file.diskspaceConsumed()/oldBR);
|
|
|
|
- updateCount(iip, 0, dsDelta, true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- file.setFileReplication(replication, iip.getLatestSnapshotId());
|
|
|
|
-
|
|
|
|
- final short newBR = file.getBlockReplication();
|
|
|
|
- // check newBR < oldBR case.
|
|
|
|
- if (newBR < oldBR) {
|
|
|
|
- long dsDelta = (newBR - oldBR)*(file.diskspaceConsumed()/newBR);
|
|
|
|
- updateCount(iip, 0, dsDelta, true);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (blockRepls != null) {
|
|
|
|
- blockRepls[0] = oldBR;
|
|
|
|
- blockRepls[1] = newBR;
|
|
|
|
- }
|
|
|
|
- return file.getBlocks();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /** Set block storage policy for a directory */
|
|
|
|
- void setStoragePolicy(INodesInPath iip, byte policyId)
|
|
|
|
- throws IOException {
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- unprotectedSetStoragePolicy(iip, policyId);
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void unprotectedSetStoragePolicy(INodesInPath iip, byte policyId)
|
|
|
|
- throws IOException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
- final INode inode = iip.getLastINode();
|
|
|
|
- if (inode == null) {
|
|
|
|
- throw new FileNotFoundException("File/Directory does not exist: "
|
|
|
|
- + iip.getPath());
|
|
|
|
- }
|
|
|
|
- final int snapshotId = iip.getLatestSnapshotId();
|
|
|
|
- if (inode.isFile()) {
|
|
|
|
- BlockStoragePolicy newPolicy = getBlockManager().getStoragePolicy(policyId);
|
|
|
|
- if (newPolicy.isCopyOnCreateFile()) {
|
|
|
|
- throw new HadoopIllegalArgumentException(
|
|
|
|
- "Policy " + newPolicy + " cannot be set after file creation.");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- BlockStoragePolicy currentPolicy =
|
|
|
|
- getBlockManager().getStoragePolicy(inode.getLocalStoragePolicyID());
|
|
|
|
-
|
|
|
|
- if (currentPolicy != null && currentPolicy.isCopyOnCreateFile()) {
|
|
|
|
- throw new HadoopIllegalArgumentException(
|
|
|
|
- "Existing policy " + currentPolicy.getName() +
|
|
|
|
- " cannot be changed after file creation.");
|
|
|
|
- }
|
|
|
|
- inode.asFile().setStoragePolicyID(policyId, snapshotId);
|
|
|
|
- } else if (inode.isDirectory()) {
|
|
|
|
- setDirStoragePolicy(inode.asDirectory(), policyId, snapshotId);
|
|
|
|
- } else {
|
|
|
|
- throw new FileNotFoundException(iip.getPath()
|
|
|
|
- + " is not a file or directory");
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void setDirStoragePolicy(INodeDirectory inode, byte policyId,
|
|
|
|
- int latestSnapshotId) throws IOException {
|
|
|
|
- List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
|
|
|
|
- XAttr xAttr = BlockStoragePolicySuite.buildXAttr(policyId);
|
|
|
|
- List<XAttr> newXAttrs = FSDirXAttrOp.setINodeXAttrs(this, existingXAttrs,
|
|
|
|
- Arrays.asList(xAttr),
|
|
|
|
- EnumSet.of(
|
|
|
|
- XAttrSetFlag.CREATE,
|
|
|
|
- XAttrSetFlag.REPLACE));
|
|
|
|
- XAttrStorage.updateINodeXAttrs(inode, newXAttrs, latestSnapshotId);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void setPermission(String src, FsPermission permission)
|
|
|
|
- throws FileNotFoundException, UnresolvedLinkException,
|
|
|
|
- QuotaExceededException, SnapshotAccessControlException {
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- unprotectedSetPermission(src, permission);
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void unprotectedSetPermission(String src, FsPermission permissions)
|
|
|
|
- throws FileNotFoundException, UnresolvedLinkException,
|
|
|
|
- QuotaExceededException, SnapshotAccessControlException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
- final INodesInPath inodesInPath = getINodesInPath4Write(src, true);
|
|
|
|
- final INode inode = inodesInPath.getLastINode();
|
|
|
|
- if (inode == null) {
|
|
|
|
- throw new FileNotFoundException("File does not exist: " + src);
|
|
|
|
- }
|
|
|
|
- int snapshotId = inodesInPath.getLatestSnapshotId();
|
|
|
|
- inode.setPermission(permissions, snapshotId);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void setOwner(String src, String username, String groupname)
|
|
|
|
- throws FileNotFoundException, UnresolvedLinkException,
|
|
|
|
- QuotaExceededException, SnapshotAccessControlException {
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- unprotectedSetOwner(src, username, groupname);
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- void unprotectedSetOwner(String src, String username, String groupname)
|
|
|
|
- throws FileNotFoundException, UnresolvedLinkException,
|
|
|
|
- QuotaExceededException, SnapshotAccessControlException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
- final INodesInPath inodesInPath = getINodesInPath4Write(src, true);
|
|
|
|
- INode inode = inodesInPath.getLastINode();
|
|
|
|
- if (inode == null) {
|
|
|
|
- throw new FileNotFoundException("File does not exist: " + src);
|
|
|
|
- }
|
|
|
|
- if (username != null) {
|
|
|
|
- inode = inode.setUser(username, inodesInPath.getLatestSnapshotId());
|
|
|
|
- }
|
|
|
|
- if (groupname != null) {
|
|
|
|
- inode.setGroup(groupname, inodesInPath.getLatestSnapshotId());
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Delete the target directory and collect the blocks under it
|
|
* Delete the target directory and collect the blocks under it
|
|
*
|
|
*
|
|
@@ -841,11 +696,6 @@ public class FSDirectory implements Closeable {
|
|
return removed;
|
|
return removed;
|
|
}
|
|
}
|
|
|
|
|
|
- byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
|
|
|
|
- return inodePolicy != BlockStoragePolicySuite.ID_UNSPECIFIED ? inodePolicy :
|
|
|
|
- parentPolicy;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Check whether the filepath could be created
|
|
* Check whether the filepath could be created
|
|
* @throws SnapshotAccessControlException if path is in RO snapshot
|
|
* @throws SnapshotAccessControlException if path is in RO snapshot
|
|
@@ -895,7 +745,7 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private void updateCount(INodesInPath iip, long nsDelta, long dsDelta,
|
|
|
|
|
|
+ void updateCount(INodesInPath iip, long nsDelta, long dsDelta,
|
|
boolean checkQuota) throws QuotaExceededException {
|
|
boolean checkQuota) throws QuotaExceededException {
|
|
updateCount(iip, iip.length() - 1, nsDelta, dsDelta, checkQuota);
|
|
updateCount(iip, iip.length() - 1, nsDelta, dsDelta, checkQuota);
|
|
}
|
|
}
|
|
@@ -1315,77 +1165,7 @@ public class FSDirectory implements Closeable {
|
|
int getInodeMapSize() {
|
|
int getInodeMapSize() {
|
|
return inodeMap.size();
|
|
return inodeMap.size();
|
|
}
|
|
}
|
|
-
|
|
|
|
- /**
|
|
|
|
- * See {@link ClientProtocol#setQuota(String, long, long)} for the contract.
|
|
|
|
- * Sets quota for for a directory.
|
|
|
|
- * @return INodeDirectory if any of the quotas have changed. null otherwise.
|
|
|
|
- * @throws FileNotFoundException if the path does not exist.
|
|
|
|
- * @throws PathIsNotDirectoryException if the path is not a directory.
|
|
|
|
- * @throws QuotaExceededException if the directory tree size is
|
|
|
|
- * greater than the given quota
|
|
|
|
- * @throws UnresolvedLinkException if a symlink is encountered in src.
|
|
|
|
- * @throws SnapshotAccessControlException if path is in RO snapshot
|
|
|
|
- */
|
|
|
|
- INodeDirectory unprotectedSetQuota(String src, long nsQuota, long dsQuota)
|
|
|
|
- throws FileNotFoundException, PathIsNotDirectoryException,
|
|
|
|
- QuotaExceededException, UnresolvedLinkException,
|
|
|
|
- SnapshotAccessControlException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
- // sanity check
|
|
|
|
- if ((nsQuota < 0 && nsQuota != HdfsConstants.QUOTA_DONT_SET &&
|
|
|
|
- nsQuota != HdfsConstants.QUOTA_RESET) ||
|
|
|
|
- (dsQuota < 0 && dsQuota != HdfsConstants.QUOTA_DONT_SET &&
|
|
|
|
- dsQuota != HdfsConstants.QUOTA_RESET)) {
|
|
|
|
- throw new IllegalArgumentException("Illegal value for nsQuota or " +
|
|
|
|
- "dsQuota : " + nsQuota + " and " +
|
|
|
|
- dsQuota);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- String srcs = normalizePath(src);
|
|
|
|
- final INodesInPath iip = getINodesInPath4Write(srcs, true);
|
|
|
|
- INodeDirectory dirNode = INodeDirectory.valueOf(iip.getLastINode(), srcs);
|
|
|
|
- if (dirNode.isRoot() && nsQuota == HdfsConstants.QUOTA_RESET) {
|
|
|
|
- throw new IllegalArgumentException("Cannot clear namespace quota on root.");
|
|
|
|
- } else { // a directory inode
|
|
|
|
- final Quota.Counts oldQuota = dirNode.getQuotaCounts();
|
|
|
|
- final long oldNsQuota = oldQuota.get(Quota.NAMESPACE);
|
|
|
|
- final long oldDsQuota = oldQuota.get(Quota.DISKSPACE);
|
|
|
|
- if (nsQuota == HdfsConstants.QUOTA_DONT_SET) {
|
|
|
|
- nsQuota = oldNsQuota;
|
|
|
|
- }
|
|
|
|
- if (dsQuota == HdfsConstants.QUOTA_DONT_SET) {
|
|
|
|
- dsQuota = oldDsQuota;
|
|
|
|
- }
|
|
|
|
- if (oldNsQuota == nsQuota && oldDsQuota == dsQuota) {
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
|
|
|
|
- final int latest = iip.getLatestSnapshotId();
|
|
|
|
- dirNode.recordModification(latest);
|
|
|
|
- dirNode.setQuota(nsQuota, dsQuota);
|
|
|
|
- return dirNode;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * See {@link ClientProtocol#setQuota(String, long, long)} for the contract.
|
|
|
|
- * @return INodeDirectory if any of the quotas have changed. null otherwise.
|
|
|
|
- * @throws SnapshotAccessControlException if path is in RO snapshot
|
|
|
|
- * @see #unprotectedSetQuota(String, long, long)
|
|
|
|
- */
|
|
|
|
- INodeDirectory setQuota(String src, long nsQuota, long dsQuota)
|
|
|
|
- throws FileNotFoundException, PathIsNotDirectoryException,
|
|
|
|
- QuotaExceededException, UnresolvedLinkException,
|
|
|
|
- SnapshotAccessControlException {
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- return unprotectedSetQuota(src, nsQuota, dsQuota);
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
long totalInodes() {
|
|
long totalInodes() {
|
|
readLock();
|
|
readLock();
|
|
try {
|
|
try {
|
|
@@ -1396,50 +1176,6 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Sets the access time on the file/directory. Logs it in the transaction log.
|
|
|
|
- */
|
|
|
|
- boolean setTimes(INode inode, long mtime, long atime, boolean force,
|
|
|
|
- int latestSnapshotId) throws QuotaExceededException {
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- return unprotectedSetTimes(inode, mtime, atime, force, latestSnapshotId);
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- boolean unprotectedSetTimes(String src, long mtime, long atime, boolean force)
|
|
|
|
- throws UnresolvedLinkException, QuotaExceededException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
- final INodesInPath i = getINodesInPath(src, true);
|
|
|
|
- return unprotectedSetTimes(i.getLastINode(), mtime, atime, force,
|
|
|
|
- i.getLatestSnapshotId());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private boolean unprotectedSetTimes(INode inode, long mtime,
|
|
|
|
- long atime, boolean force, int latest) throws QuotaExceededException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
- boolean status = false;
|
|
|
|
- if (mtime != -1) {
|
|
|
|
- inode = inode.setModificationTime(mtime, latest);
|
|
|
|
- status = true;
|
|
|
|
- }
|
|
|
|
- if (atime != -1) {
|
|
|
|
- long inodeTime = inode.getAccessTime();
|
|
|
|
-
|
|
|
|
- // if the last access time update was within the last precision interval, then
|
|
|
|
- // no need to store access time
|
|
|
|
- if (atime <= inodeTime + getFSNamesystem().getAccessTimePrecision() && !force) {
|
|
|
|
- status = false;
|
|
|
|
- } else {
|
|
|
|
- inode.setAccessTime(atime, latest);
|
|
|
|
- status = true;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- return status;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Reset the entire namespace tree.
|
|
* Reset the entire namespace tree.
|
|
*/
|
|
*/
|