HDFS-6388. HDFS integration with KeyProvider. (clamb)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1598783 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f7921030cd
commit
4054a40891
|
@ -6,6 +6,8 @@ fs-encryption (Unreleased)
|
|||
|
||||
NEW FEATURES
|
||||
|
||||
HDFS-6388. HDFS integration with KeyProvider. (clamb)
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
HADOOP-10603. Crypto input and output streams implementing Hadoop stream
|
||||
|
|
|
@ -27,6 +27,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.crypto.key.KeyProvider;
|
||||
import org.apache.hadoop.crypto.key.KeyProviderFactory;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Trash;
|
||||
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
||||
|
@ -270,6 +272,9 @@ public class NameNode implements NameNodeStatusMXBean {
|
|||
|
||||
private NameNodeRpcServer rpcServer;
|
||||
|
||||
/* The KeyProvider, if any. */
|
||||
private KeyProvider provider = null;
|
||||
|
||||
private JvmPauseMonitor pauseMonitor;
|
||||
private ObjectName nameNodeStatusBeanName;
|
||||
/**
|
||||
|
@ -581,6 +586,7 @@ public class NameNode implements NameNodeStatusMXBean {
|
|||
startHttpServer(conf);
|
||||
}
|
||||
loadNamesystem(conf);
|
||||
initializeKeyProvider(conf);
|
||||
|
||||
rpcServer = createRpcServer(conf);
|
||||
if (clientNamenodeAddress == null) {
|
||||
|
@ -699,6 +705,36 @@ public class NameNode implements NameNodeStatusMXBean {
|
|||
}
|
||||
}
|
||||
|
||||
private void initializeKeyProvider(final Configuration conf) {
|
||||
try {
|
||||
final List<KeyProvider> providers = KeyProviderFactory.getProviders(conf);
|
||||
if (providers == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (providers.size() == 0) {
|
||||
LOG.info("No KeyProviders found.");
|
||||
return;
|
||||
}
|
||||
|
||||
if (providers.size() > 1) {
|
||||
final String err =
|
||||
"Multiple KeyProviders found. Only one is permitted.";
|
||||
LOG.error(err);
|
||||
throw new RuntimeException(err);
|
||||
}
|
||||
provider = providers.get(0);
|
||||
if (provider.isTransient()) {
|
||||
final String err =
|
||||
"A KeyProvider was found but it is a transient provider.";
|
||||
LOG.error(err);
|
||||
throw new RuntimeException(err);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Exception while initializing KeyProvider", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Start NameNode.
|
||||
* <p>
|
||||
|
|
Loading…
Reference in New Issue