|
@@ -24,7 +24,13 @@ import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
+import com.google.common.base.Ticker;
|
|
|
+import com.google.common.cache.CacheBuilder;
|
|
|
+import com.google.common.cache.CacheLoader;
|
|
|
+import com.google.common.cache.LoadingCache;
|
|
|
import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
|
@@ -52,10 +58,11 @@ public class Groups {
|
|
|
private static final Log LOG = LogFactory.getLog(Groups.class);
|
|
|
|
|
|
private final GroupMappingServiceProvider impl;
|
|
|
-
|
|
|
- private final Map<String, CachedGroups> userToGroupsMap =
|
|
|
- new ConcurrentHashMap<String, CachedGroups>();
|
|
|
- private final Map<String, List<String>> staticUserToGroupsMap =
|
|
|
+
|
|
|
+ private final LoadingCache<String, List<String>> cache;
|
|
|
+ private final ConcurrentHashMap<String, Long> negativeCacheMask =
|
|
|
+ new ConcurrentHashMap<String, Long>();
|
|
|
+ private final Map<String, List<String>> staticUserToGroupsMap =
|
|
|
new HashMap<String, List<String>>();
|
|
|
private final long cacheTimeout;
|
|
|
private final long negativeCacheTimeout;
|
|
@@ -66,7 +73,7 @@ public class Groups {
|
|
|
this(conf, new Timer());
|
|
|
}
|
|
|
|
|
|
- public Groups(Configuration conf, Timer timer) {
|
|
|
+ public Groups(Configuration conf, final Timer timer) {
|
|
|
impl =
|
|
|
ReflectionUtils.newInstance(
|
|
|
conf.getClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
|
|
@@ -86,6 +93,11 @@ public class Groups {
|
|
|
parseStaticMapping(conf);
|
|
|
|
|
|
this.timer = timer;
|
|
|
+ this.cache = CacheBuilder.newBuilder()
|
|
|
+ .refreshAfterWrite(cacheTimeout, TimeUnit.MILLISECONDS)
|
|
|
+ .ticker(new TimerToTickerAdapter(timer))
|
|
|
+ .expireAfterWrite(10 * cacheTimeout, TimeUnit.MILLISECONDS)
|
|
|
+ .build(new GroupCacheLoader());
|
|
|
|
|
|
if(LOG.isDebugEnabled())
|
|
|
LOG.debug("Group mapping impl=" + impl.getClass().getName() +
|
|
@@ -123,78 +135,112 @@ public class Groups {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Determine whether the CachedGroups is expired.
|
|
|
- * @param groups cached groups for one user.
|
|
|
- * @return true if groups is expired from useToGroupsMap.
|
|
|
- */
|
|
|
- private boolean hasExpired(CachedGroups groups, long startMs) {
|
|
|
- if (groups == null) {
|
|
|
- return true;
|
|
|
- }
|
|
|
- long timeout = cacheTimeout;
|
|
|
- if (isNegativeCacheEnabled() && groups.getGroups().isEmpty()) {
|
|
|
- // This CachedGroups is in the negative cache, thus it should expire
|
|
|
- // sooner.
|
|
|
- timeout = negativeCacheTimeout;
|
|
|
- }
|
|
|
- return groups.getTimestamp() + timeout <= startMs;
|
|
|
- }
|
|
|
-
|
|
|
private boolean isNegativeCacheEnabled() {
|
|
|
return negativeCacheTimeout > 0;
|
|
|
}
|
|
|
|
|
|
+ private IOException noGroupsForUser(String user) {
|
|
|
+ return new IOException("No groups found for user " + user);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Get the group memberships of a given user.
|
|
|
+ * If the user's group is not cached, this method may block.
|
|
|
* @param user User's name
|
|
|
* @return the group memberships of the user
|
|
|
- * @throws IOException
|
|
|
+ * @throws IOException if user does not exist
|
|
|
*/
|
|
|
- public List<String> getGroups(String user) throws IOException {
|
|
|
+ public List<String> getGroups(final String user) throws IOException {
|
|
|
// No need to lookup for groups of static users
|
|
|
List<String> staticMapping = staticUserToGroupsMap.get(user);
|
|
|
if (staticMapping != null) {
|
|
|
return staticMapping;
|
|
|
}
|
|
|
- // Return cached value if available
|
|
|
- CachedGroups groups = userToGroupsMap.get(user);
|
|
|
- long startMs = timer.monotonicNow();
|
|
|
- if (!hasExpired(groups, startMs)) {
|
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Returning cached groups for '" + user + "'");
|
|
|
- }
|
|
|
- if (groups.getGroups().isEmpty()) {
|
|
|
- // Even with enabling negative cache, getGroups() has the same behavior
|
|
|
- // that throws IOException if the groups for the user is empty.
|
|
|
- throw new IOException("No groups found for user " + user);
|
|
|
+
|
|
|
+ // Check the negative cache first
|
|
|
+ if (isNegativeCacheEnabled()) {
|
|
|
+ Long expirationTime = negativeCacheMask.get(user);
|
|
|
+ if (expirationTime != null) {
|
|
|
+ if (timer.monotonicNow() < expirationTime) {
|
|
|
+ throw noGroupsForUser(user);
|
|
|
+ } else {
|
|
|
+ negativeCacheMask.remove(user, expirationTime);
|
|
|
+ }
|
|
|
}
|
|
|
- return groups.getGroups();
|
|
|
}
|
|
|
|
|
|
- // Create and cache user's groups
|
|
|
- List<String> groupList = impl.getGroups(user);
|
|
|
- long endMs = timer.monotonicNow();
|
|
|
- long deltaMs = endMs - startMs ;
|
|
|
- UserGroupInformation.metrics.addGetGroups(deltaMs);
|
|
|
- if (deltaMs > warningDeltaMs) {
|
|
|
- LOG.warn("Potential performance problem: getGroups(user=" + user +") " +
|
|
|
- "took " + deltaMs + " milliseconds.");
|
|
|
+ try {
|
|
|
+ return cache.get(user);
|
|
|
+ } catch (ExecutionException e) {
|
|
|
+ throw (IOException)e.getCause();
|
|
|
}
|
|
|
- groups = new CachedGroups(groupList, endMs);
|
|
|
- if (groups.getGroups().isEmpty()) {
|
|
|
- if (isNegativeCacheEnabled()) {
|
|
|
- userToGroupsMap.put(user, groups);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Convert millisecond times from hadoop's timer to guava's nanosecond ticker.
|
|
|
+ */
|
|
|
+ private static class TimerToTickerAdapter extends Ticker {
|
|
|
+ private Timer timer;
|
|
|
+
|
|
|
+ public TimerToTickerAdapter(Timer timer) {
|
|
|
+ this.timer = timer;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long read() {
|
|
|
+ final long NANOSECONDS_PER_MS = 1000000;
|
|
|
+ return timer.monotonicNow() * NANOSECONDS_PER_MS;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Deals with loading data into the cache.
|
|
|
+ */
|
|
|
+ private class GroupCacheLoader extends CacheLoader<String, List<String>> {
|
|
|
+ /**
|
|
|
+ * This method will block if a cache entry doesn't exist, and
|
|
|
+ * any subsequent requests for the same user will wait on this
|
|
|
+ * request to return. If a user already exists in the cache,
|
|
|
+ * this will be run in the background.
|
|
|
+ * @param user key of cache
|
|
|
+ * @return List of groups belonging to user
|
|
|
+ * @throws IOException to prevent caching negative entries
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public List<String> load(String user) throws Exception {
|
|
|
+ List<String> groups = fetchGroupList(user);
|
|
|
+
|
|
|
+ if (groups.isEmpty()) {
|
|
|
+ if (isNegativeCacheEnabled()) {
|
|
|
+ long expirationTime = timer.monotonicNow() + negativeCacheTimeout;
|
|
|
+ negativeCacheMask.put(user, expirationTime);
|
|
|
+ }
|
|
|
+
|
|
|
+ // We throw here to prevent Cache from retaining an empty group
|
|
|
+ throw noGroupsForUser(user);
|
|
|
}
|
|
|
- throw new IOException("No groups found for user " + user);
|
|
|
+
|
|
|
+ return groups;
|
|
|
}
|
|
|
- userToGroupsMap.put(user, groups);
|
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Returning fetched groups for '" + user + "'");
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Queries impl for groups belonging to the user. This could involve I/O and take awhile.
|
|
|
+ */
|
|
|
+ private List<String> fetchGroupList(String user) throws IOException {
|
|
|
+ long startMs = timer.monotonicNow();
|
|
|
+ List<String> groupList = impl.getGroups(user);
|
|
|
+ long endMs = timer.monotonicNow();
|
|
|
+ long deltaMs = endMs - startMs ;
|
|
|
+ UserGroupInformation.metrics.addGetGroups(deltaMs);
|
|
|
+ if (deltaMs > warningDeltaMs) {
|
|
|
+ LOG.warn("Potential performance problem: getGroups(user=" + user +") " +
|
|
|
+ "took " + deltaMs + " milliseconds.");
|
|
|
+ }
|
|
|
+
|
|
|
+ return groupList;
|
|
|
}
|
|
|
- return groups.getGroups();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Refresh all user-to-groups mappings.
|
|
|
*/
|
|
@@ -205,7 +251,8 @@ public class Groups {
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("Error refreshing groups cache", e);
|
|
|
}
|
|
|
- userToGroupsMap.clear();
|
|
|
+ cache.invalidateAll();
|
|
|
+ negativeCacheMask.clear();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -221,40 +268,6 @@ public class Groups {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Class to hold the cached groups
|
|
|
- */
|
|
|
- private static class CachedGroups {
|
|
|
- final long timestamp;
|
|
|
- final List<String> groups;
|
|
|
-
|
|
|
- /**
|
|
|
- * Create and initialize group cache
|
|
|
- */
|
|
|
- CachedGroups(List<String> groups, long timestamp) {
|
|
|
- this.groups = groups;
|
|
|
- this.timestamp = timestamp;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Returns time of last cache update
|
|
|
- *
|
|
|
- * @return time of last cache update
|
|
|
- */
|
|
|
- public long getTimestamp() {
|
|
|
- return timestamp;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Get list of cached groups
|
|
|
- *
|
|
|
- * @return cached groups
|
|
|
- */
|
|
|
- public List<String> getGroups() {
|
|
|
- return groups;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
private static Groups GROUPS = null;
|
|
|
|
|
|
/**
|