|
@@ -17,6 +17,8 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_ENCRYPTION_ZONE;
|
|
|
|
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
|
|
import static org.apache.hadoop.util.Time.now;
|
|
import static org.apache.hadoop.util.Time.now;
|
|
|
|
|
|
import java.io.Closeable;
|
|
import java.io.Closeable;
|
|
@@ -29,12 +31,13 @@ import java.util.List;
|
|
import java.util.ListIterator;
|
|
import java.util.ListIterator;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
|
|
+import com.google.protobuf.InvalidProtocolBufferException;
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
-import org.apache.hadoop.crypto.CryptoCodec;
|
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
|
|
+import org.apache.hadoop.fs.FileEncryptionInfo;
|
|
import org.apache.hadoop.fs.Options;
|
|
import org.apache.hadoop.fs.Options;
|
|
import org.apache.hadoop.fs.Options.Rename;
|
|
import org.apache.hadoop.fs.Options.Rename;
|
|
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
@@ -66,6 +69,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotException;
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotException;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
|
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
@@ -85,10 +90,6 @@ import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.collect.Lists;
|
|
import com.google.common.collect.Lists;
|
|
|
|
|
|
-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;
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* Both FSDirectory and FSNamesystem manage the state of the namespace.
|
|
* Both FSDirectory and FSNamesystem manage the state of the namespace.
|
|
* FSDirectory is a pure in-memory data structure, all of whose operations
|
|
* FSDirectory is a pure in-memory data structure, all of whose operations
|
|
@@ -133,7 +134,6 @@ public class FSDirectory implements Closeable {
|
|
private final INodeMap inodeMap; // Synchronized by dirLock
|
|
private final INodeMap inodeMap; // Synchronized by dirLock
|
|
private long yieldCount = 0; // keep track of lock yield count.
|
|
private long yieldCount = 0; // keep track of lock yield count.
|
|
private final int inodeXAttrsLimit; //inode xattrs max limit
|
|
private final int inodeXAttrsLimit; //inode xattrs max limit
|
|
- private final CryptoCodec codec;
|
|
|
|
|
|
|
|
// lock to protect the directory and BlockMap
|
|
// lock to protect the directory and BlockMap
|
|
private final ReentrantReadWriteLock dirLock;
|
|
private final ReentrantReadWriteLock dirLock;
|
|
@@ -200,7 +200,7 @@ public class FSDirectory implements Closeable {
|
|
this.inodeXAttrsLimit = conf.getInt(
|
|
this.inodeXAttrsLimit = conf.getInt(
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY,
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY,
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_DEFAULT);
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_DEFAULT);
|
|
- this.codec = CryptoCodec.getInstance(conf);
|
|
|
|
|
|
+
|
|
Preconditions.checkArgument(this.inodeXAttrsLimit >= 0,
|
|
Preconditions.checkArgument(this.inodeXAttrsLimit >= 0,
|
|
"Cannot set a negative limit on the number of xattrs per inode (%s).",
|
|
"Cannot set a negative limit on the number of xattrs per inode (%s).",
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY);
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY);
|
|
@@ -1470,8 +1470,8 @@ public class FSDirectory implements Closeable {
|
|
* @return object containing information regarding the file
|
|
* @return object containing information regarding the file
|
|
* or null if file not found
|
|
* or null if file not found
|
|
*/
|
|
*/
|
|
- HdfsFileStatus getFileInfo(String src, boolean resolveLink)
|
|
|
|
- throws UnresolvedLinkException {
|
|
|
|
|
|
+ HdfsFileStatus getFileInfo(String src, boolean resolveLink)
|
|
|
|
+ throws UnresolvedLinkException, IOException {
|
|
String srcs = normalizePath(src);
|
|
String srcs = normalizePath(src);
|
|
readLock();
|
|
readLock();
|
|
try {
|
|
try {
|
|
@@ -1480,6 +1480,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);
|
|
|
|
+
|
|
|
|
+ final int snapshotId = inodesInPath.getPathSnapshotId();
|
|
return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
|
|
return i == null? null: createFileStatus(HdfsFileStatus.EMPTY_NAME, i,
|
|
inodesInPath.getPathSnapshotId());
|
|
inodesInPath.getPathSnapshotId());
|
|
} finally {
|
|
} finally {
|
|
@@ -1498,7 +1500,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, null /* key */, null /* IV */);
|
|
|
|
|
|
+ HdfsFileStatus.EMPTY_NAME, -1L, 0, null);
|
|
}
|
|
}
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
@@ -2326,7 +2328,7 @@ public class FSDirectory implements Closeable {
|
|
* Create FileStatus by file INode
|
|
* Create FileStatus by file INode
|
|
*/
|
|
*/
|
|
HdfsFileStatus createFileStatus(byte[] path, INode node,
|
|
HdfsFileStatus createFileStatus(byte[] path, INode node,
|
|
- int snapshot) {
|
|
|
|
|
|
+ int snapshot) throws IOException {
|
|
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;
|
|
@@ -2338,7 +2340,9 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
int childrenNum = node.isDirectory() ?
|
|
int childrenNum = node.isDirectory() ?
|
|
node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ FileEncryptionInfo feInfo = getFileEncryptionInfo(node, snapshot);
|
|
|
|
+
|
|
return new HdfsFileStatus(
|
|
return new HdfsFileStatus(
|
|
size,
|
|
size,
|
|
node.isDirectory(),
|
|
node.isDirectory(),
|
|
@@ -2353,8 +2357,7 @@ public class FSDirectory implements Closeable {
|
|
path,
|
|
path,
|
|
node.getId(),
|
|
node.getId(),
|
|
childrenNum,
|
|
childrenNum,
|
|
- HdfsConstants.KEY, // key
|
|
|
|
- HdfsConstants.IV); // IV
|
|
|
|
|
|
+ feInfo);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -2377,16 +2380,20 @@ public class FSDirectory implements Closeable {
|
|
final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
|
|
final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
|
|
final long fileSize = !inSnapshot && isUc ?
|
|
final long fileSize = !inSnapshot && isUc ?
|
|
fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
|
|
fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
|
|
|
|
+ final FileEncryptionInfo feInfo = getFileEncryptionInfo(node, snapshot);
|
|
|
|
+
|
|
loc = getFSNamesystem().getBlockManager().createLocatedBlocks(
|
|
loc = getFSNamesystem().getBlockManager().createLocatedBlocks(
|
|
fileNode.getBlocks(), fileSize, isUc, 0L, size, false,
|
|
fileNode.getBlocks(), fileSize, isUc, 0L, size, false,
|
|
- inSnapshot);
|
|
|
|
|
|
+ inSnapshot, feInfo);
|
|
if (loc == null) {
|
|
if (loc == null) {
|
|
loc = new LocatedBlocks();
|
|
loc = new LocatedBlocks();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
int childrenNum = node.isDirectory() ?
|
|
int childrenNum = node.isDirectory() ?
|
|
node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
node.asDirectory().getChildrenNum(snapshot) : 0;
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ final FileEncryptionInfo feInfo = getFileEncryptionInfo(node, snapshot);
|
|
|
|
+
|
|
HdfsLocatedFileStatus status =
|
|
HdfsLocatedFileStatus status =
|
|
new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
|
|
new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
|
|
blocksize, node.getModificationTime(snapshot),
|
|
blocksize, node.getModificationTime(snapshot),
|
|
@@ -2394,7 +2401,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, null /* key */, null /* IV */);
|
|
|
|
|
|
+ node.getId(), loc, childrenNum, feInfo);
|
|
// 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();
|
|
@@ -2665,7 +2672,7 @@ public class FSDirectory implements Closeable {
|
|
"Attempt to create an encryption zone for a non-empty directory.");
|
|
"Attempt to create an encryption zone for a non-empty directory.");
|
|
}
|
|
}
|
|
final XAttr keyIdXAttr =
|
|
final XAttr keyIdXAttr =
|
|
- XAttrHelper.buildXAttr(CRYPTO_XATTR_KEY_ID, keyId.getBytes());
|
|
|
|
|
|
+ XAttrHelper.buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, keyId.getBytes());
|
|
List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
|
|
List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
|
|
xattrs.add(keyIdXAttr);
|
|
xattrs.add(keyIdXAttr);
|
|
unprotectedSetXAttrs(src, xattrs, EnumSet.of(XAttrSetFlag.CREATE));
|
|
unprotectedSetXAttrs(src, xattrs, EnumSet.of(XAttrSetFlag.CREATE));
|
|
@@ -2684,7 +2691,7 @@ public class FSDirectory implements Closeable {
|
|
"Attempt to delete an encryption zone for a non-empty directory.");
|
|
"Attempt to delete an encryption zone for a non-empty directory.");
|
|
}
|
|
}
|
|
final XAttr keyIdXAttr =
|
|
final XAttr keyIdXAttr =
|
|
- XAttrHelper.buildXAttr(CRYPTO_XATTR_KEY_ID, null);
|
|
|
|
|
|
+ XAttrHelper.buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, null);
|
|
List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
|
|
List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
|
|
xattrs.add(keyIdXAttr);
|
|
xattrs.add(keyIdXAttr);
|
|
final List<XAttr> removedXAttrs = unprotectedRemoveXAttrs(src, xattrs);
|
|
final List<XAttr> removedXAttrs = unprotectedRemoveXAttrs(src, xattrs);
|
|
@@ -2698,6 +2705,62 @@ public class FSDirectory implements Closeable {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Set the FileEncryptionInfo for an INode.
|
|
|
|
+ */
|
|
|
|
+ void setFileEncryptionInfo(String src, FileEncryptionInfo info)
|
|
|
|
+ throws IOException {
|
|
|
|
+ // Make the PB for the xattr
|
|
|
|
+ final HdfsProtos.FileEncryptionInfoProto proto = PBHelper.convert(info);
|
|
|
|
+ final byte[] protoBytes = proto.toByteArray();
|
|
|
|
+ final XAttr fileEncryptionAttr =
|
|
|
|
+ XAttrHelper.buildXAttr(CRYPTO_XATTR_FILE_ENCRYPTION_INFO, protoBytes);
|
|
|
|
+ final List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
|
|
|
|
+ xAttrs.add(fileEncryptionAttr);
|
|
|
|
+
|
|
|
|
+ writeLock();
|
|
|
|
+ try {
|
|
|
|
+ unprotectedSetXAttrs(src, xAttrs, EnumSet.of(XAttrSetFlag.CREATE));
|
|
|
|
+ } finally {
|
|
|
|
+ writeUnlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Return the FileEncryptionInfo for an INode, or null if the INode is not
|
|
|
|
+ * an encrypted file.
|
|
|
|
+ */
|
|
|
|
+ FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId)
|
|
|
|
+ throws IOException {
|
|
|
|
+ if (!inode.isFile()) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ readLock();
|
|
|
|
+ try {
|
|
|
|
+ List<XAttr> xAttrs = XAttrStorage.readINodeXAttrs(inode, snapshotId);
|
|
|
|
+ if (xAttrs == null) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ 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);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return null;
|
|
|
|
+ } finally {
|
|
|
|
+ readUnlock();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
void setXAttrs(final String src, final List<XAttr> xAttrs,
|
|
void setXAttrs(final String src, final List<XAttr> xAttrs,
|
|
final EnumSet<XAttrSetFlag> flag) throws IOException {
|
|
final EnumSet<XAttrSetFlag> flag) throws IOException {
|
|
writeLock();
|
|
writeLock();
|