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
This commit is contained in:
Uma Maheswara Rao G 2014-08-04 09:56:57 +00:00
parent 70c99278a9
commit 10db613389
5 changed files with 39 additions and 10 deletions

View File

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

View File

@ -45,14 +45,21 @@ public abstract class CryptoCodec implements Configurable {
/**
* 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,
CipherSuite cipherSuite) {
List<Class<? extends CryptoCodec>> klasses = getCodecClasses(
conf, cipherSuite);
if (klasses == null) {
return null;
}
CryptoCodec codec = null;
for (Class<? extends CryptoCodec> klass : klasses) {
try {
@ -82,8 +89,11 @@ public abstract class CryptoCodec implements Configurable {
/**
* Get crypto codec for algorithm/mode/padding in config value
* 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) {
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 +
cipherSuite.getConfigSuffix();
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().
split(codecString)) {
try {

View File

@ -25,6 +25,7 @@ import java.io.InputStream;
import java.io.OutputStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.LocalFileSystem;
@ -50,6 +51,11 @@ public class TestCryptoStreamsForLocalFS extends CryptoStreamsTestBase {
conf = new Configuration(false);
conf.set("fs.file.impl", LocalFileSystem.class.getName());
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);
}

View File

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

View File

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