|
@@ -17,6 +17,9 @@
|
|
|
*/
|
|
|
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.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;
|
|
@@ -76,6 +79,7 @@ import java.net.Socket;
|
|
|
import java.net.SocketAddress;
|
|
|
import java.net.URI;
|
|
|
import java.net.UnknownHostException;
|
|
|
+import java.security.GeneralSecurityException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
import java.util.EnumSet;
|
|
@@ -100,6 +104,7 @@ import org.apache.hadoop.crypto.CipherSuite;
|
|
|
import org.apache.hadoop.crypto.CryptoCodec;
|
|
|
import org.apache.hadoop.crypto.CryptoInputStream;
|
|
|
import org.apache.hadoop.crypto.CryptoOutputStream;
|
|
|
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
|
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
|
import org.apache.hadoop.fs.BlockStorageLocation;
|
|
|
import org.apache.hadoop.fs.CacheFlag;
|
|
@@ -256,7 +261,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
|
private final CryptoCodec codec;
|
|
|
@VisibleForTesting
|
|
|
List<CipherSuite> cipherSuites;
|
|
|
-
|
|
|
+ @VisibleForTesting
|
|
|
+ KeyProviderCryptoExtension provider;
|
|
|
/**
|
|
|
* DFSClient configuration
|
|
|
*/
|
|
@@ -591,7 +597,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
|
this.codec = CryptoCodec.getInstance(conf);
|
|
|
this.cipherSuites = Lists.newArrayListWithCapacity(1);
|
|
|
cipherSuites.add(codec.getCipherSuite());
|
|
|
-
|
|
|
+ provider = DFSUtil.createKeyProviderCryptoExtension(conf);
|
|
|
+ if (provider == null) {
|
|
|
+ LOG.info("No KeyProvider found.");
|
|
|
+ } else {
|
|
|
+ LOG.info("Found KeyProvider: " + provider.toString());
|
|
|
+ }
|
|
|
int numResponseToDrop = conf.getInt(
|
|
|
DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
|
|
|
DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
|
|
@@ -1291,6 +1302,25 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
|
return volumeBlockLocations;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Decrypts a EDEK by consulting the KeyProvider.
|
|
|
+ */
|
|
|
+ private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
|
|
|
+ feInfo) throws IOException {
|
|
|
+ if (provider == null) {
|
|
|
+ throw new IOException("No KeyProvider is configured, cannot access" +
|
|
|
+ " an encrypted file");
|
|
|
+ }
|
|
|
+ EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
|
|
|
+ feInfo.getEzKeyVersionName(), feInfo.getIV(),
|
|
|
+ feInfo.getEncryptedDataEncryptionKey());
|
|
|
+ try {
|
|
|
+ return provider.decryptEncryptedKey(ekv);
|
|
|
+ } catch (GeneralSecurityException e) {
|
|
|
+ throw new IOException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Wraps the stream in a CryptoInputStream if the underlying file is
|
|
|
* encrypted.
|
|
@@ -1300,13 +1330,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
|
final FileEncryptionInfo feInfo = dfsis.getFileEncryptionInfo();
|
|
|
if (feInfo != null) {
|
|
|
// File is encrypted, wrap the stream in a crypto stream.
|
|
|
+ KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
|
|
|
final CryptoInputStream cryptoIn =
|
|
|
new CryptoInputStream(dfsis, CryptoCodec.getInstance(conf,
|
|
|
- feInfo.getCipherSuite()), feInfo.getEncryptedDataEncryptionKey(),
|
|
|
+ feInfo.getCipherSuite()), decrypted.getMaterial(),
|
|
|
feInfo.getIV());
|
|
|
return new HdfsDataInputStream(cryptoIn);
|
|
|
} else {
|
|
|
- // No key/IV pair so no encryption.
|
|
|
+ // No FileEncryptionInfo so no encryption.
|
|
|
return new HdfsDataInputStream(dfsis);
|
|
|
}
|
|
|
}
|
|
@@ -1329,12 +1360,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
|
|
|
final FileEncryptionInfo feInfo = dfsos.getFileEncryptionInfo();
|
|
|
if (feInfo != null) {
|
|
|
// File is encrypted, wrap the stream in a crypto stream.
|
|
|
+ KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
|
|
|
final CryptoOutputStream cryptoOut =
|
|
|
new CryptoOutputStream(dfsos, codec,
|
|
|
- feInfo.getEncryptedDataEncryptionKey(), feInfo.getIV(), startPos);
|
|
|
+ decrypted.getMaterial(), feInfo.getIV(), startPos);
|
|
|
return new HdfsDataOutputStream(cryptoOut, statistics, startPos);
|
|
|
} else {
|
|
|
- // No key/IV present so no encryption.
|
|
|
+ // No FileEncryptionInfo present so no encryption.
|
|
|
return new HdfsDataOutputStream(dfsos, statistics, startPos);
|
|
|
}
|
|
|
}
|