HDFS-7004. Update KeyProvider instantiation to create by URI. (wang)

This commit is contained in:
Andrew Wang 2014-09-17 20:14:40 -07:00
parent bf38793ce1
commit 10e8602f32
15 changed files with 60 additions and 40 deletions

View File

@ -16,7 +16,7 @@
<!-- KMS Backend KeyProvider -->
<property>
<name>hadoop.security.key.provider.path</name>
<name>hadoop.kms.key.provider.uri</name>
<value>jceks://file@/${user.home}/kms.keystore</value>
<description>
</description>

View File

@ -40,6 +40,10 @@ public class KMSConfiguration {
public static final String KEY_ACL_PREFIX = "key.acl.";
public static final String DEFAULT_KEY_ACL_PREFIX = "default.key.acl.";
// Property to set the backing KeyProvider
public static final String KEY_PROVIDER_URI = CONFIG_PREFIX +
"key.provider.uri";
// Property to Enable/Disable Caching
public static final String KEY_CACHE_ENABLE = CONFIG_PREFIX +
"cache.enable";

View File

@ -39,6 +39,7 @@ import javax.servlet.ServletContextEvent;
import javax.servlet.ServletContextListener;
import java.io.File;
import java.net.URI;
import java.net.URL;
import java.util.List;
@ -159,17 +160,12 @@ public class KMSWebApp implements ServletContextListener {
new AccessControlList(AccessControlList.WILDCARD_ACL_VALUE));
// intializing the KeyProvider
List<KeyProvider> providers = KeyProviderFactory.getProviders(kmsConf);
if (providers.isEmpty()) {
String providerString = kmsConf.get(KMSConfiguration.KEY_PROVIDER_URI);
if (providerString == null) {
throw new IllegalStateException("No KeyProvider has been defined");
}
if (providers.size() > 1) {
LOG.warn("There is more than one KeyProvider configured '{}', using " +
"the first provider",
kmsConf.get(KeyProviderFactory.KEY_PROVIDER_PATH));
}
KeyProvider keyProvider = providers.get(0);
KeyProvider keyProvider =
KeyProviderFactory.get(new URI(providerString), kmsConf);
if (kmsConf.getBoolean(KMSConfiguration.KEY_CACHE_ENABLE,
KMSConfiguration.KEY_CACHE_ENABLE_DEFAULT)) {
long keyTimeOutMillis =

View File

@ -51,7 +51,7 @@ Hadoop Key Management Server (KMS) - Documentation Sets ${project.version}
+---+
<property>
<name>hadoop.security.key.provider.path</name>
<name>hadoop.kms.key.provider.uri</name>
<value>jceks://file@/${user.home}/kms.keystore</value>
</property>

View File

@ -166,7 +166,7 @@ public class MiniKMS {
File kmsFile = new File(kmsConfDir, "kms-site.xml");
if (!kmsFile.exists()) {
Configuration kms = new Configuration(false);
kms.set("hadoop.security.key.provider.path",
kms.set(KMSConfiguration.KEY_PROVIDER_URI,
"jceks://file@" + new Path(kmsConfDir, "kms.keystore").toUri());
kms.set("hadoop.kms.authentication.type", "simple");
kms.setBoolean(KMSConfiguration.KEY_AUTHORIZATION_ENABLE, false);

View File

@ -117,7 +117,7 @@ public class TestKMS {
protected Configuration createBaseKMSConf(File keyStoreDir) throws Exception {
Configuration conf = new Configuration(false);
conf.set("hadoop.security.key.provider.path",
conf.set(KMSConfiguration.KEY_PROVIDER_URI,
"jceks://file@" + new Path(keyStoreDir.getAbsolutePath(), "kms.keystore").toUri());
conf.set("hadoop.kms.authentication.type", "simple");
return conf;

View File

@ -473,6 +473,8 @@ Release 2.6.0 - UNRELEASED
HDFS-6843. Create FileStatus isEncrypted() method (clamb via cmccabe)
HDFS-7004. Update KeyProvider instantiation to create by URI. (wang)
OPTIMIZATIONS
HDFS-6690. Deduplicate xattr names in memory. (wang)

View File

@ -580,6 +580,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY = "dfs.data.transfer.saslproperties.resolver.class";
public static final int DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES_DEFAULT = 100;
public static final String DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES = "dfs.namenode.list.encryption.zones.num.responses";
public static final String DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri";
// Journal-node related configs. These are read on the JN side.
public static final String DFS_JOURNALNODE_EDITS_DIR_KEY = "dfs.journalnode.edits.dir";

View File

@ -1794,34 +1794,37 @@ public class DFSUtil {
* Creates a new KeyProviderCryptoExtension by wrapping the
* KeyProvider specified in the given Configuration.
*
* @param conf Configuration specifying a single, non-transient KeyProvider.
* @param conf Configuration
* @return new KeyProviderCryptoExtension, or null if no provider was found.
* @throws IOException if the KeyProvider is improperly specified in
* the Configuration
*/
public static KeyProviderCryptoExtension createKeyProviderCryptoExtension(
final Configuration conf) throws IOException {
final List<KeyProvider> providers = KeyProviderFactory.getProviders(conf);
if (providers == null || providers.size() == 0) {
final String providerUriStr =
conf.get(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, null);
// No provider set in conf
if (providerUriStr == null) {
return null;
}
if (providers.size() > 1) {
StringBuilder builder = new StringBuilder();
builder.append("Found multiple KeyProviders but only one is permitted [");
String prefix = " ";
for (KeyProvider kp: providers) {
builder.append(prefix + kp.toString());
prefix = ", ";
}
builder.append("]");
throw new IOException(builder.toString());
final URI providerUri;
try {
providerUri = new URI(providerUriStr);
} catch (URISyntaxException e) {
throw new IOException(e);
}
KeyProviderCryptoExtension provider = KeyProviderCryptoExtension
.createKeyProviderCryptoExtension(providers.get(0));
if (provider.isTransient()) {
throw new IOException("KeyProvider " + provider.toString()
KeyProvider keyProvider = KeyProviderFactory.get(providerUri, conf);
if (keyProvider == null) {
throw new IOException("Could not instantiate KeyProvider from " +
DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI + " setting of '" +
providerUriStr +"'");
}
if (keyProvider.isTransient()) {
throw new IOException("KeyProvider " + keyProvider.toString()
+ " was found but it is a transient provider.");
}
return provider;
KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
.createKeyProviderCryptoExtension(keyProvider);
return cryptoProvider;
}
}

View File

@ -2137,4 +2137,12 @@
</description>
</property>
<property>
<name>dfs.encryption.key.provider.uri</name>
<description>
The KeyProvider to use when interacting with encryption keys used
when reading and writing to an encryption zone.
</description>
</property>
</configuration>

View File

@ -85,6 +85,12 @@ Transparent Encryption in HDFS
A necessary prerequisite is an instance of the KMS, as well as a backing key store for the KMS.
See the {{{../../hadoop-kms/index.html}KMS documentation}} for more information.
** Configuring the cluster KeyProvider
*** dfs.encryption.key.provider.uri
The KeyProvider to use when interacting with encryption keys used when reading and writing to an encryption zone.
** Selecting an encryption algorithm and codec
*** hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE

View File

@ -66,7 +66,7 @@ public class TestCryptoAdminCLI extends CLITestHelperDFS {
tmpDir = new File(System.getProperty("test.build.data", "target"),
UUID.randomUUID().toString()).getAbsoluteFile();
final Path jksPath = new Path(tmpDir.toString(), "test.jks");
conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();

View File

@ -25,6 +25,7 @@ import java.io.PrintWriter;
import java.io.RandomAccessFile;
import java.io.StringReader;
import java.io.StringWriter;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.Arrays;
import java.util.List;
@ -125,7 +126,7 @@ public class TestEncryptionZones {
// Set up java key store
String testRoot = fsHelper.getTestRootDir();
testRootDir = new File(testRoot).getAbsoluteFile();
conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, getKeyProviderURI());
conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, getKeyProviderURI());
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
// Lower the batch size for testing
conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,
@ -670,7 +671,8 @@ public class TestEncryptionZones {
// Check KeyProvider state
// Flushing the KP on the NN, since it caches, and init a test one
cluster.getNamesystem().getProvider().flush();
KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
KeyProvider provider = KeyProviderFactory
.get(new URI(conf.get(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI)), conf);
List<String> keys = provider.getKeys();
assertEquals("Expected NN to have created one key per zone", 1,
keys.size());
@ -694,7 +696,7 @@ public class TestEncryptionZones {
public void testCreateEZWithNoProvider() throws Exception {
// Unset the key provider and make sure EZ ops don't work
final Configuration clusterConf = cluster.getConfiguration(0);
clusterConf.set(KeyProviderFactory.KEY_PROVIDER_PATH, "");
clusterConf.unset(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI);
cluster.restartNameNode(true);
cluster.waitActive();
final Path zone1 = new Path("/zone1");
@ -706,7 +708,7 @@ public class TestEncryptionZones {
assertExceptionContains("since no key provider is available", e);
}
final Path jksPath = new Path(testRootDir.toString(), "test.jks");
clusterConf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
clusterConf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri()
);
// Try listing EZs as well

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
import org.apache.hadoop.crypto.key.KeyProviderFactory;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.FileSystemTestHelper;
import org.apache.hadoop.fs.Path;
@ -60,7 +59,7 @@ public class TestEncryptionZonesWithHA {
fsHelper = new FileSystemTestHelper();
String testRoot = fsHelper.getTestRootDir();
testRootDir = new File(testRoot).getAbsoluteFile();
conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
JavaKeyStoreProvider.SCHEME_NAME + "://file" + testRootDir + "/test.jks"
);

View File

@ -24,7 +24,6 @@ import java.security.PrivilegedExceptionAction;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
import org.apache.hadoop.crypto.key.KeyProviderFactory;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileContextTestWrapper;
import org.apache.hadoop.fs.FileStatus;
@ -70,7 +69,7 @@ public class TestReservedRawPaths {
String testRoot = fsHelper.getTestRootDir();
File testRootDir = new File(testRoot).getAbsoluteFile();
final Path jksPath = new Path(testRootDir.toString(), "test.jks");
conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri()
);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();