This is an automated email from the ASF dual-hosted git repository.

jmckenzie pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 89c6add  Prewarm role and credentials caches to avoid timeouts at 
startup
89c6add is described below

commit 89c6addb505392ef3434597b61f83da25d13e3e2
Author: Josh McKenzie <[email protected]>
AuthorDate: Wed Sep 15 10:52:46 2021 -0400

    Prewarm role and credentials caches to avoid timeouts at startup
    
    Patch by Blake Eggleston; reviewed by Sam Tunnicliffe, Jason Brown, and 
Caleb Rackliffe for CASSANDRA-16958
    
    Co-authored by: Blake Eggleston <[email protected]>
    Co-authored by: Josh McKenzie <[email protected]>
---
 CHANGES.txt                                        |   1 +
 conf/cassandra.yaml                                |  12 +-
 src/java/org/apache/cassandra/auth/AuthCache.java  |  61 +++++-
 .../org/apache/cassandra/auth/AuthCacheMBean.java  |   2 +
 .../apache/cassandra/auth/AuthCacheService.java    |  80 ++++++++
 .../apache/cassandra/auth/AuthenticatedUser.java   |  10 +-
 .../apache/cassandra/auth/CassandraAuthorizer.java |  98 +++++++++-
 .../cassandra/auth/CassandraNetworkAuthorizer.java |  46 ++++-
 .../cassandra/auth/CassandraRoleManager.java       |  58 ++++--
 .../org/apache/cassandra/auth/IAuthorizer.java     |   3 +-
 .../apache/cassandra/auth/INetworkAuthorizer.java  |   2 +-
 .../org/apache/cassandra/auth/IRoleManager.java    |   2 +-
 .../cassandra/auth/NetworkPermissionsCache.java    |   1 +
 .../cassandra/auth/PasswordAuthenticator.java      |  44 ++++-
 .../apache/cassandra/auth/PermissionsCache.java    |   3 +-
 src/java/org/apache/cassandra/auth/Roles.java      |  18 +-
 src/java/org/apache/cassandra/auth/RolesCache.java |  13 ++
 .../cassandra/auth/jmx/AuthorizationProxy.java     |   2 +
 src/java/org/apache/cassandra/config/Config.java   |   2 +
 .../cassandra/config/DatabaseDescriptor.java       |   5 +
 .../apache/cassandra/service/CassandraDaemon.java  |   9 +-
 .../apache/cassandra/service/StorageService.java   |  24 +--
 .../cassandra/distributed/test/AuthTest.java       |   1 -
 .../org/apache/cassandra/auth/AuthCacheTest.java   | 215 +++++++++++++++------
 .../org/apache/cassandra/auth/AuthTestUtils.java   |  10 +-
 .../apache/cassandra/auth/CacheRefresherTest.java  |   2 +-
 .../cassandra/auth/CassandraAuthorizerTest.java    |   2 +
 .../auth/CassandraAuthorizerTruncatingTests.java   | 118 +++++++++++
 .../auth/CassandraNetworkAuthorizerTest.java       |   2 +-
 .../cassandra/auth/CassandraRoleManagerTest.java   |  84 +++++++-
 .../cassandra/auth/PasswordAuthenticatorTest.java  |  70 +++++--
 test/unit/org/apache/cassandra/auth/RolesTest.java |   3 +
 .../config/DatabaseDescriptorRefTest.java          |   2 +
 test/unit/org/apache/cassandra/cql3/CQLTester.java |  10 +-
 .../apache/cassandra/service/ClientStateTest.java  |   4 +
 .../InvalidateJmxPermissionsCacheTest.java         |   3 +
 .../InvalidateNetworkPermissionsCacheTest.java     |   2 +
 .../nodetool/InvalidatePermissionsCacheTest.java   |   2 +
 .../tools/nodetool/InvalidateRolesCacheTest.java   |   2 +
 39 files changed, 883 insertions(+), 145 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index e50d56c..c3d1e81 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1
+ * Prewarm role and credential caches to avoid timeouts at startup 
(CASSANDRA-16958)
  * Make capacity/validity/updateinterval/activeupdate for Auth Caches 
configurable via nodetool (CASSANDRA-17063)
  * Added startup check for read_ahead_kb setting (CASSANDRA-16436)
  * Avoid unecessary array allocations and initializations when performing 
query checks (CASSANDRA-17209)
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index b0ab525..ae7d598 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -976,10 +976,10 @@ sstable_preemptive_open_interval_in_mb: 50
 #      failures.
 #   2) operations that use too much CPU/read too much data (leading to memory 
build
 #      up) by putting a limit to how long an operation will execute.
-# For this reason, you should avoid putting these settings too high. In other 
words, 
-# if you are timing out requests because of underlying resource constraints 
then 
-# increasing the timeout will just cause more problems. Of course putting them 
too 
-# low is equally ill-advised since clients could get timeouts even for 
successful 
+# For this reason, you should avoid putting these settings too high. In other 
words,
+# if you are timing out requests because of underlying resource constraints 
then
+# increasing the timeout will just cause more problems. Of course putting them 
too
+# low is equally ill-advised since clients could get timeouts even for 
successful
 # operations just because the timeout setting is too tight.
 
 # How long the coordinator should wait for read operations to complete.
@@ -1526,6 +1526,10 @@ report_unconfirmed_repaired_data_mismatches: false
 # auth_read_consistency_level: LOCAL_QUORUM
 # auth_write_consistency_level: EACH_QUORUM
 
+# Delays on auth resolution can lead to a thundering herd problem on 
reconnects; this option will enable
+# warming of auth caches prior to node completing startup. See CASSANDRA-16958
+# auth_cache_warming_enabled: false
+
 #########################
 # EXPERIMENTAL FEATURES #
 #########################
diff --git a/src/java/org/apache/cassandra/auth/AuthCache.java 
b/src/java/org/apache/cassandra/auth/AuthCache.java
index 04d3784..66a2a4f 100644
--- a/src/java/org/apache/cassandra/auth/AuthCache.java
+++ b/src/java/org/apache/cassandra/auth/AuthCache.java
@@ -31,7 +31,9 @@ import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.function.IntConsumer;
 import java.util.function.IntSupplier;
+import java.util.function.Supplier;
 
+import com.google.common.util.concurrent.Uninterruptibles;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,6 +55,11 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
 
     public static final String MBEAN_NAME_BASE = 
"org.apache.cassandra.auth:type=";
 
+    // We expect default values on cache retries and interval to be sufficient 
for everyone but have this escape hatch
+    // just in case.
+    static final String CACHE_LOAD_RETRIES_PROPERTY = 
"cassandra.auth_cache.warming.max_retries";
+    static final String CACHE_LOAD_RETRY_INTERVAL_PROPERTY = 
"cassandra.auth_cache.warming.retry_interval_ms";
+
     private volatile ScheduledFuture cacheRefresher = null;
 
     // Keep a handle on created instances so their executors can be terminated 
cleanly
@@ -79,6 +86,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
     private final Consumer<Boolean> setActiveUpdate;
     private final BooleanSupplier getActiveUpdate;
     private final Function<K, V> loadFunction;
+    private final Supplier<Map<K, V>> bulkLoadFunction;
     private final BooleanSupplier enableCache;
 
     // Determines whether the presence of a specific value should trigger the 
invalidation of
@@ -110,6 +118,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
                         Consumer<Boolean> setActiveUpdate,
                         BooleanSupplier getActiveUpdate,
                         Function<K, V> loadFunction,
+                        Supplier<Map<K, V>> bulkLoadFunction,
                         BooleanSupplier cacheEnabledDelegate)
     {
         this(name,
@@ -122,6 +131,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
              setActiveUpdate,
              getActiveUpdate,
              loadFunction,
+             bulkLoadFunction,
              cacheEnabledDelegate,
              (k, v) -> false);
     }
@@ -151,6 +161,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
                         Consumer<Boolean> setActiveUpdate,
                         BooleanSupplier getActiveUpdate,
                         Function<K, V> loadFunction,
+                        Supplier<Map<K, V>> bulkLoadFunction,
                         BooleanSupplier cacheEnabledDelegate,
                         BiPredicate<K, V> invalidationCondition)
     {
@@ -164,6 +175,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
         this.setActiveUpdate = checkNotNull(setActiveUpdate);
         this.getActiveUpdate = checkNotNull(getActiveUpdate);
         this.loadFunction = checkNotNull(loadFunction);
+        this.bulkLoadFunction = checkNotNull(bulkLoadFunction);
         this.enableCache = checkNotNull(cacheEnabledDelegate);
         this.invalidateCondition = checkNotNull(invalidationCondition);
         init();
@@ -191,7 +203,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
     }
 
     /**
-     * Retrive all cached entries. Will call {@link LoadingCache#asMap()} 
which does not trigger "load".
+     * Retrieve all cached entries. Will call {@link LoadingCache#asMap()} 
which does not trigger "load".
      * @return a map of cached key-value pairs
      */
     public Map<K, V> getAll()
@@ -201,6 +213,7 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
 
         return Collections.unmodifiableMap(cache.asMap());
     }
+
     /**
      * Retrieve a value from the cache. Will call {@link 
LoadingCache#get(Object)} which will
      * "load" the value if it's not present, thus populating the key.
@@ -303,6 +316,11 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
         cache = initCache(cache);
     }
 
+    public long getEstimatedSize()
+    {
+        return cache == null ? 0L : cache.estimatedSize();
+    }
+
     /**
      * (Re-)initialise the underlying cache. Will update validity, max 
entries, and update interval if
      * any have changed. The underlying {@link LoadingCache} will be initiated 
based on the provided {@code loadFunction}.
@@ -382,4 +400,45 @@ public class AuthCache<K, V> implements AuthCacheMBean, 
Shutdownable
     {
         return cacheRefreshExecutor.awaitTermination(timeout, units);
     }
+
+    public void warm()
+    {
+        if (cache == null)
+        {
+            logger.info("{} cache not enabled, skipping pre-warming", name);
+            return;
+        }
+
+        int retries = Integer.getInteger(CACHE_LOAD_RETRIES_PROPERTY, 10);
+        long retryInterval = Long.getLong(CACHE_LOAD_RETRY_INTERVAL_PROPERTY, 
1000);
+
+        while (retries-- > 0)
+        {
+            try
+            {
+                Map<K, V> entries = bulkLoadFunction.get();
+                cache.putAll(entries);
+                break;
+            }
+            catch (Exception e)
+            {
+                Uninterruptibles.sleepUninterruptibly(retryInterval, 
TimeUnit.MILLISECONDS);
+            }
+        }
+    }
+
+    /*
+     * Implemented when we can provide an efficient way to bulk load all 
entries for a cache. This isn't a
+     * @FunctionalInterface due to the default impl, which is for 
IRoleManager, IAuthorizer, and INetworkAuthorizer.
+     * They all extend this interface so that implementations only need to 
provide an override if it's useful.
+     * IAuthenticator doesn't implement this interface because 
CredentialsCache is more tightly coupled to
+     * PasswordAuthenticator, which does expose a bulk loader.
+     */
+    public interface BulkLoader<K, V>
+    {
+        default Supplier<Map<K, V>> bulkLoader()
+        {
+            return Collections::emptyMap;
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/auth/AuthCacheMBean.java 
b/src/java/org/apache/cassandra/auth/AuthCacheMBean.java
index 3f6247a..e443434 100644
--- a/src/java/org/apache/cassandra/auth/AuthCacheMBean.java
+++ b/src/java/org/apache/cassandra/auth/AuthCacheMBean.java
@@ -37,4 +37,6 @@ public interface AuthCacheMBean
     public boolean getActiveUpdate();
 
     public void setActiveUpdate(boolean update);
+
+    public long getEstimatedSize();
 }
diff --git a/src/java/org/apache/cassandra/auth/AuthCacheService.java 
b/src/java/org/apache/cassandra/auth/AuthCacheService.java
new file mode 100644
index 0000000..f6ee02e
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/AuthCacheService.java
@@ -0,0 +1,80 @@
+/*
+ * 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.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@ThreadSafe
+public class AuthCacheService
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(AuthCacheService.class);
+    public static final AuthCacheService instance = new AuthCacheService();
+
+    private final Set<AuthCache<?, ?>> caches = new HashSet<>();
+    private static final AtomicBoolean cachesRegistered = new 
AtomicBoolean(false);
+
+    public synchronized void register(AuthCache<?, ?> cache)
+    {
+        Preconditions.checkNotNull(cache);
+        caches.add(cache);
+    }
+
+    public synchronized void unregister(AuthCache<?, ?> cache)
+    {
+        Preconditions.checkNotNull(cache);
+        caches.remove(cache);
+    }
+
+    public synchronized void warmCaches()
+    {
+        logger.info("Initiating bulk load of {} auth cache(s)", caches.size());
+        for (AuthCache<?, ?> cache : caches)
+        {
+            cache.warm();
+        }
+    }
+
+    /**
+     * NOTE: Can only be called once per instance run.
+     *
+     * We have a couple of static initializer functions to create caches 
scattered across various classes, some solo
+     * and some with multiple member variables. As we expect these caches to 
be created and initialized in one logical
+     * block, we tie them together and use them here.
+     *
+     * Note: We also register the PasswordAuthenticator cache with the {@link 
AuthCacheService} in it's constructor
+     */
+    @VisibleForTesting
+    public static void initializeAndRegisterCaches()
+    {
+        if (!cachesRegistered.getAndSet(true))
+        {
+            AuthenticatedUser.init();
+            Roles.init();
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java 
b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
index f0f927e..ccac0ff 100644
--- a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
+++ b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
@@ -41,8 +41,16 @@ public class AuthenticatedUser
     public static final PermissionsCache permissionsCache = new 
PermissionsCache(DatabaseDescriptor.getAuthorizer());
     public static final NetworkPermissionsCache networkPermissionsCache = new 
NetworkPermissionsCache(DatabaseDescriptor.getNetworkAuthorizer());
 
+    /** Use {@link AuthCacheService#initializeAndRegisterCaches} rather than 
calling this directly */
+    public static void init()
+    {
+        AuthCacheService.instance.register(permissionsCache);
+        AuthCacheService.instance.register(networkPermissionsCache);
+    }
+
     private final String name;
-    // primary Role of the logged in user
+
+    // Primary Role of the logged in user
     private final RoleResource role;
 
     public AuthenticatedUser(String name)
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java 
b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index c808ebc..c5e1cca 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -18,10 +18,16 @@
 package org.apache.cassandra.auth;
 
 import java.util.*;
+import java.util.function.BiConsumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Table;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,19 +36,17 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
-import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
 
-import static 
org.apache.cassandra.cql3.BatchQueryOptions.withoutPerStatementVariables;
-import static org.apache.cassandra.cql3.QueryOptions.DEFAULT;
-import static org.apache.cassandra.service.QueryState.forInternalCalls;
 import static org.apache.cassandra.utils.Clock.Global.nanoTime;
 
 /**
@@ -364,6 +368,10 @@ public class CassandraAuthorizer implements IAuthorizer
         return statement.execute(QueryState.forInternalCalls(), options, 
nanoTime());
     }
 
+    /**
+     * This is exposed so we can override the consistency level for tests that 
are single node
+     */
+    @VisibleForTesting
     UntypedResultSet process(String query, ConsistencyLevel cl) throws 
RequestExecutionException
     {
         return QueryProcessor.process(query, cl);
@@ -387,4 +395,84 @@ public class CassandraAuthorizer implements IAuthorizer
     {
         return AuthProperties.instance.getReadConsistencyLevel();
     }
-}
\ No newline at end of file
+
+    /**
+     * Get an initial set of permissions to load into the PermissionsCache at 
startup
+     * @return map of User/Resource -> Permissions for cache initialisation
+     */
+    public Supplier<Map<Pair<AuthenticatedUser, IResource>, Set<Permission>>> 
bulkLoader()
+    {
+        return () ->
+        {
+            Map<Pair<AuthenticatedUser, IResource>, Set<Permission>> entries = 
new HashMap<>();
+            String cqlTemplate = "SELECT %s, %s, %s FROM %s.%s";
+
+            logger.info("Warming permissions cache from role_permissions 
table");
+            UntypedResultSet results = process(String.format(cqlTemplate,
+                                                             ROLE, RESOURCE, 
PERMISSIONS,
+                                                             
SchemaConstants.AUTH_KEYSPACE_NAME, AuthKeyspace.ROLE_PERMISSIONS),
+                                                             
AuthProperties.instance.getReadConsistencyLevel());
+
+            // role_name -> (resource, permissions)
+            Table<String, IResource, Set<Permission>> 
individualRolePermissions = HashBasedTable.create();
+            results.forEach(row -> {
+                if (row.has(PERMISSIONS))
+                {
+                    individualRolePermissions.put(row.getString(ROLE),
+                                                  
Resources.fromName(row.getString(RESOURCE)),
+                                                  
permissions(row.getSet(PERMISSIONS, UTF8Type.instance)));
+                }
+            });
+
+            // Iterate all user level roles in the system and accumulate the 
permissions of their granted roles
+            Roles.getAllRoles().forEach(roleResource -> {
+                // If the role has login priv, accumulate the permissions of 
all its granted roles
+                if (Roles.canLogin(roleResource))
+                {
+                    // Structure to accumulate the resource -> permission 
mappings for the closure of granted roles
+                    Map<IResource, ImmutableSet.Builder<Permission>> 
userPermissions = new HashMap<>();
+                    BiConsumer<IResource, Set<Permission>> accumulator = 
accumulator(userPermissions);
+
+                    // For each role granted to this primary, lookup the 
specific resource/permissions grants
+                    // we read in the first step. We'll accumlate those in the 
userPermissions map, which we'll turn
+                    // into cache entries when we're done.
+                    // Note: we need to provide a default empty set of 
permissions for roles without any explicitly
+                    // granted to them (e.g. superusers or roles with no 
direct perms).
+                    Roles.getRoleDetails(roleResource).forEach(grantedRole ->
+                                                               
individualRolePermissions.rowMap()
+                                                                               
         .getOrDefault(grantedRole.resource.getRoleName(), 
Collections.emptyMap())
+                                                                               
         .forEach(accumulator));
+
+                    // Having iterated all the roles granted to this user, 
finalize the transitive permissions
+                    // (i.e. turn them into entries for the PermissionsCache)
+                    userPermissions.forEach((resource, builder) -> 
entries.put(cacheKey(roleResource, resource),
+                                                                               
builder.build()));
+                }
+            });
+
+            return entries;
+        };
+    }
+
+    // Helper function to group the transitive set of permissions granted
+    // to user by the specific resources to which they apply
+    private static BiConsumer<IResource, Set<Permission>> 
accumulator(Map<IResource, ImmutableSet.Builder<Permission>> accumulator)
+    {
+        return (resource, permissions) -> 
accumulator.computeIfAbsent(resource, k -> new 
ImmutableSet.Builder<>()).addAll(permissions);
+    }
+
+    private static Set<Permission> permissions(Set<String> permissionNames)
+    {
+        return 
permissionNames.stream().map(Permission::valueOf).collect(Collectors.toSet());
+    }
+
+    private static Pair<AuthenticatedUser, IResource> cacheKey(RoleResource 
role, IResource resource)
+    {
+        return cacheKey(role.getRoleName(), resource);
+    }
+
+    private static Pair<AuthenticatedUser, IResource> cacheKey(String 
roleName, IResource resource)
+    {
+        return Pair.create(new AuthenticatedUser(roleName), resource);
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java 
b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
index 4384f7a..cc08e93 100644
--- a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
@@ -18,11 +18,19 @@
 
 package org.apache.cassandra.auth;
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Set;
+import java.util.function.Supplier;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -31,7 +39,6 @@ import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -40,6 +47,7 @@ import static 
org.apache.cassandra.utils.Clock.Global.nanoTime;
 
 public class CassandraNetworkAuthorizer implements INetworkAuthorizer
 {
+    private static final Logger logger = 
LoggerFactory.getLogger(CassandraNetworkAuthorizer.class);
     private SelectStatement authorizeUserStatement = null;
 
     public void setup()
@@ -56,10 +64,13 @@ public class CassandraNetworkAuthorizer implements 
INetworkAuthorizer
         return statement.execute(forInternalCalls(), options, nanoTime());
     }
 
+    /**
+     * This is exposed so we can override the consistency level for tests that 
are single node
+     */
     @VisibleForTesting
-    void process(String query)
+    UntypedResultSet process(String query, ConsistencyLevel cl) throws 
RequestExecutionException
     {
-        QueryProcessor.process(query, 
CassandraAuthorizer.authWriteConsistencyLevel());
+        return QueryProcessor.process(query, cl);
     }
 
     private Set<String> getAuthorizedDcs(String name)
@@ -138,7 +149,7 @@ public class CassandraNetworkAuthorizer implements 
INetworkAuthorizer
                                      getSetString(permissions),
                                      role.getName());
 
-        process(query);
+        process(query, CassandraAuthorizer.authWriteConsistencyLevel());
     }
 
     public void drop(RoleResource role)
@@ -148,11 +159,36 @@ public class CassandraNetworkAuthorizer implements 
INetworkAuthorizer
                                      AuthKeyspace.NETWORK_PERMISSIONS,
                                      role.getName());
 
-        process(query);
+        process(query, CassandraAuthorizer.authWriteConsistencyLevel());
     }
 
     public void validateConfiguration() throws ConfigurationException
     {
         // noop
     }
+
+    @Override
+    public Supplier<Map<RoleResource, DCPermissions>> bulkLoader()
+    {
+        return () -> {
+            logger.info("Pre-warming datacenter permissions cache from 
network_permissions table");
+            Map<RoleResource, DCPermissions> entries = new HashMap<>();
+            UntypedResultSet rows = process(String.format("SELECT role, dcs 
FROM %s.%s",
+                                                          
SchemaConstants.AUTH_KEYSPACE_NAME,
+                                                          
AuthKeyspace.NETWORK_PERMISSIONS),
+                                            
CassandraAuthorizer.authReadConsistencyLevel());
+
+            for (UntypedResultSet.Row row : rows)
+            {
+                RoleResource role = RoleResource.role(row.getString("role"));
+                DCPermissions.Builder builder = new DCPermissions.Builder();
+                Set<String> dcs = row.getFrozenSet("dcs", UTF8Type.instance);
+                for (String dc : dcs)
+                    builder.add(dc);
+                entries.put(role, builder.build());
+            }
+
+            return entries;
+        };
+    }
 }
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java 
b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index 802b998..6e8f7d8 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -22,6 +22,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.function.Predicate;
+import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
@@ -42,7 +43,6 @@ import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -132,9 +132,6 @@ public class CassandraRoleManager implements IRoleManager
     private final Set<Option> supportedOptions;
     private final Set<Option> alterableOptions;
 
-    // Will be set to true when all nodes in the cluster are on a version 
which supports roles (i.e. 2.2+)
-    private volatile boolean isClusterReady = false;
-
     public CassandraRoleManager()
     {
         supportedOptions = DatabaseDescriptor.getAuthenticator() instanceof 
PasswordAuthenticator
@@ -149,7 +146,6 @@ public class CassandraRoleManager implements IRoleManager
     public void setup()
     {
         loadRoleStatement();
-
         scheduleSetupTask(() -> {
             setupDefaultRole();
             return null;
@@ -262,7 +258,8 @@ public class CassandraRoleManager implements IRoleManager
     {
         return collectRoles(getRole(grantee.getRoleName()),
                             includeInherited,
-                            filter())
+                            filter(),
+                            this::getRole)
                .map(r -> r.resource)
                .collect(Collectors.toSet());
     }
@@ -271,7 +268,8 @@ public class CassandraRoleManager implements IRoleManager
     {
         return collectRoles(getRole(grantee.getRoleName()),
                             true,
-                            filter())
+                            filter(),
+                            this::getRole)
                .collect(Collectors.toSet());
     }
 
@@ -363,7 +361,8 @@ public class CassandraRoleManager implements IRoleManager
         }
     }
 
-    protected static String createDefaultRoleQuery()
+    @VisibleForTesting
+    public static String createDefaultRoleQuery()
     {
         return String.format("INSERT INTO %s.%s (role, is_superuser, 
can_login, salted_hash) VALUES ('%s', true, true, '%s')",
                              SchemaConstants.AUTH_KEYSPACE_NAME,
@@ -372,7 +371,8 @@ public class CassandraRoleManager implements IRoleManager
                              escape(hashpw(DEFAULT_SUPERUSER_PASSWORD)));
     }
 
-    private static boolean hasExistingRoles() throws RequestExecutionException
+    @VisibleForTesting
+    public static boolean hasExistingRoles() throws RequestExecutionException
     {
         // Try looking up the 'cassandra' default role first, to avoid the 
range query if possible.
         String defaultSUQuery = String.format("SELECT * FROM %s.%s WHERE role 
= '%s'", SchemaConstants.AUTH_KEYSPACE_NAME, AuthKeyspace.ROLES, 
DEFAULT_SUPERUSER_NAME);
@@ -386,7 +386,6 @@ public class CassandraRoleManager implements IRoleManager
     {
         // The delay is to give the node a chance to see its peers before 
attempting the operation
         ScheduledExecutors.optionalTasks.scheduleSelfRecurring(() -> {
-            isClusterReady = true;
             try
             {
                 setupTask.call();
@@ -411,19 +410,22 @@ public class CassandraRoleManager implements IRoleManager
         }
     }
 
-    private Stream<Role> collectRoles(Role role, boolean includeInherited, 
Predicate<String> distinctFilter)
+    // Providing a function to fetch the details of granted roles allows us to 
read from the underlying tables during
+    // normal usage and fetch from a prepopulated in memory structure when 
building an initial set of roles to warm
+    // the RolesCache at startup
+    private Stream<Role> collectRoles(Role role, boolean includeInherited, 
Predicate<String> distinctFilter, Function<String, Role> loaderFunction)
     {
         if (Roles.isNullRole(role))
             return Stream.empty();
 
         if (!includeInherited)
-            return Stream.concat(Stream.of(role), 
role.memberOf.stream().map(this::getRole));
+            return Stream.concat(Stream.of(role), 
role.memberOf.stream().map(loaderFunction));
 
 
         return Stream.concat(Stream.of(role),
                              role.memberOf.stream()
                                           .filter(distinctFilter)
-                                          .flatMap(r -> 
collectRoles(getRole(r), true, distinctFilter)));
+                                          .flatMap(r -> 
collectRoles(loaderFunction.apply(r), true, distinctFilter, loaderFunction)));
     }
 
     // Used as a stateful filtering function when recursively collecting 
granted roles
@@ -551,11 +553,6 @@ public class CassandraRoleManager implements IRoleManager
     UntypedResultSet process(String query, ConsistencyLevel consistencyLevel)
     throws RequestValidationException, RequestExecutionException
     {
-        if (!isClusterReady)
-            throw new InvalidRequestException("Cannot process role related 
query as the role manager isn't yet setup. "
-                                            + "This is likely because some of 
nodes in the cluster are on version 2.1 or earlier. "
-                                            + "You need to upgrade all nodes 
to Cassandra 2.2 or more to use roles.");
-
         return QueryProcessor.process(query, consistencyLevel);
     }
 
@@ -564,4 +561,27 @@ public class CassandraRoleManager implements IRoleManager
     {
         return statement.execute(forInternalCalls(), options, nanoTime());
     }
-}
\ No newline at end of file
+
+    @Override
+    public Supplier<Map<RoleResource, Set<Role>>> bulkLoader()
+    {
+        return () ->
+        {
+            Map<RoleResource, Set<Role>> entries = new HashMap<>();
+
+            logger.info("Warming roles cache from roles table");
+            UntypedResultSet results = process("SELECT * FROM 
system_auth.roles", CassandraAuthorizer.authReadConsistencyLevel());
+
+            // Create flat temporary lookup of name -> role mappings
+            Map<String, Role> roles = new HashMap<>();
+            results.forEach(row -> roles.put(row.getString("role"), 
ROW_TO_ROLE.apply(row)));
+
+            // Iterate the flat structure and populate the fully hierarchical 
one
+            roles.forEach((key, value) ->
+                          entries.put(RoleResource.role(key),
+                                      collectRoles(value, true, filter(), 
roles::get).collect(Collectors.toSet()))
+            );
+            return entries;
+        };
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/IAuthorizer.java 
b/src/java/org/apache/cassandra/auth/IAuthorizer.java
index a023e3e..a6c5eff 100644
--- a/src/java/org/apache/cassandra/auth/IAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/IAuthorizer.java
@@ -22,11 +22,12 @@ import java.util.Set;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * Primary Cassandra authorization interface.
  */
-public interface IAuthorizer
+public interface IAuthorizer extends 
AuthCache.BulkLoader<Pair<AuthenticatedUser, IResource>, Set<Permission>>
 {
     /**
      * Whether or not the authorizer will attempt authorization.
diff --git a/src/java/org/apache/cassandra/auth/INetworkAuthorizer.java 
b/src/java/org/apache/cassandra/auth/INetworkAuthorizer.java
index 9a5a5d6..9f815ac 100644
--- a/src/java/org/apache/cassandra/auth/INetworkAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/INetworkAuthorizer.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.auth;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
 
-public interface INetworkAuthorizer
+public interface INetworkAuthorizer extends AuthCache.BulkLoader<RoleResource, 
DCPermissions>
 {
     /**
      * Whether or not the authorizer will attempt authorization.
diff --git a/src/java/org/apache/cassandra/auth/IRoleManager.java 
b/src/java/org/apache/cassandra/auth/IRoleManager.java
index 1d47bee..6a65e65 100644
--- a/src/java/org/apache/cassandra/auth/IRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/IRoleManager.java
@@ -31,7 +31,7 @@ import 
org.apache.cassandra.exceptions.RequestValidationException;
  * alteration and the granting and revoking of roles to other
  * roles.
  */
-public interface IRoleManager
+public interface IRoleManager extends AuthCache.BulkLoader<RoleResource, 
Set<Role>>
 {
 
     /**
diff --git a/src/java/org/apache/cassandra/auth/NetworkPermissionsCache.java 
b/src/java/org/apache/cassandra/auth/NetworkPermissionsCache.java
index b2e8707..1c18fed 100644
--- a/src/java/org/apache/cassandra/auth/NetworkPermissionsCache.java
+++ b/src/java/org/apache/cassandra/auth/NetworkPermissionsCache.java
@@ -35,6 +35,7 @@ public class NetworkPermissionsCache extends 
AuthCache<RoleResource, DCPermissio
               DatabaseDescriptor::setRolesCacheActiveUpdate,
               DatabaseDescriptor::getRolesCacheActiveUpdate,
               authorizer::authorize,
+              authorizer.bulkLoader(),
               () -> 
DatabaseDescriptor.getAuthenticator().requireAuthentication());
 
         MBeanWrapper.instance.registerMBean(this, MBEAN_NAME_BASE + 
DEPRECATED_CACHE_NAME);
diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java 
b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
index 1ac0088..0ce96d8 100644
--- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
@@ -20,8 +20,10 @@ package org.apache.cassandra.auth;
 import java.net.InetAddress;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Supplier;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
@@ -30,6 +32,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -56,7 +59,7 @@ import static 
org.apache.cassandra.utils.Clock.Global.nanoTime;
  * PasswordAuthenticator requires the use of CassandraRoleManager
  * for storage and retrieval of encrypted passwords.
  */
-public class PasswordAuthenticator implements IAuthenticator
+public class PasswordAuthenticator implements IAuthenticator, 
AuthCache.BulkLoader<String, String>
 {
     private static final Logger logger = 
LoggerFactory.getLogger(PasswordAuthenticator.class);
 
@@ -73,7 +76,13 @@ public class PasswordAuthenticator implements IAuthenticator
     static final byte NUL = 0;
     private SelectStatement authenticateStatement;
 
-    private CredentialsCache cache;
+    private final CredentialsCache cache;
+
+    public PasswordAuthenticator()
+    {
+        cache = new CredentialsCache(this);
+        AuthCacheService.instance.register(cache);
+    }
 
     // No anonymous access.
     public boolean requireAuthentication()
@@ -81,6 +90,25 @@ public class PasswordAuthenticator implements IAuthenticator
         return true;
     }
 
+    @Override
+    public Supplier<Map<String, String>> bulkLoader()
+    {
+        return () ->
+        {
+            Map<String, String> entries = new HashMap<>();
+
+            logger.info("Pre-warming credentials cache from roles table");
+            UntypedResultSet results = process("SELECT role, salted_hash FROM 
system_auth.roles", CassandraAuthorizer.authReadConsistencyLevel());
+            results.forEach(row -> {
+                if (row.has("salted_hash"))
+                {
+                    entries.put(row.getString("role"), 
row.getString("salted_hash"));
+                }
+            });
+            return entries;
+        };
+    }
+
     public CredentialsCache getCredentialsCache()
     {
         return cache;
@@ -100,6 +128,15 @@ public class PasswordAuthenticator implements 
IAuthenticator
         }
     }
 
+    /**
+     * This is exposed so we can override the consistency level for tests that 
are single node
+     */
+    @VisibleForTesting
+    UntypedResultSet process(String query, ConsistencyLevel cl)
+    {
+        return QueryProcessor.process(query, cl);
+    }
+
     private AuthenticatedUser authenticate(String username, String password) 
throws AuthenticationException
     {
         String hash = cache.get(username);
@@ -181,8 +218,6 @@ public class PasswordAuthenticator implements IAuthenticator
                                      SchemaConstants.AUTH_KEYSPACE_NAME,
                                      AuthKeyspace.ROLES);
         authenticateStatement = prepare(query);
-
-        cache = new CredentialsCache(this);
     }
 
     public AuthenticatedUser legacyAuthenticate(Map<String, String> 
credentials) throws AuthenticationException
@@ -290,6 +325,7 @@ public class PasswordAuthenticator implements IAuthenticator
                   DatabaseDescriptor::setCredentialsCacheActiveUpdate,
                   DatabaseDescriptor::getCredentialsCacheActiveUpdate,
                   authenticator::queryHashedPassword,
+                  authenticator.bulkLoader(),
                   () -> true,
                   (k,v) -> NO_SUCH_CREDENTIAL == v); // use a known object as 
a sentinel value. CacheRefresher will
                                                      // invalidate the key if 
the sentinel is loaded during a refresh
diff --git a/src/java/org/apache/cassandra/auth/PermissionsCache.java 
b/src/java/org/apache/cassandra/auth/PermissionsCache.java
index 9f200a0..0757b5e 100644
--- a/src/java/org/apache/cassandra/auth/PermissionsCache.java
+++ b/src/java/org/apache/cassandra/auth/PermissionsCache.java
@@ -37,7 +37,8 @@ public class PermissionsCache extends 
AuthCache<Pair<AuthenticatedUser, IResourc
               DatabaseDescriptor::setPermissionsCacheActiveUpdate,
               DatabaseDescriptor::getPermissionsCacheActiveUpdate,
               (p) -> authorizer.authorize(p.left, p.right),
-              () -> DatabaseDescriptor.getAuthorizer().requireAuthorization());
+              authorizer.bulkLoader(),
+              authorizer::requireAuthorization);
     }
 
     public Set<Permission> getPermissions(AuthenticatedUser user, IResource 
resource)
diff --git a/src/java/org/apache/cassandra/auth/Roles.java 
b/src/java/org/apache/cassandra/auth/Roles.java
index 401d208..9d3e232 100644
--- a/src/java/org/apache/cassandra/auth/Roles.java
+++ b/src/java/org/apache/cassandra/auth/Roles.java
@@ -35,8 +35,13 @@ public class Roles
 
     private static final Role NO_ROLE = new Role("", false, false, 
Collections.emptyMap(), Collections.emptySet());
 
-    public static final RolesCache cache = new 
RolesCache(DatabaseDescriptor.getRoleManager(),
-            () -> 
DatabaseDescriptor.getAuthenticator().requireAuthentication());
+    public static final RolesCache cache = new 
RolesCache(DatabaseDescriptor.getRoleManager(), () -> 
DatabaseDescriptor.getAuthenticator().requireAuthentication());
+
+    /** Use {@link AuthCacheService#initializeAndRegisterCaches} rather than 
calling this directly */
+    public static void init()
+    {
+        AuthCacheService.instance.register(cache);
+    }
 
     /**
      * Identify all roles granted to the supplied Role, including both 
directly granted
@@ -69,6 +74,15 @@ public class Roles
     }
 
     /**
+     * Enumerate all the roles in the system, preferably these will be fetched 
from the cache, which in turn
+     * may have been warmed during startup.
+     */
+    public static Set<RoleResource> getAllRoles()
+    {
+        return cache.getAllRoles();
+    }
+
+    /**
      * Returns true if the supplied role or any other role granted to it
      * (directly or indirectly) has superuser status.
      *
diff --git a/src/java/org/apache/cassandra/auth/RolesCache.java 
b/src/java/org/apache/cassandra/auth/RolesCache.java
index 05a5759..d34de19 100644
--- a/src/java/org/apache/cassandra/auth/RolesCache.java
+++ b/src/java/org/apache/cassandra/auth/RolesCache.java
@@ -25,6 +25,8 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 
 public class RolesCache extends AuthCache<RoleResource, Set<Role>> implements 
RolesCacheMBean
 {
+    private final IRoleManager roleManager;
+
     public RolesCache(IRoleManager roleManager, BooleanSupplier enableCache)
     {
         super(CACHE_NAME,
@@ -37,7 +39,9 @@ public class RolesCache extends AuthCache<RoleResource, 
Set<Role>> implements Ro
               DatabaseDescriptor::setRolesCacheActiveUpdate,
               DatabaseDescriptor::getRolesCacheActiveUpdate,
               roleManager::getRoleDetails,
+              roleManager.bulkLoader(),
               enableCache);
+        this.roleManager = roleManager;
     }
 
     /**
@@ -65,6 +69,15 @@ public class RolesCache extends AuthCache<RoleResource, 
Set<Role>> implements Ro
         return get(primaryRole);
     }
 
+    Set<RoleResource> getAllRoles()
+    {
+        // This method seems kind of unnecessary as it is only called from 
Roles::getAllRoles,
+        // but we are able to inject the RoleManager to this class, making 
testing possible. If
+        // we lose this method and did everything in Roles, we'd be dependent 
on the IRM impl
+        // supplied by DatabaseDescriptor
+        return roleManager.getAllRoles();
+    }
+
     public void invalidateRoles(String roleName)
     {
         invalidate(RoleResource.role(roleName));
diff --git a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java 
b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
index 674abe6..613a1bd 100644
--- a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
+++ b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
@@ -22,6 +22,7 @@ import java.lang.reflect.*;
 import java.security.AccessControlContext;
 import java.security.AccessController;
 import java.security.Principal;
+import java.util.Collections;
 import java.util.Set;
 import java.util.function.BooleanSupplier;
 import java.util.function.Function;
@@ -492,6 +493,7 @@ public class AuthorizationProxy implements InvocationHandler
                   DatabaseDescriptor::setPermissionsCacheActiveUpdate,
                   DatabaseDescriptor::getPermissionsCacheActiveUpdate,
                   AuthorizationProxy::loadPermissions,
+                  Collections::emptyMap,
                   () -> true);
 
             MBeanWrapper.instance.registerMBean(this, MBEAN_NAME_BASE + 
DEPRECATED_CACHE_NAME);
diff --git a/src/java/org/apache/cassandra/config/Config.java 
b/src/java/org/apache/cassandra/config/Config.java
index b88767e..f0e52a2 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -522,6 +522,8 @@ public class Config
      */
     public volatile int validation_preview_purge_head_start_in_sec = 60 * 60;
 
+    public boolean auth_cache_warming_enabled = false;
+
     // Using String instead of ConsistencyLevel here to keep static 
initialization from cascading and starting
     // threads during tool usage mode. See CASSANDRA-12988 and 
DatabaseDescriptorRefTest for details
     public volatile String auth_read_consistency_level = "LOCAL_QUORUM";
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java 
b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 869678b..3968484 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -3692,6 +3692,11 @@ public class DatabaseDescriptor
         conf.denylist_max_keys_total = value;
     }
 
+    public static boolean getAuthCacheWarmingEnabled()
+    {
+        return conf.auth_cache_warming_enabled;
+    }
+
     public static SubnetGroups getClientErrorReportingExclusions()
     {
         return conf.client_error_reporting_exclusions;
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java 
b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 16c2c2b..7c257f6 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -51,6 +51,7 @@ import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
 import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
 
 import org.apache.cassandra.audit.AuditLogManager;
+import org.apache.cassandra.auth.AuthCacheService;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryProcessor;
@@ -330,7 +331,6 @@ public class CassandraDaemon
             }
         }
 
-
         try
         {
             loadRowAndKeyCacheAsync().get();
@@ -476,6 +476,13 @@ public class CassandraDaemon
 
         initializeClientTransports();
 
+        // Ensure you've registered all caches during startup you want 
pre-warmed before this call -> be wary of adding
+        // init below this mark before completeSetup().
+        if (DatabaseDescriptor.getAuthCacheWarmingEnabled())
+            AuthCacheService.instance.warmCaches();
+        else
+            logger.info("Prewarming of auth caches is disabled");
+
         completeSetup();
     }
 
diff --git a/src/java/org/apache/cassandra/service/StorageService.java 
b/src/java/org/apache/cassandra/service/StorageService.java
index 38fa191..3090ff2 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -59,30 +59,21 @@ import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.*;
-
-import org.apache.cassandra.config.CassandraRelevantProperties;
-import org.apache.cassandra.concurrent.*;
-import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
-import org.apache.cassandra.fql.FullQueryLogger;
-import org.apache.cassandra.fql.FullQueryLoggerOptions;
-import org.apache.cassandra.fql.FullQueryLoggerOptionsCompositeData;
-import org.apache.cassandra.io.util.File;
-import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
-import org.apache.cassandra.utils.concurrent.Future;
-import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 import org.apache.commons.lang3.StringUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.audit.AuditLogManager;
 import org.apache.cassandra.audit.AuditLogOptions;
+import org.apache.cassandra.auth.AuthCacheService;
 import org.apache.cassandra.auth.AuthKeyspace;
 import org.apache.cassandra.auth.AuthSchemaChangeListener;
 import org.apache.cassandra.batchlog.BatchlogManager;
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Duration;
+import org.apache.cassandra.concurrent.*;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLog;
@@ -92,15 +83,21 @@ import 
org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
 import org.apache.cassandra.dht.Token.TokenFactory;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.fql.FullQueryLogger;
+import org.apache.cassandra.fql.FullQueryLoggerOptions;
+import org.apache.cassandra.fql.FullQueryLoggerOptionsCompositeData;
 import org.apache.cassandra.gms.*;
 import org.apache.cassandra.hints.HintsService;
 import org.apache.cassandra.io.sstable.SSTableLoader;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.VersionAndType;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.*;
+import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.net.*;
 import org.apache.cassandra.repair.*;
@@ -123,6 +120,8 @@ import org.apache.cassandra.tracing.TraceKeyspace;
 import org.apache.cassandra.transport.ClientResourceLimits;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.concurrent.Future;
+import org.apache.cassandra.utils.concurrent.ImmediateFuture;
 import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
 import org.apache.cassandra.utils.logging.LoggingSupportFactory;
 import org.apache.cassandra.utils.progress.ProgressEvent;
@@ -1224,6 +1223,7 @@ public class StorageService extends 
NotificationBroadcasterSupport implements IE
             DatabaseDescriptor.getAuthenticator().setup();
             DatabaseDescriptor.getAuthorizer().setup();
             DatabaseDescriptor.getNetworkAuthorizer().setup();
+            AuthCacheService.initializeAndRegisterCaches();
             Schema.instance.registerListener(new AuthSchemaChangeListener());
             authSetupComplete = true;
         }
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/AuthTest.java 
b/test/distributed/org/apache/cassandra/distributed/test/AuthTest.java
index 4f75080..27ab634 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/AuthTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/AuthTest.java
@@ -31,7 +31,6 @@ import static org.junit.Assert.assertTrue;
 
 public class AuthTest extends TestBaseImpl
 {
-
     /**
      * Simply tests that initialisation of a test Instance results in
      * StorageService.instance.doAuthSetup being called as the regular
diff --git a/test/unit/org/apache/cassandra/auth/AuthCacheTest.java 
b/test/unit/org/apache/cassandra/auth/AuthCacheTest.java
index bf3d7bb..0e22346 100644
--- a/test/unit/org/apache/cassandra/auth/AuthCacheTest.java
+++ b/test/unit/org/apache/cassandra/auth/AuthCacheTest.java
@@ -17,11 +17,16 @@
  */
 package org.apache.cassandra.auth;
 
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.BooleanSupplier;
 import java.util.function.Function;
 import java.util.function.IntConsumer;
 import java.util.function.IntSupplier;
+import java.util.function.Supplier;
 
 import org.junit.Test;
 
@@ -41,27 +46,23 @@ public class AuthCacheTest
     private int validity = 2000;
     private boolean isCacheEnabled = true;
 
+    private final int MAX_ENTRIES = 10;
+
     @Test
     public void testCacheLoaderIsCalledOnFirst()
     {
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
-
-        int result = authCache.get("10");
-
-        assertEquals(10, result);
+        TestCache authCache = newCache();
+        assertEquals(10, (int)authCache.get("10"));
         assertEquals(1, loadCounter);
     }
 
     @Test
     public void testCacheLoaderIsNotCalledOnSecond()
     {
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
+        TestCache authCache = newCache();
         authCache.get("10");
         assertEquals(1, loadCounter);
-
-        int result = authCache.get("10");
-
-        assertEquals(10, result);
+        assertEquals(10, (int)authCache.get("10"));
         assertEquals(1, loadCounter);
     }
 
@@ -69,12 +70,9 @@ public class AuthCacheTest
     public void testCacheLoaderIsAlwaysCalledWhenDisabled()
     {
         isCacheEnabled = false;
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
-
+        TestCache authCache = newCache();
         authCache.get("10");
-        int result = authCache.get("10");
-
-        assertEquals(10, result);
+        assertEquals(10, (int)authCache.get("10"));
         assertEquals(2, loadCounter);
     }
 
@@ -82,45 +80,42 @@ public class AuthCacheTest
     public void testCacheLoaderIsAlwaysCalledWhenValidityIsZero()
     {
         setValidity(0);
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
-
+        TestCache authCache = newCache();
         authCache.get("10");
-        int result = authCache.get("10");
-
-        assertEquals(10, result);
+        assertEquals(10, (int)authCache.get("10"));
         assertEquals(2, loadCounter);
     }
 
     @Test
     public void testCacheLoaderIsCalledAfterFullInvalidate()
     {
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
+        TestCache authCache = newCache();
         authCache.get("10");
-
-        authCache.invalidate();
-        int result = authCache.get("10");
-
-        assertEquals(10, result);
+        authCache.get("11");
         assertEquals(2, loadCounter);
+        authCache.invalidate();
+        assertEquals(10, (int)authCache.get("10"));
+        assertEquals(11, (int)authCache.get("11"));
+        assertEquals(4, loadCounter);
     }
 
     @Test
     public void testCacheLoaderIsCalledAfterInvalidateKey()
     {
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
+        TestCache authCache = newCache();
         authCache.get("10");
-
-        authCache.invalidate("10");
-        int result = authCache.get("10");
-
-        assertEquals(10, result);
+        authCache.get("11"); // second key that should not be invalidated
         assertEquals(2, loadCounter);
+        authCache.invalidate("10");
+        assertEquals(10, (int)authCache.get("10"));
+        assertEquals(11, (int)authCache.get("11"));
+        assertEquals(3, loadCounter);
     }
 
     @Test
     public void testCacheLoaderIsCalledAfterReset()
     {
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
+        TestCache authCache = newCache();
         authCache.get("10");
 
         authCache.cache = null;
@@ -134,7 +129,7 @@ public class AuthCacheTest
     public void testThatZeroValidityTurnOffCaching()
     {
         setValidity(0);
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
+        TestCache authCache = newCache();
         authCache.get("10");
         int result = authCache.get("10");
 
@@ -147,8 +142,7 @@ public class AuthCacheTest
     public void testThatRaisingValidityTurnOnCaching()
     {
         setValidity(0);
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
-
+        TestCache authCache = newCache();
         authCache.setValidity(2000);
         authCache.cache = authCache.initCache(null);
 
@@ -159,8 +153,7 @@ public class AuthCacheTest
     public void testDisableCache()
     {
         isCacheEnabled = false;
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
-
+        TestCache authCache = newCache();
         assertNull(authCache.cache);
     }
 
@@ -168,8 +161,7 @@ public class AuthCacheTest
     public void testDynamicallyEnableCache()
     {
         isCacheEnabled = false;
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
-
+        TestCache authCache = newCache();
         isCacheEnabled = true;
         authCache.cache = authCache.initCache(null);
 
@@ -179,8 +171,7 @@ public class AuthCacheTest
     @Test
     public void testDefaultPolicies()
     {
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
-
+        TestCache authCache = newCache();
         assertTrue(authCache.cache.policy().expireAfterWrite().isPresent());
         assertTrue(authCache.cache.policy().refreshAfterWrite().isPresent());
         assertTrue(authCache.cache.policy().eviction().isPresent());
@@ -189,8 +180,7 @@ public class AuthCacheTest
     @Test(expected = UnavailableException.class)
     public void testCassandraExceptionPassThroughWhenCacheEnabled()
     {
-        TestCache<String, Integer> cache = new TestCache<>(s -> { throw 
UnavailableException.create(ConsistencyLevel.QUORUM, 3, 1); }, 
this::setValidity, () -> validity, () -> isCacheEnabled);
-
+        TestCache cache = newCache(s -> { throw 
UnavailableException.create(ConsistencyLevel.QUORUM, 3, 1); });
         cache.get("expect-exception");
     }
 
@@ -198,8 +188,7 @@ public class AuthCacheTest
     public void testCassandraExceptionPassThroughWhenCacheDisable()
     {
         isCacheEnabled = false;
-        TestCache<String, Integer> cache = new TestCache<>(s -> { throw 
UnavailableException.create(ConsistencyLevel.QUORUM, 3, 1); }, 
this::setValidity, () -> validity, () -> isCacheEnabled);
-
+        TestCache cache = newCache(s -> { throw 
UnavailableException.create(ConsistencyLevel.QUORUM, 3, 1); });
         cache.get("expect-exception");
     }
 
@@ -207,7 +196,7 @@ public class AuthCacheTest
     public void testCassandraExceptionPassThroughWhenCacheRefreshed() throws 
InterruptedException
     {
         setValidity(50);
-        TestCache<String, Integer> cache = new 
TestCache<>(this::countingLoaderWithException, this::setValidity, () -> 
validity, () -> isCacheEnabled);
+        TestCache cache = new TestCache(this::countingLoaderWithException, 
this::emptyBulkLoader, this::setValidity, () -> validity, () -> isCacheEnabled);
         cache.get("10");
 
         // wait until the cached record expires
@@ -227,10 +216,78 @@ public class AuthCacheTest
     }
 
     @Test
+    public void warmCacheUsingEntryProvider()
+    {
+        AtomicBoolean provided = new AtomicBoolean(false);
+        Supplier<Map<String, Integer>> bulkLoader = () -> {
+            provided.set(true);
+            return Collections.singletonMap("0", 0);
+        };
+        TestCache cache = newCache(bulkLoader);
+        cache.warm();
+        assertEquals(1, cache.getEstimatedSize());
+        assertEquals(0, (int)cache.get("0")); // warmed entry
+        assertEquals(0, loadCounter);
+        assertEquals(10, (int)cache.get("10")); // cold entry
+        assertEquals(1, loadCounter);
+        assertTrue(provided.get());
+    }
+
+    @Test
+    public void warmCacheIsSafeIfCachingIsDisabled()
+    {
+        isCacheEnabled = false;
+        TestCache cache = newCache(() -> Collections.singletonMap("0", 0));
+        cache.warm();
+        assertEquals(0, cache.getEstimatedSize());
+    }
+
+    @Test
+    public void providerSuppliesMoreEntriesThanCapacity()
+    {
+        Supplier<Map<String, Integer>> bulkLoader = () -> {
+            Map<String, Integer> entries = new HashMap<>();
+            for (int i = 0; i < MAX_ENTRIES * 2; i++)
+                entries.put(Integer.toString(i), i);
+            return entries;
+        };
+        TestCache cache = new TestCache(this::countingLoader,
+                                        bulkLoader,
+                                        this::setValidity,
+                                        () -> validity,
+                                        () -> isCacheEnabled);
+        cache.warm();
+        cache.cleanup(); // Force the cleanup task rather than waiting for it 
to be scheduled to get accurate count
+        assertEquals(MAX_ENTRIES, cache.getEstimatedSize());
+    }
+
+    @Test
+    public void handleProviderErrorDuringWarming()
+    {
+        System.setProperty(AuthCache.CACHE_LOAD_RETRIES_PROPERTY, "3");
+        System.setProperty(AuthCache.CACHE_LOAD_RETRY_INTERVAL_PROPERTY, "0");
+        final AtomicInteger attempts = new AtomicInteger(0);
+
+        Supplier<Map<String, Integer>> bulkLoader = () -> {
+            if (attempts.incrementAndGet() < 3)
+                throw new RuntimeException("BOOM");
+
+            return Collections.singletonMap("0", 99);
+        };
+
+        TestCache cache = newCache(bulkLoader);
+        cache.warm();
+        assertEquals(1, cache.getEstimatedSize());
+        assertEquals(99, (int)cache.get("0"));
+        // We should have made 3 attempts to get the initial entries
+        assertEquals(3, attempts.get());
+    }
+
+    @Test
     public void testCacheLoaderIsNotCalledOnGetAllWhenCacheIsDisabled()
     {
         isCacheEnabled = false;
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
+        TestCache authCache = new TestCache(this::countingLoader, 
this::emptyBulkLoader, this::setValidity, () -> validity, () -> isCacheEnabled);
         authCache.get("10");
         Map<String, Integer> result = authCache.getAll();
 
@@ -243,7 +300,7 @@ public class AuthCacheTest
     @Test
     public void testCacheLoaderIsNotCalledOnGetAllWhenCacheIsEmpty()
     {
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
+        TestCache authCache = new TestCache(this::countingLoader, 
this::emptyBulkLoader, this::setValidity, () -> validity, () -> isCacheEnabled);
 
         Map<String, Integer> result = authCache.getAll();
 
@@ -254,7 +311,7 @@ public class AuthCacheTest
     @Test
     public void testCacheLoaderIsNotCalledOnGetAllWhenCacheIsNotEmpty()
     {
-        TestCache<String, Integer> authCache = new 
TestCache<>(this::countingLoader, this::setValidity, () -> validity, () -> 
isCacheEnabled);
+        TestCache authCache = new TestCache(this::countingLoader, 
this::emptyBulkLoader, this::setValidity, () -> validity, () -> isCacheEnabled);
         authCache.get("10");
         Map<String, Integer> result = authCache.getAll();
 
@@ -284,23 +341,65 @@ public class AuthCacheTest
         return loadedValue;
     }
 
-    private static class TestCache<K, V> extends AuthCache<K, V>
+    private Map<String, Integer> emptyBulkLoader()
+    {
+        return Collections.emptyMap();
+    }
+
+    private TestCache newCache()
+    {
+        return new TestCache(this::countingLoader,
+                             this::emptyBulkLoader,
+                             this::setValidity,
+                             () -> validity,
+                             () -> isCacheEnabled);
+    }
+
+    private TestCache newCache(Function<String, Integer> loadFunction)
+    {
+        return new TestCache(loadFunction,
+                             this::emptyBulkLoader,
+                             this::setValidity,
+                             () -> validity,
+                             () -> isCacheEnabled);
+    }
+
+    private TestCache newCache(Supplier<Map<String, Integer>> bulkLoader)
+    {
+        return new TestCache(this::countingLoader,
+                             bulkLoader,
+                             this::setValidity,
+                             () -> validity,
+                             () -> isCacheEnabled);
+    }
+
+    private static class TestCache extends AuthCache<String, Integer>
     {
         private static int nameCounter = 0; // Allow us to create many 
instances of cache with same name prefix
 
-        TestCache(Function<K, V> loadFunction, IntConsumer 
setValidityDelegate, IntSupplier getValidityDelegate, BooleanSupplier 
cacheEnabledDelegate)
+        TestCache(Function<String, Integer> loadFunction,
+                  Supplier<Map<String, Integer>> bulkLoadFunction,
+                  IntConsumer setValidityDelegate,
+                  IntSupplier getValidityDelegate,
+                  BooleanSupplier cacheEnabledDelegate)
         {
             super("TestCache" + nameCounter++,
                   setValidityDelegate,
                   getValidityDelegate,
-                  (updateInterval) -> {},
-                  () -> 1000,
-                  (maxEntries) -> {},
-                  () -> 10,
-                  (updateActiveUpdate) -> {},
-                  () -> false,
+                  (updateInterval) -> {},               // set update interval
+                  () -> 1000,                           // get update interval
+                  (MAX_ENTRIES) -> {},                   // set max entries
+                  () -> 10,                             // get max entries
+                  (updateActiveUpdate) -> {},           // set active update 
enabled
+                  () -> false,                          // get active update 
enabled
                   loadFunction,
+                  bulkLoadFunction,
                   cacheEnabledDelegate);
         }
+
+        void cleanup()
+        {
+            cache.cleanUp();
+        }
     }
 }
diff --git a/test/unit/org/apache/cassandra/auth/AuthTestUtils.java 
b/test/unit/org/apache/cassandra/auth/AuthTestUtils.java
index 18e95a4..c78520e 100644
--- a/test/unit/org/apache/cassandra/auth/AuthTestUtils.java
+++ b/test/unit/org/apache/cassandra/auth/AuthTestUtils.java
@@ -107,9 +107,9 @@ public class AuthTestUtils
         }
 
         @Override
-        void process(String query)
+        UntypedResultSet process(String query, ConsistencyLevel cl)
         {
-            QueryProcessor.executeInternal(query);
+            return QueryProcessor.executeInternal(query);
         }
     }
 
@@ -120,6 +120,12 @@ public class AuthTestUtils
         {
             return statement.executeLocally(QueryState.forInternalCalls(), 
options);
         }
+
+        @Override
+        UntypedResultSet process(String query, ConsistencyLevel cl)
+        {
+            return QueryProcessor.executeInternal(query);
+        }
     }
 
     public static class NoAuthSetupAuthorizationProxy extends 
AuthorizationProxy
diff --git a/test/unit/org/apache/cassandra/auth/CacheRefresherTest.java 
b/test/unit/org/apache/cassandra/auth/CacheRefresherTest.java
index 3340d82..7600c29 100644
--- a/test/unit/org/apache/cassandra/auth/CacheRefresherTest.java
+++ b/test/unit/org/apache/cassandra/auth/CacheRefresherTest.java
@@ -85,4 +85,4 @@ public class CacheRefresherTest
         refresher.run();
         Assert.assertEquals("body", cache.get("some"));
     }
-}
\ No newline at end of file
+}
diff --git a/test/unit/org/apache/cassandra/auth/CassandraAuthorizerTest.java 
b/test/unit/org/apache/cassandra/auth/CassandraAuthorizerTest.java
index 45f7e39..643b2df 100644
--- a/test/unit/org/apache/cassandra/auth/CassandraAuthorizerTest.java
+++ b/test/unit/org/apache/cassandra/auth/CassandraAuthorizerTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.auth;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.CassandraRelevantProperties;
 import org.apache.cassandra.cql3.CQLTester;
 
 import static java.lang.String.format;
@@ -35,6 +36,7 @@ public class CassandraAuthorizerTest extends CQLTester
     @BeforeClass
     public static void setupClass()
     {
+        
CassandraRelevantProperties.ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.setBoolean(true);
         CQLTester.setUpClass();
         requireAuthentication();
         requireNetwork();
diff --git 
a/test/unit/org/apache/cassandra/auth/CassandraAuthorizerTruncatingTests.java 
b/test/unit/org/apache/cassandra/auth/CassandraAuthorizerTruncatingTests.java
new file mode 100644
index 0000000..3605239
--- /dev/null
+++ 
b/test/unit/org/apache/cassandra/auth/CassandraAuthorizerTruncatingTests.java
@@ -0,0 +1,118 @@
+/*
+ * 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.util.EnumSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.utils.Pair;
+
+import static org.apache.cassandra.auth.AuthTestUtils.ALL_ROLES;
+import static 
org.apache.cassandra.auth.AuthTestUtils.LocalCassandraRoleManager;
+import static org.apache.cassandra.auth.AuthTestUtils.ROLE_B;
+import static org.apache.cassandra.auth.AuthTestUtils.ROLE_B_1;
+import static org.apache.cassandra.auth.AuthTestUtils.ROLE_B_2;
+import static org.apache.cassandra.auth.AuthTestUtils.ROLE_C;
+import static org.apache.cassandra.auth.AuthTestUtils.ROLE_C_1;
+import static org.apache.cassandra.auth.AuthTestUtils.ROLE_C_2;
+import static org.apache.cassandra.auth.AuthTestUtils.grantRolesTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * For Authorizer based tests where we need to fully truncate the roles, 
members, and permissions between tests
+ */
+public class CassandraAuthorizerTruncatingTests extends CQLTester
+{
+    @BeforeClass
+    public static void setupClass()
+    {
+        
CassandraRelevantProperties.ORG_APACHE_CASSANDRA_DISABLE_MBEAN_REGISTRATION.setBoolean(true);
+        DatabaseDescriptor.daemonInitialization();
+        DatabaseDescriptor.setAuthorizer(new StubAuthorizer());
+        requireAuthentication();
+        requireNetwork();
+    }
+
+    @Before
+    public void setup()
+    {
+        ColumnFamilyStore.getIfExists(SchemaConstants.AUTH_KEYSPACE_NAME, 
AuthKeyspace.ROLES).truncateBlocking();
+        ColumnFamilyStore.getIfExists(SchemaConstants.AUTH_KEYSPACE_NAME, 
AuthKeyspace.ROLE_MEMBERS).truncateBlocking();
+        ColumnFamilyStore.getIfExists(SchemaConstants.AUTH_KEYSPACE_NAME, 
AuthKeyspace.ROLE_PERMISSIONS).truncateBlocking();
+    }
+
+    @Test
+    public void testBulkLoadingForAuthCache()
+    {
+        IResource table1 = Resources.fromName("data/ks1/t1");
+        IResource table2 = Resources.fromName("data/ks2/t2");
+
+        // Setup a hierarchy of roles. ROLE_B is granted LOGIN privs, ROLE_B_1 
and ROLE_B_2.
+        // ROLE_C is granted LOGIN along with ROLE_C_1 & ROLE_C_2
+        IRoleManager roleManager = new LocalCassandraRoleManager();
+        roleManager.setup();
+
+        for (RoleResource role : ALL_ROLES)
+            roleManager.createRole(AuthenticatedUser.ANONYMOUS_USER, role, new 
RoleOptions());
+
+        RoleOptions withLogin = new RoleOptions();
+        withLogin.setOption(IRoleManager.Option.LOGIN, Boolean.TRUE);
+        roleManager.alterRole(AuthenticatedUser.ANONYMOUS_USER, ROLE_B, 
withLogin);
+        roleManager.alterRole(AuthenticatedUser.ANONYMOUS_USER, ROLE_C, 
withLogin);
+        grantRolesTo(roleManager, ROLE_B, ROLE_B_1, ROLE_B_2);
+        grantRolesTo(roleManager, ROLE_C, ROLE_C_1, ROLE_C_2);
+
+        CassandraAuthorizer authorizer = new CassandraAuthorizer();
+        // Granted on ks1.t1: B1 -> {SELECT, MODIFY}, B2 -> {AUTHORIZE}, so B 
-> {SELECT, MODIFY, AUTHORIZE}
+        authorizer.grant(AuthenticatedUser.SYSTEM_USER, 
EnumSet.of(Permission.SELECT, Permission.MODIFY), table1, ROLE_B_1);
+        authorizer.grant(AuthenticatedUser.SYSTEM_USER, 
EnumSet.of(Permission.AUTHORIZE), table1, ROLE_B_2);
+
+        // Granted on ks2.t2: C1 -> {SELECT, MODIFY}, C2 -> {AUTHORIZE}, so C 
-> {SELECT, MODIFY, AUTHORIZE}
+        authorizer.grant(AuthenticatedUser.SYSTEM_USER, 
EnumSet.of(Permission.SELECT, Permission.MODIFY), table2, ROLE_C_1);
+        authorizer.grant(AuthenticatedUser.SYSTEM_USER, 
EnumSet.of(Permission.AUTHORIZE), table2, ROLE_C_2);
+
+        Map<Pair<AuthenticatedUser, IResource>, Set<Permission>> cacheEntries 
= authorizer.bulkLoader().get();
+
+        // Only ROLE_B and ROLE_C have LOGIN privs, so only they should be in 
the cached
+        assertEquals(2, cacheEntries.size());
+        assertEquals(EnumSet.of(Permission.SELECT, Permission.MODIFY, 
Permission.AUTHORIZE),
+                     cacheEntries.get(Pair.create(new 
AuthenticatedUser(ROLE_B.getRoleName()), table1)));
+        assertEquals(EnumSet.of(Permission.SELECT, Permission.MODIFY, 
Permission.AUTHORIZE),
+                     cacheEntries.get(Pair.create(new 
AuthenticatedUser(ROLE_C.getRoleName()), table2)));
+    }
+
+    @Test
+    public void testBulkLoadingForAuthCachWithEmptyTable()
+    {
+        CassandraAuthorizer authorizer = new CassandraAuthorizer();
+        Map<Pair<AuthenticatedUser, IResource>, Set<Permission>> cacheEntries 
= authorizer.bulkLoader().get();
+        assertTrue(cacheEntries.isEmpty());
+    }
+}
diff --git 
a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java 
b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
index bfa89ec..3a208f8 100644
--- a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
+++ b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
@@ -66,7 +66,7 @@ public class CassandraNetworkAuthorizerTest
                                new AuthTestUtils.LocalPasswordAuthenticator(),
                                new AuthTestUtils.LocalCassandraAuthorizer(),
                                new 
AuthTestUtils.LocalCassandraNetworkAuthorizer());
-
+        AuthCacheService.initializeAndRegisterCaches();
         setupSuperUser();
     }
 
diff --git a/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java 
b/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java
index 1df57ea..7b6b910 100644
--- a/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java
+++ b/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java
@@ -18,23 +18,28 @@
 
 package org.apache.cassandra.auth;
 
+import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.Iterables;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
 
 import static org.apache.cassandra.auth.AuthTestUtils.*;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class CassandraRoleManagerTest
 {
-
     @BeforeClass
     public static void setupClass()
     {
@@ -43,6 +48,19 @@ public class CassandraRoleManagerTest
         SchemaLoader.createKeyspace(SchemaConstants.AUTH_KEYSPACE_NAME,
                                     KeyspaceParams.simple(1),
                                     
Iterables.toArray(AuthKeyspace.metadata().tables, TableMetadata.class));
+        // We start StorageService because confirmFastRoleSetup confirms that 
CassandraRoleManager will
+        // take a faster path once the cluster is already setup, which 
includes checking MessagingService
+        // and issuing queries with QueryProcessor.process, which uses 
TokenMetadata
+        DatabaseDescriptor.daemonInitialization();
+        StorageService.instance.initServer(0);
+        AuthCacheService.initializeAndRegisterCaches();
+    }
+
+    @Before
+    public void setup() throws Exception
+    {
+        ColumnFamilyStore.getIfExists(SchemaConstants.AUTH_KEYSPACE_NAME, 
AuthKeyspace.ROLES).truncateBlocking();
+        ColumnFamilyStore.getIfExists(SchemaConstants.AUTH_KEYSPACE_NAME, 
AuthKeyspace.ROLE_MEMBERS).truncateBlocking();
     }
 
     @Test
@@ -85,4 +103,68 @@ public class CassandraRoleManagerTest
         long after = getRolesReadCount();
         assertEquals(granted.size(), after - before);
     }
+
+    @Test
+    public void confirmFastRoleSetup()
+    {
+        IRoleManager roleManager = new 
AuthTestUtils.LocalCassandraRoleManager();
+        roleManager.setup();
+        for (RoleResource r : ALL_ROLES)
+            roleManager.createRole(AuthenticatedUser.ANONYMOUS_USER, r, new 
RoleOptions());
+
+        CassandraRoleManager crm = new CassandraRoleManager();
+
+        assertTrue("Expected the role manager to have existing roles before 
CassandraRoleManager setup", CassandraRoleManager.hasExistingRoles());
+    }
+
+    @Test
+    public void warmCacheLoadsAllEntries()
+    {
+        IRoleManager roleManager = new 
AuthTestUtils.LocalCassandraRoleManager();
+        roleManager.setup();
+        for (RoleResource r : ALL_ROLES)
+            roleManager.createRole(AuthenticatedUser.ANONYMOUS_USER, r, new 
RoleOptions());
+
+        // Multi level role hierarchy
+        grantRolesTo(roleManager, ROLE_B, ROLE_B_1, ROLE_B_2, ROLE_B_3);
+        grantRolesTo(roleManager, ROLE_C, ROLE_C_1, ROLE_C_2, ROLE_C_3);
+
+        // Use CassandraRoleManager to get entries for pre-warming a cache, 
then verify those entries
+        CassandraRoleManager crm = new CassandraRoleManager();
+        crm.setup();
+        Map<RoleResource, Set<Role>> cacheEntries = crm.bulkLoader().get();
+
+        Set<Role> roleBRoles = cacheEntries.get(ROLE_B);
+        assertRoleSet(roleBRoles, ROLE_B, ROLE_B_1, ROLE_B_2, ROLE_B_3);
+
+        Set<Role> roleCRoles = cacheEntries.get(ROLE_C);
+        assertRoleSet(roleCRoles, ROLE_C, ROLE_C_1, ROLE_C_2, ROLE_C_3);
+
+        for (RoleResource r : ALL_ROLES)
+        {
+            // We already verified ROLE_B and ROLE_C
+            if (r.equals(ROLE_B) || r.equals(ROLE_C))
+                continue;
+
+            // Check the cache entries for the roles without any further grants
+            assertRoleSet(cacheEntries.get(r), r);
+        }
+    }
+
+    @Test
+    public void warmCacheWithEmptyTable()
+    {
+        CassandraRoleManager crm = new CassandraRoleManager();
+        crm.setup();
+        Map<RoleResource, Set<Role>> cacheEntries = crm.bulkLoader().get();
+        assertTrue(cacheEntries.isEmpty());
+    }
+
+    private void assertRoleSet(Set<Role> actual, RoleResource...expected)
+    {
+        assertEquals(expected.length, actual.size());
+
+        for (RoleResource expectedRole : expected)
+            assertTrue(actual.stream().anyMatch(role -> 
role.resource.equals(expectedRole)));
+    }
 }
diff --git a/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java 
b/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java
index 4aa268e..e1033ff 100644
--- a/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java
+++ b/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java
@@ -19,35 +19,58 @@ package org.apache.cassandra.auth;
 
 
 import java.nio.charset.StandardCharsets;
+import java.util.Map;
 
-import com.google.common.collect.Iterables;
-import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.mindrot.jbcrypt.BCrypt;
+
 import com.datastax.driver.core.Authenticator;
 import com.datastax.driver.core.EndPoint;
 import com.datastax.driver.core.PlainTextAuthProvider;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.SchemaConstants;
-import org.apache.cassandra.schema.TableMetadata;
-
-import static org.apache.cassandra.auth.CassandraRoleManager.*;
-import static org.apache.cassandra.auth.PasswordAuthenticator.*;
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.auth.AuthTestUtils.ALL_ROLES;
+import static 
org.apache.cassandra.auth.CassandraRoleManager.DEFAULT_SUPERUSER_PASSWORD;
+import static 
org.apache.cassandra.auth.CassandraRoleManager.getGensaltLogRounds;
+import static org.apache.cassandra.auth.PasswordAuthenticator.SaslNegotiator;
+import static org.apache.cassandra.auth.PasswordAuthenticator.checkpw;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.mindrot.jbcrypt.BCrypt.hashpw;
 import static org.mindrot.jbcrypt.BCrypt.gensalt;
+import static org.mindrot.jbcrypt.BCrypt.hashpw;
+
+import static 
org.apache.cassandra.auth.CassandraRoleManager.GENSALT_LOG2_ROUNDS_PROPERTY;
 
 public class PasswordAuthenticatorTest extends CQLTester
 {
+    private final static PasswordAuthenticator authenticator = new 
PasswordAuthenticator();
 
-    private static PasswordAuthenticator authenticator = new 
PasswordAuthenticator();
+    @BeforeClass
+    public static void setupClass() throws Exception
+    {
+        SchemaLoader.loadSchema();
+        DatabaseDescriptor.daemonInitialization();
+        StorageService.instance.initServer(0);
+    }
+
+    @Before
+    public void setup() throws Exception
+    {
+        ColumnFamilyStore.getIfExists(SchemaConstants.AUTH_KEYSPACE_NAME, 
AuthKeyspace.ROLES).truncateBlocking();
+        ColumnFamilyStore.getIfExists(SchemaConstants.AUTH_KEYSPACE_NAME, 
AuthKeyspace.ROLE_MEMBERS).truncateBlocking();
+    }
 
     @Test
     public void testCheckpw()
@@ -112,7 +135,6 @@ public class PasswordAuthenticatorTest extends CQLTester
         }
     }
 
-
     @Test(expected = AuthenticationException.class)
     public void testEmptyUsername()
     {
@@ -161,18 +183,28 @@ public class PasswordAuthenticatorTest extends CQLTester
         negotiator.getAuthenticatedUser();
     }
 
-    @BeforeClass
-    public static void setUp()
+    @Test
+    public void warmCacheLoadsAllEntriesFromTables()
     {
-        SchemaLoader.createKeyspace(SchemaConstants.AUTH_KEYSPACE_NAME,
-                                    KeyspaceParams.simple(1),
-                                    
Iterables.toArray(AuthKeyspace.metadata().tables, TableMetadata.class));
-        authenticator.setup();
+        IRoleManager roleManager = new 
AuthTestUtils.LocalCassandraRoleManager();
+        roleManager.setup();
+        for (RoleResource r : ALL_ROLES)
+        {
+            RoleOptions options = new RoleOptions();
+            options.setOption(IRoleManager.Option.PASSWORD, "hash_for_" + 
r.getRoleName());
+            roleManager.createRole(AuthenticatedUser.ANONYMOUS_USER, r, 
options);
+        }
+
+        Map<String, String> cacheEntries = authenticator.bulkLoader().get();
+
+        assertEquals(ALL_ROLES.length, cacheEntries.size());
+        cacheEntries.forEach((username, hash) -> 
assertTrue(BCrypt.checkpw("hash_for_" + username, hash)));
     }
 
-    @AfterClass
-    public static void tearDown()
+    @Test
+    public void warmCacheWithEmptyTable()
     {
-        schemaChange("DROP KEYSPACE " + SchemaConstants.AUTH_KEYSPACE_NAME);
+        Map<String, String> cacheEntries = authenticator.bulkLoader().get();
+        assertTrue(cacheEntries.isEmpty());
     }
 }
diff --git a/test/unit/org/apache/cassandra/auth/RolesTest.java 
b/test/unit/org/apache/cassandra/auth/RolesTest.java
index bba861d..5eced33 100644
--- a/test/unit/org/apache/cassandra/auth/RolesTest.java
+++ b/test/unit/org/apache/cassandra/auth/RolesTest.java
@@ -54,6 +54,9 @@ public class RolesTest
         for (RoleResource role : ALL_ROLES)
             roleManager.createRole(AuthenticatedUser.ANONYMOUS_USER, role, new 
RoleOptions());
         grantRolesTo(roleManager, ROLE_A, ROLE_B, ROLE_C);
+
+        roleManager.setup();
+        AuthCacheService.initializeAndRegisterCaches();
     }
 
     @Test
diff --git 
a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java 
b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
index 9fd1128..7eb9ed8 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
@@ -61,6 +61,8 @@ public class DatabaseDescriptorRefTest
     "org.apache.cassandra.audit.BinLogAuditLogger",
     "org.apache.cassandra.audit.IAuditLogger",
     "org.apache.cassandra.auth.AllowAllInternodeAuthenticator",
+    "org.apache.cassandra.auth.AuthCache$BulkLoader",
+    "org.apache.cassandra.auth.Cacheable",
     "org.apache.cassandra.auth.IInternodeAuthenticator",
     "org.apache.cassandra.auth.IAuthenticator",
     "org.apache.cassandra.auth.IAuthorizer",
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java 
b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 92762ce..e5395e1 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -46,9 +46,6 @@ import com.google.common.base.Objects;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableSet;
 
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.io.util.File;
-
 import org.junit.*;
 
 import org.slf4j.Logger;
@@ -62,16 +59,19 @@ import 
com.datastax.driver.core.exceptions.UnauthorizedException;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.ServerTestUtils;
+import org.apache.cassandra.auth.AuthCacheService;
 import org.apache.cassandra.auth.AuthKeyspace;
 import org.apache.cassandra.auth.AuthSchemaChangeListener;
 import org.apache.cassandra.auth.AuthTestUtils;
 import org.apache.cassandra.auth.IRoleManager;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry;
 import org.apache.cassandra.db.virtual.VirtualSchemaKeyspace;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.index.SecondaryIndexManager;
-import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.io.util.File;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
@@ -483,6 +483,8 @@ public abstract class CQLTester
         DatabaseDescriptor.getAuthorizer().setup();
         DatabaseDescriptor.getNetworkAuthorizer().setup();
         Schema.instance.registerListener(new AuthSchemaChangeListener());
+
+        AuthCacheService.initializeAndRegisterCaches();
     }
 
     /**
diff --git a/test/unit/org/apache/cassandra/service/ClientStateTest.java 
b/test/unit/org/apache/cassandra/service/ClientStateTest.java
index 014bc2d..56d0893 100644
--- a/test/unit/org/apache/cassandra/service/ClientStateTest.java
+++ b/test/unit/org/apache/cassandra/service/ClientStateTest.java
@@ -27,6 +27,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.auth.AuthCacheService;
 import org.apache.cassandra.auth.AuthKeyspace;
 import org.apache.cassandra.auth.AuthTestUtils;
 import org.apache.cassandra.auth.AuthenticatedUser;
@@ -54,7 +55,10 @@ public class ClientStateTest
         SchemaLoader.createKeyspace(SchemaConstants.AUTH_KEYSPACE_NAME,
                                     KeyspaceParams.simple(1),
                                     
Iterables.toArray(AuthKeyspace.metadata().tables, TableMetadata.class));
+
+        AuthCacheService.initializeAndRegisterCaches();
     }
+
     @AfterClass
     public static void afterClass()
     {
diff --git 
a/test/unit/org/apache/cassandra/tools/nodetool/InvalidateJmxPermissionsCacheTest.java
 
b/test/unit/org/apache/cassandra/tools/nodetool/InvalidateJmxPermissionsCacheTest.java
index edb6578..f44a274 100644
--- 
a/test/unit/org/apache/cassandra/tools/nodetool/InvalidateJmxPermissionsCacheTest.java
+++ 
b/test/unit/org/apache/cassandra/tools/nodetool/InvalidateJmxPermissionsCacheTest.java
@@ -24,6 +24,7 @@ import javax.security.auth.Subject;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.auth.AuthCacheService;
 import org.apache.cassandra.auth.AuthTestUtils;
 import org.apache.cassandra.auth.AuthenticatedUser;
 import org.apache.cassandra.auth.CassandraPrincipal;
@@ -62,6 +63,8 @@ public class InvalidateJmxPermissionsCacheTest extends 
CQLTester
         authorizer.grant(AuthenticatedUser.SYSTEM_USER, jmxPermissions, 
rootJmxResource, ROLE_A);
         authorizer.grant(AuthenticatedUser.SYSTEM_USER, jmxPermissions, 
rootJmxResource, ROLE_B);
 
+        AuthCacheService.initializeAndRegisterCaches();
+
         startJMXServer();
     }
 
diff --git 
a/test/unit/org/apache/cassandra/tools/nodetool/InvalidateNetworkPermissionsCacheTest.java
 
b/test/unit/org/apache/cassandra/tools/nodetool/InvalidateNetworkPermissionsCacheTest.java
index d68994d..c54e526 100644
--- 
a/test/unit/org/apache/cassandra/tools/nodetool/InvalidateNetworkPermissionsCacheTest.java
+++ 
b/test/unit/org/apache/cassandra/tools/nodetool/InvalidateNetworkPermissionsCacheTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.tools.nodetool;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.auth.AuthCacheService;
 import org.apache.cassandra.auth.AuthTestUtils;
 import org.apache.cassandra.auth.AuthenticatedUser;
 import org.apache.cassandra.auth.IRoleManager;
@@ -44,6 +45,7 @@ public class InvalidateNetworkPermissionsCacheTest extends 
CQLTester
         IRoleManager roleManager = DatabaseDescriptor.getRoleManager();
         roleManager.createRole(AuthenticatedUser.SYSTEM_USER, ROLE_A, 
AuthTestUtils.getLoginRoleOptions());
         roleManager.createRole(AuthenticatedUser.SYSTEM_USER, ROLE_B, 
AuthTestUtils.getLoginRoleOptions());
+        AuthCacheService.initializeAndRegisterCaches();
 
         startJMXServer();
     }
diff --git 
a/test/unit/org/apache/cassandra/tools/nodetool/InvalidatePermissionsCacheTest.java
 
b/test/unit/org/apache/cassandra/tools/nodetool/InvalidatePermissionsCacheTest.java
index fdd0bf7..e6d2ba1 100644
--- 
a/test/unit/org/apache/cassandra/tools/nodetool/InvalidatePermissionsCacheTest.java
+++ 
b/test/unit/org/apache/cassandra/tools/nodetool/InvalidatePermissionsCacheTest.java
@@ -28,6 +28,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.auth.AuthCacheService;
 import org.apache.cassandra.auth.AuthTestUtils;
 import org.apache.cassandra.auth.AuthenticatedUser;
 import org.apache.cassandra.auth.DataResource;
@@ -59,6 +60,7 @@ public class InvalidatePermissionsCacheTest extends CQLTester
         IRoleManager roleManager = DatabaseDescriptor.getRoleManager();
         roleManager.createRole(AuthenticatedUser.SYSTEM_USER, ROLE_A, 
AuthTestUtils.getLoginRoleOptions());
         roleManager.createRole(AuthenticatedUser.SYSTEM_USER, ROLE_B, 
AuthTestUtils.getLoginRoleOptions());
+        AuthCacheService.initializeAndRegisterCaches();
 
         List<IResource> resources = Arrays.asList(
                 DataResource.root(),
diff --git 
a/test/unit/org/apache/cassandra/tools/nodetool/InvalidateRolesCacheTest.java 
b/test/unit/org/apache/cassandra/tools/nodetool/InvalidateRolesCacheTest.java
index 99cfb38..bb8fb92 100644
--- 
a/test/unit/org/apache/cassandra/tools/nodetool/InvalidateRolesCacheTest.java
+++ 
b/test/unit/org/apache/cassandra/tools/nodetool/InvalidateRolesCacheTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.tools.nodetool;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.auth.AuthCacheService;
 import org.apache.cassandra.auth.AuthTestUtils;
 import org.apache.cassandra.auth.AuthenticatedUser;
 import org.apache.cassandra.auth.IRoleManager;
@@ -44,6 +45,7 @@ public class InvalidateRolesCacheTest extends CQLTester
         IRoleManager roleManager = DatabaseDescriptor.getRoleManager();
         roleManager.createRole(AuthenticatedUser.SYSTEM_USER, ROLE_A, 
AuthTestUtils.getLoginRoleOptions());
         roleManager.createRole(AuthenticatedUser.SYSTEM_USER, ROLE_B, 
AuthTestUtils.getLoginRoleOptions());
+        AuthCacheService.initializeAndRegisterCaches();
 
         startJMXServer();
     }

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to