|
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
|
|
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_ENCRYPTION_ZONE;
|
|
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
|
|
import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
|
|
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
|
|
|
|
import static org.apache.hadoop.util.Time.now;
|
|
import static org.apache.hadoop.util.Time.now;
|
|
|
|
|
|
import java.io.Closeable;
|
|
import java.io.Closeable;
|
|
@@ -30,7 +29,6 @@ import java.util.ArrayList;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.EnumSet;
|
|
import java.util.EnumSet;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
-import java.util.ListIterator;
|
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import com.google.protobuf.InvalidProtocolBufferException;
|
|
import com.google.protobuf.InvalidProtocolBufferException;
|
|
@@ -122,10 +120,6 @@ public class FSDirectory implements Closeable {
|
|
public final static String DOT_INODES_STRING = ".inodes";
|
|
public final static String DOT_INODES_STRING = ".inodes";
|
|
public final static byte[] DOT_INODES =
|
|
public final static byte[] DOT_INODES =
|
|
DFSUtil.string2Bytes(DOT_INODES_STRING);
|
|
DFSUtil.string2Bytes(DOT_INODES_STRING);
|
|
- private final XAttr KEYID_XATTR =
|
|
|
|
- XAttrHelper.buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, null);
|
|
|
|
- private final XAttr UNREADABLE_BY_SUPERUSER_XATTR =
|
|
|
|
- XAttrHelper.buildXAttr(SECURITY_XATTR_UNREADABLE_BY_SUPERUSER, null);
|
|
|
|
|
|
|
|
INodeDirectory rootDir;
|
|
INodeDirectory rootDir;
|
|
private final FSNamesystem namesystem;
|
|
private final FSNamesystem namesystem;
|
|
@@ -136,6 +130,7 @@ public class FSDirectory implements Closeable {
|
|
private final int contentCountLimit; // max content summary counts per run
|
|
private final int contentCountLimit; // max content summary counts per run
|
|
private final INodeMap inodeMap; // Synchronized by dirLock
|
|
private final INodeMap inodeMap; // Synchronized by dirLock
|
|
private long yieldCount = 0; // keep track of lock yield count.
|
|
private long yieldCount = 0; // keep track of lock yield count.
|
|
|
|
+
|
|
private final int inodeXAttrsLimit; //inode xattrs max limit
|
|
private final int inodeXAttrsLimit; //inode xattrs max limit
|
|
|
|
|
|
// lock to protect the directory and BlockMap
|
|
// lock to protect the directory and BlockMap
|
|
@@ -148,6 +143,8 @@ public class FSDirectory implements Closeable {
|
|
* ACL-related operations.
|
|
* ACL-related operations.
|
|
*/
|
|
*/
|
|
private final boolean aclsEnabled;
|
|
private final boolean aclsEnabled;
|
|
|
|
+ private final boolean xattrsEnabled;
|
|
|
|
+ private final int xattrMaxSize;
|
|
private final String fsOwnerShortUserName;
|
|
private final String fsOwnerShortUserName;
|
|
private final String supergroup;
|
|
private final String supergroup;
|
|
private final INodeId inodeId;
|
|
private final INodeId inodeId;
|
|
@@ -213,6 +210,18 @@ public class FSDirectory implements Closeable {
|
|
DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,
|
|
DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,
|
|
DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
|
|
DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_DEFAULT);
|
|
LOG.info("ACLs enabled? " + aclsEnabled);
|
|
LOG.info("ACLs enabled? " + aclsEnabled);
|
|
|
|
+ this.xattrsEnabled = conf.getBoolean(
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY,
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_DEFAULT);
|
|
|
|
+ LOG.info("XAttrs enabled? " + xattrsEnabled);
|
|
|
|
+ this.xattrMaxSize = conf.getInt(
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY,
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_DEFAULT);
|
|
|
|
+ Preconditions.checkArgument(xattrMaxSize >= 0,
|
|
|
|
+ "Cannot set a negative value for the maximum size of an xattr (%s).",
|
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_MAX_XATTR_SIZE_KEY);
|
|
|
|
+ final String unlimited = xattrMaxSize == 0 ? " (unlimited)" : "";
|
|
|
|
+ LOG.info("Maximum size of an xattr: " + xattrMaxSize + unlimited);
|
|
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 ?
|
|
@@ -274,6 +283,10 @@ public class FSDirectory implements Closeable {
|
|
boolean isAclsEnabled() {
|
|
boolean isAclsEnabled() {
|
|
return aclsEnabled;
|
|
return aclsEnabled;
|
|
}
|
|
}
|
|
|
|
+ boolean isXattrsEnabled() {
|
|
|
|
+ return xattrsEnabled;
|
|
|
|
+ }
|
|
|
|
+ int getXattrMaxSize() { return xattrMaxSize; }
|
|
|
|
|
|
int getLsLimit() {
|
|
int getLsLimit() {
|
|
return lsLimit;
|
|
return lsLimit;
|
|
@@ -283,6 +296,10 @@ public class FSDirectory implements Closeable {
|
|
return contentCountLimit;
|
|
return contentCountLimit;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ int getInodeXAttrsLimit() {
|
|
|
|
+ return inodeXAttrsLimit;
|
|
|
|
+ }
|
|
|
|
+
|
|
FSEditLog getEditLog() {
|
|
FSEditLog getEditLog() {
|
|
return editLog;
|
|
return editLog;
|
|
}
|
|
}
|
|
@@ -613,8 +630,11 @@ public class FSDirectory implements Closeable {
|
|
int latestSnapshotId) throws IOException {
|
|
int latestSnapshotId) throws IOException {
|
|
List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
|
|
List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
|
|
XAttr xAttr = BlockStoragePolicySuite.buildXAttr(policyId);
|
|
XAttr xAttr = BlockStoragePolicySuite.buildXAttr(policyId);
|
|
- List<XAttr> newXAttrs = setINodeXAttrs(existingXAttrs, Arrays.asList(xAttr),
|
|
|
|
- EnumSet.of(XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE));
|
|
|
|
|
|
+ List<XAttr> newXAttrs = FSDirXAttrOp.setINodeXAttrs(this, existingXAttrs,
|
|
|
|
+ Arrays.asList(xAttr),
|
|
|
|
+ EnumSet.of(
|
|
|
|
+ XAttrSetFlag.CREATE,
|
|
|
|
+ XAttrSetFlag.REPLACE));
|
|
XAttrStorage.updateINodeXAttrs(inode, newXAttrs, latestSnapshotId);
|
|
XAttrStorage.updateINodeXAttrs(inode, newXAttrs, latestSnapshotId);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1560,90 +1580,6 @@ public class FSDirectory implements Closeable {
|
|
return addINode(path, symlink) ? symlink : null;
|
|
return addINode(path, symlink) ? symlink : null;
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Removes a list of XAttrs from an inode at a path.
|
|
|
|
- *
|
|
|
|
- * @param src path of inode
|
|
|
|
- * @param toRemove XAttrs to be removed
|
|
|
|
- * @return List of XAttrs that were removed
|
|
|
|
- * @throws IOException if the inode does not exist, if quota is exceeded
|
|
|
|
- */
|
|
|
|
- List<XAttr> removeXAttrs(final String src, final List<XAttr> toRemove)
|
|
|
|
- throws IOException {
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- return unprotectedRemoveXAttrs(src, toRemove);
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- List<XAttr> unprotectedRemoveXAttrs(final String src,
|
|
|
|
- final List<XAttr> toRemove) throws IOException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
- INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
|
|
|
|
- INode inode = resolveLastINode(src, iip);
|
|
|
|
- int snapshotId = iip.getLatestSnapshotId();
|
|
|
|
- List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
|
|
|
|
- List<XAttr> removedXAttrs = Lists.newArrayListWithCapacity(toRemove.size());
|
|
|
|
- List<XAttr> newXAttrs = filterINodeXAttrs(existingXAttrs, toRemove,
|
|
|
|
- removedXAttrs);
|
|
|
|
- if (existingXAttrs.size() != newXAttrs.size()) {
|
|
|
|
- XAttrStorage.updateINodeXAttrs(inode, newXAttrs, snapshotId);
|
|
|
|
- return removedXAttrs;
|
|
|
|
- }
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Filter XAttrs from a list of existing XAttrs. Removes matched XAttrs from
|
|
|
|
- * toFilter and puts them into filtered. Upon completion,
|
|
|
|
- * toFilter contains the filter XAttrs that were not found, while
|
|
|
|
- * fitleredXAttrs contains the XAttrs that were found.
|
|
|
|
- *
|
|
|
|
- * @param existingXAttrs Existing XAttrs to be filtered
|
|
|
|
- * @param toFilter XAttrs to filter from the existing XAttrs
|
|
|
|
- * @param filtered Return parameter, XAttrs that were filtered
|
|
|
|
- * @return List of XAttrs that does not contain filtered XAttrs
|
|
|
|
- */
|
|
|
|
- @VisibleForTesting
|
|
|
|
- List<XAttr> filterINodeXAttrs(final List<XAttr> existingXAttrs,
|
|
|
|
- final List<XAttr> toFilter, final List<XAttr> filtered)
|
|
|
|
- throws AccessControlException {
|
|
|
|
- if (existingXAttrs == null || existingXAttrs.isEmpty() ||
|
|
|
|
- toFilter == null || toFilter.isEmpty()) {
|
|
|
|
- return existingXAttrs;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Populate a new list with XAttrs that pass the filter
|
|
|
|
- List<XAttr> newXAttrs =
|
|
|
|
- Lists.newArrayListWithCapacity(existingXAttrs.size());
|
|
|
|
- for (XAttr a : existingXAttrs) {
|
|
|
|
- boolean add = true;
|
|
|
|
- for (ListIterator<XAttr> it = toFilter.listIterator(); it.hasNext()
|
|
|
|
- ;) {
|
|
|
|
- XAttr filter = it.next();
|
|
|
|
- Preconditions.checkArgument(!KEYID_XATTR.equalsIgnoreValue(filter),
|
|
|
|
- "The encryption zone xattr should never be deleted.");
|
|
|
|
- if (UNREADABLE_BY_SUPERUSER_XATTR.equalsIgnoreValue(filter)) {
|
|
|
|
- throw new AccessControlException("The xattr '" +
|
|
|
|
- SECURITY_XATTR_UNREADABLE_BY_SUPERUSER + "' can not be deleted.");
|
|
|
|
- }
|
|
|
|
- if (a.equalsIgnoreValue(filter)) {
|
|
|
|
- add = false;
|
|
|
|
- it.remove();
|
|
|
|
- filtered.add(filter);
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- if (add) {
|
|
|
|
- newXAttrs.add(a);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- return newXAttrs;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
boolean isInAnEZ(INodesInPath iip)
|
|
boolean isInAnEZ(INodesInPath iip)
|
|
throws UnresolvedLinkException, SnapshotAccessControlException {
|
|
throws UnresolvedLinkException, SnapshotAccessControlException {
|
|
readLock();
|
|
readLock();
|
|
@@ -1709,7 +1645,8 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
writeLock();
|
|
writeLock();
|
|
try {
|
|
try {
|
|
- unprotectedSetXAttrs(src, xAttrs, EnumSet.of(XAttrSetFlag.CREATE));
|
|
|
|
|
|
+ FSDirXAttrOp.unprotectedSetXAttrs(this, src, xAttrs,
|
|
|
|
+ EnumSet.of(XAttrSetFlag.CREATE));
|
|
} finally {
|
|
} finally {
|
|
writeUnlock();
|
|
writeUnlock();
|
|
}
|
|
}
|
|
@@ -1752,8 +1689,9 @@ public class FSDirectory implements Closeable {
|
|
final CipherSuite suite = encryptionZone.getSuite();
|
|
final CipherSuite suite = encryptionZone.getSuite();
|
|
final String keyName = encryptionZone.getKeyName();
|
|
final String keyName = encryptionZone.getKeyName();
|
|
|
|
|
|
- XAttr fileXAttr = unprotectedGetXAttrByName(inode, snapshotId,
|
|
|
|
- CRYPTO_XATTR_FILE_ENCRYPTION_INFO);
|
|
|
|
|
|
+ XAttr fileXAttr = FSDirXAttrOp.unprotectedGetXAttrByName(inode,
|
|
|
|
+ snapshotId,
|
|
|
|
+ CRYPTO_XATTR_FILE_ENCRYPTION_INFO);
|
|
|
|
|
|
if (fileXAttr == null) {
|
|
if (fileXAttr == null) {
|
|
NameNode.LOG.warn("Could not find encryption XAttr for file " +
|
|
NameNode.LOG.warn("Could not find encryption XAttr for file " +
|
|
@@ -1775,173 +1713,6 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- void setXAttrs(final String src, final List<XAttr> xAttrs,
|
|
|
|
- final EnumSet<XAttrSetFlag> flag) throws IOException {
|
|
|
|
- writeLock();
|
|
|
|
- try {
|
|
|
|
- unprotectedSetXAttrs(src, xAttrs, flag);
|
|
|
|
- } finally {
|
|
|
|
- writeUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- INode unprotectedSetXAttrs(final String src, final List<XAttr> xAttrs,
|
|
|
|
- final EnumSet<XAttrSetFlag> flag)
|
|
|
|
- throws QuotaExceededException, IOException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
- INodesInPath iip = getINodesInPath4Write(normalizePath(src), true);
|
|
|
|
- INode inode = resolveLastINode(src, iip);
|
|
|
|
- int snapshotId = iip.getLatestSnapshotId();
|
|
|
|
- List<XAttr> existingXAttrs = XAttrStorage.readINodeXAttrs(inode);
|
|
|
|
- List<XAttr> newXAttrs = setINodeXAttrs(existingXAttrs, xAttrs, flag);
|
|
|
|
- final boolean isFile = inode.isFile();
|
|
|
|
-
|
|
|
|
- for (XAttr xattr : newXAttrs) {
|
|
|
|
- final String xaName = XAttrHelper.getPrefixName(xattr);
|
|
|
|
-
|
|
|
|
- /*
|
|
|
|
- * If we're adding the encryption zone xattr, then add src to the list
|
|
|
|
- * of encryption zones.
|
|
|
|
- */
|
|
|
|
- if (CRYPTO_XATTR_ENCRYPTION_ZONE.equals(xaName)) {
|
|
|
|
- final HdfsProtos.ZoneEncryptionInfoProto ezProto =
|
|
|
|
- HdfsProtos.ZoneEncryptionInfoProto.parseFrom(xattr.getValue());
|
|
|
|
- ezManager.addEncryptionZone(inode.getId(),
|
|
|
|
- PBHelper.convert(ezProto.getSuite()),
|
|
|
|
- PBHelper.convert(ezProto.getCryptoProtocolVersion()),
|
|
|
|
- ezProto.getKeyName());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (!isFile && SECURITY_XATTR_UNREADABLE_BY_SUPERUSER.equals(xaName)) {
|
|
|
|
- throw new IOException("Can only set '" +
|
|
|
|
- SECURITY_XATTR_UNREADABLE_BY_SUPERUSER + "' on a file.");
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- XAttrStorage.updateINodeXAttrs(inode, newXAttrs, snapshotId);
|
|
|
|
- return inode;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- List<XAttr> setINodeXAttrs(final List<XAttr> existingXAttrs,
|
|
|
|
- final List<XAttr> toSet, final EnumSet<XAttrSetFlag> flag)
|
|
|
|
- throws IOException {
|
|
|
|
- // Check for duplicate XAttrs in toSet
|
|
|
|
- // We need to use a custom comparator, so using a HashSet is not suitable
|
|
|
|
- for (int i = 0; i < toSet.size(); i++) {
|
|
|
|
- for (int j = i + 1; j < toSet.size(); j++) {
|
|
|
|
- if (toSet.get(i).equalsIgnoreValue(toSet.get(j))) {
|
|
|
|
- throw new IOException("Cannot specify the same XAttr to be set " +
|
|
|
|
- "more than once");
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Count the current number of user-visible XAttrs for limit checking
|
|
|
|
- int userVisibleXAttrsNum = 0; // Number of user visible xAttrs
|
|
|
|
-
|
|
|
|
- // The XAttr list is copied to an exactly-sized array when it's stored,
|
|
|
|
- // so there's no need to size it precisely here.
|
|
|
|
- int newSize = (existingXAttrs != null) ? existingXAttrs.size() : 0;
|
|
|
|
- newSize += toSet.size();
|
|
|
|
- List<XAttr> xAttrs = Lists.newArrayListWithCapacity(newSize);
|
|
|
|
-
|
|
|
|
- // Check if the XAttr already exists to validate with the provided flag
|
|
|
|
- for (XAttr xAttr: toSet) {
|
|
|
|
- boolean exist = false;
|
|
|
|
- if (existingXAttrs != null) {
|
|
|
|
- for (XAttr a : existingXAttrs) {
|
|
|
|
- if (a.equalsIgnoreValue(xAttr)) {
|
|
|
|
- exist = true;
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- XAttrSetFlag.validate(xAttr.getName(), exist, flag);
|
|
|
|
- // add the new XAttr since it passed validation
|
|
|
|
- xAttrs.add(xAttr);
|
|
|
|
- if (isUserVisible(xAttr)) {
|
|
|
|
- userVisibleXAttrsNum++;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // Add the existing xattrs back in, if they weren't already set
|
|
|
|
- if (existingXAttrs != null) {
|
|
|
|
- for (XAttr existing : existingXAttrs) {
|
|
|
|
- boolean alreadySet = false;
|
|
|
|
- for (XAttr set : toSet) {
|
|
|
|
- if (set.equalsIgnoreValue(existing)) {
|
|
|
|
- alreadySet = true;
|
|
|
|
- break;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- if (!alreadySet) {
|
|
|
|
- xAttrs.add(existing);
|
|
|
|
- if (isUserVisible(existing)) {
|
|
|
|
- userVisibleXAttrsNum++;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if (userVisibleXAttrsNum > inodeXAttrsLimit) {
|
|
|
|
- throw new IOException("Cannot add additional XAttr to inode, "
|
|
|
|
- + "would exceed limit of " + inodeXAttrsLimit);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- return xAttrs;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private boolean isUserVisible(XAttr xAttr) {
|
|
|
|
- if (xAttr.getNameSpace() == XAttr.NameSpace.USER ||
|
|
|
|
- xAttr.getNameSpace() == XAttr.NameSpace.TRUSTED) {
|
|
|
|
- return true;
|
|
|
|
- }
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- List<XAttr> getXAttrs(String src) throws IOException {
|
|
|
|
- String srcs = normalizePath(src);
|
|
|
|
- readLock();
|
|
|
|
- try {
|
|
|
|
- INodesInPath iip = getLastINodeInPath(srcs, true);
|
|
|
|
- INode inode = resolveLastINode(src, iip);
|
|
|
|
- int snapshotId = iip.getPathSnapshotId();
|
|
|
|
- return unprotectedGetXAttrs(inode, snapshotId);
|
|
|
|
- } finally {
|
|
|
|
- readUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- List<XAttr> getXAttrs(INode inode, int snapshotId) throws IOException {
|
|
|
|
- readLock();
|
|
|
|
- try {
|
|
|
|
- return unprotectedGetXAttrs(inode, snapshotId);
|
|
|
|
- } finally {
|
|
|
|
- readUnlock();
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private List<XAttr> unprotectedGetXAttrs(INode inode, int snapshotId)
|
|
|
|
- throws IOException {
|
|
|
|
- return XAttrStorage.readINodeXAttrs(inode, snapshotId);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private XAttr unprotectedGetXAttrByName(INode inode, int snapshotId,
|
|
|
|
- String xAttrName)
|
|
|
|
- throws IOException {
|
|
|
|
- List<XAttr> xAttrs = XAttrStorage.readINodeXAttrs(inode, snapshotId);
|
|
|
|
- if (xAttrs == null) {
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
- for (XAttr x : xAttrs) {
|
|
|
|
- if (XAttrHelper.getPrefixName(x)
|
|
|
|
- .equals(xAttrName)) {
|
|
|
|
- return x;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
static INode resolveLastINode(String src, INodesInPath iip)
|
|
static INode resolveLastINode(String src, INodesInPath iip)
|
|
throws FileNotFoundException {
|
|
throws FileNotFoundException {
|
|
INode[] inodes = iip.getINodes();
|
|
INode[] inodes = iip.getINodes();
|