|
@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.util.Diff;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
import com.google.common.primitives.SignedBytes;
|
|
|
//import org.apache.hadoop.hdfs.util.EnumCounters;
|
|
|
|
|
@@ -54,141 +53,47 @@ import com.google.common.primitives.SignedBytes;
|
|
|
public abstract class INode implements Diff.Element<byte[]> {
|
|
|
public static final Log LOG = LogFactory.getLog(INode.class);
|
|
|
|
|
|
- private static enum PermissionStatusFormat {
|
|
|
- MODE(0, 16),
|
|
|
- GROUP(MODE.OFFSET + MODE.LENGTH, 25),
|
|
|
- USER(GROUP.OFFSET + GROUP.LENGTH, 23);
|
|
|
+ /** parent is either an {@link INodeDirectory} or an {@link INodeReference}.*/
|
|
|
+ private INode parent = null;
|
|
|
|
|
|
- final int OFFSET;
|
|
|
- final int LENGTH; //bit length
|
|
|
- final long MASK;
|
|
|
-
|
|
|
- PermissionStatusFormat(int offset, int length) {
|
|
|
- OFFSET = offset;
|
|
|
- LENGTH = length;
|
|
|
- MASK = ((-1L) >>> (64 - LENGTH)) << OFFSET;
|
|
|
- }
|
|
|
-
|
|
|
- long retrieve(long record) {
|
|
|
- return (record & MASK) >>> OFFSET;
|
|
|
- }
|
|
|
-
|
|
|
- long combine(long bits, long record) {
|
|
|
- return (record & ~MASK) | (bits << OFFSET);
|
|
|
- }
|
|
|
-
|
|
|
- /** Encode the {@link PermissionStatus} to a long. */
|
|
|
- static long toLong(PermissionStatus ps) {
|
|
|
- long permission = 0L;
|
|
|
- final int user = SerialNumberManager.INSTANCE.getUserSerialNumber(
|
|
|
- ps.getUserName());
|
|
|
- permission = USER.combine(user, permission);
|
|
|
- final int group = SerialNumberManager.INSTANCE.getGroupSerialNumber(
|
|
|
- ps.getGroupName());
|
|
|
- permission = GROUP.combine(group, permission);
|
|
|
- final int mode = ps.getPermission().toShort();
|
|
|
- permission = MODE.combine(mode, permission);
|
|
|
- return permission;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * The inode id
|
|
|
- */
|
|
|
- final private long id;
|
|
|
-
|
|
|
- /**
|
|
|
- * The inode name is in java UTF8 encoding;
|
|
|
- * The name in HdfsFileStatus should keep the same encoding as this.
|
|
|
- * if this encoding is changed, implicitly getFileInfo and listStatus in
|
|
|
- * clientProtocol are changed; The decoding at the client
|
|
|
- * side should change accordingly.
|
|
|
- */
|
|
|
- private byte[] name = null;
|
|
|
- /**
|
|
|
- * Permission encoded using {@link PermissionStatusFormat}.
|
|
|
- * Codes other than {@link #clonePermissionStatus(INode)}
|
|
|
- * and {@link #updatePermissionStatus(PermissionStatusFormat, long)}
|
|
|
- * should not modify it.
|
|
|
- */
|
|
|
- private long permission = 0L;
|
|
|
- private INodeDirectory parent = null;
|
|
|
- private long modificationTime = 0L;
|
|
|
- private long accessTime = 0L;
|
|
|
-
|
|
|
- private INode(long id, byte[] name, long permission, INodeDirectory parent,
|
|
|
- long modificationTime, long accessTime) {
|
|
|
- this.id = id;
|
|
|
- this.name = name;
|
|
|
- this.permission = permission;
|
|
|
+ INode(INode parent) {
|
|
|
this.parent = parent;
|
|
|
- this.modificationTime = modificationTime;
|
|
|
- this.accessTime = accessTime;
|
|
|
- }
|
|
|
-
|
|
|
- INode(long id, byte[] name, PermissionStatus permissions,
|
|
|
- long modificationTime, long accessTime) {
|
|
|
- this(id, name, PermissionStatusFormat.toLong(permissions), null,
|
|
|
- modificationTime, accessTime);
|
|
|
- }
|
|
|
-
|
|
|
- /** @param other Other node to be copied */
|
|
|
- INode(INode other) {
|
|
|
- this(other.id, other.name, other.permission, other.parent,
|
|
|
- other.modificationTime, other.accessTime);
|
|
|
}
|
|
|
|
|
|
/** Get inode id */
|
|
|
- public long getId() {
|
|
|
- return this.id;
|
|
|
- }
|
|
|
+ public abstract long getId();
|
|
|
|
|
|
/**
|
|
|
* Check whether this is the root inode.
|
|
|
*/
|
|
|
- boolean isRoot() {
|
|
|
- return name.length == 0;
|
|
|
+ final boolean isRoot() {
|
|
|
+ return getLocalNameBytes().length == 0;
|
|
|
}
|
|
|
|
|
|
- /** Clone the {@link PermissionStatus}. */
|
|
|
- void clonePermissionStatus(INode that) {
|
|
|
- this.permission = that.permission;
|
|
|
- }
|
|
|
/** Get the {@link PermissionStatus} */
|
|
|
- public final PermissionStatus getPermissionStatus(Snapshot snapshot) {
|
|
|
- return new PermissionStatus(getUserName(snapshot), getGroupName(snapshot),
|
|
|
- getFsPermission(snapshot));
|
|
|
- }
|
|
|
+ abstract PermissionStatus getPermissionStatus(Snapshot snapshot);
|
|
|
+
|
|
|
/** The same as getPermissionStatus(null). */
|
|
|
- public final PermissionStatus getPermissionStatus() {
|
|
|
+ final PermissionStatus getPermissionStatus() {
|
|
|
return getPermissionStatus(null);
|
|
|
}
|
|
|
- private void updatePermissionStatus(PermissionStatusFormat f, long n) {
|
|
|
- this.permission = f.combine(n, permission);
|
|
|
- }
|
|
|
+
|
|
|
/**
|
|
|
* @param snapshot
|
|
|
* if it is not null, get the result from the given snapshot;
|
|
|
* otherwise, get the result from the current inode.
|
|
|
* @return user name
|
|
|
*/
|
|
|
- public final String getUserName(Snapshot snapshot) {
|
|
|
- if (snapshot != null) {
|
|
|
- return getSnapshotINode(snapshot).getUserName();
|
|
|
- }
|
|
|
+ abstract String getUserName(Snapshot snapshot);
|
|
|
|
|
|
- int n = (int)PermissionStatusFormat.USER.retrieve(permission);
|
|
|
- return SerialNumberManager.INSTANCE.getUser(n);
|
|
|
- }
|
|
|
/** The same as getUserName(null). */
|
|
|
public final String getUserName() {
|
|
|
return getUserName(null);
|
|
|
}
|
|
|
+
|
|
|
/** Set user */
|
|
|
- final void setUser(String user) {
|
|
|
- int n = SerialNumberManager.INSTANCE.getUserSerialNumber(user);
|
|
|
- updatePermissionStatus(PermissionStatusFormat.USER, n);
|
|
|
- }
|
|
|
+ abstract void setUser(String user);
|
|
|
+
|
|
|
/** Set user */
|
|
|
final INode setUser(String user, Snapshot latest)
|
|
|
throws QuotaExceededException {
|
|
@@ -202,23 +107,16 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
* otherwise, get the result from the current inode.
|
|
|
* @return group name
|
|
|
*/
|
|
|
- public final String getGroupName(Snapshot snapshot) {
|
|
|
- if (snapshot != null) {
|
|
|
- return getSnapshotINode(snapshot).getGroupName();
|
|
|
- }
|
|
|
+ abstract String getGroupName(Snapshot snapshot);
|
|
|
|
|
|
- int n = (int)PermissionStatusFormat.GROUP.retrieve(permission);
|
|
|
- return SerialNumberManager.INSTANCE.getGroup(n);
|
|
|
- }
|
|
|
/** The same as getGroupName(null). */
|
|
|
public final String getGroupName() {
|
|
|
return getGroupName(null);
|
|
|
}
|
|
|
+
|
|
|
/** Set group */
|
|
|
- final void setGroup(String group) {
|
|
|
- int n = SerialNumberManager.INSTANCE.getGroupSerialNumber(group);
|
|
|
- updatePermissionStatus(PermissionStatusFormat.GROUP, n);
|
|
|
- }
|
|
|
+ abstract void setGroup(String group);
|
|
|
+
|
|
|
/** Set group */
|
|
|
final INode setGroup(String group, Snapshot latest)
|
|
|
throws QuotaExceededException {
|
|
@@ -226,32 +124,23 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
nodeToUpdate.setGroup(group);
|
|
|
return nodeToUpdate;
|
|
|
}
|
|
|
+
|
|
|
/**
|
|
|
* @param snapshot
|
|
|
* if it is not null, get the result from the given snapshot;
|
|
|
* otherwise, get the result from the current inode.
|
|
|
* @return permission.
|
|
|
*/
|
|
|
- public final FsPermission getFsPermission(Snapshot snapshot) {
|
|
|
- if (snapshot != null) {
|
|
|
- return getSnapshotINode(snapshot).getFsPermission();
|
|
|
- }
|
|
|
-
|
|
|
- return new FsPermission(
|
|
|
- (short)PermissionStatusFormat.MODE.retrieve(permission));
|
|
|
- }
|
|
|
+ abstract FsPermission getFsPermission(Snapshot snapshot);
|
|
|
+
|
|
|
/** The same as getFsPermission(null). */
|
|
|
public final FsPermission getFsPermission() {
|
|
|
return getFsPermission(null);
|
|
|
}
|
|
|
- protected short getFsPermissionShort() {
|
|
|
- return (short)PermissionStatusFormat.MODE.retrieve(permission);
|
|
|
- }
|
|
|
+
|
|
|
/** Set the {@link FsPermission} of this {@link INode} */
|
|
|
- void setPermission(FsPermission permission) {
|
|
|
- final short mode = permission.toShort();
|
|
|
- updatePermissionStatus(PermissionStatusFormat.MODE, mode);
|
|
|
- }
|
|
|
+ abstract void setPermission(FsPermission permission);
|
|
|
+
|
|
|
/** Set the {@link FsPermission} of this {@link INode} */
|
|
|
INode setPermission(FsPermission permission, Snapshot latest)
|
|
|
throws QuotaExceededException {
|
|
@@ -270,10 +159,24 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
|
|
|
/** Is this inode in the latest snapshot? */
|
|
|
public final boolean isInLatestSnapshot(final Snapshot latest) {
|
|
|
- return latest != null
|
|
|
- && (parent == null
|
|
|
- || (parent.isInLatestSnapshot(latest)
|
|
|
- && this == parent.getChild(getLocalNameBytes(), latest)));
|
|
|
+ if (latest == null) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ final INodeDirectory parentDir = getParent();
|
|
|
+ if (parentDir == null) { // root
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ if (!parentDir.isInLatestSnapshot(latest)) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ final INode child = parentDir.getChild(getLocalNameBytes(), latest);
|
|
|
+ if (this == child) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ if (child == null || !(child.isReference())) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ return this == child.asReference().getReferredINode();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -289,6 +192,17 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
abstract INode recordModification(final Snapshot latest)
|
|
|
throws QuotaExceededException;
|
|
|
|
|
|
+ /** Check whether it's a reference. */
|
|
|
+ public boolean isReference() {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Cast this inode to an {@link INodeReference}. */
|
|
|
+ public INodeReference asReference() {
|
|
|
+ throw new IllegalStateException("Current inode is not a reference: "
|
|
|
+ + this.toDetailString());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Check whether it's a file.
|
|
|
*/
|
|
@@ -315,6 +229,19 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
+ this.toDetailString());
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Check whether it's a symlink
|
|
|
+ */
|
|
|
+ public boolean isSymlink() {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Cast this inode to an {@link INodeSymlink}. */
|
|
|
+ public INodeSymlink asSymlink() {
|
|
|
+ throw new IllegalStateException("Current inode is not a symlink: "
|
|
|
+ + this.toDetailString());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Clean the subtree under this inode and collect the blocks from the descents
|
|
|
* for further block deletion/update. The current inode can either resides in
|
|
@@ -420,8 +347,9 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
*/
|
|
|
public void addSpaceConsumed(long nsDelta, long dsDelta)
|
|
|
throws QuotaExceededException {
|
|
|
- if (parent != null) {
|
|
|
- parent.addSpaceConsumed(nsDelta, dsDelta);
|
|
|
+ final INodeDirectory parentDir = getParent();
|
|
|
+ if (parentDir != null) {
|
|
|
+ parentDir.addSpaceConsumed(nsDelta, dsDelta);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -460,7 +388,8 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
/**
|
|
|
* @return null if the local name is null; otherwise, return the local name.
|
|
|
*/
|
|
|
- public String getLocalName() {
|
|
|
+ public final String getLocalName() {
|
|
|
+ final byte[] name = getLocalNameBytes();
|
|
|
return name == null? null: DFSUtil.bytes2String(name);
|
|
|
}
|
|
|
|
|
@@ -468,21 +397,17 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
* @return null if the local name is null;
|
|
|
* otherwise, return the local name byte array.
|
|
|
*/
|
|
|
- public byte[] getLocalNameBytes() {
|
|
|
- return name;
|
|
|
- }
|
|
|
+ public abstract byte[] getLocalNameBytes();
|
|
|
|
|
|
@Override
|
|
|
- public byte[] getKey() {
|
|
|
+ public final byte[] getKey() {
|
|
|
return getLocalNameBytes();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Set local file name
|
|
|
*/
|
|
|
- public void setLocalName(byte[] name) {
|
|
|
- this.name = name;
|
|
|
- }
|
|
|
+ public abstract void setLocalName(byte[] name);
|
|
|
|
|
|
public String getFullPathName() {
|
|
|
// Get the full path name of this inode.
|
|
@@ -492,7 +417,7 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
/**
|
|
|
* @return The full path name represented in a list of byte array
|
|
|
*/
|
|
|
- public byte[][] getRelativePathNameBytes(INode ancestor) {
|
|
|
+ public final byte[][] getRelativePathNameBytes(INode ancestor) {
|
|
|
return FSDirectory.getRelativePathNameBytes(this, ancestor);
|
|
|
}
|
|
|
|
|
@@ -514,25 +439,37 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
|
|
|
@VisibleForTesting
|
|
|
public String toDetailString() {
|
|
|
+ final INodeDirectory p = getParent();
|
|
|
return toStringWithObjectType()
|
|
|
- + ", parent=" + (parent == null? null: parent.toStringWithObjectType());
|
|
|
+ + ", parent=" + (p == null? null: p.toStringWithObjectType());
|
|
|
+ }
|
|
|
+
|
|
|
+ /** @return the parent directory */
|
|
|
+ public final INodeDirectory getParent() {
|
|
|
+ return parent == null? null
|
|
|
+ : parent.isReference()? getParentReference().getParent(): parent.asDirectory();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Get parent directory
|
|
|
- * @return parent INode
|
|
|
+ * @return the parent as a reference if this is a referred inode;
|
|
|
+ * otherwise, return null.
|
|
|
*/
|
|
|
- public final INodeDirectory getParent() {
|
|
|
- return this.parent;
|
|
|
+ public INodeReference getParentReference() {
|
|
|
+ return parent == null || !parent.isReference()? null: (INodeReference)parent;
|
|
|
}
|
|
|
|
|
|
/** Set parent directory */
|
|
|
- public void setParent(INodeDirectory parent) {
|
|
|
+ public final void setParent(INodeDirectory parent) {
|
|
|
+ this.parent = parent;
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Set container. */
|
|
|
+ public final void setParentReference(INodeReference parent) {
|
|
|
this.parent = parent;
|
|
|
}
|
|
|
|
|
|
/** Clear references to other objects. */
|
|
|
- public void clearReferences() {
|
|
|
+ public void clear() {
|
|
|
setParent(null);
|
|
|
}
|
|
|
|
|
@@ -542,13 +479,7 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
* otherwise, get the result from the current inode.
|
|
|
* @return modification time.
|
|
|
*/
|
|
|
- public final long getModificationTime(Snapshot snapshot) {
|
|
|
- if (snapshot != null) {
|
|
|
- return getSnapshotINode(snapshot).modificationTime;
|
|
|
- }
|
|
|
-
|
|
|
- return this.modificationTime;
|
|
|
- }
|
|
|
+ abstract long getModificationTime(Snapshot snapshot);
|
|
|
|
|
|
/** The same as getModificationTime(null). */
|
|
|
public final long getModificationTime() {
|
|
@@ -556,23 +487,12 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
}
|
|
|
|
|
|
/** Update modification time if it is larger than the current value. */
|
|
|
- public final INode updateModificationTime(long mtime, Snapshot latest)
|
|
|
- throws QuotaExceededException {
|
|
|
- Preconditions.checkState(isDirectory());
|
|
|
- if (mtime <= modificationTime) {
|
|
|
- return this;
|
|
|
- }
|
|
|
- return setModificationTime(mtime, latest);
|
|
|
- }
|
|
|
-
|
|
|
- void cloneModificationTime(INode that) {
|
|
|
- this.modificationTime = that.modificationTime;
|
|
|
- }
|
|
|
+ public abstract INode updateModificationTime(long mtime, Snapshot latest)
|
|
|
+ throws QuotaExceededException;
|
|
|
|
|
|
/** Set the last modification time of inode. */
|
|
|
- public final void setModificationTime(long modificationTime) {
|
|
|
- this.modificationTime = modificationTime;
|
|
|
- }
|
|
|
+ public abstract void setModificationTime(long modificationTime);
|
|
|
+
|
|
|
/** Set the last modification time of inode. */
|
|
|
public final INode setModificationTime(long modificationTime, Snapshot latest)
|
|
|
throws QuotaExceededException {
|
|
@@ -587,13 +507,7 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
* otherwise, get the result from the current inode.
|
|
|
* @return access time
|
|
|
*/
|
|
|
- public final long getAccessTime(Snapshot snapshot) {
|
|
|
- if (snapshot != null) {
|
|
|
- return getSnapshotINode(snapshot).accessTime;
|
|
|
- }
|
|
|
-
|
|
|
- return accessTime;
|
|
|
- }
|
|
|
+ abstract long getAccessTime(Snapshot snapshot);
|
|
|
|
|
|
/** The same as getAccessTime(null). */
|
|
|
public final long getAccessTime() {
|
|
@@ -603,31 +517,18 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
/**
|
|
|
* Set last access time of inode.
|
|
|
*/
|
|
|
- public void setAccessTime(long accessTime) {
|
|
|
- this.accessTime = accessTime;
|
|
|
- }
|
|
|
+ public abstract void setAccessTime(long accessTime);
|
|
|
+
|
|
|
/**
|
|
|
* Set last access time of inode.
|
|
|
*/
|
|
|
- public INode setAccessTime(long accessTime, Snapshot latest)
|
|
|
+ public final INode setAccessTime(long accessTime, Snapshot latest)
|
|
|
throws QuotaExceededException {
|
|
|
final INode nodeToUpdate = recordModification(latest);
|
|
|
nodeToUpdate.setAccessTime(accessTime);
|
|
|
return nodeToUpdate;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Check whether it's a symlink
|
|
|
- */
|
|
|
- public boolean isSymlink() {
|
|
|
- return false;
|
|
|
- }
|
|
|
-
|
|
|
- /** Cast this inode to an {@link INodeSymlink}. */
|
|
|
- public INodeSymlink asSymlink() {
|
|
|
- throw new IllegalStateException("Current inode is not a symlink: "
|
|
|
- + this.toDetailString());
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Breaks file path into components.
|
|
@@ -683,6 +584,7 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
|
|
|
@Override
|
|
|
public final int compareTo(byte[] bytes) {
|
|
|
+ final byte[] name = getLocalNameBytes();
|
|
|
final byte[] left = name == null? DFSUtil.EMPTY_BYTES: name;
|
|
|
final byte[] right = bytes == null? DFSUtil.EMPTY_BYTES: bytes;
|
|
|
return SignedBytes.lexicographicalComparator().compare(left, right);
|
|
@@ -696,12 +598,13 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
if (that == null || !(that instanceof INode)) {
|
|
|
return false;
|
|
|
}
|
|
|
- return Arrays.equals(this.name, ((INode)that).name);
|
|
|
+ return Arrays.equals(this.getLocalNameBytes(),
|
|
|
+ ((INode)that).getLocalNameBytes());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public final int hashCode() {
|
|
|
- return Arrays.hashCode(this.name);
|
|
|
+ return Arrays.hashCode(getLocalNameBytes());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -728,7 +631,9 @@ public abstract class INode implements Diff.Element<byte[]> {
|
|
|
out.print(" (");
|
|
|
out.print(getObjectString());
|
|
|
out.print("), parent=");
|
|
|
- out.print(parent == null? null: parent.getLocalName() + "/");
|
|
|
+
|
|
|
+ final INodeDirectory p = getParent();
|
|
|
+ out.print(p == null? null: p.getLocalName() + "/");
|
|
|
out.print(", " + getPermissionStatus(snapshot));
|
|
|
}
|
|
|
|