You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jm...@apache.org on 2021/10/08 12:50:12 UTC

[cassandra] branch trunk updated: Allow reverse iteration order during resource permissions checking

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 5ab1531  Allow reverse iteration order during resource permissions checking
5ab1531 is described below

commit 5ab15316e1cd83c129f3a66ba89e0fdcb60e4dfd
Author: Josh McKenzie <jm...@apache.org>
AuthorDate: Wed Sep 15 16:29:01 2021 -0400

    Allow reverse iteration order during resource permissions checking
    
    Patch by Sam Tunnicliffe; reviewed by Blake Eggleston, Jeff Jirsa, and Aleksei Zotov for CASSANDRA-17016
    
    Co-authored by Sam Tunnicliffe <sa...@beobal.com>
    Co-authored by Josh McKenzie <jm...@apache.org>
---
 CHANGES.txt                                        |   1 +
 conf/cassandra.yaml                                |   7 ++
 src/java/org/apache/cassandra/config/Config.java   |   2 +
 .../cassandra/config/DatabaseDescriptor.java       |  10 ++
 .../org/apache/cassandra/service/ClientState.java  |   9 +-
 .../apache/cassandra/service/ClientStateTest.java  | 111 +++++++++++++++++++++
 6 files changed, 139 insertions(+), 1 deletion(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 0208fc4..efcbb5d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1
+ * Allow reverse iteration of resources during permissions checking (CASSANDRA-17016)
  * Add feature to verify correct ownership of attached locations on disk at startup (CASSANDRA-16879)
  * Make SSLContext creation pluggable/extensible (CASSANDRA-16666)
  * Add soft/hard limits to local reads to protect against reading too much data in a single query (CASSANDRA-16896)
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 5322bff..fb0263d 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -143,6 +143,13 @@ role_manager: CassandraRoleManager
 #   increase system_auth keyspace replication factor if you use this authorizer.
 network_authorizer: AllowAllNetworkAuthorizer
 
+# Depending on the auth strategy of the cluster, it can be beneficial to iterate
+# from root to table (root -> ks -> table) instead of table to root (table -> ks -> root).
+# As the auth entries are whitelisting, once a permission is found you know it to be
+# valid. We default to false as the legacy behavior is to query at the table level then
+# move back up to the root. See CASSANDRA-17016 for details.
+# traverse_auth_from_root = false
+
 # Validity period for roles cache (fetching granted roles can be an expensive
 # operation depending on the role manager, CassandraRoleManager is one example)
 # Granted roles are cached for authenticated sessions in AuthenticatedUser and
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 8c498e4..f7c0635 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -146,6 +146,8 @@ public class Config
     public boolean listen_on_broadcast_address = false;
     public String internode_authenticator;
 
+    public boolean traverse_auth_from_root = false;
+
     /*
      * RPC address and interface refer to the address/interface used for the native protocol used to communicate with
      * clients. It's still called RPC in some places even though Thrift RPC is gone. If you see references to native
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 9d08a62..3341b43 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1228,6 +1228,16 @@ public class DatabaseDescriptor
         DatabaseDescriptor.networkAuthorizer = networkAuthorizer;
     }
 
+    public static void setAuthFromRoot(boolean fromRoot)
+    {
+        conf.traverse_auth_from_root = fromRoot;
+    }
+
+    public static boolean getAuthFromRoot()
+    {
+        return conf.traverse_auth_from_root;
+    }
+
     public static IRoleManager getRoleManager()
     {
         return roleManager;
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index ed53a20..4333473 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -22,10 +22,13 @@ import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.util.Arrays;
 import java.util.HashSet;
+import java.util.List;
+import java.util.ListIterator;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
+import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -430,7 +433,11 @@ public class ClientState
 
     private void ensurePermissionOnResourceChain(Permission perm, IResource resource)
     {
-        for (IResource r : Resources.chain(resource))
+        List<? extends IResource> resources = Resources.chain(resource);
+        if (DatabaseDescriptor.getAuthFromRoot())
+            resources = Lists.reverse(resources);
+
+        for (IResource r : resources)
             if (authorize(r).contains(perm))
                 return;
 
diff --git a/test/unit/org/apache/cassandra/service/ClientStateTest.java b/test/unit/org/apache/cassandra/service/ClientStateTest.java
new file mode 100644
index 0000000..62f538c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/ClientStateTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.service;
+
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.Iterables;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.auth.AuthKeyspace;
+import org.apache.cassandra.auth.AuthTestUtils;
+import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.auth.CassandraAuthorizer;
+import org.apache.cassandra.auth.DataResource;
+import org.apache.cassandra.auth.IResource;
+import org.apache.cassandra.auth.IRoleManager;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.auth.Roles;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class ClientStateTest
+{
+    @BeforeClass
+    public static void beforeClass()
+    {
+        System.setProperty("org.apache.cassandra.disable_mbean_registration", "true");
+        SchemaLoader.prepareServer();
+        DatabaseDescriptor.setAuthFromRoot(true);
+        // 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));
+    }
+    @AfterClass
+    public static void afterClass()
+    {
+        System.clearProperty("org.apache.cassandra.disable_mbean_registration");
+    }
+
+    @Test
+    public void permissionsCheckStartsAtHeadOfResourceChain() throws Exception {
+        // verify that when performing a permissions check, we start from the
+        // root IResource in the applicable hierarchy and proceed to the more
+        // granular resources until we find the required permission (or until
+        // we reach the end of the resource chain). This is because our typical
+        // usage is to grant blanket permissions on the root resources to users
+        // and so we save lookups, cache misses and cache space by traversing in
+        // this order. e.g. for DataResources, we typically grant perms on the
+        // 'data' resource, so when looking up a users perms on a specific table
+        // it makes sense to follow: data -> keyspace -> table
+
+        final AtomicInteger getPermissionsRequestCount = new AtomicInteger(0);
+        final IResource rootResource = DataResource.root();
+        final IResource tableResource = DataResource.table("test_ks", "test_table");
+        final AuthenticatedUser testUser = new AuthenticatedUser("test_user")
+        {
+            public Set<Permission> getPermissions(IResource resource)
+            {
+                getPermissionsRequestCount.incrementAndGet();
+                if (resource.equals(rootResource))
+                    return Permission.ALL;
+
+                fail(String.format("Permissions requested for unexpected resource %s", resource));
+                // need a return to make the compiler happy
+                return null;
+            }
+
+            public boolean canLogin() { return true; }
+        };
+
+        IRoleManager roleManager = new AuthTestUtils.LocalCassandraRoleManager();
+        roleManager.setup();
+        Roles.initRolesCache(roleManager, () -> true);
+
+        // finally, need to configure CassandraAuthorizer so we don't shortcircuit out of the authz process
+        DatabaseDescriptor.setAuthorizer(new CassandraAuthorizer());
+
+        // check permissions on the table, which should check for the root resource first
+        // & return successfully without needing to proceed further
+        ClientState state = ClientState.forInternalCalls();
+        state.login(testUser);
+        state.ensurePermission(Permission.SELECT, tableResource);
+        assertEquals(1, getPermissionsRequestCount.get());
+    }
+}
\ No newline at end of file

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