HDFS-16518: Add shutdownhook to invalidate the KeyProviders in the cache
Fixes #4100 Signed-off-by: Owen O'Malley <oomalley@linkedin.com>
This commit is contained in:
parent
f8314cd469
commit
4f85c9a73b
@ -26,6 +26,7 @@
|
|||||||
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.fs.CommonConfigurationKeysPublic;
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.util.KMSUtil;
|
import org.apache.hadoop.util.KMSUtil;
|
||||||
|
|
||||||
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||||
@ -34,6 +35,7 @@
|
|||||||
import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener;
|
import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener;
|
||||||
import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalNotification;
|
import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalNotification;
|
||||||
|
|
||||||
|
import org.apache.hadoop.util.ShutdownHookManager;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
@ -65,6 +67,9 @@ public void onRemoval(
|
|||||||
}
|
}
|
||||||
})
|
})
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
ShutdownHookManager.get().addShutdownHook(new KeyProviderCacheFinalizer(),
|
||||||
|
SHUTDOWN_HOOK_PRIORITY);
|
||||||
}
|
}
|
||||||
|
|
||||||
public KeyProvider get(final Configuration conf,
|
public KeyProvider get(final Configuration conf,
|
||||||
@ -85,6 +90,26 @@ public KeyProvider call() throws Exception {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static final int SHUTDOWN_HOOK_PRIORITY = FileSystem.SHUTDOWN_HOOK_PRIORITY - 1;
|
||||||
|
|
||||||
|
private class KeyProviderCacheFinalizer implements Runnable {
|
||||||
|
@Override
|
||||||
|
public synchronized void run() {
|
||||||
|
invalidateCache();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Invalidate cache. KeyProviders in the cache will be closed by cache hook.
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
synchronized void invalidateCache() {
|
||||||
|
LOG.debug("Invalidating all cached KeyProviders.");
|
||||||
|
if (cache != null) {
|
||||||
|
cache.invalidateAll();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private URI createKeyProviderURI(Configuration conf) {
|
private URI createKeyProviderURI(Configuration conf) {
|
||||||
final String providerUriStr = conf.getTrimmed(
|
final String providerUriStr = conf.getTrimmed(
|
||||||
CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
|
CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
|
||||||
|
@ -32,6 +32,8 @@ public class TestKeyProviderCache {
|
|||||||
|
|
||||||
public static class DummyKeyProvider extends KeyProvider {
|
public static class DummyKeyProvider extends KeyProvider {
|
||||||
|
|
||||||
|
public static int CLOSE_CALL_COUNT = 0;
|
||||||
|
|
||||||
public DummyKeyProvider(Configuration conf) {
|
public DummyKeyProvider(Configuration conf) {
|
||||||
super(conf);
|
super(conf);
|
||||||
}
|
}
|
||||||
@ -76,6 +78,10 @@ public KeyVersion rollNewVersion(String name, byte[] material)
|
|||||||
public void flush() throws IOException {
|
public void flush() throws IOException {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
CLOSE_CALL_COUNT += 1;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Factory extends KeyProviderFactory {
|
public static class Factory extends KeyProviderFactory {
|
||||||
@ -124,6 +130,9 @@ public void testCache() throws Exception {
|
|||||||
Assert.assertFalse("Same KeyProviders returned !!",
|
Assert.assertFalse("Same KeyProviders returned !!",
|
||||||
keyProvider1 == keyProvider4);
|
keyProvider1 == keyProvider4);
|
||||||
|
|
||||||
|
kpCache.invalidateCache();
|
||||||
|
Assert.assertEquals("Expected number of closing calls doesn't match",
|
||||||
|
3, DummyKeyProvider.CLOSE_CALL_COUNT);
|
||||||
}
|
}
|
||||||
|
|
||||||
private URI getKeyProviderUriFromConf(Configuration conf) {
|
private URI getKeyProviderUriFromConf(Configuration conf) {
|
||||||
|
Loading…
Reference in New Issue
Block a user