diff --git a/hadoop-common-project/hadoop-common/CHANGES-fs-encryption.txt b/hadoop-common-project/hadoop-common/CHANGES-fs-encryption.txt index c8dc9d4670..82f68d7516 100644 --- a/hadoop-common-project/hadoop-common/CHANGES-fs-encryption.txt +++ b/hadoop-common-project/hadoop-common/CHANGES-fs-encryption.txt @@ -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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index 9cdad26eb2..504e3ed201 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -27,6 +27,8 @@ 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 long getProtocolVersion(String protocol, private NameNodeRpcServer rpcServer; + /* The KeyProvider, if any. */ + private KeyProvider provider = null; + private JvmPauseMonitor pauseMonitor; private ObjectName nameNodeStatusBeanName; /** @@ -581,6 +586,7 @@ protected void initialize(Configuration conf) throws IOException { startHttpServer(conf); } loadNamesystem(conf); + initializeKeyProvider(conf); rpcServer = createRpcServer(conf); if (clientNamenodeAddress == null) { @@ -699,6 +705,36 @@ private void stopHttpServer() { } } + private void initializeKeyProvider(final Configuration conf) { + try { + final List 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. *