|
@@ -37,6 +37,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.crypto.CipherSuite;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
|
import org.apache.hadoop.fs.FileEncryptionInfo;
|
|
@@ -1402,9 +1403,10 @@ public class FSDirectory implements Closeable {
|
|
|
if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
|
|
|
return getSnapshotsListing(srcs, startAfter);
|
|
|
}
|
|
|
- final INodesInPath inodesInPath = getLastINodeInPath(srcs);
|
|
|
+ final INodesInPath inodesInPath = getINodesInPath(srcs, true);
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
final int snapshot = inodesInPath.getPathSnapshotId();
|
|
|
- final INode targetNode = inodesInPath.getLastINode();
|
|
|
+ final INode targetNode = inodes[inodes.length - 1];
|
|
|
if (targetNode == null)
|
|
|
return null;
|
|
|
byte parentStoragePolicy = isSuperUser ?
|
|
@@ -1414,7 +1416,7 @@ public class FSDirectory implements Closeable {
|
|
|
return new DirectoryListing(
|
|
|
new HdfsFileStatus[]{createFileStatus(HdfsFileStatus.EMPTY_NAME,
|
|
|
targetNode, needLocation, parentStoragePolicy, snapshot,
|
|
|
- isRawPath)}, 0);
|
|
|
+ isRawPath, inodesInPath)}, 0);
|
|
|
}
|
|
|
|
|
|
final INodeDirectory dirInode = targetNode.asDirectory();
|
|
@@ -1431,7 +1433,7 @@ public class FSDirectory implements Closeable {
|
|
|
cur.getLocalStoragePolicyID(): BlockStoragePolicy.ID_UNSPECIFIED;
|
|
|
listing[i] = createFileStatus(cur.getLocalNameBytes(), cur, needLocation,
|
|
|
getStoragePolicyID(curPolicy, parentStoragePolicy), snapshot,
|
|
|
- isRawPath);
|
|
|
+ isRawPath, inodesInPath);
|
|
|
listingCnt++;
|
|
|
if (needLocation) {
|
|
|
// Once we hit lsLimit locations, stop.
|
|
@@ -1482,7 +1484,8 @@ public class FSDirectory implements Closeable {
|
|
|
for (int i = 0; i < numOfListing; i++) {
|
|
|
Root sRoot = snapshots.get(i + skipSize).getRoot();
|
|
|
listing[i] = createFileStatus(sRoot.getLocalNameBytes(), sRoot,
|
|
|
- BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID, false);
|
|
|
+ BlockStoragePolicy.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
|
|
|
+ false, null);
|
|
|
}
|
|
|
return new DirectoryListing(
|
|
|
listing, snapshots.size() - skipSize - numOfListing);
|
|
@@ -1505,12 +1508,14 @@ public class FSDirectory implements Closeable {
|
|
|
if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
|
|
|
return getFileInfo4DotSnapshot(srcs);
|
|
|
}
|
|
|
- final INodesInPath inodesInPath = getLastINodeInPath(srcs, resolveLink);
|
|
|
- final INode i = inodesInPath.getINode(0);
|
|
|
+ final INodesInPath inodesInPath = getINodesInPath(srcs, resolveLink);
|
|
|
+ final INode[] inodes = inodesInPath.getINodes();
|
|
|
+ final INode i = inodes[inodes.length - 1];
|
|
|
byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
|
|
|
i.getStoragePolicyID() : BlockStoragePolicy.ID_UNSPECIFIED;
|
|
|
return i == null ? null : createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
|
|
|
- policyId, inodesInPath.getPathSnapshotId(), isRawPath);
|
|
|
+ policyId, inodesInPath.getPathSnapshotId(), isRawPath,
|
|
|
+ inodesInPath);
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
@@ -2162,8 +2167,17 @@ public class FSDirectory implements Closeable {
|
|
|
for (XAttr xattr : xattrs) {
|
|
|
final String xaName = XAttrHelper.getPrefixName(xattr);
|
|
|
if (CRYPTO_XATTR_ENCRYPTION_ZONE.equals(xaName)) {
|
|
|
- ezManager.unprotectedAddEncryptionZone(inode.getId(),
|
|
|
- new String(xattr.getValue()));
|
|
|
+ try {
|
|
|
+ final HdfsProtos.ZoneEncryptionInfoProto ezProto =
|
|
|
+ HdfsProtos.ZoneEncryptionInfoProto.parseFrom(
|
|
|
+ xattr.getValue());
|
|
|
+ ezManager.unprotectedAddEncryptionZone(inode.getId(),
|
|
|
+ PBHelper.convert(ezProto.getSuite()),
|
|
|
+ ezProto.getKeyName());
|
|
|
+ } catch (InvalidProtocolBufferException e) {
|
|
|
+ NameNode.LOG.warn("Error parsing protocol buffer of " +
|
|
|
+ "EZ XAttr " + xattr.getName());
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -2355,12 +2369,15 @@ public class FSDirectory implements Closeable {
|
|
|
* @throws IOException if any error occurs
|
|
|
*/
|
|
|
private HdfsFileStatus createFileStatus(byte[] path, INode node,
|
|
|
- boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath)
|
|
|
+ boolean needLocation, byte storagePolicy, int snapshot,
|
|
|
+ boolean isRawPath, INodesInPath iip)
|
|
|
throws IOException {
|
|
|
if (needLocation) {
|
|
|
- return createLocatedFileStatus(path, node, storagePolicy, snapshot, isRawPath);
|
|
|
+ return createLocatedFileStatus(path, node, storagePolicy, snapshot,
|
|
|
+ isRawPath, iip);
|
|
|
} else {
|
|
|
- return createFileStatus(path, node, storagePolicy, snapshot, isRawPath);
|
|
|
+ return createFileStatus(path, node, storagePolicy, snapshot,
|
|
|
+ isRawPath, iip);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2368,14 +2385,14 @@ public class FSDirectory implements Closeable {
|
|
|
* Create FileStatus by file INode
|
|
|
*/
|
|
|
HdfsFileStatus createFileStatus(byte[] path, INode node, byte storagePolicy,
|
|
|
- int snapshot, boolean isRawPath) throws IOException {
|
|
|
+ int snapshot, boolean isRawPath, INodesInPath iip) throws IOException {
|
|
|
long size = 0; // length is zero for directories
|
|
|
short replication = 0;
|
|
|
long blocksize = 0;
|
|
|
final boolean isEncrypted;
|
|
|
|
|
|
final FileEncryptionInfo feInfo = isRawPath ? null :
|
|
|
- getFileEncryptionInfo(node, snapshot);
|
|
|
+ getFileEncryptionInfo(node, snapshot, iip);
|
|
|
|
|
|
if (node.isFile()) {
|
|
|
final INodeFile fileNode = node.asFile();
|
|
@@ -2413,7 +2430,8 @@ public class FSDirectory implements Closeable {
|
|
|
* Create FileStatus with location info by file INode
|
|
|
*/
|
|
|
private HdfsLocatedFileStatus createLocatedFileStatus(byte[] path, INode node,
|
|
|
- byte storagePolicy, int snapshot, boolean isRawPath) throws IOException {
|
|
|
+ byte storagePolicy, int snapshot, boolean isRawPath,
|
|
|
+ INodesInPath iip) throws IOException {
|
|
|
assert hasReadLock();
|
|
|
long size = 0; // length is zero for directories
|
|
|
short replication = 0;
|
|
@@ -2421,7 +2439,7 @@ public class FSDirectory implements Closeable {
|
|
|
LocatedBlocks loc = null;
|
|
|
final boolean isEncrypted;
|
|
|
final FileEncryptionInfo feInfo = isRawPath ? null :
|
|
|
- getFileEncryptionInfo(node, snapshot);
|
|
|
+ getFileEncryptionInfo(node, snapshot, iip);
|
|
|
if (node.isFile()) {
|
|
|
final INodeFile fileNode = node.asFile();
|
|
|
size = fileNode.computeFileSize(snapshot);
|
|
@@ -2746,11 +2764,11 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- XAttr createEncryptionZone(String src, String keyName)
|
|
|
+ XAttr createEncryptionZone(String src, CipherSuite suite, String keyName)
|
|
|
throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
- return ezManager.createEncryptionZone(src, keyName);
|
|
|
+ return ezManager.createEncryptionZone(src, suite, keyName);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -2781,7 +2799,8 @@ public class FSDirectory implements Closeable {
|
|
|
void setFileEncryptionInfo(String src, FileEncryptionInfo info)
|
|
|
throws IOException {
|
|
|
// Make the PB for the xattr
|
|
|
- final HdfsProtos.FileEncryptionInfoProto proto = PBHelper.convert(info);
|
|
|
+ final HdfsProtos.PerFileEncryptionInfoProto proto =
|
|
|
+ PBHelper.convertPerFileEncInfo(info);
|
|
|
final byte[] protoBytes = proto.toByteArray();
|
|
|
final XAttr fileEncryptionAttr =
|
|
|
XAttrHelper.buildXAttr(CRYPTO_XATTR_FILE_ENCRYPTION_INFO, protoBytes);
|
|
@@ -2797,35 +2816,64 @@ public class FSDirectory implements Closeable {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Return the FileEncryptionInfo for an INode, or null if the INode is not
|
|
|
- * an encrypted file.
|
|
|
- */
|
|
|
- FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId)
|
|
|
- throws IOException {
|
|
|
+ * This function combines the per-file encryption info (obtained
|
|
|
+ * from the inode's XAttrs), and the encryption info from its zone, and
|
|
|
+ * returns a consolidated FileEncryptionInfo instance. Null is returned
|
|
|
+ * for non-encrypted files.
|
|
|
+ *
|
|
|
+ * @param inode inode of the file
|
|
|
+ * @param snapshotId ID of the snapshot that
|
|
|
+ * we want to get encryption info from
|
|
|
+ * @param iip inodes in the path containing the file, passed in to
|
|
|
+ * avoid obtaining the list of inodes again; if iip is
|
|
|
+ * null then the list of inodes will be obtained again
|
|
|
+ * @return consolidated file encryption info; null for non-encrypted files
|
|
|
+ */
|
|
|
+ FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId,
|
|
|
+ INodesInPath iip) throws IOException {
|
|
|
if (!inode.isFile()) {
|
|
|
return null;
|
|
|
}
|
|
|
readLock();
|
|
|
try {
|
|
|
- List<XAttr> xAttrs = XAttrStorage.readINodeXAttrs(inode, snapshotId);
|
|
|
- if (xAttrs == null) {
|
|
|
- return null;
|
|
|
+ if (iip == null) {
|
|
|
+ iip = getINodesInPath(inode.getFullPathName(), true);
|
|
|
}
|
|
|
- for (XAttr x : xAttrs) {
|
|
|
- if (XAttrHelper.getPrefixName(x)
|
|
|
- .equals(CRYPTO_XATTR_FILE_ENCRYPTION_INFO)) {
|
|
|
- try {
|
|
|
- HdfsProtos.FileEncryptionInfoProto proto =
|
|
|
- HdfsProtos.FileEncryptionInfoProto.parseFrom(x.getValue());
|
|
|
- FileEncryptionInfo feInfo = PBHelper.convert(proto);
|
|
|
- return feInfo;
|
|
|
- } catch (InvalidProtocolBufferException e) {
|
|
|
- throw new IOException("Could not parse file encryption info for " +
|
|
|
- "inode " + inode, e);
|
|
|
- }
|
|
|
+ EncryptionZone encryptionZone = getEZForPath(iip);
|
|
|
+ if (encryptionZone == null ||
|
|
|
+ encryptionZone.equals(EncryptionZoneManager.NULL_EZ)) {
|
|
|
+ // not an encrypted file
|
|
|
+ return null;
|
|
|
+ } else if(encryptionZone.getPath() == null
|
|
|
+ || encryptionZone.getPath().isEmpty()) {
|
|
|
+ if (NameNode.LOG.isDebugEnabled()) {
|
|
|
+ NameNode.LOG.debug("Encryption zone " +
|
|
|
+ encryptionZone.getPath() + " does not have a valid path.");
|
|
|
}
|
|
|
}
|
|
|
- return null;
|
|
|
+
|
|
|
+ CipherSuite suite = encryptionZone.getSuite();
|
|
|
+ String keyName = encryptionZone.getKeyName();
|
|
|
+
|
|
|
+ XAttr fileXAttr = unprotectedGetXAttrByName(inode, snapshotId,
|
|
|
+ CRYPTO_XATTR_FILE_ENCRYPTION_INFO);
|
|
|
+
|
|
|
+ if (fileXAttr == null) {
|
|
|
+ NameNode.LOG.warn("Could not find encryption XAttr for file " +
|
|
|
+ inode.getFullPathName() + " in encryption zone " +
|
|
|
+ encryptionZone.getPath());
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ HdfsProtos.PerFileEncryptionInfoProto fileProto =
|
|
|
+ HdfsProtos.PerFileEncryptionInfoProto.parseFrom(
|
|
|
+ fileXAttr.getValue());
|
|
|
+ return PBHelper.convert(fileProto, suite, keyName);
|
|
|
+ } catch (InvalidProtocolBufferException e) {
|
|
|
+ throw new IOException("Could not parse file encryption info for " +
|
|
|
+ "inode " + inode, e);
|
|
|
+ }
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
@@ -2860,7 +2908,11 @@ public class FSDirectory implements Closeable {
|
|
|
* of encryption zones.
|
|
|
*/
|
|
|
if (CRYPTO_XATTR_ENCRYPTION_ZONE.equals(xaName)) {
|
|
|
- ezManager.addEncryptionZone(inode.getId(), new String(xattr.getValue()));
|
|
|
+ final HdfsProtos.ZoneEncryptionInfoProto ezProto =
|
|
|
+ HdfsProtos.ZoneEncryptionInfoProto.parseFrom(xattr.getValue());
|
|
|
+ ezManager.addEncryptionZone(inode.getId(),
|
|
|
+ PBHelper.convert(ezProto.getSuite()),
|
|
|
+ ezProto.getKeyName());
|
|
|
}
|
|
|
|
|
|
if (!isFile && SECURITY_XATTR_UNREADABLE_BY_SUPERUSER.equals(xaName)) {
|
|
@@ -2977,6 +3029,22 @@ public class FSDirectory implements Closeable {
|
|
|
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;
|
|
|
+ }
|
|
|
+
|
|
|
private static INode resolveLastINode(String src, INodesInPath iip)
|
|
|
throws FileNotFoundException {
|
|
|
INode inode = iip.getLastINode();
|