HADOOP-10237. JavaKeyStoreProvider needs to set keystore permissions
correctly. (Larry McCay via omalley) Conflicts: hadoop-common-project/hadoop-common/CHANGES.txt git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1619509 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
48289796d2
commit
90c9ffcb3c
|
@ -188,6 +188,9 @@ Release 2.6.0 - UNRELEASED
|
||||||
HADOOP-10141. Create KeyProvider API to separate encryption key storage
|
HADOOP-10141. Create KeyProvider API to separate encryption key storage
|
||||||
from the applications. (omalley)
|
from the applications. (omalley)
|
||||||
|
|
||||||
|
HADOOP-10237. JavaKeyStoreProvider needs to set keystore permissions
|
||||||
|
correctly. (Larry McCay via omalley)
|
||||||
|
|
||||||
Release 2.5.0 - 2014-08-11
|
Release 2.5.0 - 2014-08-11
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -21,9 +21,10 @@ package org.apache.hadoop.crypto.key;
|
||||||
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;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
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.ObjectInputStream;
|
import java.io.ObjectInputStream;
|
||||||
|
@ -68,6 +69,7 @@ public class JavaKeyStoreProvider extends KeyProvider {
|
||||||
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 KeyStore keyStore;
|
private final KeyStore keyStore;
|
||||||
private final char[] password;
|
private final char[] password;
|
||||||
private boolean changed = false;
|
private boolean changed = false;
|
||||||
|
@ -87,8 +89,14 @@ public class JavaKeyStoreProvider extends KeyProvider {
|
||||||
try {
|
try {
|
||||||
keyStore = KeyStore.getInstance(SCHEME_NAME);
|
keyStore = KeyStore.getInstance(SCHEME_NAME);
|
||||||
if (fs.exists(path)) {
|
if (fs.exists(path)) {
|
||||||
|
// save off permissions in case we need to
|
||||||
|
// rewrite the keystore in flush()
|
||||||
|
FileStatus s = fs.getFileStatus(path);
|
||||||
|
permissions = s.getPermission();
|
||||||
|
|
||||||
keyStore.load(fs.open(path), password);
|
keyStore.load(fs.open(path), password);
|
||||||
} else {
|
} else {
|
||||||
|
permissions = new FsPermission("700");
|
||||||
// required to create an empty keystore. *sigh*
|
// required to create an empty keystore. *sigh*
|
||||||
keyStore.load(null, password);
|
keyStore.load(null, password);
|
||||||
}
|
}
|
||||||
|
@ -277,7 +285,7 @@ public class JavaKeyStoreProvider extends KeyProvider {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// write out the keystore
|
// write out the keystore
|
||||||
FSDataOutputStream out = fs.create(path, true);
|
FSDataOutputStream out = FileSystem.create(fs, path, permissions);
|
||||||
try {
|
try {
|
||||||
keyStore.store(out, password);
|
keyStore.store(out, password);
|
||||||
} catch (KeyStoreException e) {
|
} catch (KeyStoreException e) {
|
||||||
|
|
|
@ -19,9 +19,14 @@ package org.apache.hadoop.crypto.key;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.net.URI;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
|
import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
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;
|
||||||
|
@ -193,10 +198,43 @@ public class TestKeyProviderFactory {
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
final String ourUrl =
|
final String ourUrl =
|
||||||
JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks";
|
JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks";
|
||||||
|
|
||||||
File file = new File(tmpDir, "test.jks");
|
File file = new File(tmpDir, "test.jks");
|
||||||
file.delete();
|
file.delete();
|
||||||
conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, ourUrl);
|
conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, ourUrl);
|
||||||
checkSpecificProvider(conf, ourUrl);
|
checkSpecificProvider(conf, ourUrl);
|
||||||
|
Path path = KeyProvider.unnestUri(new URI(ourUrl));
|
||||||
|
FileSystem fs = path.getFileSystem(conf);
|
||||||
|
FileStatus s = fs.getFileStatus(path);
|
||||||
|
assertTrue(s.getPermission().toString().equals("rwx------"));
|
||||||
assertTrue(file + " should exist", file.isFile());
|
assertTrue(file + " should exist", file.isFile());
|
||||||
|
|
||||||
|
// check permission retention after explicit change
|
||||||
|
fs.setPermission(path, new FsPermission("777"));
|
||||||
|
checkPermissionRetention(conf, ourUrl, path);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void checkPermissionRetention(Configuration conf, String ourUrl, Path path) throws Exception {
|
||||||
|
KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
|
||||||
|
// let's add a new key and flush and check that permissions are still set to 777
|
||||||
|
byte[] key = new byte[32];
|
||||||
|
for(int i =0; i < key.length; ++i) {
|
||||||
|
key[i] = (byte) i;
|
||||||
|
}
|
||||||
|
// create a new key
|
||||||
|
try {
|
||||||
|
provider.createKey("key5", key, KeyProvider.options(conf));
|
||||||
|
} catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
provider.flush();
|
||||||
|
// get a new instance of the provider to ensure it was saved correctly
|
||||||
|
provider = KeyProviderFactory.getProviders(conf).get(0);
|
||||||
|
assertArrayEquals(key, provider.getCurrentKey("key5").getMaterial());
|
||||||
|
|
||||||
|
FileSystem fs = path.getFileSystem(conf);
|
||||||
|
FileStatus s = fs.getFileStatus(path);
|
||||||
|
assertTrue("Permissions should have been retained from the preexisting keystore.", s.getPermission().toString().equals("rwxrwxrwx"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue