|
@@ -21,7 +21,9 @@ import org.apache.commons.codec.binary.Base64;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.crypto.key.KeyProvider;
|
|
import org.apache.hadoop.crypto.key.KeyProvider;
|
|
|
|
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
|
|
import org.apache.hadoop.crypto.key.KeyProviderFactory;
|
|
import org.apache.hadoop.crypto.key.KeyProviderFactory;
|
|
|
|
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.security.ProviderUtils;
|
|
import org.apache.hadoop.security.ProviderUtils;
|
|
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
|
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
|
@@ -33,6 +35,7 @@ import org.apache.http.client.utils.URIBuilder;
|
|
import org.codehaus.jackson.map.ObjectMapper;
|
|
import org.codehaus.jackson.map.ObjectMapper;
|
|
|
|
|
|
import javax.net.ssl.HttpsURLConnection;
|
|
import javax.net.ssl.HttpsURLConnection;
|
|
|
|
+
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
import java.io.OutputStream;
|
|
import java.io.OutputStream;
|
|
@@ -40,6 +43,7 @@ import java.io.OutputStreamWriter;
|
|
import java.io.Writer;
|
|
import java.io.Writer;
|
|
import java.lang.reflect.Constructor;
|
|
import java.lang.reflect.Constructor;
|
|
import java.net.HttpURLConnection;
|
|
import java.net.HttpURLConnection;
|
|
|
|
+import java.net.SocketTimeoutException;
|
|
import java.net.URI;
|
|
import java.net.URI;
|
|
import java.net.URISyntaxException;
|
|
import java.net.URISyntaxException;
|
|
import java.net.URL;
|
|
import java.net.URL;
|
|
@@ -50,14 +54,22 @@ import java.text.MessageFormat;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Date;
|
|
import java.util.Date;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
|
|
+import java.util.LinkedList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
+import java.util.Queue;
|
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
|
+
|
|
|
|
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
|
|
|
|
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
|
|
|
|
+
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
|
|
|
/**
|
|
/**
|
|
* KMS client <code>KeyProvider</code> implementation.
|
|
* KMS client <code>KeyProvider</code> implementation.
|
|
*/
|
|
*/
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
-public class KMSClientProvider extends KeyProvider {
|
|
|
|
|
|
+public class KMSClientProvider extends KeyProvider implements CryptoExtension {
|
|
|
|
|
|
public static final String SCHEME_NAME = "kms";
|
|
public static final String SCHEME_NAME = "kms";
|
|
|
|
|
|
@@ -78,6 +90,73 @@ public class KMSClientProvider extends KeyProvider {
|
|
public static final String TIMEOUT_ATTR = CONFIG_PREFIX + "timeout";
|
|
public static final String TIMEOUT_ATTR = CONFIG_PREFIX + "timeout";
|
|
public static final int DEFAULT_TIMEOUT = 60;
|
|
public static final int DEFAULT_TIMEOUT = 60;
|
|
|
|
|
|
|
|
+ private final ValueQueue<EncryptedKeyVersion> encKeyVersionQueue;
|
|
|
|
+
|
|
|
|
+ private class EncryptedQueueRefiller implements
|
|
|
|
+ ValueQueue.QueueRefiller<EncryptedKeyVersion> {
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void fillQueueForKey(String keyName,
|
|
|
|
+ Queue<EncryptedKeyVersion> keyQueue, int numEKVs) throws IOException {
|
|
|
|
+ checkNotNull(keyName, "keyName");
|
|
|
|
+ Map<String, String> params = new HashMap<String, String>();
|
|
|
|
+ params.put(KMSRESTConstants.EEK_OP, KMSRESTConstants.EEK_GENERATE);
|
|
|
|
+ params.put(KMSRESTConstants.EEK_NUM_KEYS, "" + numEKVs);
|
|
|
|
+ URL url = createURL(KMSRESTConstants.KEY_RESOURCE, keyName,
|
|
|
|
+ KMSRESTConstants.EEK_SUB_RESOURCE, params);
|
|
|
|
+ HttpURLConnection conn = createConnection(url, HTTP_GET);
|
|
|
|
+ conn.setRequestProperty(CONTENT_TYPE, APPLICATION_JSON_MIME);
|
|
|
|
+ List response = call(conn, null,
|
|
|
|
+ HttpURLConnection.HTTP_OK, List.class);
|
|
|
|
+ List<EncryptedKeyVersion> ekvs =
|
|
|
|
+ parseJSONEncKeyVersion(keyName, response);
|
|
|
|
+ keyQueue.addAll(ekvs);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static class KMSEncryptedKeyVersion extends EncryptedKeyVersion {
|
|
|
|
+ public KMSEncryptedKeyVersion(String keyName, String keyVersionName,
|
|
|
|
+ byte[] iv, String encryptedVersionName, byte[] keyMaterial) {
|
|
|
|
+ super(keyName, keyVersionName, iv, new KMSKeyVersion(null,
|
|
|
|
+ encryptedVersionName, keyMaterial));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @SuppressWarnings("rawtypes")
|
|
|
|
+ private static List<EncryptedKeyVersion>
|
|
|
|
+ parseJSONEncKeyVersion(String keyName, List valueList) {
|
|
|
|
+ List<EncryptedKeyVersion> ekvs = new LinkedList<EncryptedKeyVersion>();
|
|
|
|
+ if (!valueList.isEmpty()) {
|
|
|
|
+ for (Object values : valueList) {
|
|
|
|
+ Map valueMap = (Map) values;
|
|
|
|
+
|
|
|
|
+ String versionName = checkNotNull(
|
|
|
|
+ (String) valueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
|
|
|
|
+ KMSRESTConstants.VERSION_NAME_FIELD);
|
|
|
|
+
|
|
|
|
+ byte[] iv = Base64.decodeBase64(checkNotNull(
|
|
|
|
+ (String) valueMap.get(KMSRESTConstants.IV_FIELD),
|
|
|
|
+ KMSRESTConstants.IV_FIELD));
|
|
|
|
+
|
|
|
|
+ Map encValueMap = checkNotNull((Map)
|
|
|
|
+ valueMap.get(KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD),
|
|
|
|
+ KMSRESTConstants.ENCRYPTED_KEY_VERSION_FIELD);
|
|
|
|
+
|
|
|
|
+ String encVersionName = checkNotNull((String)
|
|
|
|
+ encValueMap.get(KMSRESTConstants.VERSION_NAME_FIELD),
|
|
|
|
+ KMSRESTConstants.VERSION_NAME_FIELD);
|
|
|
|
+
|
|
|
|
+ byte[] encKeyMaterial = Base64.decodeBase64(checkNotNull((String)
|
|
|
|
+ encValueMap.get(KMSRESTConstants.MATERIAL_FIELD),
|
|
|
|
+ KMSRESTConstants.MATERIAL_FIELD));
|
|
|
|
+
|
|
|
|
+ ekvs.add(new KMSEncryptedKeyVersion(keyName, versionName, iv,
|
|
|
|
+ encVersionName, encKeyMaterial));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return ekvs;
|
|
|
|
+ }
|
|
|
|
+
|
|
private static KeyVersion parseJSONKeyVersion(Map valueMap) {
|
|
private static KeyVersion parseJSONKeyVersion(Map valueMap) {
|
|
KeyVersion keyVersion = null;
|
|
KeyVersion keyVersion = null;
|
|
if (!valueMap.isEmpty()) {
|
|
if (!valueMap.isEmpty()) {
|
|
@@ -208,6 +287,28 @@ public class KMSClientProvider extends KeyProvider {
|
|
}
|
|
}
|
|
int timeout = conf.getInt(TIMEOUT_ATTR, DEFAULT_TIMEOUT);
|
|
int timeout = conf.getInt(TIMEOUT_ATTR, DEFAULT_TIMEOUT);
|
|
configurator = new TimeoutConnConfigurator(timeout, sslFactory);
|
|
configurator = new TimeoutConnConfigurator(timeout, sslFactory);
|
|
|
|
+ encKeyVersionQueue =
|
|
|
|
+ new ValueQueue<KeyProviderCryptoExtension.EncryptedKeyVersion>(
|
|
|
|
+ conf.getInt(
|
|
|
|
+ CommonConfigurationKeysPublic.KMS_CLIENT_ENC_KEY_CACHE_SIZE,
|
|
|
|
+ CommonConfigurationKeysPublic.
|
|
|
|
+ KMS_CLIENT_ENC_KEY_CACHE_SIZE_DEFAULT),
|
|
|
|
+ conf.getFloat(
|
|
|
|
+ CommonConfigurationKeysPublic.
|
|
|
|
+ KMS_CLIENT_ENC_KEY_CACHE_LOW_WATERMARK,
|
|
|
|
+ CommonConfigurationKeysPublic.
|
|
|
|
+ KMS_CLIENT_ENC_KEY_CACHE_LOW_WATERMARK_DEFAULT),
|
|
|
|
+ conf.getInt(
|
|
|
|
+ CommonConfigurationKeysPublic.
|
|
|
|
+ KMS_CLIENT_ENC_KEY_CACHE_EXPIRY_MS,
|
|
|
|
+ CommonConfigurationKeysPublic.
|
|
|
|
+ KMS_CLIENT_ENC_KEY_CACHE_EXPIRY_DEFAULT),
|
|
|
|
+ conf.getInt(
|
|
|
|
+ CommonConfigurationKeysPublic.
|
|
|
|
+ KMS_CLIENT_ENC_KEY_CACHE_NUM_REFILL_THREADS,
|
|
|
|
+ CommonConfigurationKeysPublic.
|
|
|
|
+ KMS_CLIENT_ENC_KEY_CACHE_NUM_REFILL_THREADS_DEFAULT),
|
|
|
|
+ new EncryptedQueueRefiller());
|
|
}
|
|
}
|
|
|
|
|
|
private String createServiceURL(URL url) throws IOException {
|
|
private String createServiceURL(URL url) throws IOException {
|
|
@@ -527,6 +628,51 @@ public class KMSClientProvider extends KeyProvider {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public EncryptedKeyVersion generateEncryptedKey(
|
|
|
|
+ String encryptionKeyName) throws IOException, GeneralSecurityException {
|
|
|
|
+ try {
|
|
|
|
+ return encKeyVersionQueue.getNext(encryptionKeyName);
|
|
|
|
+ } catch (ExecutionException e) {
|
|
|
|
+ if (e.getCause() instanceof SocketTimeoutException) {
|
|
|
|
+ throw (SocketTimeoutException)e.getCause();
|
|
|
|
+ }
|
|
|
|
+ throw new IOException(e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @SuppressWarnings("rawtypes")
|
|
|
|
+ @Override
|
|
|
|
+ public KeyVersion decryptEncryptedKey(
|
|
|
|
+ EncryptedKeyVersion encryptedKeyVersion) throws IOException,
|
|
|
|
+ GeneralSecurityException {
|
|
|
|
+ checkNotNull(encryptedKeyVersion.getKeyVersionName(), "versionName");
|
|
|
|
+ checkNotNull(encryptedKeyVersion.getIv(), "iv");
|
|
|
|
+ Preconditions.checkArgument(encryptedKeyVersion.getEncryptedKey()
|
|
|
|
+ .getVersionName().equals(KeyProviderCryptoExtension.EEK),
|
|
|
|
+ "encryptedKey version name must be '%s', is '%s'",
|
|
|
|
+ KeyProviderCryptoExtension.EK, encryptedKeyVersion.getEncryptedKey()
|
|
|
|
+ .getVersionName());
|
|
|
|
+ checkNotNull(encryptedKeyVersion.getEncryptedKey(), "encryptedKey");
|
|
|
|
+ Map<String, String> params = new HashMap<String, String>();
|
|
|
|
+ params.put(KMSRESTConstants.EEK_OP, KMSRESTConstants.EEK_DECRYPT);
|
|
|
|
+ Map<String, Object> jsonPayload = new HashMap<String, Object>();
|
|
|
|
+ jsonPayload.put(KMSRESTConstants.NAME_FIELD,
|
|
|
|
+ encryptedKeyVersion.getKeyName());
|
|
|
|
+ jsonPayload.put(KMSRESTConstants.IV_FIELD, Base64.encodeBase64String(
|
|
|
|
+ encryptedKeyVersion.getIv()));
|
|
|
|
+ jsonPayload.put(KMSRESTConstants.MATERIAL_FIELD, Base64.encodeBase64String(
|
|
|
|
+ encryptedKeyVersion.getEncryptedKey().getMaterial()));
|
|
|
|
+ URL url = createURL(KMSRESTConstants.KEY_VERSION_RESOURCE,
|
|
|
|
+ encryptedKeyVersion.getKeyVersionName(),
|
|
|
|
+ KMSRESTConstants.EEK_SUB_RESOURCE, params);
|
|
|
|
+ HttpURLConnection conn = createConnection(url, HTTP_POST);
|
|
|
|
+ conn.setRequestProperty(CONTENT_TYPE, APPLICATION_JSON_MIME);
|
|
|
|
+ Map response =
|
|
|
|
+ call(conn, jsonPayload, HttpURLConnection.HTTP_OK, Map.class);
|
|
|
|
+ return parseJSONKeyVersion(response);
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public List<KeyVersion> getKeyVersions(String name) throws IOException {
|
|
public List<KeyVersion> getKeyVersions(String name) throws IOException {
|
|
checkNotEmpty(name, "name");
|
|
checkNotEmpty(name, "name");
|
|
@@ -570,4 +716,14 @@ public class KMSClientProvider extends KeyProvider {
|
|
// the server should not keep in memory state on behalf of clients either.
|
|
// the server should not keep in memory state on behalf of clients either.
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public void warmUpEncryptedKeys(String... keyNames)
|
|
|
|
+ throws IOException {
|
|
|
|
+ try {
|
|
|
|
+ encKeyVersionQueue.initializeQueuesForKeys(keyNames);
|
|
|
|
+ } catch (ExecutionException e) {
|
|
|
|
+ throw new IOException(e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|