HADOOP-10428. JavaKeyStoreProvider should accept keystore password via configuration falling back to ENV VAR. (tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1586216 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Alejandro Abdelnur 2014-04-10 05:09:12 +00:00
parent 791f2f6f1b
commit b4f0259f13
5 changed files with 97 additions and 12 deletions

View File

@ -135,6 +135,9 @@ Trunk (Unreleased)
HADOOP-10429. KeyStores should have methods to generate the materials HADOOP-10429. KeyStores should have methods to generate the materials
themselves, KeyShell should use them. (tucu) themselves, KeyShell should use them. (tucu)
HADOOP-10428. JavaKeyStoreProvider should accept keystore password via
configuration falling back to ENV VAR. (tucu)
BUG FIXES BUG FIXES
HADOOP-9451. Fault single-layer config if node group topology is enabled. HADOOP-9451. Fault single-layer config if node group topology is enabled.

View File

@ -483,6 +483,7 @@
<exclude>src/test/resources/test.har/_index</exclude> <exclude>src/test/resources/test.har/_index</exclude>
<exclude>src/test/resources/test.har/_masterindex</exclude> <exclude>src/test/resources/test.har/_masterindex</exclude>
<exclude>src/test/resources/test.har/part-0</exclude> <exclude>src/test/resources/test.har/part-0</exclude>
<exclude>src/test/resources/javakeystoreprovider.password</exclude>
</excludes> </excludes>
</configuration> </configuration>
</plugin> </plugin>

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.crypto.key; package org.apache.hadoop.crypto.key;
import org.apache.commons.io.IOUtils;
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.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
@ -27,10 +28,12 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import javax.crypto.spec.SecretKeySpec; import javax.crypto.spec.SecretKeySpec;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream;
import java.io.ObjectInputStream; import java.io.ObjectInputStream;
import java.io.ObjectOutputStream; import java.io.ObjectOutputStream;
import java.io.Serializable; import java.io.Serializable;
import java.net.URI; import java.net.URI;
import java.net.URL;
import java.security.Key; import java.security.Key;
import java.security.KeyStore; import java.security.KeyStore;
import java.security.KeyStoreException; import java.security.KeyStoreException;
@ -52,10 +55,21 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
* any Hadoop FileSystem using the following name mangling: * any Hadoop FileSystem using the following name mangling:
* jks://hdfs@nn1.example.com/my/keys.jks -> hdfs://nn1.example.com/my/keys.jks * jks://hdfs@nn1.example.com/my/keys.jks -> hdfs://nn1.example.com/my/keys.jks
* jks://file/home/owen/keys.jks -> file:///home/owen/keys.jks * jks://file/home/owen/keys.jks -> file:///home/owen/keys.jks
* * <p/>
* The password for the keystore is taken from the HADOOP_KEYSTORE_PASSWORD * If the <code>HADOOP_KEYSTORE_PASSWORD</code> environment variable is set,
* environment variable with a default of 'none'. * its value is used as the password for the keystore.
* * <p/>
* If the <code>HADOOP_KEYSTORE_PASSWORD</code> environment variable is not set,
* the password for the keystore is read from file specified in the
* {@link #KEYSTORE_PASSWORD_FILE_KEY} configuration property. The password file
* is looked up in Hadoop's configuration directory via the classpath.
* <p/>
* <b>NOTE:</b> Make sure the password in the password file does not have an
* ENTER at the end, else it won't be valid for the Java KeyStore.
* <p/>
* If the environment variable, nor the property are not set, the password used
* is 'none'.
* <p/>
* It is expected for encrypted InputFormats and OutputFormats to copy the keys * It is expected for encrypted InputFormats and OutputFormats to copy the keys
* from the original provider into the job's Credentials object, which is * from the original provider into the job's Credentials object, which is
* accessed via the UserProvider. Therefore, this provider won't be used by * accessed via the UserProvider. Therefore, this provider won't be used by
@ -65,16 +79,20 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
public class JavaKeyStoreProvider extends KeyProvider { public class JavaKeyStoreProvider extends KeyProvider {
private static final String KEY_METADATA = "KeyMetadata"; private static final String KEY_METADATA = "KeyMetadata";
public static final String SCHEME_NAME = "jceks"; public static final String SCHEME_NAME = "jceks";
public static final String KEYSTORE_PASSWORD_NAME =
public static final String KEYSTORE_PASSWORD_FILE_KEY =
"hadoop.security.keystore.java-keystore-provider.password-file";
public static final String KEYSTORE_PASSWORD_ENV_VAR =
"HADOOP_KEYSTORE_PASSWORD"; "HADOOP_KEYSTORE_PASSWORD";
public static final String KEYSTORE_PASSWORD_DEFAULT = "none"; public static final char[] KEYSTORE_PASSWORD_DEFAULT = "none".toCharArray();
private final URI uri; private final URI uri;
private final Path path; private final Path path;
private final FileSystem fs; private final FileSystem fs;
private final FsPermission permissions; private final FsPermission permissions;
private final KeyStore keyStore; private final KeyStore keyStore;
private final char[] password; private char[] password;
private boolean changed = false; private boolean changed = false;
private Lock readLock; private Lock readLock;
private Lock writeLock; private Lock writeLock;
@ -85,12 +103,29 @@ public class JavaKeyStoreProvider extends KeyProvider {
this.uri = uri; this.uri = uri;
path = unnestUri(uri); path = unnestUri(uri);
fs = path.getFileSystem(conf); fs = path.getFileSystem(conf);
// Get the password from the user's environment // Get the password file from the conf, if not present from the user's
String pw = System.getenv(KEYSTORE_PASSWORD_NAME); // environment var
if (pw == null) { if (System.getenv().containsKey(KEYSTORE_PASSWORD_ENV_VAR)) {
pw = KEYSTORE_PASSWORD_DEFAULT; password = System.getenv(KEYSTORE_PASSWORD_ENV_VAR).toCharArray();
}
if (password == null) {
String pwFile = conf.get(KEYSTORE_PASSWORD_FILE_KEY);
if (pwFile != null) {
ClassLoader cl = Thread.currentThread().getContextClassLoader();
URL pwdFile = cl.getResource(pwFile);
if (pwdFile != null) {
InputStream is = pwdFile.openStream();
try {
password = IOUtils.toCharArray(is);
} finally {
is.close();
}
}
}
}
if (password == null) {
password = KEYSTORE_PASSWORD_DEFAULT;
} }
password = pw.toCharArray();
try { try {
keyStore = KeyStore.getInstance(SCHEME_NAME); keyStore = KeyStore.getInstance(SCHEME_NAME);
if (fs.exists(path)) { if (fs.exists(path)) {

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertArrayEquals;
@ -237,4 +238,48 @@ public class TestKeyProviderFactory {
FileStatus s = fs.getFileStatus(path); FileStatus s = fs.getFileStatus(path);
assertTrue("Permissions should have been retained from the preexisting keystore.", s.getPermission().toString().equals("rwxrwxrwx")); assertTrue("Permissions should have been retained from the preexisting keystore.", s.getPermission().toString().equals("rwxrwxrwx"));
} }
@Test
public void testJksProviderPasswordViaConfig() throws Exception {
Configuration conf = new Configuration();
final String ourUrl =
JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks";
File file = new File(tmpDir, "test.jks");
file.delete();
try {
conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, ourUrl);
conf.set(JavaKeyStoreProvider.KEYSTORE_PASSWORD_FILE_KEY,
"javakeystoreprovider.password");
KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
provider.createKey("key3", new byte[32], KeyProvider.options(conf));
provider.flush();
} catch (Exception ex) {
Assert.fail("could not create keystore with password file");
}
KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
Assert.assertNotNull(provider.getCurrentKey("key3"));
try {
conf.set(JavaKeyStoreProvider.KEYSTORE_PASSWORD_FILE_KEY, "bar");
KeyProviderFactory.getProviders(conf).get(0);
Assert.fail("using non existing password file, it should fail");
} catch (IOException ex) {
//NOP
}
try {
conf.set(JavaKeyStoreProvider.KEYSTORE_PASSWORD_FILE_KEY, "core-site.xml");
KeyProviderFactory.getProviders(conf).get(0);
Assert.fail("using different password file, it should fail");
} catch (IOException ex) {
//NOP
}
try {
conf.unset(JavaKeyStoreProvider.KEYSTORE_PASSWORD_FILE_KEY);
KeyProviderFactory.getProviders(conf).get(0);
Assert.fail("No password file property, env not set, it should fail");
} catch (IOException ex) {
//NOP
}
}
} }