Kaynağa Gözat

HADOOP-10886. CryptoCodec#getCodecclasses throws NPE when configurations not loaded. Contributed by Uma Maheswara Rao G.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1615523 13f79535-47bb-0310-9956-ffa450edef68
Uma Maheswara Rao G 10 yıl önce
ebeveyn
işleme
10db613389

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES-fs-encryption.txt

@@ -51,3 +51,6 @@ fs-encryption (Unreleased)
   BUG FIXES
   BUG FIXES
 
 
     HADOOP-10871. incorrect prototype in OpensslSecureRandom.c (cmccabe)
     HADOOP-10871. incorrect prototype in OpensslSecureRandom.c (cmccabe)
+
+    HADOOP-10886. CryptoCodec#getCodecclasses throws NPE when configurations not 
+    loaded. (umamahesh)

+ 20 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java

@@ -45,14 +45,21 @@ public abstract class CryptoCodec implements Configurable {
   
   
   /**
   /**
    * Get crypto codec for specified algorithm/mode/padding.
    * Get crypto codec for specified algorithm/mode/padding.
-   * @param conf the configuration
-   * @param CipherSuite algorithm/mode/padding
-   * @return CryptoCodec the codec object
+   * 
+   * @param conf
+   *          the configuration
+   * @param CipherSuite
+   *          algorithm/mode/padding
+   * @return CryptoCodec the codec object. Null value will be returned if no
+   *         crypto codec classes with cipher suite configured.
    */
    */
   public static CryptoCodec getInstance(Configuration conf, 
   public static CryptoCodec getInstance(Configuration conf, 
       CipherSuite cipherSuite) {
       CipherSuite cipherSuite) {
     List<Class<? extends CryptoCodec>> klasses = getCodecClasses(
     List<Class<? extends CryptoCodec>> klasses = getCodecClasses(
         conf, cipherSuite);
         conf, cipherSuite);
+    if (klasses == null) {
+      return null;
+    }
     CryptoCodec codec = null;
     CryptoCodec codec = null;
     for (Class<? extends CryptoCodec> klass : klasses) {
     for (Class<? extends CryptoCodec> klass : klasses) {
       try {
       try {
@@ -80,10 +87,13 @@ public abstract class CryptoCodec implements Configurable {
   }
   }
   
   
   /**
   /**
-   * Get crypto codec for algorithm/mode/padding in config value 
+   * Get crypto codec for algorithm/mode/padding in config value
    * hadoop.security.crypto.cipher.suite
    * hadoop.security.crypto.cipher.suite
-   * @param conf the configuration
-   * @return CryptoCodec the codec object
+   * 
+   * @param conf
+   *          the configuration
+   * @return CryptoCodec the codec object Null value will be returned if no
+   *         crypto codec classes with cipher suite configured.
    */
    */
   public static CryptoCodec getInstance(Configuration conf) {
   public static CryptoCodec getInstance(Configuration conf) {
     String name = conf.get(HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY, 
     String name = conf.get(HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY, 
@@ -97,6 +107,10 @@ public abstract class CryptoCodec implements Configurable {
     String configName = HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX + 
     String configName = HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX + 
         cipherSuite.getConfigSuffix();
         cipherSuite.getConfigSuffix();
     String codecString = conf.get(configName);
     String codecString = conf.get(configName);
+    if (codecString == null) {
+      LOG.warn("No crypto codec classes with cipher suite configured.");
+      return null;
+    }
     for (String c : Splitter.on(',').trimResults().omitEmptyStrings().
     for (String c : Splitter.on(',').trimResults().omitEmptyStrings().
         split(codecString)) {
         split(codecString)) {
       try {
       try {

+ 6 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java

@@ -25,6 +25,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -50,6 +51,11 @@ public class TestCryptoStreamsForLocalFS extends CryptoStreamsTestBase {
     conf = new Configuration(false);
     conf = new Configuration(false);
     conf.set("fs.file.impl", LocalFileSystem.class.getName());
     conf.set("fs.file.impl", LocalFileSystem.class.getName());
     fileSys = FileSystem.getLocal(conf);
     fileSys = FileSystem.getLocal(conf);
+    conf.set(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
+            + CipherSuite.AES_CTR_NOPADDING.getConfigSuffix(),
+        OpensslAesCtrCryptoCodec.class.getName() + ","
+            + JceAesCtrCryptoCodec.class.getName());
     codec = CryptoCodec.getInstance(conf);
     codec = CryptoCodec.getInstance(conf);
   }
   }
   
   

+ 10 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -598,7 +598,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
         DFSUtil.getRandom().nextInt()  + "_" + Thread.currentThread().getId();
         DFSUtil.getRandom().nextInt()  + "_" + Thread.currentThread().getId();
     this.codec = CryptoCodec.getInstance(conf);
     this.codec = CryptoCodec.getInstance(conf);
     this.cipherSuites = Lists.newArrayListWithCapacity(1);
     this.cipherSuites = Lists.newArrayListWithCapacity(1);
-    cipherSuites.add(codec.getCipherSuite());
+    if (codec != null) {
+      cipherSuites.add(codec.getCipherSuite());
+    }
     provider = DFSUtil.createKeyProviderCryptoExtension(conf);
     provider = DFSUtil.createKeyProviderCryptoExtension(conf);
     if (provider == null) {
     if (provider == null) {
       LOG.info("No KeyProvider found.");
       LOG.info("No KeyProvider found.");
@@ -1333,9 +1335,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if (feInfo != null) {
     if (feInfo != null) {
       // File is encrypted, wrap the stream in a crypto stream.
       // File is encrypted, wrap the stream in a crypto stream.
       KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
       KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
+      CryptoCodec codec = CryptoCodec
+          .getInstance(conf, feInfo.getCipherSuite());
+      Preconditions.checkNotNull(codec == null,
+          "No crypto codec classes with cipher suite configured.");
       final CryptoInputStream cryptoIn =
       final CryptoInputStream cryptoIn =
-          new CryptoInputStream(dfsis, CryptoCodec.getInstance(conf, 
-              feInfo.getCipherSuite()), decrypted.getMaterial(),
+          new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
               feInfo.getIV());
               feInfo.getIV());
       return new HdfsDataInputStream(cryptoIn);
       return new HdfsDataInputStream(cryptoIn);
     } else {
     } else {
@@ -1361,6 +1366,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       FileSystem.Statistics statistics, long startPos) throws IOException {
       FileSystem.Statistics statistics, long startPos) throws IOException {
     final FileEncryptionInfo feInfo = dfsos.getFileEncryptionInfo();
     final FileEncryptionInfo feInfo = dfsos.getFileEncryptionInfo();
     if (feInfo != null) {
     if (feInfo != null) {
+      Preconditions.checkNotNull(codec == null,
+          "No crypto codec classes with cipher suite configured.");
       // File is encrypted, wrap the stream in a crypto stream.
       // File is encrypted, wrap the stream in a crypto stream.
       KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
       KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
       final CryptoOutputStream cryptoOut =
       final CryptoOutputStream cryptoOut =

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -38,7 +38,6 @@ import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.List;
 import java.util.Random;
 import java.util.Random;
-import java.util.concurrent.CancellationException;
 
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.commons.logging.impl.Log4JLogger;