You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sa...@apache.org on 2016/02/17 18:08:03 UTC

cassandra git commit: Refactor auth caches and add credentials cache

Repository: cassandra
Updated Branches:
  refs/heads/trunk 7871c36e3 -> 186777aac


Refactor auth caches and add credentials cache

Patch by Sam Tunnicliffe; reviewed by Mike Adamson for CASSANDRA-7715


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/186777aa
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/186777aa
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/186777aa

Branch: refs/heads/trunk
Commit: 186777aac55b1aa95c5c9e9cdaff2b981cf9d5af
Parents: 7871c36
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Wed Feb 17 13:41:18 2016 +0000
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Feb 17 17:03:11 2016 +0000

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  24 +++
 conf/cassandra.yaml                             |  33 ++-
 conf/jvm.options                                |   5 +
 .../org/apache/cassandra/auth/AuthCache.java    | 207 +++++++++++++++++++
 .../apache/cassandra/auth/AuthCacheMBean.java   |  36 ++++
 .../cassandra/auth/PasswordAuthenticator.java   | 111 ++++++++--
 .../apache/cassandra/auth/PermissionsCache.java | 125 ++---------
 .../cassandra/auth/PermissionsCacheMBean.java   |  15 +-
 .../org/apache/cassandra/auth/RolesCache.java   | 124 ++---------
 .../apache/cassandra/auth/RolesCacheMBean.java  |  17 +-
 .../org/apache/cassandra/config/Config.java     |   7 +-
 .../cassandra/config/DatabaseDescriptor.java    |  77 ++++++-
 13 files changed, 495 insertions(+), 287 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 978f20a..f3248df 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.4
+ * Add caching of encrypted credentials in PasswordAuthenticator (CASSANDRA-7715)
  * fix SASI memtable switching on flush (CASSANDRA-11159)
  * Remove duplicate offline compaction tracking (CASSANDRA-11148)
  * fix EQ semantics of analyzed SASI indexes (CASSANDRA-11130)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 9dd4e25..68ccefb 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -13,6 +13,30 @@ restore snapshots created with the previous major version using the
 'sstableloader' tool. You can upgrade the file format of your snapshots
 using the provided 'sstableupgrade' tool.
 
+3.4
+===
+
+New features
+------------
+    - Internal authentication now supports caching of encrypted credentials.
+      Reference cassandra.yaml:credentials_validity_in_ms
+    - Remote configuration of auth caches via JMX can be disabled using the
+      the system property cassandra.disable_auth_caches_remote_configuration
+
+Upgrading
+---------
+    - Nothing specific to 3.4 but please see previous versions upgrading section,
+      especially if you are upgrading from 2.2.
+
+Deprecation
+-----------
+    - The mbean interfaces org.apache.cassandra.auth.PermissionsCacheMBean and
+      org.apache.cassandra.auth.RolesCacheMBean are deprecated in favor of
+      org.apache.cassandra.auth.AuthCacheMBean. This generalized interface is
+      common across all caches in the auth subsystem. The specific mbean interfaces
+      for each individual cache will be removed in a subsequent major version.
+
+
 3.2
 ===
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index d6bcace..5663113 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -120,11 +120,11 @@ authorizer: AllowAllAuthorizer
 #   increase system_auth keyspace replication factor if you use this role manager.
 role_manager: CassandraRoleManager
 
-# Validity period for roles cache (fetching permissions can be an
-# expensive operation depending on the authorizer). Granted roles are cached for
-# authenticated sessions in AuthenticatedUser and after the period specified
-# here, become eligible for (async) reload.
-# Defaults to 2000, set to 0 to disable.
+# Validity period for roles cache (fetching granted roles can be an expensive
+# operation depending on the role manager, CassandraRoleManager is one example)
+# Granted roles are cached for authenticated sessions in AuthenticatedUser and
+# after the period specified here, become eligible for (async) reload.
+# Defaults to 2000, set to 0 to disable caching entirely.
 # Will be disabled automatically for AllowAllAuthenticator.
 roles_validity_in_ms: 2000
 
@@ -134,7 +134,7 @@ roles_validity_in_ms: 2000
 # completes. If roles_validity_in_ms is non-zero, then this must be
 # also.
 # Defaults to the same value as roles_validity_in_ms.
-# roles_update_interval_in_ms: 1000
+# roles_update_interval_in_ms: 2000
 
 # Validity period for permissions cache (fetching permissions can be an
 # expensive operation depending on the authorizer, CassandraAuthorizer is
@@ -148,7 +148,26 @@ permissions_validity_in_ms: 2000
 # completes. If permissions_validity_in_ms is non-zero, then this must be
 # also.
 # Defaults to the same value as permissions_validity_in_ms.
-# permissions_update_interval_in_ms: 1000
+# permissions_update_interval_in_ms: 2000
+
+# Validity period for credentials cache. This cache is tightly coupled to
+# the provided PasswordAuthenticator implementation of IAuthenticator. If
+# another IAuthenticator implementation is configured, this cache will not
+# be automatically used and so the following settings will have no effect.
+# Please note, credentials are cached in their encrypted form, so while
+# activating this cache may reduce the number of queries made to the
+# underlying table, it may not  bring a significant reduction in the
+# latency of individual authentication attempts.
+# Defaults to 2000, set to 0 to disable credentials caching.
+credentials_validity_in_ms: 2000
+
+# Refresh interval for credentials cache (if enabled).
+# After this interval, cache entries become eligible for refresh. Upon next
+# access, an async reload is scheduled and the old value returned until it
+# completes. If credentials_validity_in_ms is non-zero, then this must be
+# also.
+# Defaults to the same value as credentials_validity_in_ms.
+# credentials_update_interval_in_ms: 2000
 
 # The partitioner is responsible for distributing groups of rows (by
 # partition key) across nodes in the cluster.  You should leave this

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/conf/jvm.options
----------------------------------------------------------------------
diff --git a/conf/jvm.options b/conf/jvm.options
index 4aec619..692d06b 100644
--- a/conf/jvm.options
+++ b/conf/jvm.options
@@ -75,6 +75,11 @@
 # strategies and benchmark write performance differences without affecting the production workload. 
 #-Dcassandra.write_survey=true
 
+# To disable configuration via JMX of auth caches (such as those for credentials, permissions and
+# roles). This will mean those config options can only be set (persistently) in cassandra.yaml
+# and will require a restart for new values to take effect.
+#-Dcassandra.disable_auth_caches_remote_configuration=true
+
 ########################
 # GENERAL JVM SETTINGS #
 ########################

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/src/java/org/apache/cassandra/auth/AuthCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthCache.java b/src/java/org/apache/cassandra/auth/AuthCache.java
new file mode 100644
index 0000000..3e33bbd
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/AuthCache.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.auth;
+
+import java.lang.management.ManagementFactory;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListenableFutureTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+
+public class AuthCache<K, V> implements AuthCacheMBean
+{
+    private static final Logger logger = LoggerFactory.getLogger(AuthCache.class);
+
+    private static final String MBEAN_NAME_BASE = "org.apache.cassandra.auth:type=";
+
+    private volatile LoadingCache<K, V> cache;
+    private ThreadPoolExecutor cacheRefreshExecutor;
+
+    private final String name;
+    private final Consumer<Integer> setValidityDelegate;
+    private final Supplier<Integer> getValidityDelegate;
+    private final Consumer<Integer> setUpdateIntervalDelegate;
+    private final Supplier<Integer> getUpdateIntervalDelegate;
+    private final Consumer<Integer> setMaxEntriesDelegate;
+    private final Supplier<Integer> getMaxEntriesDelegate;
+    private final Function<K, V> loadFunction;
+    private final Supplier<Boolean> enableCache;
+
+    protected AuthCache(String name,
+                        Consumer<Integer> setValidityDelegate,
+                        Supplier<Integer> getValidityDelegate,
+                        Consumer<Integer> setUpdateIntervalDelegate,
+                        Supplier<Integer> getUpdateIntervalDelegate,
+                        Consumer<Integer> setMaxEntriesDelegate,
+                        Supplier<Integer> getMaxEntriesDelegate,
+                        Function<K, V> loadFunction,
+                        Supplier<Boolean> enableCache)
+    {
+        this.name = name;
+        this.setValidityDelegate = setValidityDelegate;
+        this.getValidityDelegate = getValidityDelegate;
+        this.setUpdateIntervalDelegate = setUpdateIntervalDelegate;
+        this.getUpdateIntervalDelegate = getUpdateIntervalDelegate;
+        this.setMaxEntriesDelegate = setMaxEntriesDelegate;
+        this.getMaxEntriesDelegate = getMaxEntriesDelegate;
+        this.loadFunction = loadFunction;
+        this.enableCache = enableCache;
+        init();
+    }
+
+    protected void init()
+    {
+        this.cacheRefreshExecutor = new DebuggableThreadPoolExecutor(name + "Refresh", Thread.NORM_PRIORITY);
+        this.cache = initCache(null);
+        try
+        {
+            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+            mbs.registerMBean(this, getObjectName());
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    protected ObjectName getObjectName() throws MalformedObjectNameException
+    {
+        return new ObjectName(MBEAN_NAME_BASE + name);
+    }
+
+    public V get(K k) throws ExecutionException
+    {
+        if (cache == null)
+            return loadFunction.apply(k);
+
+        return cache.get(k);
+    }
+
+    public void invalidate()
+    {
+        cache = initCache(null);
+    }
+
+    public void invalidate(K k)
+    {
+        if (cache != null)
+            cache.invalidate(k);
+    }
+
+    public void setValidity(int validityPeriod)
+    {
+        if (Boolean.getBoolean("cassandra.disable_auth_caches_remote_configuration"))
+            throw new UnsupportedOperationException("Remote configuration of auth caches is disabled");
+
+        setValidityDelegate.accept(validityPeriod);
+        cache = initCache(cache);
+    }
+
+    public int getValidity()
+    {
+        return getValidityDelegate.get();
+    }
+
+    public void setUpdateInterval(int updateInterval)
+    {
+        if (Boolean.getBoolean("cassandra.disable_auth_caches_remote_configuration"))
+            throw new UnsupportedOperationException("Remote configuration of auth caches is disabled");
+
+        setUpdateIntervalDelegate.accept(updateInterval);
+        cache = initCache(cache);
+    }
+
+    public int getUpdateInterval()
+    {
+        return getUpdateIntervalDelegate.get();
+    }
+
+    public void setMaxEntries(int maxEntries)
+    {
+        if (Boolean.getBoolean("cassandra.disable_auth_caches_remote_configuration"))
+            throw new UnsupportedOperationException("Remote configuration of auth caches is disabled");
+
+        setMaxEntriesDelegate.accept(maxEntries);
+        cache = initCache(cache);
+    }
+
+    public int getMaxEntries()
+    {
+        return getMaxEntriesDelegate.get();
+    }
+
+    private LoadingCache<K, V> initCache(LoadingCache<K, V> existing)
+    {
+        if (!enableCache.get())
+            return null;
+
+        if (getValidity() <= 0)
+            return null;
+
+        logger.info("(Re)initializing {} (validity period/update interval/max entries) ({}/{}/{})",
+                    name, getValidity(), getUpdateInterval(), getMaxEntries());
+
+        LoadingCache<K, V> newcache = CacheBuilder.newBuilder()
+                           .refreshAfterWrite(getUpdateInterval(), TimeUnit.MILLISECONDS)
+                           .expireAfterWrite(getValidity(), TimeUnit.MILLISECONDS)
+                           .maximumSize(getMaxEntries())
+                           .build(new CacheLoader<K, V>()
+                           {
+                               public V load(K k) throws Exception
+                               {
+                                   return loadFunction.apply(k);
+                               }
+
+                               public ListenableFuture<V> reload(final K k, final V oldV)
+                               {
+                                   ListenableFutureTask<V> task = ListenableFutureTask.create(() -> {
+                                       try
+                                       {
+                                           return loadFunction.apply(k);
+                                       }
+                                       catch (Exception e)
+                                       {
+                                           logger.trace("Error performing async refresh of auth data in {}", name, e);
+                                           throw e;
+                                       }
+                                   });
+                                   cacheRefreshExecutor.execute(task);
+                                   return task;
+                               }
+                           });
+        if (existing != null)
+            newcache.putAll(existing.asMap());
+        return newcache;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/src/java/org/apache/cassandra/auth/AuthCacheMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthCacheMBean.java b/src/java/org/apache/cassandra/auth/AuthCacheMBean.java
new file mode 100644
index 0000000..43fb88e
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/AuthCacheMBean.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.auth;
+
+public interface AuthCacheMBean
+{
+    public void invalidate();
+
+    public void setValidity(int validityPeriod);
+
+    public int getValidity();
+
+    public void setUpdateInterval(int updateInterval);
+
+    public int getUpdateInterval();
+
+    public void setMaxEntries(int maxEntries);
+
+    public int getMaxEntries();
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
index 3bee0e3..3714523 100644
--- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
@@ -22,18 +22,23 @@ import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ExecutionException;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.UncheckedExecutionException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.statements.SelectStatement;
-import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.exceptions.AuthenticationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -68,6 +73,8 @@ public class PasswordAuthenticator implements IAuthenticator
     public static final String LEGACY_CREDENTIALS_TABLE = "credentials";
     private SelectStatement legacyAuthenticateStatement;
 
+    private CredentialsCache cache;
+
     // No anonymous access.
     public boolean requireAuthentication()
     {
@@ -78,17 +85,60 @@ public class PasswordAuthenticator implements IAuthenticator
     {
         try
         {
+            String hash = cache.get(username);
+            if (!BCrypt.checkpw(password, hash))
+                throw new AuthenticationException("Username and/or password are incorrect");
+
+            return new AuthenticatedUser(username);
+        }
+        catch (ExecutionException | UncheckedExecutionException e)
+        {
+            // the credentials were somehow invalid - either a non-existent role, or one without a defined password
+            if (e.getCause() instanceof NoSuchCredentialsException)
+                throw new AuthenticationException("Username and/or password are incorrect");
+
+            // an unanticipated exception occured whilst querying the credentials table
+            if (e.getCause() instanceof RequestExecutionException)
+            {
+                logger.trace("Error performing internal authentication", e);
+                throw new AuthenticationException(e.getMessage());
+            }
+
+            throw new RuntimeException(e);
+        }
+    }
+
+    private String queryHashedPassword(String username) throws NoSuchCredentialsException
+    {
+        try
+        {
             // If the legacy users table exists try to verify credentials there. This is to handle the case
             // where the cluster is being upgraded and so is running with mixed versions of the authn tables
             SelectStatement authenticationStatement = Schema.instance.getCFMetaData(AuthKeyspace.NAME, LEGACY_CREDENTIALS_TABLE) == null
                                                     ? authenticateStatement
                                                     : legacyAuthenticateStatement;
-            return doAuthenticate(username, password, authenticationStatement);
+
+            ResultMessage.Rows rows =
+                authenticationStatement.execute(QueryState.forInternalCalls(),
+                                                QueryOptions.forInternalCalls(consistencyForRole(username),
+                                                                              Lists.newArrayList(ByteBufferUtil.bytes(username))));
+
+            // If either a non-existent role name was supplied, or no credentials
+            // were found for that role we don't want to cache the result so we throw
+            // a specific, but unchecked, exception to keep LoadingCache happy.
+            if (rows.result.isEmpty())
+                throw new NoSuchCredentialsException();
+
+            UntypedResultSet result = UntypedResultSet.create(rows.result);
+            if (!result.one().has(SALTED_HASH))
+                throw new NoSuchCredentialsException();
+
+            return result.one().getString(SALTED_HASH);
         }
         catch (RequestExecutionException e)
         {
             logger.trace("Error performing internal authentication", e);
-            throw new AuthenticationException(e.toString());
+            throw e;
         }
     }
 
@@ -118,6 +168,8 @@ public class PasswordAuthenticator implements IAuthenticator
                                   LEGACY_CREDENTIALS_TABLE);
             legacyAuthenticateStatement = prepare(query);
         }
+
+        cache = new CredentialsCache(this);
     }
 
     public AuthenticatedUser legacyAuthenticate(Map<String, String> credentials) throws AuthenticationException
@@ -138,21 +190,7 @@ public class PasswordAuthenticator implements IAuthenticator
         return new PlainTextSaslAuthenticator();
     }
 
-    private AuthenticatedUser doAuthenticate(String username, String password, SelectStatement authenticationStatement)
-    throws RequestExecutionException, AuthenticationException
-    {
-        ResultMessage.Rows rows = authenticationStatement.execute(QueryState.forInternalCalls(),
-                                                                  QueryOptions.forInternalCalls(consistencyForRole(username),
-                                                                                                Lists.newArrayList(ByteBufferUtil.bytes(username))));
-        UntypedResultSet result = UntypedResultSet.create(rows.result);
-
-        if ((result.isEmpty() || !result.one().has(SALTED_HASH)) || !BCrypt.checkpw(password, result.one().getString(SALTED_HASH)))
-            throw new AuthenticationException("Username and/or password are incorrect");
-
-        return new AuthenticatedUser(username);
-    }
-
-    private SelectStatement prepare(String query)
+    private static SelectStatement prepare(String query)
     {
         return (SelectStatement) QueryProcessor.getStatement(query, ClientState.forInternalCalls()).statement;
     }
@@ -191,9 +229,8 @@ public class PasswordAuthenticator implements IAuthenticator
          * a user being authorized to act on behalf of another with this IAuthenticator).
          *
          * @param bytes encoded credentials string sent by the client
-         * @return map containing the username/password pairs in the form an IAuthenticator
-         * would expect
-         * @throws javax.security.sasl.SaslException
+         * @throws org.apache.cassandra.exceptions.AuthenticationException if either the
+         *         authnId or password is null
          */
         private void decodeCredentials(byte[] bytes) throws AuthenticationException
         {
@@ -222,4 +259,36 @@ public class PasswordAuthenticator implements IAuthenticator
             password = new String(pass, StandardCharsets.UTF_8);
         }
     }
+
+    private static class CredentialsCache extends AuthCache<String, String> implements CredentialsCacheMBean
+    {
+        private CredentialsCache(PasswordAuthenticator authenticator)
+        {
+            super("CredentialsCache",
+                  DatabaseDescriptor::setCredentialsValidity,
+                  DatabaseDescriptor::getCredentialsValidity,
+                  DatabaseDescriptor::setCredentialsUpdateInterval,
+                  DatabaseDescriptor::getCredentialsUpdateInterval,
+                  DatabaseDescriptor::setCredentialsCacheMaxEntries,
+                  DatabaseDescriptor::getCredentialsCacheMaxEntries,
+                  authenticator::queryHashedPassword,
+                  () -> true);
+        }
+
+        public void invalidateCredentials(String roleName)
+        {
+            invalidate(roleName);
+        }
+    }
+
+    public static interface CredentialsCacheMBean extends AuthCacheMBean
+    {
+        public void invalidateCredentials(String roleName);
+    }
+
+    // Just a marker so we can identify that invalid credentials were the
+    // cause of a loading exception from the cache
+    private static final class NoSuchCredentialsException extends RuntimeException
+    {
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/src/java/org/apache/cassandra/auth/PermissionsCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/PermissionsCache.java b/src/java/org/apache/cassandra/auth/PermissionsCache.java
index 95aa398..875c473 100644
--- a/src/java/org/apache/cassandra/auth/PermissionsCache.java
+++ b/src/java/org/apache/cassandra/auth/PermissionsCache.java
@@ -17,137 +17,36 @@
  */
 package org.apache.cassandra.auth;
 
-import java.lang.management.ManagementFactory;
 import java.util.Set;
-import java.util.concurrent.*;
+import java.util.concurrent.ExecutionException;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListenableFutureTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.utils.Pair;
 
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-public class PermissionsCache implements PermissionsCacheMBean
+public class PermissionsCache extends AuthCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> implements PermissionsCacheMBean
 {
-    private static final Logger logger = LoggerFactory.getLogger(PermissionsCache.class);
-
-    private final String MBEAN_NAME = "org.apache.cassandra.auth:type=PermissionsCache";
-
-    private final ThreadPoolExecutor cacheRefreshExecutor = new DebuggableThreadPoolExecutor("PermissionsCacheRefresh",
-                                                                                             Thread.NORM_PRIORITY);
-    private final IAuthorizer authorizer;
-    private volatile LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> cache;
-
     public PermissionsCache(IAuthorizer authorizer)
     {
-        this.authorizer = authorizer;
-        this.cache = initCache(null);
-        try
-        {
-            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-            mbs.registerMBean(this, new ObjectName(MBEAN_NAME));
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
+        super("PermissionsCache",
+              DatabaseDescriptor::setPermissionsValidity,
+              DatabaseDescriptor::getPermissionsValidity,
+              DatabaseDescriptor::setPermissionsUpdateInterval,
+              DatabaseDescriptor::getPermissionsUpdateInterval,
+              DatabaseDescriptor::setPermissionsCacheMaxEntries,
+              DatabaseDescriptor::getPermissionsCacheMaxEntries,
+              (p) -> authorizer.authorize(p.left, p.right),
+              () -> DatabaseDescriptor.getAuthorizer().requireAuthorization());
     }
 
     public Set<Permission> getPermissions(AuthenticatedUser user, IResource resource)
     {
-        if (cache == null)
-            return authorizer.authorize(user, resource);
-
         try
         {
-            return cache.get(Pair.create(user, resource));
+            return get(Pair.create(user, resource));
         }
         catch (ExecutionException e)
         {
             throw new RuntimeException(e);
         }
     }
-
-    public void invalidate()
-    {
-        cache = initCache(null);
-    }
-
-    public void setValidity(int validityPeriod)
-    {
-        DatabaseDescriptor.setPermissionsValidity(validityPeriod);
-        cache = initCache(cache);
-    }
-
-    public int getValidity()
-    {
-        return DatabaseDescriptor.getPermissionsValidity();
-    }
-
-    public void setUpdateInterval(int updateInterval)
-    {
-        DatabaseDescriptor.setPermissionsUpdateInterval(updateInterval);
-        cache = initCache(cache);
-    }
-
-    public int getUpdateInterval()
-    {
-        return DatabaseDescriptor.getPermissionsUpdateInterval();
-    }
-
-    private LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> initCache(
-                                                             LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> existing)
-    {
-        if (!authorizer.requireAuthorization())
-            return null;
-
-        if (DatabaseDescriptor.getPermissionsValidity() <= 0)
-            return null;
-
-        LoadingCache<Pair<AuthenticatedUser, IResource>, Set<Permission>> newcache = CacheBuilder.newBuilder()
-                           .refreshAfterWrite(DatabaseDescriptor.getPermissionsUpdateInterval(), TimeUnit.MILLISECONDS)
-                           .expireAfterWrite(DatabaseDescriptor.getPermissionsValidity(), TimeUnit.MILLISECONDS)
-                           .maximumSize(DatabaseDescriptor.getPermissionsCacheMaxEntries())
-                           .build(new CacheLoader<Pair<AuthenticatedUser, IResource>, Set<Permission>>()
-                           {
-                               public Set<Permission> load(Pair<AuthenticatedUser, IResource> userResource)
-                               {
-                                   return authorizer.authorize(userResource.left, userResource.right);
-                               }
-
-                               public ListenableFuture<Set<Permission>> reload(final Pair<AuthenticatedUser, IResource> userResource,
-                                                                               final Set<Permission> oldValue)
-                               {
-                                   ListenableFutureTask<Set<Permission>> task = ListenableFutureTask.create(new Callable<Set<Permission>>()
-                                   {
-                                       public Set<Permission>call() throws Exception
-                                       {
-                                           try
-                                           {
-                                               return authorizer.authorize(userResource.left, userResource.right);
-                                           }
-                                           catch (Exception e)
-                                           {
-                                               logger.trace("Error performing async refresh of user permissions", e);
-                                               throw e;
-                                           }
-                                       }
-                                   });
-                                   cacheRefreshExecutor.execute(task);
-                                   return task;
-                               }
-                           });
-        if (existing != null)
-            newcache.putAll(existing.asMap());
-        return newcache;
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/src/java/org/apache/cassandra/auth/PermissionsCacheMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/PermissionsCacheMBean.java b/src/java/org/apache/cassandra/auth/PermissionsCacheMBean.java
index d07c98f..d370d06 100644
--- a/src/java/org/apache/cassandra/auth/PermissionsCacheMBean.java
+++ b/src/java/org/apache/cassandra/auth/PermissionsCacheMBean.java
@@ -17,15 +17,10 @@
  */
 package org.apache.cassandra.auth;
 
-public interface PermissionsCacheMBean
+/**
+ * Retained since CASSANDRA-7715 for backwards compatibility of MBean interface
+ * classes. This should be removed in the next major version (4.0)
+ */
+public interface PermissionsCacheMBean extends AuthCacheMBean
 {
-    public void invalidate();
-
-    public void setValidity(int validityPeriod);
-
-    public int getValidity();
-
-    public void setUpdateInterval(int updateInterval);
-
-    public int getUpdateInterval();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/src/java/org/apache/cassandra/auth/RolesCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/RolesCache.java b/src/java/org/apache/cassandra/auth/RolesCache.java
index 2694173..8b9c322 100644
--- a/src/java/org/apache/cassandra/auth/RolesCache.java
+++ b/src/java/org/apache/cassandra/auth/RolesCache.java
@@ -17,135 +17,35 @@
  */
 package org.apache.cassandra.auth;
 
-import java.lang.management.ManagementFactory;
 import java.util.Set;
-import java.util.concurrent.*;
+import java.util.concurrent.ExecutionException;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListenableFutureTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-public class RolesCache implements RolesCacheMBean
+public class RolesCache extends AuthCache<RoleResource, Set<RoleResource>> implements RolesCacheMBean
 {
-    private static final Logger logger = LoggerFactory.getLogger(RolesCache.class);
-
-    private final String MBEAN_NAME = "org.apache.cassandra.auth:type=RolesCache";
-    private final ThreadPoolExecutor cacheRefreshExecutor = new DebuggableThreadPoolExecutor("RolesCacheRefresh",
-                                                                                             Thread.NORM_PRIORITY);
-    private final IRoleManager roleManager;
-    private volatile LoadingCache<RoleResource, Set<RoleResource>> cache;
-
     public RolesCache(IRoleManager roleManager)
     {
-        this.roleManager = roleManager;
-        this.cache = initCache(null);
-        try
-        {
-            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-            mbs.registerMBean(this, new ObjectName(MBEAN_NAME));
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
+        super("RolesCache",
+              DatabaseDescriptor::setRolesValidity,
+              DatabaseDescriptor::getRolesValidity,
+              DatabaseDescriptor::setRolesUpdateInterval,
+              DatabaseDescriptor::getRolesUpdateInterval,
+              DatabaseDescriptor::setRolesCacheMaxEntries,
+              DatabaseDescriptor::getRolesCacheMaxEntries,
+              (r) -> roleManager.getRoles(r, true),
+              () -> DatabaseDescriptor.getAuthenticator().requireAuthentication());
     }
 
     public Set<RoleResource> getRoles(RoleResource role)
     {
-        if (cache == null)
-            return roleManager.getRoles(role, true);
-
         try
         {
-            return cache.get(role);
+            return get(role);
         }
         catch (ExecutionException e)
         {
             throw new RuntimeException(e);
         }
     }
-
-    public void invalidate()
-    {
-        cache = initCache(null);
-    }
-
-    public void setValidity(int validityPeriod)
-    {
-        DatabaseDescriptor.setRolesValidity(validityPeriod);
-        cache = initCache(cache);
-    }
-
-    public int getValidity()
-    {
-        return DatabaseDescriptor.getRolesValidity();
-    }
-
-    public void setUpdateInterval(int updateInterval)
-    {
-        DatabaseDescriptor.setRolesUpdateInterval(updateInterval);
-        cache = initCache(cache);
-    }
-
-    public int getUpdateInterval()
-    {
-        return DatabaseDescriptor.getRolesUpdateInterval();
-    }
-
-
-    private LoadingCache<RoleResource, Set<RoleResource>> initCache(LoadingCache<RoleResource, Set<RoleResource>> existing)
-    {
-        if (!DatabaseDescriptor.getAuthenticator().requireAuthentication())
-            return null;
-
-        if (DatabaseDescriptor.getRolesValidity() <= 0)
-            return null;
-
-        LoadingCache<RoleResource, Set<RoleResource>> newcache = CacheBuilder.newBuilder()
-                .refreshAfterWrite(DatabaseDescriptor.getRolesUpdateInterval(), TimeUnit.MILLISECONDS)
-                .expireAfterWrite(DatabaseDescriptor.getRolesValidity(), TimeUnit.MILLISECONDS)
-                .maximumSize(DatabaseDescriptor.getRolesCacheMaxEntries())
-                .build(new CacheLoader<RoleResource, Set<RoleResource>>()
-                {
-                    public Set<RoleResource> load(RoleResource primaryRole)
-                    {
-                        return roleManager.getRoles(primaryRole, true);
-                    }
-
-                    public ListenableFuture<Set<RoleResource>> reload(final RoleResource primaryRole,
-                                                                      final Set<RoleResource> oldValue)
-                    {
-                        ListenableFutureTask<Set<RoleResource>> task;
-                        task = ListenableFutureTask.create(new Callable<Set<RoleResource>>()
-                        {
-                            public Set<RoleResource> call() throws Exception
-                            {
-                                try
-                                {
-                                    return roleManager.getRoles(primaryRole, true);
-                                } catch (Exception e)
-                                {
-                                    logger.trace("Error performing async refresh of user roles", e);
-                                    throw e;
-                                }
-                            }
-                        });
-                        cacheRefreshExecutor.execute(task);
-                        return task;
-                    }
-                });
-        if (existing != null)
-            newcache.putAll(existing.asMap());
-        return newcache;
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/src/java/org/apache/cassandra/auth/RolesCacheMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/RolesCacheMBean.java b/src/java/org/apache/cassandra/auth/RolesCacheMBean.java
index cf270e6..06482d7 100644
--- a/src/java/org/apache/cassandra/auth/RolesCacheMBean.java
+++ b/src/java/org/apache/cassandra/auth/RolesCacheMBean.java
@@ -17,15 +17,10 @@
  */
 package org.apache.cassandra.auth;
 
-public interface RolesCacheMBean
+/**
+ * Retained since CASSANDRA-7715 for backwards compatibility of MBean interface
+ * classes. This should be removed in the next major version (4.0)
+ */
+public interface RolesCacheMBean extends AuthCacheMBean
 {
-    public void invalidate();
-
-    public void setValidity(int validityPeriod);
-
-    public int getValidity();
-
-    public void setUpdateInterval(int updateInterval);
-
-    public int getUpdateInterval();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 9277df9..c783960 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -43,11 +43,14 @@ public class Config
     public String authorizer;
     public String role_manager;
     public volatile int permissions_validity_in_ms = 2000;
-    public int permissions_cache_max_entries = 1000;
+    public volatile int permissions_cache_max_entries = 1000;
     public volatile int permissions_update_interval_in_ms = -1;
     public volatile int roles_validity_in_ms = 2000;
-    public int roles_cache_max_entries = 1000;
+    public volatile int roles_cache_max_entries = 1000;
     public volatile int roles_update_interval_in_ms = -1;
+    public volatile int credentials_validity_in_ms = 2000;
+    public volatile int credentials_cache_max_entries = 1000;
+    public volatile int credentials_update_interval_in_ms = -1;
 
     /* Hashing strategy Random or OPHF */
     public String partitioner;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/186777aa/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 5c2a5c9..a3a9b4a 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -321,6 +321,19 @@ public class DatabaseDescriptor
         if (conf.authenticator != null)
             authenticator = FBUtilities.newAuthenticator(conf.authenticator);
 
+        // the configuration options regarding credentials caching are only guaranteed to
+        // work with PasswordAuthenticator, so log a message if some other authenticator
+        // is in use and non-default values are detected
+        if (!(authenticator instanceof PasswordAuthenticator)
+            && (conf.credentials_update_interval_in_ms != -1
+                || conf.credentials_validity_in_ms != 2000
+                || conf.credentials_cache_max_entries != 1000))
+        {
+            logger.info("Configuration options credentials_update_interval_in_ms, credentials_validity_in_ms and " +
+                        "credentials_cache_max_entries may not be applicable for the configured authenticator ({})",
+                        authenticator.getClass().getName());
+        }
+
         if (conf.authorizer != null)
             authorizer = FBUtilities.newAuthorizer(conf.authorizer);
 
@@ -754,16 +767,26 @@ public class DatabaseDescriptor
         conf.permissions_validity_in_ms = timeout;
     }
 
+    public static int getPermissionsUpdateInterval()
+    {
+        return conf.permissions_update_interval_in_ms == -1
+             ? conf.permissions_validity_in_ms
+             : conf.permissions_update_interval_in_ms;
+    }
+
+    public static void setPermissionsUpdateInterval(int updateInterval)
+    {
+        conf.permissions_update_interval_in_ms = updateInterval;
+    }
+
     public static int getPermissionsCacheMaxEntries()
     {
         return conf.permissions_cache_max_entries;
     }
 
-    public static int getPermissionsUpdateInterval()
+    public static int setPermissionsCacheMaxEntries(int maxEntries)
     {
-        return conf.permissions_update_interval_in_ms == -1
-             ? conf.permissions_validity_in_ms
-             : conf.permissions_update_interval_in_ms;
+        return conf.permissions_cache_max_entries = maxEntries;
     }
 
     public static int getRolesValidity()
@@ -776,11 +799,6 @@ public class DatabaseDescriptor
         conf.roles_validity_in_ms = validity;
     }
 
-    public static int getRolesCacheMaxEntries()
-    {
-        return conf.roles_cache_max_entries;
-    }
-
     public static int getRolesUpdateInterval()
     {
         return conf.roles_update_interval_in_ms == -1
@@ -793,9 +811,46 @@ public class DatabaseDescriptor
         conf.roles_update_interval_in_ms = interval;
     }
 
-    public static void setPermissionsUpdateInterval(int updateInterval)
+    public static int getRolesCacheMaxEntries()
     {
-        conf.permissions_update_interval_in_ms = updateInterval;
+        return conf.roles_cache_max_entries;
+    }
+
+    public static int setRolesCacheMaxEntries(int maxEntries)
+    {
+        return conf.roles_cache_max_entries = maxEntries;
+    }
+
+    public static int getCredentialsValidity()
+    {
+        return conf.credentials_validity_in_ms;
+    }
+
+    public static void setCredentialsValidity(int timeout)
+    {
+        conf.credentials_validity_in_ms = timeout;
+    }
+
+    public static int getCredentialsUpdateInterval()
+    {
+        return conf.credentials_update_interval_in_ms == -1
+               ? conf.credentials_validity_in_ms
+               : conf.credentials_update_interval_in_ms;
+    }
+
+    public static void setCredentialsUpdateInterval(int updateInterval)
+    {
+        conf.credentials_update_interval_in_ms = updateInterval;
+    }
+
+    public static int getCredentialsCacheMaxEntries()
+    {
+        return conf.credentials_cache_max_entries;
+    }
+
+    public static int setCredentialsCacheMaxEntries(int maxEntries)
+    {
+        return conf.credentials_cache_max_entries = maxEntries;
     }
 
     public static int getThriftFramedTransportSize()