You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sa...@apache.org on 2018/09/01 20:17:50 UTC

cassandra git commit: Improve RolesCache to include detailed role info

Repository: cassandra
Updated Branches:
  refs/heads/trunk f83bd5ac2 -> cc12665bb


Improve RolesCache to include detailed role info

Patch by Sam Tunnicliffe; reviewed by Jay Zhuang


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

Branch: refs/heads/trunk
Commit: cc12665bb7645d17ba70edcf952ee6a1ea63127b
Parents: f83bd5a
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Wed Apr 25 12:11:34 2018 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Sat Sep 1 21:12:52 2018 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/cassandra/auth/AuthCache.java    |  13 ++
 .../cassandra/auth/AuthenticatedUser.java       |  31 ++-
 .../cassandra/auth/CassandraAuthorizer.java     |   7 +-
 .../cassandra/auth/CassandraRoleManager.java    | 230 ++++++++-----------
 .../org/apache/cassandra/auth/IRoleManager.java |  18 ++
 src/java/org/apache/cassandra/auth/Role.java    |  72 ++++++
 src/java/org/apache/cassandra/auth/Roles.java   | 132 ++++++++++-
 .../org/apache/cassandra/auth/RolesCache.java   |  34 ++-
 .../apache/cassandra/service/ClientState.java   |   5 +-
 .../auth/CassandraNetworkAuthorizerTest.java    |  36 +--
 .../auth/CassandraRoleManagerTest.java          |  88 +++++++
 .../apache/cassandra/auth/RoleTestUtils.java    |  85 +++++++
 .../org/apache/cassandra/auth/RolesTest.java    |  95 ++++++++
 14 files changed, 669 insertions(+), 178 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1ba9975..a7468f4 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Extend RolesCache to include detailed role info (CASSANDRA-14497)
  * Add fqltool compare (CASSANDRA-14619)
  * Add fqltool replay (CASSANDRA-14618)
  * Log keyspace in full query log (CASSANDRA-14656)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/src/java/org/apache/cassandra/auth/AuthCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthCache.java b/src/java/org/apache/cassandra/auth/AuthCache.java
index 3954230..d6ff0b0 100644
--- a/src/java/org/apache/cassandra/auth/AuthCache.java
+++ b/src/java/org/apache/cassandra/auth/AuthCache.java
@@ -89,6 +89,19 @@ public class AuthCache<K, V> implements AuthCacheMBean
         }
     }
 
+    protected void unregisterMBean()
+    {
+        try
+        {
+            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+            mbs.unregisterMBean(getObjectName());
+        }
+        catch (Exception e)
+        {
+            logger.warn("Error unregistering {} cache mbean", name, e);
+        }
+    }
+
     protected ObjectName getObjectName() throws MalformedObjectNameException
     {
         return new ObjectName(MBEAN_NAME_BASE + name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
index 3d7c078..9f22bea 100644
--- a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
+++ b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
@@ -94,18 +94,46 @@ public class AuthenticatedUser
     /**
      * Get the roles that have been granted to the user via the IRoleManager
      *
-     * @return a list of roles that have been granted to the user
+     * @return a set of identifiers for the roles that have been granted to the user
      */
     public Set<RoleResource> getRoles()
     {
         return Roles.getRoles(role);
     }
 
+    /**
+     * Get the detailed info on roles granted to the user via IRoleManager
+     *
+     * @return a set of Role objects detailing the roles granted to the user
+     */
+    public Set<Role> getRoleDetails()
+    {
+       return Roles.getRoleDetails(role);
+    }
+
     public Set<Permission> getPermissions(IResource resource)
     {
         return permissionsCache.getPermissions(this, resource);
     }
 
+    /**
+     * Check whether this user has login privileges.
+     * LOGIN is not inherited from granted roles, so must be directly granted to the primary role for this user
+     *
+     * @return true if the user is permitted to login, false otherwise.
+     */
+    public boolean canLogin()
+    {
+        return Roles.canLogin(getPrimaryRole());
+    }
+
+    /**
+     * Verify that there is not DC level restriction on this user accessing this node.
+     * Further extends the login privilege check by verifying that the primary role for this user is permitted
+     * to perform operations in the local (to this node) datacenter. Like LOGIN, this is not inherited from
+     * granted roles.
+     * @return true if the user is permitted to access nodes in this node's datacenter, false otherwise
+     */
     public boolean hasLocalAccess()
     {
         return networkAuthCache.get(this.getPrimaryRole()).canAccess(Datacenters.thisDatacenter());
@@ -136,4 +164,5 @@ public class AuthenticatedUser
     {
         return Objects.hashCode(name);
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index cebde13..238b5b5 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -68,9 +68,10 @@ public class CassandraAuthorizer implements IAuthorizer
 
         Set<Permission> permissions = EnumSet.noneOf(Permission.class);
 
-        for (RoleResource role: user.getRoles())
-            addPermissionsForRole(permissions, resource, role);
-
+        // Even though we only care about the RoleResource here, we use getRoleDetails as
+        // it saves a Set creation in RolesCache
+        for (Role role: user.getRoleDetails())
+            addPermissionsForRole(permissions, resource, role.resource);
         return permissions;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index f5dd457..ebb7d5f 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -20,12 +20,14 @@ package org.apache.cassandra.auth;
 import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.*;
-import com.google.common.base.Objects;
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -79,27 +81,24 @@ public class CassandraRoleManager implements IRoleManager
     static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";
 
     // Transform a row in the AuthKeyspace.ROLES to a Role instance
-    private static final Function<UntypedResultSet.Row, Role> ROW_TO_ROLE = new Function<UntypedResultSet.Row, Role>()
+    private static final Function<UntypedResultSet.Row, Role> ROW_TO_ROLE = row ->
     {
-        public Role apply(UntypedResultSet.Row row)
+        try
         {
-            try
-            {
-                return new Role(row.getString("role"),
-                         row.getBoolean("is_superuser"),
-                         row.getBoolean("can_login"),
-                         row.has("member_of") ? row.getSet("member_of", UTF8Type.instance)
-                                              : Collections.<String>emptySet());
-            }
-            // Failing to deserialize a boolean in is_superuser or can_login will throw an NPE
-            catch (NullPointerException e)
-            {
-                logger.warn("An invalid value has been detected in the {} table for role {}. If you are " +
-                            "unable to login, you may need to disable authentication and confirm " +
-                            "that values in that table are accurate", AuthKeyspace.ROLES, row.getString("role"));
-                throw new RuntimeException(String.format("Invalid metadata has been detected for role %s", row.getString("role")), e);
-            }
-
+            return new Role(row.getString("role"),
+                            row.getBoolean("is_superuser"),
+                            row.getBoolean("can_login"),
+                            Collections.emptyMap(),
+                            row.has("member_of") ? row.getSet("member_of", UTF8Type.instance)
+                                                 : Collections.<String>emptySet());
+        }
+        // Failing to deserialize a boolean in is_superuser or can_login will throw an NPE
+        catch (NullPointerException e)
+        {
+            logger.warn("An invalid value has been detected in the {} table for role {}. If you are " +
+                        "unable to login, you may need to disable authentication and confirm " +
+                        "that values in that table are accurate", AuthKeyspace.ROLES, row.getString("role"));
+            throw new RuntimeException(String.format("Invalid metadata has been detected for role %s", row.getString("role")), e);
         }
     };
 
@@ -117,9 +116,6 @@ public class CassandraRoleManager implements IRoleManager
          return rounds;
     }
 
-    // NullObject returned when a supplied role name not found in AuthKeyspace.ROLES
-    private static final Role NULL_ROLE = new Role(null, false, false, Collections.<String>emptySet());
-
     private SelectStatement loadRoleStatement;
 
     private final Set<Option> supportedOptions;
@@ -193,8 +189,7 @@ public class CassandraRoleManager implements IRoleManager
     {
         // Unlike most of the other data access methods here, this does not use a
         // prepared statement in order to allow the set of assignments to be variable.
-        String assignments = Joiner.on(',').join(Iterables.filter(optionsToAssignments(options.getOptions()),
-                                                                  Predicates.notNull()));
+        String assignments = optionsToAssignments(options.getOptions());
         if (!Strings.isNullOrEmpty(assignments))
         {
             process(String.format("UPDATE %s.%s SET %s WHERE role = '%s'",
@@ -244,29 +239,33 @@ public class CassandraRoleManager implements IRoleManager
                 consistencyForRole(role.getRoleName()));
     }
 
-    public Set<RoleResource> getRoles(RoleResource grantee, boolean includeInherited) throws RequestValidationException, RequestExecutionException
+    public Set<RoleResource> getRoles(RoleResource grantee, boolean includeInherited)
+    throws RequestValidationException, RequestExecutionException
     {
-        Set<RoleResource> roles = new HashSet<>();
-        Role role = getRole(grantee.getRoleName());
-        if (!role.equals(NULL_ROLE))
-        {
-            roles.add(RoleResource.role(role.name));
-            collectRoles(role, roles, includeInherited);
-        }
-        return roles;
+        return collectRoles(getRole(grantee.getRoleName()),
+                            includeInherited,
+                            filter())
+               .map(r -> r.resource)
+               .collect(Collectors.toSet());
+    }
+
+    public Set<Role> getRoleDetails(RoleResource grantee)
+    {
+        return collectRoles(getRole(grantee.getRoleName()),
+                            true,
+                            filter())
+               .collect(Collectors.toSet());
     }
 
     public Set<RoleResource> getAllRoles() throws RequestValidationException, RequestExecutionException
     {
-        UntypedResultSet rows = process(String.format("SELECT role from %s.%s", SchemaConstants.AUTH_KEYSPACE_NAME, AuthKeyspace.ROLES), ConsistencyLevel.QUORUM);
-        Iterable<RoleResource> roles = Iterables.transform(rows, new Function<UntypedResultSet.Row, RoleResource>()
-        {
-            public RoleResource apply(UntypedResultSet.Row row)
-            {
-                return RoleResource.role(row.getString("role"));
-            }
-        });
-        return ImmutableSet.<RoleResource>builder().addAll(roles).build();
+        ImmutableSet.Builder<RoleResource> builder = ImmutableSet.builder();
+        UntypedResultSet rows = process(String.format("SELECT role from %s.%s",
+                                                      SchemaConstants.AUTH_KEYSPACE_NAME,
+                                                      AuthKeyspace.ROLES),
+                                        ConsistencyLevel.QUORUM);
+        rows.forEach(row -> builder.add(RoleResource.role(row.getString("role"))));
+        return builder.build();
     }
 
     public boolean isSuper(RoleResource role)
@@ -286,7 +285,7 @@ public class CassandraRoleManager implements IRoleManager
 
     public boolean isExistingRole(RoleResource role)
     {
-        return getRole(role.getRoleName()) != NULL_ROLE;
+        return !Roles.isNullRole(getRole(role.getRoleName()));
     }
 
     public Set<? extends IResource> protectedResources()
@@ -340,23 +339,19 @@ public class CassandraRoleManager implements IRoleManager
                || !QueryProcessor.process(allUsersQuery, ConsistencyLevel.QUORUM).isEmpty();
     }
 
-    private void scheduleSetupTask(final Callable<Void> setupTask)
+    protected void scheduleSetupTask(final Callable<Void> setupTask)
     {
         // The delay is to give the node a chance to see its peers before attempting the operation
-        ScheduledExecutors.optionalTasks.schedule(new Runnable()
-        {
-            public void run()
+        ScheduledExecutors.optionalTasks.schedule(() -> {
+            isClusterReady = true;
+            try
             {
-                isClusterReady = true;
-                try
-                {
-                    setupTask.call();
-                }
-                catch (Exception e)
-                {
-                    logger.info("Setup task failed with error, rescheduling");
-                    scheduleSetupTask(setupTask);
-                }
+                setupTask.call();
+            }
+            catch (Exception e)
+            {
+                logger.info("Setup task failed with error, rescheduling");
+                scheduleSetupTask(setupTask);
             }
         }, AuthKeyspace.SUPERUSER_SETUP_DELAY, TimeUnit.MILLISECONDS);
     }
@@ -373,32 +368,31 @@ public class CassandraRoleManager implements IRoleManager
         }
     }
 
-    /*
-     * Retrieve all roles granted to the given role. includeInherited specifies
-     * whether to include only those roles granted directly or all inherited roles.
-     */
-    private void collectRoles(Role role, Set<RoleResource> collected, boolean includeInherited) throws RequestValidationException, RequestExecutionException
+    private Stream<Role> collectRoles(Role role, boolean includeInherited, Predicate<String> distinctFilter)
     {
-        for (String memberOf : role.memberOf)
-        {
-            Role granted = getRole(memberOf);
-            if (granted.equals(NULL_ROLE))
-                continue;
-            collected.add(RoleResource.role(granted.name));
-            if (includeInherited)
-                collectRoles(granted, collected, true);
-        }
+        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()
+                                          .filter(distinctFilter)
+                                          .flatMap(r -> collectRoles(getRole(r), true, distinctFilter)));
     }
 
-    @VisibleForTesting
-    ResultMessage.Rows select(SelectStatement statement, QueryOptions options)
+    // Used as a stateful filtering function when recursively collecting granted roles
+    private static Predicate<String> filter()
     {
-        return statement.execute(QueryState.forInternalCalls(), options, System.nanoTime());
+        final Set<String> seen = new HashSet<>();
+        return seen::add;
     }
 
     /*
      * Get a single Role instance given the role name. This never returns null, instead it
-     * uses the null object NULL_ROLE when a role with the given name cannot be found. So
+     * uses a null object when a role with the given name cannot be found. So
      * it's always safe to call methods on the returned object without risk of NPE.
      */
     private Role getRole(String name)
@@ -407,7 +401,7 @@ public class CassandraRoleManager implements IRoleManager
                                                              Collections.singletonList(ByteBufferUtil.bytes(name)));
         ResultMessage.Rows rows = select(loadRoleStatement, options);
         if (rows.result.isEmpty())
-            return NULL_ROLE;
+            return Roles.nullRole();
 
         return ROW_TO_ROLE.apply(UntypedResultSet.create(rows.result).one());
     }
@@ -458,27 +452,26 @@ public class CassandraRoleManager implements IRoleManager
      * Convert a map of Options from a CREATE/ALTER statement into
      * assignment clauses used to construct a CQL UPDATE statement
      */
-    private Iterable<String> optionsToAssignments(Map<Option, Object> options)
-    {
-        return Iterables.transform(
-                                  options.entrySet(),
-                                  new Function<Map.Entry<Option, Object>, String>()
-                                  {
-                                      public String apply(Map.Entry<Option, Object> entry)
-                                      {
-                                          switch (entry.getKey())
-                                          {
-                                              case LOGIN:
-                                                  return String.format("can_login = %s", entry.getValue());
-                                              case SUPERUSER:
-                                                  return String.format("is_superuser = %s", entry.getValue());
-                                              case PASSWORD:
-                                                  return String.format("salted_hash = '%s'", escape(hashpw((String) entry.getValue())));
-                                              default:
-                                                  return null;
-                                          }
-                                      }
-                                  });
+    private String optionsToAssignments(Map<Option, Object> options)
+    {
+        return options.entrySet()
+                      .stream()
+                      .map(entry ->
+                           {
+                               switch (entry.getKey())
+                               {
+                                   case LOGIN:
+                                       return String.format("can_login = %s", entry.getValue());
+                                   case SUPERUSER:
+                                       return String.format("is_superuser = %s", entry.getValue());
+                                   case PASSWORD:
+                                       return String.format("salted_hash = '%s'", escape(hashpw((String) entry.getValue())));
+                                   default:
+                                       return null;
+                               }
+                           })
+                      .filter(Objects::nonNull)
+                      .collect(Collectors.joining(","));
     }
 
     protected static ConsistencyLevel consistencyForRole(String role)
@@ -505,7 +498,8 @@ public class CassandraRoleManager implements IRoleManager
      * should use QueryProcessor.process directly.
      */
     @VisibleForTesting
-    UntypedResultSet process(String query, ConsistencyLevel consistencyLevel) throws RequestValidationException, RequestExecutionException
+    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. "
@@ -515,36 +509,10 @@ public class CassandraRoleManager implements IRoleManager
         return QueryProcessor.process(query, consistencyLevel);
     }
 
-    private static final class Role
+    @VisibleForTesting
+    ResultMessage.Rows select(SelectStatement statement, QueryOptions options)
     {
-        private String name;
-        private final boolean isSuper;
-        private final boolean canLogin;
-        private Set<String> memberOf;
-
-        private Role(String name, boolean isSuper, boolean canLogin, Set<String> memberOf)
-        {
-            this.name = name;
-            this.isSuper = isSuper;
-            this.canLogin = canLogin;
-            this.memberOf = memberOf;
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o)
-                return true;
-
-            if (!(o instanceof Role))
-                return false;
-
-            Role r = (Role) o;
-            return Objects.equal(name, r.name);
-        }
-
-        public int hashCode()
-        {
-            return Objects.hashCode(name);
-        }
+        return statement.execute(QueryState.forInternalCalls(), options, System.nanoTime());
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/src/java/org/apache/cassandra/auth/IRoleManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/IRoleManager.java b/src/java/org/apache/cassandra/auth/IRoleManager.java
index b27681d..1d47bee 100644
--- a/src/java/org/apache/cassandra/auth/IRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/IRoleManager.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.auth;
 
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
@@ -135,6 +136,23 @@ public interface IRoleManager
     Set<RoleResource> getRoles(RoleResource grantee, boolean includeInherited) throws RequestValidationException, RequestExecutionException;
 
     /**
+     * Used to retrieve detailed role info on the full set of roles granted to a grantee.
+     * This method was not part of the V1 IRoleManager API, so a default impl is supplied which uses the V1
+     * methods to retrieve the detailed role info for the grantee. This is essentially what clients of this interface
+     * would have to do themselves. Implementations can provide optimized versions of this method where the details
+     * can be retrieved more efficiently.
+     *
+     * @param grantee identifies the role whose granted roles are retrieved
+     * @return A set of Role objects detailing the roles granted to the grantee, either directly or through inheritance.
+     */
+     default Set<Role> getRoleDetails(RoleResource grantee)
+     {
+         return getRoles(grantee, true).stream()
+                                       .map(roleResource -> Roles.fromRoleResource(roleResource, this))
+                                       .collect(Collectors.toSet());
+     }
+
+    /**
      * Called during the execution of an unqualified LIST ROLES query.
      * Returns the total set of distinct roles in the system.
      *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/src/java/org/apache/cassandra/auth/Role.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/Role.java b/src/java/org/apache/cassandra/auth/Role.java
new file mode 100644
index 0000000..e98cc7d
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/Role.java
@@ -0,0 +1,72 @@
+/*
+ * 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.Map;
+import java.util.Set;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+public class Role
+{
+    /**
+     * Represents a user or group in the auth subsystem.
+     * Roles may be members of other roles, but circular graphs of roles are not permitted.
+     * The reason that memberOf is a Set<String> and not Set<Role> is to simplify loading
+     * for IRoleManager implementations (in particular, CassandraRoleManager)
+     */
+
+    public final RoleResource resource ;
+    public final boolean isSuper;
+    public final boolean canLogin;
+    public final Set<String> memberOf;
+    public final Map<String, String> options;
+
+    public Role(String name, boolean isSuperUser, boolean canLogin, Map<String, String> options, Set<String> memberOf)
+    {
+        this.resource = RoleResource.role(name);
+        this.isSuper = isSuperUser;
+        this.canLogin = canLogin;
+        this.memberOf = ImmutableSet.copyOf(memberOf);
+        this.options = ImmutableMap.copyOf(options);
+    }
+
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof Role))
+            return false;
+
+        Role r = (Role)o;
+        return Objects.equal(resource, r.resource)
+               && Objects.equal(isSuper, r.isSuper)
+               && Objects.equal(canLogin, r.canLogin)
+               && Objects.equal(memberOf, r.memberOf)
+               && Objects.equal(options, r.options);
+    }
+
+    public int hashCode()
+    {
+        return Objects.hashCode(resource, isSuper, canLogin, memberOf, options);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/src/java/org/apache/cassandra/auth/Roles.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/Roles.java b/src/java/org/apache/cassandra/auth/Roles.java
index 854d1f2..22eb0d3 100644
--- a/src/java/org/apache/cassandra/auth/Roles.java
+++ b/src/java/org/apache/cassandra/auth/Roles.java
@@ -17,24 +17,68 @@
  */
 package org.apache.cassandra.auth;
 
+import java.util.Collections;
+import java.util.Map;
 import java.util.Set;
+import java.util.function.BooleanSupplier;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 
 public class Roles
 {
-    private static final RolesCache cache = new RolesCache(DatabaseDescriptor.getRoleManager());
+    private static final Role NO_ROLE = new Role("", false, false, Collections.emptyMap(), Collections.emptySet());
+
+    private static RolesCache cache;
+    static
+    {
+        initRolesCache(DatabaseDescriptor.getRoleManager(),
+                       () -> DatabaseDescriptor.getAuthenticator().requireAuthentication());
+    }
+
+    @VisibleForTesting
+    public static void initRolesCache(IRoleManager roleManager, BooleanSupplier enableCache)
+    {
+        if (cache != null)
+            cache.unregisterMBean();
+        cache = new RolesCache(roleManager, enableCache);
+    }
+
+    @VisibleForTesting
+    public static void clearCache()
+    {
+        cache.invalidate();
+    }
 
     /**
-     * Get all roles granted to the supplied Role, including both directly granted
+     * Identify all roles granted to the supplied Role, including both directly granted
      * and inherited roles.
-     * The returned roles may be cached if {@code roles_validity_in_ms > 0}
+     * This method is used where we mainly just care about *which* roles are granted to a given role,
+     * including when looking up or listing permissions for a role on a given resource.
      *
      * @param primaryRole the Role
      * @return set of all granted Roles for the primary Role
      */
     public static Set<RoleResource> getRoles(RoleResource primaryRole)
     {
+        return cache.getRoleResources(primaryRole);
+    }
+
+    /**
+     * Get detailed info on all the roles granted to the role identified by the supplied RoleResource.
+     * This includes superuser status and login privileges for the primary role and all roles granted directly
+     * to it or inherited.
+     * The returnred roles may be cached if roles_validity_in_ms > 0
+     * This method is used where we need to know specific attributes of the collection of granted roles, i.e.
+     * when checking for superuser status which may be inherited from *any* granted role.
+     *
+     * @param primaryRole identifies the role
+     * @return set of detailed info for all of the roles granted to the primary
+     */
+    public static Set<Role> getRoleDetails(RoleResource primaryRole)
+    {
         return cache.getRoles(primaryRole);
     }
 
@@ -47,10 +91,86 @@ public class Roles
      */
     public static boolean hasSuperuserStatus(RoleResource role)
     {
-        IRoleManager roleManager = DatabaseDescriptor.getRoleManager();
-        for (RoleResource r : cache.getRoles(role))
-            if (roleManager.isSuper(r))
+        for (Role r : getRoleDetails(role))
+            if (r.isSuper)
                 return true;
+
         return false;
     }
+
+    /**
+     * Returns true if the supplied role has the login privilege. This cannot be inherited, so
+     * returns true iff the named role has that bit set.
+     * @param role the role identifier
+     * @return true if the role has the canLogin privilege, false otherwise
+     */
+    public static boolean canLogin(final RoleResource role)
+    {
+        for (Role r : getRoleDetails(role))
+            if (r.resource.equals(role))
+                return r.canLogin;
+
+        return false;
+    }
+
+    /**
+     * Returns the map of custom options for the named role. These options are not inherited from granted roles, but
+     * are set directly.
+     * @param role the role identifier
+     * @return map of option_name -> value. If no options are set for the named role, the map will be empty
+     * but never null.
+     */
+    public static Map<String, String> getOptions(RoleResource role)
+    {
+        for (Role r : getRoleDetails(role))
+            if (r.resource.equals(role))
+                return r.options;
+
+        return NO_ROLE.options;
+    }
+
+   /**
+    * Return the NullObject Role instance which can be safely used to indicate no information is available
+    * when querying for a specific named role.
+    * @return singleton null role object
+    */
+   public static Role nullRole()
+   {
+       return NO_ROLE;
+   }
+
+   /**
+    * Just a convenience method which compares a role instance with the null object version, indicating if the
+    * return from some query/lookup method was a valid Role or indicates that the role does not exist.
+    * @param role
+    * @return true if the supplied role is the null role instance, false otherwise.
+    */
+   public static boolean isNullRole(Role role)
+   {
+       return NO_ROLE.equals(role);
+   }
+
+
+   /**
+    * Constructs a Role object from a RoleResource, using the methods of the supplied IRoleManager.
+    * This is used by the default implementation of IRoleManager#getRoleDetails so that IRoleManager impls
+    * which don't implement an optimized getRoleDetails remain compatible. Depending on the IRoleManager
+    * implementation this could be quite heavyweight, so should not be used on any hot path.
+    *
+    * @param resource identifies the role
+    * @param roleManager provides lookup functions to retrieve role info
+    * @return Role object including superuser status, login privilege, custom options and the set of roles
+    * granted to identified role.
+    */
+   public static Role fromRoleResource(RoleResource resource, IRoleManager roleManager)
+   {
+       return new Role(resource.getName(),
+                       roleManager.isSuper(resource),
+                       roleManager.canLogin(resource),
+                       roleManager.getCustomOptions(resource),
+                       roleManager.getRoles(resource, false)
+                                  .stream()
+                                  .map(RoleResource::getRoleName)
+                                  .collect(Collectors.toSet()));
+   }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/src/java/org/apache/cassandra/auth/RolesCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/RolesCache.java b/src/java/org/apache/cassandra/auth/RolesCache.java
index a02828a..cc178ce 100644
--- a/src/java/org/apache/cassandra/auth/RolesCache.java
+++ b/src/java/org/apache/cassandra/auth/RolesCache.java
@@ -18,12 +18,14 @@
 package org.apache.cassandra.auth;
 
 import java.util.Set;
+import java.util.function.BooleanSupplier;
+import java.util.stream.Collectors;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 
-public class RolesCache extends AuthCache<RoleResource, Set<RoleResource>> implements RolesCacheMBean
+public class RolesCache extends AuthCache<RoleResource, Set<Role>> implements RolesCacheMBean
 {
-    public RolesCache(IRoleManager roleManager)
+    public RolesCache(IRoleManager roleManager, BooleanSupplier enableCache)
     {
         super("RolesCache",
               DatabaseDescriptor::setRolesValidity,
@@ -32,12 +34,32 @@ public class RolesCache extends AuthCache<RoleResource, Set<RoleResource>> imple
               DatabaseDescriptor::getRolesUpdateInterval,
               DatabaseDescriptor::setRolesCacheMaxEntries,
               DatabaseDescriptor::getRolesCacheMaxEntries,
-              (r) -> roleManager.getRoles(r, true),
-              () -> DatabaseDescriptor.getAuthenticator().requireAuthentication());
+              roleManager::getRoleDetails,
+              enableCache);
     }
 
-    public Set<RoleResource> getRoles(RoleResource role)
+    /**
+     * Read or return from the cache the Set of the RoleResources identifying the roles granted to the primary resource
+     * @see Roles#getRoles(RoleResource)
+     * @param primaryRole identifier for the primary role
+     * @return the set of identifiers of all the roles granted to (directly or through inheritance) the primary role
+     */
+    Set<RoleResource> getRoleResources(RoleResource primaryRole)
     {
-        return get(role);
+        return get(primaryRole).stream()
+                               .map(r -> r.resource)
+                               .collect(Collectors.toSet());
+    }
+
+    /**
+     * Read or return from cache the set of Role objects representing the roles granted to the primary resource
+     * @see Roles#getRoleDetails(RoleResource)
+     * @param primaryRole identifier for the primary role
+     * @return the set of Role objects containing info of all roles granted to (directly or through inheritance)
+     * the primary role.
+     */
+    Set<Role> getRoles(RoleResource primaryRole)
+    {
+        return get(primaryRole);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index cb06161..26ed271 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -323,10 +323,7 @@ public class ClientState
      */
     public void login(AuthenticatedUser user)
     {
-        // Login privilege is not inherited via granted roles, so just
-        // verify that the role with the credentials that were actually
-        // supplied has it
-        if (user.isAnonymous() || DatabaseDescriptor.getRoleManager().canLogin(user.getPrimaryRole()))
+        if (user.isAnonymous() || user.canLogin())
             this.user = user;
         else
             throw new AuthenticationException(String.format("%s is not permitted to log in", user.getName()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
index bf50094..c24a769 100644
--- a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
+++ b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
@@ -40,17 +40,16 @@ import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.CreateRoleStatement;
 import org.apache.cassandra.cql3.statements.DropRoleStatement;
 import org.apache.cassandra.cql3.statements.SelectStatement;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
 
 import static org.apache.cassandra.auth.AuthKeyspace.NETWORK_PERMISSIONS;
+import static org.apache.cassandra.auth.RoleTestUtils.LocalCassandraRoleManager;
 import static org.apache.cassandra.schema.SchemaConstants.AUTH_KEYSPACE_NAME;
 
 public class CassandraNetworkAuthorizerTest
@@ -74,19 +73,6 @@ public class CassandraNetworkAuthorizerTest
         }
     }
 
-    private static class LocalCassandraRoleManager extends CassandraRoleManager
-    {
-        ResultMessage.Rows select(SelectStatement statement, QueryOptions options)
-        {
-            return statement.executeLocally(QueryState.forInternalCalls(), options);
-        }
-
-        UntypedResultSet process(String query, ConsistencyLevel consistencyLevel) throws RequestValidationException, RequestExecutionException
-        {
-            return QueryProcessor.executeInternal(query);
-        }
-    }
-
     private static class LocalCassandraNetworkAuthorizer extends CassandraNetworkAuthorizer
     {
         ResultMessage.Rows select(SelectStatement statement, QueryOptions options)
@@ -122,17 +108,11 @@ public class CassandraNetworkAuthorizerTest
     }
 
     @Before
-    public void clear() throws Exception
+    public void clear()
     {
         Keyspace.open(AUTH_KEYSPACE_NAME).getColumnFamilyStore(NETWORK_PERMISSIONS).truncateBlocking();
     }
 
-
-    private static UntypedResultSet query(String q)
-    {
-        return QueryProcessor.executeInternal(q);
-    }
-
     private static void assertNoDcPermRow(String username)
     {
         String query = String.format("SELECT dcs FROM %s.%s WHERE role = '%s'",
@@ -175,6 +155,9 @@ public class CassandraNetworkAuthorizerTest
                || statement instanceof AlterRoleStatement
                || statement instanceof DropRoleStatement;
         AuthenticationStatement authStmt = (AuthenticationStatement) statement;
+
+        // invalidate roles cache so that any changes to the underlying roles are picked up
+        Roles.clearCache();
         authStmt.execute(getClientState());
     }
 
@@ -185,7 +168,7 @@ public class CassandraNetworkAuthorizerTest
     }
 
     @Test
-    public void create() throws Exception
+    public void create()
     {
         String username = createName();
 
@@ -197,7 +180,7 @@ public class CassandraNetworkAuthorizerTest
     }
 
     @Test
-    public void alter() throws Exception
+    public void alter()
     {
 
         String username = createName();
@@ -237,7 +220,7 @@ public class CassandraNetworkAuthorizerTest
     }
 
     @Test
-    public void superUser() throws Exception
+    public void superUser()
     {
         String username = createName();
         auth("CREATE ROLE %s WITH password = 'password' AND LOGIN = true AND ACCESS TO DATACENTERS {'dc1'}", username);
@@ -249,11 +232,10 @@ public class CassandraNetworkAuthorizerTest
     }
 
     @Test
-    public void cantLogin() throws Exception
+    public void cantLogin()
     {
         String username = createName();
         auth("CREATE ROLE %s", username);
         Assert.assertEquals(DCPermissions.none(), dcPerms(username));
-
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java b/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java
new file mode 100644
index 0000000..6583c49
--- /dev/null
+++ b/test/unit/org/apache/cassandra/auth/CassandraRoleManagerTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.Set;
+
+import com.google.common.collect.Iterables;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static org.apache.cassandra.auth.RoleTestUtils.*;
+import static org.junit.Assert.assertEquals;
+
+public class CassandraRoleManagerTest
+{
+
+    @BeforeClass
+    public static void setupClass()
+    {
+        SchemaLoader.prepareServer();
+        // create the system_auth keyspace so the IRoleManager can function as normal
+        SchemaLoader.createKeyspace(SchemaConstants.AUTH_KEYSPACE_NAME,
+                                    KeyspaceParams.simple(1),
+                                    Iterables.toArray(AuthKeyspace.metadata().tables, TableMetadata.class));
+    }
+
+    @Test
+    public void getGrantedRolesImplMinimizesReads()
+    {
+        // IRoleManager::getRoleDetails was not in the initial API, so a default impl
+        // was added which uses the existing methods on IRoleManager as primitive to
+        // construct the Role objects. While this will work for any IRoleManager impl
+        // it is inefficient, so CassandraRoleManager has its own implementation which
+        // collects all of the necessary info with a single query for each granted role.
+        // This just tests that that is the case, i.e. we perform 1 read per role in the
+        // transitive set of granted roles
+        IRoleManager roleManager = new LocalCassandraRoleManager();
+        roleManager.setup();
+        for (RoleResource r : ALL_ROLES)
+            roleManager.createRole(AuthenticatedUser.ANONYMOUS_USER, r, new RoleOptions());
+
+        // simple role with no grants
+        fetchRolesAndCheckReadCount(roleManager, ROLE_A);
+        // single level of grants
+        grantRolesTo(roleManager, ROLE_A, ROLE_B, ROLE_C);
+        fetchRolesAndCheckReadCount(roleManager, ROLE_A);
+
+        // 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);
+        fetchRolesAndCheckReadCount(roleManager, ROLE_A);
+
+        // Check that when granted roles appear multiple times in parallel levels of the hierarchy, we don't
+        // do redundant reads. E.g. here role_b_1, role_b_2 and role_b3 are granted to both role_b and role_c
+        // but we only want to actually read them once
+        grantRolesTo(roleManager, ROLE_C, ROLE_B_1, ROLE_B_2, ROLE_B_3);
+        fetchRolesAndCheckReadCount(roleManager, ROLE_A);
+    }
+
+    private void fetchRolesAndCheckReadCount(IRoleManager roleManager, RoleResource primaryRole)
+    {
+        long before = getReadCount();
+        Set<Role> granted = roleManager.getRoleDetails(primaryRole);
+        long after = getReadCount();
+        assertEquals(granted.size(), after - before);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/test/unit/org/apache/cassandra/auth/RoleTestUtils.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/auth/RoleTestUtils.java b/test/unit/org/apache/cassandra/auth/RoleTestUtils.java
new file mode 100644
index 0000000..e2d1006
--- /dev/null
+++ b/test/unit/org/apache/cassandra/auth/RoleTestUtils.java
@@ -0,0 +1,85 @@
+/*
+ * 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.concurrent.Callable;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+
+public class RoleTestUtils
+{
+
+    public static final RoleResource ROLE_A = RoleResource.role("role_a");
+    public static final RoleResource ROLE_B = RoleResource.role("role_b");
+    public static final RoleResource ROLE_B_1 = RoleResource.role("role_b_1");
+    public static final RoleResource ROLE_B_2 = RoleResource.role("role_b_2");
+    public static final RoleResource ROLE_B_3 = RoleResource.role("role_b_3");
+    public static final RoleResource ROLE_C = RoleResource.role("role_c");
+    public static final RoleResource ROLE_C_1 = RoleResource.role("role_c_1");
+    public static final RoleResource ROLE_C_2 = RoleResource.role("role_c_2");
+    public static final RoleResource ROLE_C_3 = RoleResource.role("role_c_3");
+    public static final RoleResource[] ALL_ROLES  = new RoleResource[] {ROLE_A,
+                                                                        ROLE_B, ROLE_B_1, ROLE_B_2, ROLE_B_3,
+                                                                        ROLE_C, ROLE_C_1, ROLE_C_2, ROLE_C_3};
+    /**
+     * This just extends the internal IRoleManager implementation to ensure that
+     * all access to underlying tables is made via
+     * QueryProcessor.executeOnceInternal/CQLStatement.executeInternal and not
+     * StorageProxy so that it can be used in unit tests.
+     */
+    public static class LocalCassandraRoleManager extends CassandraRoleManager
+    {
+        ResultMessage.Rows select(SelectStatement statement, QueryOptions options)
+        {
+            return statement.executeLocally(QueryState.forInternalCalls(), options);
+        }
+
+        UntypedResultSet process(String query, ConsistencyLevel consistencyLevel)
+        {
+            return QueryProcessor.executeInternal(query);
+        }
+
+        protected void scheduleSetupTask(final Callable<Void> setupTask)
+        {
+            // skip data migration or setting up default role for tests
+        }
+    }
+
+    public static void grantRolesTo(IRoleManager roleManager, RoleResource grantee, RoleResource...granted)
+    {
+        for(RoleResource toGrant : granted)
+            roleManager.grantRole(AuthenticatedUser.ANONYMOUS_USER, toGrant, grantee);
+    }
+
+    public static long getReadCount()
+    {
+        ColumnFamilyStore rolesTable = Keyspace.open(SchemaConstants.AUTH_KEYSPACE_NAME).getColumnFamilyStore(AuthKeyspace.ROLES);
+        return rolesTable.metric.readLatency.latency.getCount();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cc12665b/test/unit/org/apache/cassandra/auth/RolesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/auth/RolesTest.java b/test/unit/org/apache/cassandra/auth/RolesTest.java
new file mode 100644
index 0000000..94322a7
--- /dev/null
+++ b/test/unit/org/apache/cassandra/auth/RolesTest.java
@@ -0,0 +1,95 @@
+/*
+ * 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.Set;
+
+import com.google.common.collect.Iterables;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static org.apache.cassandra.auth.RoleTestUtils.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class RolesTest
+{
+
+    @BeforeClass
+    public static void setupClass()
+    {
+        SchemaLoader.prepareServer();
+        // create the system_auth keyspace so the IRoleManager can function as normal
+        SchemaLoader.createKeyspace(SchemaConstants.AUTH_KEYSPACE_NAME,
+                                    KeyspaceParams.simple(1),
+                                    Iterables.toArray(AuthKeyspace.metadata().tables, TableMetadata.class));
+
+        IRoleManager roleManager = new LocalCassandraRoleManager();
+        roleManager.setup();
+        Roles.initRolesCache(roleManager, () -> true);
+        for (RoleResource role : ALL_ROLES)
+            roleManager.createRole(AuthenticatedUser.ANONYMOUS_USER, role, new RoleOptions());
+        grantRolesTo(roleManager, ROLE_A, ROLE_B, ROLE_C);
+    }
+
+    @Test
+    public void superuserStatusIsCached()
+    {
+        boolean hasSuper = Roles.hasSuperuserStatus(ROLE_A);
+        long count = getReadCount();
+
+        assertEquals(hasSuper, Roles.hasSuperuserStatus(ROLE_A));
+        assertEquals(count, getReadCount());
+    }
+
+    @Test
+    public void loginPrivilegeIsCached()
+    {
+        boolean canLogin = Roles.canLogin(ROLE_A);
+        long count = getReadCount();
+
+        assertEquals(canLogin, Roles.canLogin(ROLE_A));
+        assertEquals(count, getReadCount());
+    }
+
+    @Test
+    public void grantedRoleDetailsAreCached()
+    {
+        Iterable<Role> granted = Roles.getRoleDetails(ROLE_A);
+        long count = getReadCount();
+
+        assertTrue(Iterables.elementsEqual(granted, Roles.getRoleDetails(ROLE_A)));
+        assertEquals(count, getReadCount());
+    }
+
+    @Test
+    public void grantedRoleResourcesAreCached()
+    {
+        Set<RoleResource> granted = Roles.getRoles(ROLE_A);
+        long count = getReadCount();
+
+        assertEquals(granted, Roles.getRoles(ROLE_A));
+        assertEquals(count, getReadCount());
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org