Author: clamb
Date: Fri May 30 23:53:45 2014
New Revision: 1598783

URL: http://svn.apache.org/r1598783
Log:
HDFS-6388. HDFS integration with KeyProvider. (clamb)

Modified:
    
hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

Modified: 
hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1598783&r1=1598782&r2=1598783&view=diff
==============================================================================
--- 
hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
 (original)
+++ 
hadoop/common/branches/fs-encryption/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
 Fri May 30 23:53:45 2014
@@ -27,6 +27,8 @@ import org.apache.commons.logging.LogFac
 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 NameNod
   
   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 NameNod
       startHttpServer(conf);
     }
     loadNamesystem(conf);
+    initializeKeyProvider(conf);
 
     rpcServer = createRpcServer(conf);
     if (clientNamenodeAddress == null) {
@@ -699,6 +705,36 @@ public class NameNode implements NameNod
     }
   }
 
+  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>


Reply via email to