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]