HDFS-6834. Improve the configuration guidance in DFSClient when there are no Codec classes found in configs. Contributed by Uma Maheswara Rao G.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1616721 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
041b8326a1
commit
1fa2d6c4ba
|
@ -76,6 +76,9 @@ fs-encryption (Unreleased)
|
|||
|
||||
HDFS-6394. HDFS encryption documentation. (wang)
|
||||
|
||||
HDFS-6834. Improve the configuration guidance in DFSClient when there
|
||||
are no Codec classes found in configs. (umamahesh)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs;
|
|||
import static org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
|
||||
import static org.apache.hadoop.crypto.key.KeyProviderCryptoExtension
|
||||
.EncryptedKeyVersion;
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY;
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT;
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
|
||||
|
@ -1338,8 +1340,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|||
KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
|
||||
CryptoCodec codec = CryptoCodec
|
||||
.getInstance(conf, feInfo.getCipherSuite());
|
||||
Preconditions.checkNotNull(codec == null,
|
||||
"No crypto codec classes with cipher suite configured.");
|
||||
if (codec == null) {
|
||||
throw new IOException("No configuration found for the cipher suite "
|
||||
+ feInfo.getCipherSuite().getConfigSuffix() + " prefixed with "
|
||||
+ HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
|
||||
+ ". Please see the example configuration "
|
||||
+ "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
|
||||
+ "at core-default.xml for details.");
|
||||
}
|
||||
final CryptoInputStream cryptoIn =
|
||||
new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
|
||||
feInfo.getIV());
|
||||
|
@ -1367,8 +1375,14 @@ 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.");
|
||||
if (codec == null) {
|
||||
throw new IOException("No configuration found for the cipher suite "
|
||||
+ HADOOP_SECURITY_CRYPTO_CIPHER_SUITE_KEY + " value prefixed with "
|
||||
+ HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
|
||||
+ ". Please see the example configuration "
|
||||
+ "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
|
||||
+ "at core-default.xml for details.");
|
||||
}
|
||||
// File is encrypted, wrap the stream in a crypto stream.
|
||||
KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
|
||||
final CryptoOutputStream cryptoOut =
|
||||
|
|
Loading…
Reference in New Issue