001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hdfs; 019 020import java.io.IOException; 021import java.net.URI; 022import java.net.URISyntaxException; 023import java.util.concurrent.Callable; 024import java.util.concurrent.ExecutionException; 025import java.util.concurrent.TimeUnit; 026 027import org.apache.commons.logging.Log; 028import org.apache.commons.logging.LogFactory; 029import org.apache.hadoop.conf.Configuration; 030import org.apache.hadoop.crypto.key.KeyProvider; 031 032import com.google.common.annotations.VisibleForTesting; 033import com.google.common.cache.Cache; 034import com.google.common.cache.CacheBuilder; 035import com.google.common.cache.RemovalListener; 036import com.google.common.cache.RemovalNotification; 037 038public class KeyProviderCache { 039 040 public static final Log LOG = LogFactory.getLog(KeyProviderCache.class); 041 042 private final Cache<URI, KeyProvider> cache; 043 044 public KeyProviderCache(long expiryMs) { 045 cache = CacheBuilder.newBuilder() 046 .expireAfterAccess(expiryMs, TimeUnit.MILLISECONDS) 047 .removalListener(new RemovalListener<URI, KeyProvider>() { 048 @Override 049 public void onRemoval( 050 RemovalNotification<URI, KeyProvider> notification) { 051 try { 052 notification.getValue().close(); 053 } catch (Throwable e) { 054 LOG.error( 055 "Error closing KeyProvider with uri [" 056 + notification.getKey() + "]", e); 057 ; 058 } 059 } 060 }) 061 .build(); 062 } 063 064 public KeyProvider get(final Configuration conf) { 065 URI kpURI = createKeyProviderURI(conf); 066 if (kpURI == null) { 067 return null; 068 } 069 try { 070 return cache.get(kpURI, new Callable<KeyProvider>() { 071 @Override 072 public KeyProvider call() throws Exception { 073 return DFSUtil.createKeyProvider(conf); 074 } 075 }); 076 } catch (Exception e) { 077 LOG.error("Could not create KeyProvider for DFSClient !!", e.getCause()); 078 return null; 079 } 080 } 081 082 private URI createKeyProviderURI(Configuration conf) { 083 final String providerUriStr = 084 conf.get(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, null); 085 // No provider set in conf 086 if (providerUriStr == null) { 087 LOG.error("Could not find uri with key [" 088 + DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI 089 + "] to create a keyProvider !!"); 090 return null; 091 } 092 final URI providerUri; 093 try { 094 providerUri = new URI(providerUriStr); 095 } catch (URISyntaxException e) { 096 LOG.error("KeyProvider URI string is invalid [" + providerUriStr 097 + "]!!", e.getCause()); 098 return null; 099 } 100 return providerUri; 101 } 102 103 @VisibleForTesting 104 public void setKeyProvider(Configuration conf, KeyProvider keyProvider) 105 throws IOException { 106 URI uri = createKeyProviderURI(conf); 107 cache.put(uri, keyProvider); 108 } 109}