|
@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
|
|
import org.apache.hadoop.fs.permission.AclStatus;
|
|
import org.apache.hadoop.fs.permission.AclStatus;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
|
+import org.apache.hadoop.hdfs.BlockStoragePolicy;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
@@ -946,10 +947,9 @@ public class FSDirectory implements Closeable {
|
|
return file.getBlocks();
|
|
return file.getBlocks();
|
|
}
|
|
}
|
|
|
|
|
|
- /** Set block storage policy for a file */
|
|
|
|
|
|
+ /** Set block storage policy for a directory */
|
|
void setStoragePolicy(String src, byte policyId)
|
|
void setStoragePolicy(String src, byte policyId)
|
|
- throws SnapshotAccessControlException, UnresolvedLinkException,
|
|
|
|
- FileNotFoundException, QuotaExceededException {
|
|
|
|
|
|
+ throws IOException {
|
|
writeLock();
|
|
writeLock();
|
|
try {
|
|
try {
|
|
unprotectedSetStoragePolicy(src, policyId);
|
|
unprotectedSetStoragePolicy(src, policyId);
|
|
@@ -959,13 +959,30 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
|
|
|
|
void unprotectedSetStoragePolicy(String src, byte policyId)
|
|
void unprotectedSetStoragePolicy(String src, byte policyId)
|
|
- throws SnapshotAccessControlException, UnresolvedLinkException,
|
|
|
|
- FileNotFoundException, QuotaExceededException {
|
|
|
|
|
|
+ throws IOException {
|
|
assert hasWriteLock();
|
|
assert hasWriteLock();
|
|
final INodesInPath iip = getINodesInPath4Write(src, true);
|
|
final INodesInPath iip = getINodesInPath4Write(src, true);
|
|
- // TODO: currently we only support setting storage policy on a file
|
|
|
|
- final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
|
|
|
|
- inode.setStoragePolicyID(policyId, iip.getLatestSnapshotId());
|
|
|
|
|
|
+ final INode inode = iip.getLastINode();
|
|
|
|
+ if (inode == null) {
|
|
|
|
+ throw new FileNotFoundException("File/Directory does not exist: " + src);
|
|
|
|
+ }
|
|
|
|
+ final int snapshotId = iip.getLatestSnapshotId();
|
|
|
|
+ if (inode.isFile()) {
|
|
|
|
+ inode.asFile().setStoragePolicyID(policyId, snapshotId);
|
|
|
|
+ } else if (inode.isDirectory()) {
|
|
|
|
+ setDirStoragePolicy(inode.asDirectory(), policyId, snapshotId);
|
|
|
|
+ } else {
|
|
|
|
+ throw new FileNotFoundException(src + " 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 = BlockStoragePolicy.buildXAttr(policyId);
|
|
|
|
+ List<XAttr> newXAttrs = setINodeXAttrs(existingXAttrs, Arrays.asList(xAttr),
|
|
|
|
+ EnumSet.of(XAttrSetFlag.CREATE, XAttrSetFlag.REPLACE));
|
|
|
|
+ XAttrStorage.updateINodeXAttrs(inode, newXAttrs, latestSnapshotId);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1313,7 +1330,8 @@ public class FSDirectory implements Closeable {
|
|
* @return a partial listing starting after startAfter
|
|
* @return a partial listing starting after startAfter
|
|
*/
|
|
*/
|
|
DirectoryListing getListing(String src, byte[] startAfter,
|
|
DirectoryListing getListing(String src, byte[] startAfter,
|
|
- boolean needLocation) throws UnresolvedLinkException, IOException {
|
|
|
|
|
|
+ boolean needLocation, boolean isSuperUser)
|
|
|
|
+ throws UnresolvedLinkException, IOException {
|
|
String srcs = normalizePath(src);
|
|
String srcs = normalizePath(src);
|
|
|
|
|
|
readLock();
|
|
readLock();
|
|
@@ -1321,16 +1339,19 @@ public class FSDirectory implements Closeable {
|
|
if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
|
|
if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
|
|
return getSnapshotsListing(srcs, startAfter);
|
|
return getSnapshotsListing(srcs, startAfter);
|
|
}
|
|
}
|
|
- final INodesInPath inodesInPath = getLastINodeInPath(srcs, true);
|
|
|
|
|
|
+ final INodesInPath inodesInPath = getINodesInPath(srcs, true);
|
|
final int snapshot = inodesInPath.getPathSnapshotId();
|
|
final int snapshot = inodesInPath.getPathSnapshotId();
|
|
- final INode targetNode = inodesInPath.getINode(0);
|
|
|
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
|
+ final INode targetNode = inodes[inodes.length - 1];
|
|
|
|
+ byte parentStoragePolicy = isSuperUser ? getStoragePolicy(inodes,
|
|
|
|
+ snapshot) : BlockStoragePolicy.ID_UNSPECIFIED;
|
|
if (targetNode == null)
|
|
if (targetNode == null)
|
|
return null;
|
|
return null;
|
|
|
|
|
|
if (!targetNode.isDirectory()) {
|
|
if (!targetNode.isDirectory()) {
|
|
return new DirectoryListing(
|
|
return new DirectoryListing(
|
|
new HdfsFileStatus[]{createFileStatus(HdfsFileStatus.EMPTY_NAME,
|
|
new HdfsFileStatus[]{createFileStatus(HdfsFileStatus.EMPTY_NAME,
|
|
- targetNode, needLocation, snapshot)}, 0);
|
|
|
|
|
|
+ targetNode, needLocation, parentStoragePolicy, snapshot)}, 0);
|
|
}
|
|
}
|
|
|
|
|
|
final INodeDirectory dirInode = targetNode.asDirectory();
|
|
final INodeDirectory dirInode = targetNode.asDirectory();
|
|
@@ -1343,8 +1364,10 @@ public class FSDirectory implements Closeable {
|
|
HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
|
|
HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
|
|
for (int i=0; i<numOfListing && locationBudget>0; i++) {
|
|
for (int i=0; i<numOfListing && locationBudget>0; i++) {
|
|
INode cur = contents.get(startChild+i);
|
|
INode cur = contents.get(startChild+i);
|
|
|
|
+ byte curPolicy = cur.getStoragePolicyID(snapshot);
|
|
listing[i] = createFileStatus(cur.getLocalNameBytes(), cur,
|
|
listing[i] = createFileStatus(cur.getLocalNameBytes(), cur,
|
|
- needLocation, snapshot);
|
|
|
|
|
|
+ needLocation, curPolicy != BlockStoragePolicy.ID_UNSPECIFIED ?
|
|
|
|
+ curPolicy : parentStoragePolicy, snapshot);
|
|
listingCnt++;
|
|
listingCnt++;
|
|
if (needLocation) {
|
|
if (needLocation) {
|
|
// Once we hit lsLimit locations, stop.
|
|
// Once we hit lsLimit locations, stop.
|
|
@@ -1395,7 +1418,7 @@ public class FSDirectory implements Closeable {
|
|
for (int i = 0; i < numOfListing; i++) {
|
|
for (int i = 0; i < numOfListing; i++) {
|
|
Root sRoot = snapshots.get(i + skipSize).getRoot();
|
|
Root sRoot = snapshots.get(i + skipSize).getRoot();
|
|
listing[i] = createFileStatus(sRoot.getLocalNameBytes(), sRoot,
|
|
listing[i] = createFileStatus(sRoot.getLocalNameBytes(), sRoot,
|
|
- Snapshot.CURRENT_STATE_ID);
|
|
|
|
|
|
+ BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID);
|
|
}
|
|
}
|
|
return new DirectoryListing(
|
|
return new DirectoryListing(
|
|
listing, snapshots.size() - skipSize - numOfListing);
|
|
listing, snapshots.size() - skipSize - numOfListing);
|
|
@@ -1417,8 +1440,8 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
final INodesInPath inodesInPath = getLastINodeInPath(srcs, resolveLink);
|
|
final INodesInPath inodesInPath = getLastINodeInPath(srcs, resolveLink);
|
|
final INode i = inodesInPath.getINode(0);
|
|
final INode i = inodesInPath.getINode(0);
|
|
- return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
|
|
|
|
- inodesInPath.getPathSnapshotId());
|
|
|
|
|
|
+ return i == null ? null : createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
|
|
|
|
+ BlockStoragePolicy.ID_UNSPECIFIED, inodesInPath.getPathSnapshotId());
|
|
} finally {
|
|
} finally {
|
|
readUnlock();
|
|
readUnlock();
|
|
}
|
|
}
|
|
@@ -1435,7 +1458,7 @@ public class FSDirectory implements Closeable {
|
|
throws UnresolvedLinkException {
|
|
throws UnresolvedLinkException {
|
|
if (getINode4DotSnapshot(src) != null) {
|
|
if (getINode4DotSnapshot(src) != null) {
|
|
return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
|
|
return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
|
|
- HdfsFileStatus.EMPTY_NAME, -1L, 0);
|
|
|
|
|
|
+ HdfsFileStatus.EMPTY_NAME, -1L, 0, BlockStoragePolicy.ID_UNSPECIFIED);
|
|
}
|
|
}
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
@@ -2247,18 +2270,19 @@ public class FSDirectory implements Closeable {
|
|
* @throws IOException if any error occurs
|
|
* @throws IOException if any error occurs
|
|
*/
|
|
*/
|
|
private HdfsFileStatus createFileStatus(byte[] path, INode node,
|
|
private HdfsFileStatus createFileStatus(byte[] path, INode node,
|
|
- boolean needLocation, int snapshot) throws IOException {
|
|
|
|
|
|
+ boolean needLocation, byte storagePolicy, int snapshot) throws IOException {
|
|
if (needLocation) {
|
|
if (needLocation) {
|
|
- return createLocatedFileStatus(path, node, snapshot);
|
|
|
|
|
|
+ return createLocatedFileStatus(path, node, storagePolicy, snapshot);
|
|
} else {
|
|
} else {
|
|
- return createFileStatus(path, node, snapshot);
|
|
|
|
|
|
+ return createFileStatus(path, node, storagePolicy, snapshot);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Create FileStatus by file INode
|
|
* Create FileStatus by file INode
|
|
*/
|
|
*/
|
|
- HdfsFileStatus createFileStatus(byte[] path, INode node,
|
|
|
|
- int snapshot) {
|
|
|
|
|
|
+ HdfsFileStatus createFileStatus(byte[] path, INode node, byte storagePolicy,
|
|
|
|
+ int snapshot) {
|
|
long size = 0; // length is zero for directories
|
|
long size = 0; // length is zero for directories
|
|
short replication = 0;
|
|
short replication = 0;
|
|
long blocksize = 0;
|
|
long blocksize = 0;
|
|
@@ -2270,7 +2294,7 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
int childrenNum = node.isDirectory() ?
|
|
int childrenNum = node.isDirectory() ?
|
|
node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
-
|
|
|
|
|
|
+
|
|
return new HdfsFileStatus(
|
|
return new HdfsFileStatus(
|
|
size,
|
|
size,
|
|
node.isDirectory(),
|
|
node.isDirectory(),
|
|
@@ -2284,14 +2308,24 @@ public class FSDirectory implements Closeable {
|
|
node.isSymlink() ? node.asSymlink().getSymlink() : null,
|
|
node.isSymlink() ? node.asSymlink().getSymlink() : null,
|
|
path,
|
|
path,
|
|
node.getId(),
|
|
node.getId(),
|
|
- childrenNum);
|
|
|
|
|
|
+ childrenNum, storagePolicy);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private byte getStoragePolicy(INode[] inodes, int snapshotId) {
|
|
|
|
+ for (int i = inodes.length - 1; i >= 0; i--) {
|
|
|
|
+ byte policy = inodes[i].getStoragePolicyID(snapshotId);
|
|
|
|
+ if (policy != BlockStoragePolicy.ID_UNSPECIFIED) {
|
|
|
|
+ return policy;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return BlockStoragePolicy.ID_UNSPECIFIED;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Create FileStatus with location info by file INode
|
|
* Create FileStatus with location info by file INode
|
|
*/
|
|
*/
|
|
private HdfsLocatedFileStatus createLocatedFileStatus(byte[] path,
|
|
private HdfsLocatedFileStatus createLocatedFileStatus(byte[] path,
|
|
- INode node, int snapshot) throws IOException {
|
|
|
|
|
|
+ INode node, byte storagePolicy, int snapshot) throws IOException {
|
|
assert hasReadLock();
|
|
assert hasReadLock();
|
|
long size = 0; // length is zero for directories
|
|
long size = 0; // length is zero for directories
|
|
short replication = 0;
|
|
short replication = 0;
|
|
@@ -2316,7 +2350,7 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
int childrenNum = node.isDirectory() ?
|
|
int childrenNum = node.isDirectory() ?
|
|
node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
-
|
|
|
|
|
|
+
|
|
HdfsLocatedFileStatus status =
|
|
HdfsLocatedFileStatus status =
|
|
new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
|
|
new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
|
|
blocksize, node.getModificationTime(snapshot),
|
|
blocksize, node.getModificationTime(snapshot),
|
|
@@ -2324,7 +2358,7 @@ public class FSDirectory implements Closeable {
|
|
getPermissionForFileStatus(node, snapshot),
|
|
getPermissionForFileStatus(node, snapshot),
|
|
node.getUserName(snapshot), node.getGroupName(snapshot),
|
|
node.getUserName(snapshot), node.getGroupName(snapshot),
|
|
node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
|
|
node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
|
|
- node.getId(), loc, childrenNum);
|
|
|
|
|
|
+ node.getId(), loc, childrenNum, storagePolicy);
|
|
// Set caching information for the located blocks.
|
|
// Set caching information for the located blocks.
|
|
if (loc != null) {
|
|
if (loc != null) {
|
|
CacheManager cacheManager = namesystem.getCacheManager();
|
|
CacheManager cacheManager = namesystem.getCacheManager();
|