|
@@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.crypto.key.KeyProvider;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.util.KMSUtil;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
@@ -34,6 +35,7 @@ import com.google.common.cache.CacheBuilder;
|
|
|
import com.google.common.cache.RemovalListener;
|
|
|
import com.google.common.cache.RemovalNotification;
|
|
|
|
|
|
+import org.apache.hadoop.util.ShutdownHookManager;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -65,6 +67,9 @@ public class KeyProviderCache {
|
|
|
}
|
|
|
})
|
|
|
.build();
|
|
|
+
|
|
|
+ ShutdownHookManager.get().addShutdownHook(new KeyProviderCacheFinalizer(),
|
|
|
+ SHUTDOWN_HOOK_PRIORITY);
|
|
|
}
|
|
|
|
|
|
public KeyProvider get(final Configuration conf,
|
|
@@ -85,6 +90,26 @@ public class KeyProviderCache {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ 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) {
|
|
|
final String providerUriStr = conf.getTrimmed(
|
|
|
CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
|