|
@@ -60,8 +60,8 @@ 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.INodeDirectory.INodesInPath;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotAccessControlException;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotAccessControlException;
|
|
|
import org.apache.hadoop.hdfs.util.ByteArray;
|
|
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
|
|
|
|
@@ -82,7 +82,9 @@ public class FSDirectory implements Closeable {
|
|
|
final INodeDirectoryWithQuota r = new INodeDirectoryWithQuota(
|
|
|
INodeDirectory.ROOT_NAME,
|
|
|
namesystem.createFsOwnerPermissions(new FsPermission((short)0755)));
|
|
|
- return INodeDirectorySnapshottable.newInstance(r, 0);
|
|
|
+ final INodeDirectorySnapshottable s = new INodeDirectorySnapshottable(r);
|
|
|
+ s.setSnapshotQuota(0);
|
|
|
+ return s;
|
|
|
}
|
|
|
|
|
|
INodeDirectoryWithQuota rootDir;
|
|
@@ -379,11 +381,9 @@ public class FSDirectory implements Closeable {
|
|
|
*/
|
|
|
void closeFile(String path, INodeFile file) {
|
|
|
waitForReady();
|
|
|
- long now = now();
|
|
|
writeLock();
|
|
|
try {
|
|
|
// file is closed
|
|
|
- file.setModificationTime(now);
|
|
|
fsImage.getEditLog().logCloseFile(path, file);
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.closeFile: "
|
|
@@ -563,7 +563,7 @@ public class FSDirectory implements Closeable {
|
|
|
|
|
|
boolean added = false;
|
|
|
INode srcChild = null;
|
|
|
- String srcChildName = null;
|
|
|
+ byte[] srcChildName = null;
|
|
|
try {
|
|
|
// remove src
|
|
|
srcChild = removeLastINode(srcInodesInPath);
|
|
@@ -573,7 +573,7 @@ public class FSDirectory implements Closeable {
|
|
|
+ " because the source can not be removed");
|
|
|
return false;
|
|
|
}
|
|
|
- srcChildName = srcChild.getLocalName();
|
|
|
+ srcChildName = srcChild.getLocalNameBytes();
|
|
|
srcChild.setLocalName(dstComponents[dstInodes.length-1]);
|
|
|
|
|
|
// add src to the destination
|
|
@@ -585,8 +585,10 @@ public class FSDirectory implements Closeable {
|
|
|
+ src + " is renamed to " + dst);
|
|
|
}
|
|
|
// update modification time of dst and the parent of src
|
|
|
- srcInodes[srcInodes.length-2].updateModificationTime(timestamp);
|
|
|
- dstInodes[dstInodes.length-2].updateModificationTime(timestamp);
|
|
|
+ srcInodes[srcInodes.length-2].updateModificationTime(timestamp,
|
|
|
+ srcInodesInPath.getLatestSnapshot());
|
|
|
+ dstInodes[dstInodes.length-2].updateModificationTime(timestamp,
|
|
|
+ dstInodesInPath.getLatestSnapshot());
|
|
|
// update moved leases with new filename
|
|
|
getFSNamesystem().unprotectedChangeLease(src, dst);
|
|
|
return true;
|
|
@@ -734,13 +736,13 @@ public class FSDirectory implements Closeable {
|
|
|
+ error);
|
|
|
throw new IOException(error);
|
|
|
}
|
|
|
- final String srcChildName = removedSrc.getLocalName();
|
|
|
- String dstChildName = null;
|
|
|
+ final byte[] srcChildName = removedSrc.getLocalNameBytes();
|
|
|
+ byte[] dstChildName = null;
|
|
|
INode removedDst = null;
|
|
|
try {
|
|
|
if (dstInode != null) { // dst exists remove it
|
|
|
removedDst = removeLastINode(dstInodesInPath);
|
|
|
- dstChildName = removedDst.getLocalName();
|
|
|
+ dstChildName = removedDst.getLocalNameBytes();
|
|
|
}
|
|
|
|
|
|
removedSrc.setLocalName(dstComponents[dstInodes.length - 1]);
|
|
@@ -752,8 +754,10 @@ public class FSDirectory implements Closeable {
|
|
|
"DIR* FSDirectory.unprotectedRenameTo: " + src
|
|
|
+ " is renamed to " + dst);
|
|
|
}
|
|
|
- srcInodes[srcInodes.length - 2].updateModificationTime(timestamp);
|
|
|
- dstInodes[dstInodes.length - 2].updateModificationTime(timestamp);
|
|
|
+ srcInodes[srcInodes.length - 2].updateModificationTime(timestamp,
|
|
|
+ srcInodesInPath.getLatestSnapshot());
|
|
|
+ dstInodes[dstInodes.length - 2].updateModificationTime(timestamp,
|
|
|
+ dstInodesInPath.getLatestSnapshot());
|
|
|
// update moved lease with new filename
|
|
|
getFSNamesystem().unprotectedChangeLease(src, dst);
|
|
|
|
|
@@ -829,7 +833,7 @@ public class FSDirectory implements Closeable {
|
|
|
long dsDelta = (replication - oldRepl) * (fileNode.diskspaceConsumed()/oldRepl);
|
|
|
updateCount(inodesInPath, inodes.length-1, 0, dsDelta, true);
|
|
|
|
|
|
- fileNode.setFileReplication(replication);
|
|
|
+ fileNode.setFileReplication(replication, inodesInPath.getLatestSnapshot());
|
|
|
|
|
|
if (oldReplication != null) {
|
|
|
oldReplication[0] = oldRepl;
|
|
@@ -899,11 +903,12 @@ public class FSDirectory implements Closeable {
|
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
|
SnapshotAccessControlException {
|
|
|
assert hasWriteLock();
|
|
|
- INode inode = rootDir.getMutableNode(src, true);
|
|
|
+ final INodesInPath inodesInPath = rootDir.getMutableINodesInPath(src, true);
|
|
|
+ final INode inode = inodesInPath.getLastINode();
|
|
|
if (inode == null) {
|
|
|
throw new FileNotFoundException("File does not exist: " + src);
|
|
|
}
|
|
|
- inode.setPermission(permissions);
|
|
|
+ inode.setPermission(permissions, inodesInPath.getLatestSnapshot());
|
|
|
}
|
|
|
|
|
|
void setOwner(String src, String username, String groupname)
|
|
@@ -922,15 +927,16 @@ public class FSDirectory implements Closeable {
|
|
|
throws FileNotFoundException, UnresolvedLinkException,
|
|
|
SnapshotAccessControlException {
|
|
|
assert hasWriteLock();
|
|
|
- INode inode = rootDir.getMutableNode(src, true);
|
|
|
+ final INodesInPath inodesInPath = rootDir.getMutableINodesInPath(src, true);
|
|
|
+ final INode inode = inodesInPath.getLastINode();
|
|
|
if (inode == null) {
|
|
|
throw new FileNotFoundException("File does not exist: " + src);
|
|
|
}
|
|
|
if (username != null) {
|
|
|
- inode.setUser(username);
|
|
|
+ inode.setUser(username, inodesInPath.getLatestSnapshot());
|
|
|
}
|
|
|
if (groupname != null) {
|
|
|
- inode.setGroup(groupname);
|
|
|
+ inode.setGroup(groupname, inodesInPath.getLatestSnapshot());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -973,6 +979,7 @@ public class FSDirectory implements Closeable {
|
|
|
final INode[] trgINodes = trgINodesInPath.getINodes();
|
|
|
INodeFile trgInode = (INodeFile) trgINodes[trgINodes.length-1];
|
|
|
INodeDirectory trgParent = (INodeDirectory)trgINodes[trgINodes.length-2];
|
|
|
+ final Snapshot trgLatestSnapshot = trgINodesInPath.getLatestSnapshot();
|
|
|
|
|
|
INodeFile [] allSrcInodes = new INodeFile[srcs.length];
|
|
|
int i = 0;
|
|
@@ -990,12 +997,12 @@ public class FSDirectory implements Closeable {
|
|
|
if(nodeToRemove == null) continue;
|
|
|
|
|
|
nodeToRemove.setBlocks(null);
|
|
|
- trgParent.removeChild(nodeToRemove);
|
|
|
+ trgParent.removeChild(nodeToRemove, trgLatestSnapshot);
|
|
|
count++;
|
|
|
}
|
|
|
|
|
|
- trgInode.setModificationTime(timestamp);
|
|
|
- trgParent.updateModificationTime(timestamp);
|
|
|
+ trgInode.setModificationTime(timestamp, trgLatestSnapshot);
|
|
|
+ trgParent.updateModificationTime(timestamp, trgLatestSnapshot);
|
|
|
// update quota on the parent directory ('count' files removed, 0 space)
|
|
|
unprotectedUpdateCount(trgINodesInPath, trgINodes.length-1, -count, 0);
|
|
|
}
|
|
@@ -1125,16 +1132,23 @@ public class FSDirectory implements Closeable {
|
|
|
BlocksMapUpdateInfo collectedBlocks, long mtime) {
|
|
|
assert hasWriteLock();
|
|
|
|
|
|
- final INode[] inodes = inodesInPath.getINodes();
|
|
|
- INode targetNode = inodes[inodes.length-1];
|
|
|
// Remove the node from the namespace
|
|
|
- targetNode = removeLastINode(inodesInPath);
|
|
|
+ final INode targetNode = removeLastINode(inodesInPath);
|
|
|
if (targetNode == null) {
|
|
|
return 0;
|
|
|
}
|
|
|
// set the parent's modification time
|
|
|
- inodes[inodes.length - 2].updateModificationTime(mtime);
|
|
|
- int filesRemoved = targetNode.collectSubtreeBlocksAndClear(collectedBlocks);
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
+ final Snapshot latestSnapshot = inodesInPath.getLatestSnapshot();
|
|
|
+ final INodeDirectory parent = (INodeDirectory)inodes[inodes.length - 2];
|
|
|
+ parent.updateModificationTime(mtime, latestSnapshot);
|
|
|
+
|
|
|
+ final INode snapshotCopy = parent.getChild(targetNode.getLocalNameBytes(),
|
|
|
+ latestSnapshot);
|
|
|
+ // if snapshotCopy == targetNode, it means that the file is also stored in
|
|
|
+ // a snapshot so that the block should not be removed.
|
|
|
+ final int filesRemoved = snapshotCopy == targetNode? 0
|
|
|
+ : targetNode.collectSubtreeBlocksAndClear(collectedBlocks);
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
|
|
|
+ targetNode.getFullPathName() + " is removed");
|
|
@@ -1166,35 +1180,21 @@ public class FSDirectory implements Closeable {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Replaces the specified INodeDirectory.
|
|
|
- */
|
|
|
- public void replaceINodeDirectory(String path, INodeDirectory oldnode,
|
|
|
- INodeDirectory newnode) throws IOException {
|
|
|
- writeLock();
|
|
|
- try {
|
|
|
- unprotectedReplaceINode(path, oldnode, newnode);
|
|
|
- // Note that the parent of the children of the oldnode is already updated
|
|
|
- } finally {
|
|
|
- writeUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Replaces the specified INodeFile with the specified one.
|
|
|
*/
|
|
|
public void replaceINodeFile(String path, INodeFile oldnode,
|
|
|
- INodeFile newnode) throws IOException {
|
|
|
+ INodeFile newnode, Snapshot latest) throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
- unprotectedReplaceINodeFile(path, oldnode, newnode);
|
|
|
+ unprotectedReplaceINodeFile(path, oldnode, newnode, latest);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private void unprotectedReplaceINode(String path, INode oldnode,
|
|
|
- INode newnode) throws IOException {
|
|
|
+ INode newnode, Snapshot latest) throws IOException {
|
|
|
Preconditions.checkState(hasWriteLock());
|
|
|
|
|
|
INodeDirectory parent = oldnode.getParent();
|
|
@@ -1206,20 +1206,19 @@ public class FSDirectory implements Closeable {
|
|
|
throw new IOException(mess);
|
|
|
}
|
|
|
|
|
|
- final INode removed = parent.removeChild(oldnode);
|
|
|
+ final INode removed = parent.removeChild(oldnode, latest);
|
|
|
Preconditions.checkState(removed == oldnode,
|
|
|
"removed != oldnode=%s, removed=%s", oldnode, removed);
|
|
|
|
|
|
parent = oldnode.getParent();
|
|
|
oldnode.setParent(null);
|
|
|
- parent.addChild(newnode, true);
|
|
|
-
|
|
|
+ parent.addChild(newnode, true, latest);
|
|
|
}
|
|
|
|
|
|
void unprotectedReplaceINodeFile(String path, INodeFile oldnode,
|
|
|
- INodeFile newnode)
|
|
|
- throws IOException, UnresolvedLinkException {
|
|
|
- unprotectedReplaceINode(path, oldnode, newnode);
|
|
|
+ INodeFile newnode, Snapshot latest
|
|
|
+ ) throws IOException, UnresolvedLinkException {
|
|
|
+ unprotectedReplaceINode(path, oldnode, newnode, latest);
|
|
|
newnode.setLocalName(oldnode.getLocalNameBytes());
|
|
|
|
|
|
/* Currently oldnode and newnode are assumed to contain the same
|
|
@@ -1248,6 +1247,7 @@ public class FSDirectory implements Closeable {
|
|
|
readLock();
|
|
|
try {
|
|
|
final INodesInPath inodesInPath = rootDir.getINodesInPath(srcs, true);
|
|
|
+ final Snapshot snapshot = inodesInPath.getPathSnapshot();
|
|
|
final INode targetNode = inodesInPath.getINode(0);
|
|
|
if (targetNode == null)
|
|
|
return null;
|
|
@@ -1255,19 +1255,20 @@ public class FSDirectory implements Closeable {
|
|
|
if (!targetNode.isDirectory()) {
|
|
|
return new DirectoryListing(
|
|
|
new HdfsFileStatus[]{createFileStatus(HdfsFileStatus.EMPTY_NAME,
|
|
|
- targetNode, needLocation)}, 0);
|
|
|
+ targetNode, needLocation, snapshot)}, 0);
|
|
|
}
|
|
|
|
|
|
INodeDirectory dirInode = (INodeDirectory)targetNode;
|
|
|
final ReadOnlyList<INode> contents = dirInode.getChildrenList(
|
|
|
inodesInPath.getPathSnapshot());
|
|
|
- int startChild = dirInode.nextChild(startAfter);
|
|
|
+ int startChild = INodeDirectory.nextChild(contents, startAfter);
|
|
|
int totalNumChildren = contents.size();
|
|
|
int numOfListing = Math.min(totalNumChildren-startChild, this.lsLimit);
|
|
|
HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
|
|
|
for (int i=0; i<numOfListing; i++) {
|
|
|
INode cur = contents.get(startChild+i);
|
|
|
- listing[i] = createFileStatus(cur.getLocalNameBytes(), cur, needLocation);
|
|
|
+ listing[i] = createFileStatus(cur.getLocalNameBytes(), cur,
|
|
|
+ needLocation, snapshot);
|
|
|
}
|
|
|
return new DirectoryListing(
|
|
|
listing, totalNumChildren-startChild-numOfListing);
|
|
@@ -1287,13 +1288,10 @@ public class FSDirectory implements Closeable {
|
|
|
String srcs = normalizePath(src);
|
|
|
readLock();
|
|
|
try {
|
|
|
- INode targetNode = rootDir.getNode(srcs, resolveLink);
|
|
|
- if (targetNode == null) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- else {
|
|
|
- return createFileStatus(HdfsFileStatus.EMPTY_NAME, targetNode);
|
|
|
- }
|
|
|
+ final INodesInPath inodesInPath = rootDir.getINodesInPath(srcs, resolveLink);
|
|
|
+ final INode i = inodesInPath.getINode(0);
|
|
|
+ return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
|
|
|
+ inodesInPath.getPathSnapshot());
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
@@ -1668,8 +1666,7 @@ public class FSDirectory implements Closeable {
|
|
|
private boolean addINode(String src, INode child
|
|
|
) throws QuotaExceededException, UnresolvedLinkException {
|
|
|
byte[][] components = INode.getPathComponents(src);
|
|
|
- byte[] path = components[components.length-1];
|
|
|
- child.setLocalName(path);
|
|
|
+ child.setLocalName(components[components.length-1]);
|
|
|
cacheName(child);
|
|
|
writeLock();
|
|
|
try {
|
|
@@ -1834,7 +1831,8 @@ public class FSDirectory implements Closeable {
|
|
|
if (inodes[pos-1] == null) {
|
|
|
throw new NullPointerException("Panic: parent does not exist");
|
|
|
}
|
|
|
- final boolean added = ((INodeDirectory)inodes[pos-1]).addChild(child, true);
|
|
|
+ final boolean added = ((INodeDirectory)inodes[pos-1]).addChild(child, true,
|
|
|
+ inodesInPath.getLatestSnapshot());
|
|
|
if (!added) {
|
|
|
updateCount(inodesInPath, pos, -counts.getNsCount(), -counts.getDsCount(), true);
|
|
|
}
|
|
@@ -1858,7 +1856,8 @@ public class FSDirectory implements Closeable {
|
|
|
private INode removeLastINode(final INodesInPath inodesInPath) {
|
|
|
final INode[] inodes = inodesInPath.getINodes();
|
|
|
final int pos = inodes.length - 1;
|
|
|
- INode removedNode = ((INodeDirectory)inodes[pos-1]).removeChild(inodes[pos]);
|
|
|
+ INode removedNode = ((INodeDirectory)inodes[pos-1]).removeChild(
|
|
|
+ inodes[pos], inodesInPath.getLatestSnapshot());
|
|
|
if (removedNode != null) {
|
|
|
INode.DirCounts counts = new INode.DirCounts();
|
|
|
removedNode.spaceConsumedInTree(counts);
|
|
@@ -1999,9 +1998,8 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
|
|
|
String srcs = normalizePath(src);
|
|
|
- final INode[] inodes = rootDir.getMutableINodesInPath(srcs, true)
|
|
|
- .getINodes();
|
|
|
- INodeDirectory dirNode = INodeDirectory.valueOf(inodes[inodes.length-1], srcs);
|
|
|
+ final INodesInPath iip = rootDir.getMutableINodesInPath(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
|
|
@@ -2014,24 +2012,17 @@ public class FSDirectory implements Closeable {
|
|
|
dsQuota = oldDsQuota;
|
|
|
}
|
|
|
|
|
|
+ final Snapshot latest = iip.getLatestSnapshot();
|
|
|
if (dirNode instanceof INodeDirectoryWithQuota) {
|
|
|
// a directory with quota; so set the quota to the new value
|
|
|
- ((INodeDirectoryWithQuota)dirNode).setQuota(nsQuota, dsQuota);
|
|
|
- if (!dirNode.isQuotaSet()) {
|
|
|
+ ((INodeDirectoryWithQuota)dirNode).setQuota(nsQuota, dsQuota, latest);
|
|
|
+ if (!dirNode.isQuotaSet() && latest == null) {
|
|
|
// will not come here for root because root's nsQuota is always set
|
|
|
- INodeDirectory newNode = new INodeDirectory(dirNode, true);
|
|
|
- INodeDirectory parent = (INodeDirectory)inodes[inodes.length-2];
|
|
|
- dirNode = newNode;
|
|
|
- parent.replaceChild(newNode);
|
|
|
+ return dirNode.replaceSelf4INodeDirectory();
|
|
|
}
|
|
|
} else {
|
|
|
// a non-quota directory; so replace it with a directory with quota
|
|
|
- final INodeDirectoryWithQuota newNode = new INodeDirectoryWithQuota(
|
|
|
- dirNode, true, nsQuota, dsQuota);
|
|
|
- // non-root directory node; parent != null
|
|
|
- INodeDirectory parent = (INodeDirectory)inodes[inodes.length-2];
|
|
|
- dirNode = newNode;
|
|
|
- parent.replaceChild(newNode);
|
|
|
+ return dirNode.replaceSelf4Quota(latest, oldNsQuota, oldDsQuota);
|
|
|
}
|
|
|
return (oldNsQuota != nsQuota || oldDsQuota != dsQuota) ? dirNode : null;
|
|
|
}
|
|
@@ -2070,11 +2061,12 @@ public class FSDirectory implements Closeable {
|
|
|
/**
|
|
|
* Sets the access time on the file/directory. Logs it in the transaction log.
|
|
|
*/
|
|
|
- void setTimes(String src, INode inode, long mtime, long atime, boolean force) {
|
|
|
+ void setTimes(String src, INode inode, long mtime, long atime, boolean force,
|
|
|
+ Snapshot latest) {
|
|
|
boolean status = false;
|
|
|
writeLock();
|
|
|
try {
|
|
|
- status = unprotectedSetTimes(src, inode, mtime, atime, force);
|
|
|
+ status = unprotectedSetTimes(src, inode, mtime, atime, force, latest);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -2086,27 +2078,28 @@ public class FSDirectory implements Closeable {
|
|
|
boolean unprotectedSetTimes(String src, long mtime, long atime, boolean force)
|
|
|
throws UnresolvedLinkException {
|
|
|
assert hasWriteLock();
|
|
|
- INode inode = getINode(src);
|
|
|
- return unprotectedSetTimes(src, inode, mtime, atime, force);
|
|
|
+ final INodesInPath i = getINodesInPath(src);
|
|
|
+ return unprotectedSetTimes(src, i.getLastINode(), mtime, atime, force,
|
|
|
+ i.getLatestSnapshot());
|
|
|
}
|
|
|
|
|
|
private boolean unprotectedSetTimes(String src, INode inode, long mtime,
|
|
|
- long atime, boolean force) {
|
|
|
+ long atime, boolean force, Snapshot latest) {
|
|
|
assert hasWriteLock();
|
|
|
boolean status = false;
|
|
|
if (mtime != -1) {
|
|
|
- inode.setModificationTime(mtime);
|
|
|
+ inode.setModificationTime(mtime, latest);
|
|
|
status = true;
|
|
|
}
|
|
|
if (atime != -1) {
|
|
|
- long inodeTime = inode.getAccessTime();
|
|
|
+ long inodeTime = inode.getAccessTime(null);
|
|
|
|
|
|
// 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);
|
|
|
+ inode.setAccessTime(atime, latest);
|
|
|
status = true;
|
|
|
}
|
|
|
}
|
|
@@ -2137,17 +2130,18 @@ public class FSDirectory implements Closeable {
|
|
|
* @throws IOException if any error occurs
|
|
|
*/
|
|
|
private HdfsFileStatus createFileStatus(byte[] path, INode node,
|
|
|
- boolean needLocation) throws IOException {
|
|
|
+ boolean needLocation, Snapshot snapshot) throws IOException {
|
|
|
if (needLocation) {
|
|
|
- return createLocatedFileStatus(path, node);
|
|
|
+ return createLocatedFileStatus(path, node, snapshot);
|
|
|
} else {
|
|
|
- return createFileStatus(path, node);
|
|
|
+ return createFileStatus(path, node, snapshot);
|
|
|
}
|
|
|
}
|
|
|
/**
|
|
|
* Create FileStatus by file INode
|
|
|
*/
|
|
|
- private HdfsFileStatus createFileStatus(byte[] path, INode node) {
|
|
|
+ private HdfsFileStatus createFileStatus(byte[] path, INode node,
|
|
|
+ Snapshot snapshot) {
|
|
|
long size = 0; // length is zero for directories
|
|
|
short replication = 0;
|
|
|
long blocksize = 0;
|
|
@@ -2162,11 +2156,11 @@ public class FSDirectory implements Closeable {
|
|
|
node.isDirectory(),
|
|
|
replication,
|
|
|
blocksize,
|
|
|
- node.getModificationTime(),
|
|
|
- node.getAccessTime(),
|
|
|
- node.getFsPermission(),
|
|
|
- node.getUserName(),
|
|
|
- node.getGroupName(),
|
|
|
+ node.getModificationTime(snapshot),
|
|
|
+ node.getAccessTime(snapshot),
|
|
|
+ node.getFsPermission(snapshot),
|
|
|
+ node.getUserName(snapshot),
|
|
|
+ node.getGroupName(snapshot),
|
|
|
node.isSymlink() ? ((INodeSymlink)node).getSymlink() : null,
|
|
|
path);
|
|
|
}
|
|
@@ -2175,7 +2169,7 @@ public class FSDirectory implements Closeable {
|
|
|
* Create FileStatus with location info by file INode
|
|
|
*/
|
|
|
private HdfsLocatedFileStatus createLocatedFileStatus(
|
|
|
- byte[] path, INode node) throws IOException {
|
|
|
+ byte[] path, INode node, Snapshot snapshot) throws IOException {
|
|
|
assert hasReadLock();
|
|
|
long size = 0; // length is zero for directories
|
|
|
short replication = 0;
|
|
@@ -2198,11 +2192,11 @@ public class FSDirectory implements Closeable {
|
|
|
node.isDirectory(),
|
|
|
replication,
|
|
|
blocksize,
|
|
|
- node.getModificationTime(),
|
|
|
- node.getAccessTime(),
|
|
|
- node.getFsPermission(),
|
|
|
- node.getUserName(),
|
|
|
- node.getGroupName(),
|
|
|
+ node.getModificationTime(snapshot),
|
|
|
+ node.getAccessTime(snapshot),
|
|
|
+ node.getFsPermission(snapshot),
|
|
|
+ node.getUserName(snapshot),
|
|
|
+ node.getGroupName(snapshot),
|
|
|
node.isSymlink() ? ((INodeSymlink)node).getSymlink() : null,
|
|
|
path,
|
|
|
loc);
|