|
@@ -26,11 +26,11 @@ import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.List;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
-import java.util.concurrent.locks.Condition;
|
|
|
+import java.util.ListIterator;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
+import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.crypto.CryptoCodec;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
@@ -89,15 +89,14 @@ import static org.apache.hadoop.hdfs.protocol.HdfsConstants.CRYPTO_XATTR_KEY_ID;
|
|
|
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.CRYPTO_XATTR_IV;
|
|
|
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.CRYPTO_XATTR_KEY_VERSION_ID;
|
|
|
|
|
|
-/*************************************************
|
|
|
- * FSDirectory stores the filesystem directory state.
|
|
|
- * It handles writing/loading values to disk, and logging
|
|
|
- * changes as we go.
|
|
|
- *
|
|
|
- * It keeps the filename->blockset mapping always-current
|
|
|
- * and logged to disk.
|
|
|
- *
|
|
|
- *************************************************/
|
|
|
+/**
|
|
|
+ * Both FSDirectory and FSNamesystem manage the state of the namespace.
|
|
|
+ * FSDirectory is a pure in-memory data structure, all of whose operations
|
|
|
+ * happen entirely in memory. In contrast, FSNamesystem persists the operations
|
|
|
+ * to the disk.
|
|
|
+ * @see org.apache.hadoop.hdfs.server.namenode.FSNamesystem
|
|
|
+ **/
|
|
|
+@InterfaceAudience.Private
|
|
|
public class FSDirectory implements Closeable {
|
|
|
private static INodeDirectorySnapshottable createRoot(FSNamesystem namesystem) {
|
|
|
final INodeDirectory r = new INodeDirectory(
|
|
@@ -126,7 +125,6 @@ public class FSDirectory implements Closeable {
|
|
|
INodeDirectory rootDir;
|
|
|
FSImage fsImage;
|
|
|
private final FSNamesystem namesystem;
|
|
|
- private volatile boolean ready = false;
|
|
|
private volatile boolean skipQuotaCheck = false; //skip while consuming edits
|
|
|
private final int maxComponentLength;
|
|
|
private final int maxDirItems;
|
|
@@ -139,7 +137,6 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
// lock to protect the directory and BlockMap
|
|
|
private final ReentrantReadWriteLock dirLock;
|
|
|
- private final Condition cond;
|
|
|
|
|
|
// utility methods to acquire and release read lock and write lock
|
|
|
void readLock() {
|
|
@@ -182,7 +179,6 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
FSDirectory(FSImage fsImage, FSNamesystem ns, Configuration conf) {
|
|
|
this.dirLock = new ReentrantReadWriteLock(true); // fair
|
|
|
- this.cond = dirLock.writeLock().newCondition();
|
|
|
rootDir = createRoot(ns);
|
|
|
inodeMap = INodeMap.newInstance(rootDir);
|
|
|
this.fsImage = fsImage;
|
|
@@ -239,38 +235,6 @@ public class FSDirectory implements Closeable {
|
|
|
return rootDir;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Notify that loading of this FSDirectory is complete, and
|
|
|
- * it is ready for use
|
|
|
- */
|
|
|
- void imageLoadComplete() {
|
|
|
- Preconditions.checkState(!ready, "FSDirectory already loaded");
|
|
|
- setReady();
|
|
|
- }
|
|
|
-
|
|
|
- void setReady() {
|
|
|
- if(ready) return;
|
|
|
- writeLock();
|
|
|
- try {
|
|
|
- setReady(true);
|
|
|
- this.nameCache.initialized();
|
|
|
- cond.signalAll();
|
|
|
- } finally {
|
|
|
- writeUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- //This is for testing purposes only
|
|
|
- @VisibleForTesting
|
|
|
- boolean isReady() {
|
|
|
- return ready;
|
|
|
- }
|
|
|
-
|
|
|
- // exposed for unit tests
|
|
|
- protected void setReady(boolean flag) {
|
|
|
- ready = flag;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Shutdown the filestore
|
|
|
*/
|
|
@@ -279,22 +243,12 @@ public class FSDirectory implements Closeable {
|
|
|
fsImage.close();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Block until the object is ready to be used.
|
|
|
- */
|
|
|
- void waitForReady() {
|
|
|
- if (!ready) {
|
|
|
- writeLock();
|
|
|
- try {
|
|
|
- while (!ready) {
|
|
|
- try {
|
|
|
- cond.await(5000, TimeUnit.MILLISECONDS);
|
|
|
- } catch (InterruptedException ignored) {
|
|
|
- }
|
|
|
- }
|
|
|
- } finally {
|
|
|
- writeUnlock();
|
|
|
- }
|
|
|
+ void markNameCacheInitialized() {
|
|
|
+ writeLock();
|
|
|
+ try {
|
|
|
+ nameCache.initialized();
|
|
|
+ } finally {
|
|
|
+ writeUnlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -320,7 +274,6 @@ public class FSDirectory implements Closeable {
|
|
|
String clientMachine, DatanodeDescriptor clientNode)
|
|
|
throws FileAlreadyExistsException, QuotaExceededException,
|
|
|
UnresolvedLinkException, SnapshotAccessControlException, AclException {
|
|
|
- waitForReady();
|
|
|
|
|
|
long modTime = now();
|
|
|
INodeFile newNode = new INodeFile(namesystem.allocateNewInodeId(), null,
|
|
@@ -350,6 +303,7 @@ public class FSDirectory implements Closeable {
|
|
|
String path,
|
|
|
PermissionStatus permissions,
|
|
|
List<AclEntry> aclEntries,
|
|
|
+ List<XAttr> xAttrs,
|
|
|
short replication,
|
|
|
long modificationTime,
|
|
|
long atime,
|
|
@@ -376,6 +330,10 @@ public class FSDirectory implements Closeable {
|
|
|
AclStorage.updateINodeAcl(newNode, aclEntries,
|
|
|
Snapshot.CURRENT_STATE_ID);
|
|
|
}
|
|
|
+ if (xAttrs != null) {
|
|
|
+ XAttrStorage.updateINodeXAttrs(newNode, xAttrs,
|
|
|
+ Snapshot.CURRENT_STATE_ID);
|
|
|
+ }
|
|
|
return newNode;
|
|
|
}
|
|
|
} catch (IOException e) {
|
|
@@ -393,8 +351,6 @@ public class FSDirectory implements Closeable {
|
|
|
*/
|
|
|
BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block,
|
|
|
DatanodeStorageInfo[] targets) throws IOException {
|
|
|
- waitForReady();
|
|
|
-
|
|
|
writeLock();
|
|
|
try {
|
|
|
final INodeFile fileINode = inodesInPath.getLastINode().asFile();
|
|
@@ -432,8 +388,6 @@ public class FSDirectory implements Closeable {
|
|
|
boolean removeBlock(String path, INodeFile fileNode, Block block)
|
|
|
throws IOException {
|
|
|
Preconditions.checkArgument(fileNode.isUnderConstruction());
|
|
|
- waitForReady();
|
|
|
-
|
|
|
writeLock();
|
|
|
try {
|
|
|
return unprotectedRemoveBlock(path, fileNode, block);
|
|
@@ -477,7 +431,6 @@ public class FSDirectory implements Closeable {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: "
|
|
|
+src+" to "+dst);
|
|
|
}
|
|
|
- waitForReady();
|
|
|
writeLock();
|
|
|
try {
|
|
|
if (!unprotectedRenameTo(src, dst, mtime))
|
|
@@ -500,7 +453,6 @@ public class FSDirectory implements Closeable {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: " + src
|
|
|
+ " to " + dst);
|
|
|
}
|
|
|
- waitForReady();
|
|
|
writeLock();
|
|
|
try {
|
|
|
if (unprotectedRenameTo(src, dst, mtime, options)) {
|
|
@@ -1032,7 +984,6 @@ public class FSDirectory implements Closeable {
|
|
|
Block[] setReplication(String src, short replication, short[] blockRepls)
|
|
|
throws QuotaExceededException, UnresolvedLinkException,
|
|
|
SnapshotAccessControlException {
|
|
|
- waitForReady();
|
|
|
writeLock();
|
|
|
try {
|
|
|
return unprotectedSetReplication(src, replication, blockRepls);
|
|
@@ -1155,7 +1106,6 @@ public class FSDirectory implements Closeable {
|
|
|
writeLock();
|
|
|
try {
|
|
|
// actual move
|
|
|
- waitForReady();
|
|
|
unprotectedConcat(target, srcs, timestamp);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -1238,7 +1188,6 @@ public class FSDirectory implements Closeable {
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: " + src);
|
|
|
}
|
|
|
- waitForReady();
|
|
|
final long filesRemoved;
|
|
|
writeLock();
|
|
|
try {
|
|
@@ -1711,7 +1660,7 @@ public class FSDirectory implements Closeable {
|
|
|
long nsDelta, long dsDelta, boolean checkQuota)
|
|
|
throws QuotaExceededException {
|
|
|
assert hasWriteLock();
|
|
|
- if (!ready) {
|
|
|
+ if (!namesystem.isImageLoaded()) {
|
|
|
//still initializing. do not check or update quotas.
|
|
|
return;
|
|
|
}
|
|
@@ -1904,7 +1853,7 @@ public class FSDirectory implements Closeable {
|
|
|
*/
|
|
|
private void verifyQuotaForRename(INode[] src, INode[] dst)
|
|
|
throws QuotaExceededException {
|
|
|
- if (!ready || skipQuotaCheck) {
|
|
|
+ if (!namesystem.isImageLoaded() || skipQuotaCheck) {
|
|
|
// Do not check quota if edits log is still being processed
|
|
|
return;
|
|
|
}
|
|
@@ -1960,7 +1909,7 @@ public class FSDirectory implements Closeable {
|
|
|
void verifyINodeName(byte[] childName) throws HadoopIllegalArgumentException {
|
|
|
if (Arrays.equals(HdfsConstants.DOT_SNAPSHOT_DIR_BYTES, childName)) {
|
|
|
String s = "\"" + HdfsConstants.DOT_SNAPSHOT_DIR + "\" is a reserved name.";
|
|
|
- if (!ready) {
|
|
|
+ if (!namesystem.isImageLoaded()) {
|
|
|
s += " Please rename it before upgrade.";
|
|
|
}
|
|
|
throw new HadoopIllegalArgumentException(s);
|
|
@@ -1987,7 +1936,7 @@ public class FSDirectory implements Closeable {
|
|
|
getFullPathName((INode[])parentPath, pos - 1): (String)parentPath;
|
|
|
final PathComponentTooLongException e = new PathComponentTooLongException(
|
|
|
maxComponentLength, length, p, DFSUtil.bytes2String(childName));
|
|
|
- if (ready) {
|
|
|
+ if (namesystem.isImageLoaded()) {
|
|
|
throw e;
|
|
|
} else {
|
|
|
// Do not throw if edits log is still being processed
|
|
@@ -2011,7 +1960,7 @@ public class FSDirectory implements Closeable {
|
|
|
if (count >= maxDirItems) {
|
|
|
final MaxDirectoryItemsExceededException e
|
|
|
= new MaxDirectoryItemsExceededException(maxDirItems, count);
|
|
|
- if (ready) {
|
|
|
+ if (namesystem.isImageLoaded()) {
|
|
|
e.setPathName(getFullPathName(pathComponents, pos - 1));
|
|
|
throw e;
|
|
|
} else {
|
|
@@ -2347,7 +2296,6 @@ public class FSDirectory implements Closeable {
|
|
|
void reset() {
|
|
|
writeLock();
|
|
|
try {
|
|
|
- setReady(false);
|
|
|
rootDir = createRoot(getFSNamesystem());
|
|
|
inodeMap.clear();
|
|
|
addToInodeMap(rootDir);
|
|
@@ -2631,45 +2579,81 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- XAttr removeXAttr(String src, XAttr xAttr) throws IOException {
|
|
|
+ /**
|
|
|
+ * 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 unprotectedRemoveXAttr(src, xAttr);
|
|
|
+ return unprotectedRemoveXAttrs(src, toRemove);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- XAttr unprotectedRemoveXAttr(String src,
|
|
|
- XAttr xAttr) throws IOException {
|
|
|
+
|
|
|
+ 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> newXAttrs = filterINodeXAttr(existingXAttrs, xAttr);
|
|
|
+ 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 xAttr;
|
|
|
+ return removedXAttrs;
|
|
|
}
|
|
|
return null;
|
|
|
}
|
|
|
-
|
|
|
- List<XAttr> filterINodeXAttr(List<XAttr> existingXAttrs,
|
|
|
- XAttr xAttr) throws QuotaExceededException {
|
|
|
- if (existingXAttrs == null || existingXAttrs.isEmpty()) {
|
|
|
+
|
|
|
+ /**
|
|
|
+ * 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) {
|
|
|
+ if (existingXAttrs == null || existingXAttrs.isEmpty() ||
|
|
|
+ toFilter == null || toFilter.isEmpty()) {
|
|
|
return existingXAttrs;
|
|
|
}
|
|
|
-
|
|
|
- List<XAttr> xAttrs = Lists.newArrayListWithCapacity(existingXAttrs.size());
|
|
|
+
|
|
|
+ // Populate a new list with XAttrs that pass the filter
|
|
|
+ List<XAttr> newXAttrs =
|
|
|
+ Lists.newArrayListWithCapacity(existingXAttrs.size());
|
|
|
for (XAttr a : existingXAttrs) {
|
|
|
- if (!(a.getNameSpace() == xAttr.getNameSpace()
|
|
|
- && a.getName().equals(xAttr.getName()))) {
|
|
|
- xAttrs.add(a);
|
|
|
+ boolean add = true;
|
|
|
+ for (ListIterator<XAttr> it = toFilter.listIterator(); it.hasNext()
|
|
|
+ ;) {
|
|
|
+ XAttr filter = it.next();
|
|
|
+ if (a.equalsIgnoreValue(filter)) {
|
|
|
+ add = false;
|
|
|
+ it.remove();
|
|
|
+ filtered.add(filter);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (add) {
|
|
|
+ newXAttrs.add(a);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- return xAttrs;
|
|
|
+
|
|
|
+ return newXAttrs;
|
|
|
}
|
|
|
|
|
|
XAttr createEncryptionZone(String src, String keyId)
|
|
@@ -2682,14 +2666,16 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
final XAttr keyIdXAttr =
|
|
|
XAttrHelper.buildXAttr(CRYPTO_XATTR_KEY_ID, keyId.getBytes());
|
|
|
- unprotectedSetXAttr(src, keyIdXAttr, EnumSet.of(XAttrSetFlag.CREATE));
|
|
|
+ List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
|
|
|
+ xattrs.add(keyIdXAttr);
|
|
|
+ unprotectedSetXAttrs(src, xattrs, EnumSet.of(XAttrSetFlag.CREATE));
|
|
|
return keyIdXAttr;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- XAttr deleteEncryptionZone(String src)
|
|
|
+ List<XAttr> deleteEncryptionZone(String src)
|
|
|
throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
@@ -2699,71 +2685,107 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
final XAttr keyIdXAttr =
|
|
|
XAttrHelper.buildXAttr(CRYPTO_XATTR_KEY_ID, null);
|
|
|
- final XAttr removedXAttr = unprotectedRemoveXAttr(src, keyIdXAttr);
|
|
|
- if (removedXAttr == null) {
|
|
|
+ List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
|
|
|
+ xattrs.add(keyIdXAttr);
|
|
|
+ final List<XAttr> removedXAttrs = unprotectedRemoveXAttrs(src, xattrs);
|
|
|
+ if (removedXAttrs == null || removedXAttrs.isEmpty()) {
|
|
|
throw new IOException(
|
|
|
src + " does not appear to be the root of an encryption zone");
|
|
|
}
|
|
|
- return removedXAttr;
|
|
|
+ return removedXAttrs;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
|
|
|
- throws IOException {
|
|
|
+ void setXAttrs(final String src, final List<XAttr> xAttrs,
|
|
|
+ final EnumSet<XAttrSetFlag> flag) throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
- unprotectedSetXAttr(src, xAttr, flag);
|
|
|
+ unprotectedSetXAttrs(src, xAttrs, flag);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void unprotectedSetXAttr(String src, XAttr xAttr,
|
|
|
- EnumSet<XAttrSetFlag> flag) throws IOException {
|
|
|
+ void 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 = setINodeXAttr(existingXAttrs, xAttr, flag);
|
|
|
+ List<XAttr> newXAttrs = setINodeXAttrs(existingXAttrs, xAttrs, flag);
|
|
|
XAttrStorage.updateINodeXAttrs(inode, newXAttrs, snapshotId);
|
|
|
}
|
|
|
-
|
|
|
- List<XAttr> setINodeXAttr(List<XAttr> existingXAttrs, XAttr xAttr,
|
|
|
- EnumSet<XAttrSetFlag> flag) throws QuotaExceededException, IOException {
|
|
|
- List<XAttr> xAttrs = Lists.newArrayListWithCapacity(
|
|
|
- existingXAttrs != null ? existingXAttrs.size() + 1 : 1);
|
|
|
+
|
|
|
+ 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
|
|
|
- boolean exist = false;
|
|
|
+
|
|
|
+ // 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 a: existingXAttrs) {
|
|
|
- if ((a.getNameSpace() == xAttr.getNameSpace()
|
|
|
- && a.getName().equals(xAttr.getName()))) {
|
|
|
- exist = true;
|
|
|
- } else {
|
|
|
- xAttrs.add(a);
|
|
|
-
|
|
|
- if (isUserVisible(a)) {
|
|
|
+ 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++;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- XAttrSetFlag.validate(xAttr.getName(), exist, flag);
|
|
|
- xAttrs.add(xAttr);
|
|
|
-
|
|
|
- if (isUserVisible(xAttr)) {
|
|
|
- userVisibleXAttrsNum++;
|
|
|
- }
|
|
|
-
|
|
|
+
|
|
|
if (userVisibleXAttrsNum > inodeXAttrsLimit) {
|
|
|
throw new IOException("Cannot add additional XAttr to inode, "
|
|
|
+ "would exceed limit of " + inodeXAttrsLimit);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
return xAttrs;
|
|
|
}
|
|
|
|