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/09/24 15:18:11 UTC

[cassandra] branch trunk updated: Evaluate consistency levels of auth reads

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 9f15ec6  Evaluate consistency levels of auth reads
9f15ec6 is described below

commit 9f15ec6de11c57d5fff02fe08639b647fc0749e8
Author: Josh McKenzie <jm...@apache.org>
AuthorDate: Thu Sep 2 13:58:16 2021 -0400

    Evaluate consistency levels of auth reads
    
    Patch by Jason Brown; reviewed by Matthew Byrd, Sankalp Kohli, and Benjamin Lerer for CASSANDRA-12988
    
    Co-authored by Jason Brown (jasedbrown@gmail.com)
    Co-authored by Josh McKenzie (jmckenzie@apache.org)
---
 CHANGES.txt                                        |  1 +
 NEWS.txt                                           |  3 +
 conf/cassandra.yaml                                |  4 ++
 .../org/apache/cassandra/auth/AuthProperties.java  | 70 ++++++++++++++++++++++
 .../cassandra/auth/AuthPropertiesMXBean.java       | 29 +++++++++
 .../apache/cassandra/auth/CassandraAuthorizer.java | 38 ++++++++----
 .../cassandra/auth/CassandraNetworkAuthorizer.java |  6 +-
 .../cassandra/auth/CassandraRoleManager.java       | 58 +++++++++++-------
 .../cassandra/auth/PasswordAuthenticator.java      |  4 +-
 src/java/org/apache/cassandra/config/Config.java   |  5 ++
 .../cassandra/config/DatabaseDescriptor.java       | 20 +++++++
 .../apache/cassandra/auth/AuthPropertiesTest.java  | 53 ++++++++++++++++
 .../org/apache/cassandra/auth/AuthTestUtils.java   |  2 +-
 .../auth/CassandraNetworkAuthorizerTest.java       |  8 +--
 test/unit/org/apache/cassandra/auth/RolesTest.java | 21 +++++++
 15 files changed, 279 insertions(+), 43 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index ed95f95..b89bb31 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1
+ * Allow configuration of consistency levels on auth operations (CASSANDRA-12988)
  * Add number of sstables in a compaction to compactionstats output (CASSANDRA-16844)
  * Upgrade Caffeine to 2.9.2 (CASSANDRA-15153)
  * Allow DELETE and TRUNCATE to work on Virtual Tables if the implementation allows it (CASSANDRA-16806)
diff --git a/NEWS.txt b/NEWS.txt
index 443a5c4..77a3e1b 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -140,6 +140,9 @@ New features
       See CASSANDRA-10190 for details.
     - Support for server side DESCRIBE statements has been added. See CASSANDRA-14825
     - It is now possible to rate limit snapshot creation/clearing. See CASSANDRA-13019
+    - Authentication reads and writes have been changed from a mix of ONE, LOCAL_ONE, and QUORUM
+      to LOCAL_QUORUM on reads and EACH_QUORUM on writes. This is configurable via cassandra.yaml with
+      auth_read_consistency_level and auth_write_consistency_level respectively. See CASSANDRA-12988.
 
 Upgrading
 ---------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 25f6eb1..a868a4a 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -1426,6 +1426,10 @@ report_unconfirmed_repaired_data_mismatches: false
 # table_count_warn_threshold: 150
 # keyspace_count_warn_threshold: 40
 
+# configure the read and write consistency levels for modifications to auth tables
+# auth_read_consistency_level: LOCAL_QUORUM
+# auth_write_consistency_level: EACH_QUORUM
+
 #########################
 # EXPERIMENTAL FEATURES #
 #########################
diff --git a/src/java/org/apache/cassandra/auth/AuthProperties.java b/src/java/org/apache/cassandra/auth/AuthProperties.java
new file mode 100644
index 0000000..036cbe2
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/AuthProperties.java
@@ -0,0 +1,70 @@
+/*
+ * 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 javax.management.ObjectName;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.utils.MBeanWrapper;
+
+public class AuthProperties implements AuthPropertiesMXBean
+{
+    public static AuthProperties instance = new AuthProperties(DatabaseDescriptor.getAuthWriteConsistencyLevel(),
+                                                               DatabaseDescriptor.getAuthReadConsistencyLevel(),
+                                                               true);
+
+    public AuthProperties(ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel, boolean registerMBean)
+    {
+        setWriteConsistencyLevel(writeConsistencyLevel);
+        setReadConsistencyLevel(readConsistencyLevel);
+
+        if (registerMBean)
+        {
+            try
+            {
+                MBeanWrapper.instance.registerMBean(this, new ObjectName("org.apache.cassandra.auth:type=AuthProperties"));
+            }
+            catch (Exception e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    public void setWriteConsistencyLevel(ConsistencyLevel cl)
+    {
+        DatabaseDescriptor.setAuthWriteConsistencyLevel(cl);
+    }
+
+    public ConsistencyLevel getWriteConsistencyLevel()
+    {
+        return DatabaseDescriptor.getAuthWriteConsistencyLevel();
+    }
+
+    public void setReadConsistencyLevel(ConsistencyLevel cl)
+    {
+        DatabaseDescriptor.setAuthReadConsistencyLevel(cl);
+    }
+
+    public ConsistencyLevel getReadConsistencyLevel()
+    {
+        return DatabaseDescriptor.getAuthReadConsistencyLevel();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/auth/AuthPropertiesMXBean.java b/src/java/org/apache/cassandra/auth/AuthPropertiesMXBean.java
new file mode 100644
index 0000000..f52a380
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/AuthPropertiesMXBean.java
@@ -0,0 +1,29 @@
+/*
+ * 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 org.apache.cassandra.db.ConsistencyLevel;
+
+public interface AuthPropertiesMXBean
+{
+    void setReadConsistencyLevel(ConsistencyLevel cl);
+    ConsistencyLevel getReadConsistencyLevel();
+    void setWriteConsistencyLevel(ConsistencyLevel cl);
+    ConsistencyLevel getWriteConsistencyLevel();
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index 37ad60a..318b7f3 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -109,7 +109,8 @@ public class CassandraAuthorizer implements IAuthorizer
             UntypedResultSet rows = process(String.format("SELECT resource FROM %s.%s WHERE role = '%s'",
                                                           SchemaConstants.AUTH_KEYSPACE_NAME,
                                                           AuthKeyspace.ROLE_PERMISSIONS,
-                                                          escape(revokee.getRoleName())));
+                                                          escape(revokee.getRoleName())),
+                                            authReadConsistencyLevel());
 
             List<CQLStatement> statements = new ArrayList<>();
             for (UntypedResultSet.Row row : rows)
@@ -148,7 +149,8 @@ public class CassandraAuthorizer implements IAuthorizer
             UntypedResultSet rows = process(String.format("SELECT role FROM %s.%s WHERE resource = '%s'",
                                                           SchemaConstants.AUTH_KEYSPACE_NAME,
                                                           AuthKeyspace.RESOURCE_ROLE_INDEX,
-                                                          escape(droppedResource.getName())));
+                                                          escape(droppedResource.getName())),
+                                            authReadConsistencyLevel());
 
             List<CQLStatement> statements = new ArrayList<>();
             for (UntypedResultSet.Row row : rows)
@@ -189,7 +191,7 @@ public class CassandraAuthorizer implements IAuthorizer
     private void addPermissionsForRole(Set<Permission> permissions, IResource resource, RoleResource role)
     throws RequestExecutionException, RequestValidationException
     {
-        QueryOptions options = QueryOptions.forInternalCalls(ConsistencyLevel.LOCAL_ONE,
+        QueryOptions options = QueryOptions.forInternalCalls(authReadConsistencyLevel(),
                                                              Lists.newArrayList(ByteBufferUtil.bytes(role.getRoleName()),
                                                                                 ByteBufferUtil.bytes(resource.getName())));
 
@@ -216,7 +218,8 @@ public class CassandraAuthorizer implements IAuthorizer
                               op,
                               "'" + StringUtils.join(permissions, "','") + "'",
                               escape(role.getRoleName()),
-                              escape(resource.getName())));
+                              escape(resource.getName())),
+                authWriteConsistencyLevel());
     }
 
     // Removes an entry from the inverted index table (from resource -> role with defined permissions)
@@ -226,7 +229,8 @@ public class CassandraAuthorizer implements IAuthorizer
                               SchemaConstants.AUTH_KEYSPACE_NAME,
                               AuthKeyspace.RESOURCE_ROLE_INDEX,
                               escape(resource.getName()),
-                              escape(role.getRoleName())));
+                              escape(role.getRoleName())),
+                authWriteConsistencyLevel());
     }
 
     // Adds an entry to the inverted index table (from resource -> role with defined permissions)
@@ -236,7 +240,8 @@ public class CassandraAuthorizer implements IAuthorizer
                               SchemaConstants.AUTH_KEYSPACE_NAME,
                               AuthKeyspace.RESOURCE_ROLE_INDEX,
                               escape(resource.getName()),
-                              escape(role.getRoleName())));
+                              escape(role.getRoleName())),
+                authWriteConsistencyLevel());
     }
 
     // 'of' can be null - in that case everyone's permissions have been requested. Otherwise only single user's.
@@ -270,7 +275,7 @@ public class CassandraAuthorizer implements IAuthorizer
     throws RequestExecutionException
     {
         Set<PermissionDetails> details = new HashSet<>();
-        for (UntypedResultSet.Row row : process(buildListQuery(resource, role)))
+        for (UntypedResultSet.Row row : process(buildListQuery(resource, role), authReadConsistencyLevel()))
         {
             if (row.has(PERMISSIONS))
             {
@@ -350,16 +355,27 @@ public class CassandraAuthorizer implements IAuthorizer
         return statement.execute(QueryState.forInternalCalls(), options, System.nanoTime());
     }
 
-    UntypedResultSet process(String query) throws RequestExecutionException
+    UntypedResultSet process(String query, ConsistencyLevel cl) throws RequestExecutionException
     {
-        return QueryProcessor.process(query, ConsistencyLevel.LOCAL_ONE);
+        return QueryProcessor.process(query, cl);
     }
 
     void processBatch(BatchStatement statement)
     {
+        QueryOptions options = QueryOptions.forInternalCalls(authWriteConsistencyLevel(), Collections.emptyList());
         QueryProcessor.instance.processBatch(statement,
                                              QueryState.forInternalCalls(),
-                                             BatchQueryOptions.withoutPerStatementVariables(QueryOptions.DEFAULT),
+                                             BatchQueryOptions.withoutPerStatementVariables(options),
                                              System.nanoTime());
     }
-}
+
+    public static ConsistencyLevel authWriteConsistencyLevel()
+    {
+        return AuthProperties.instance.getWriteConsistencyLevel();
+    }
+
+    public static ConsistencyLevel authReadConsistencyLevel()
+    {
+        return AuthProperties.instance.getReadConsistencyLevel();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
index 6fdcd69..107cd85 100644
--- a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java
@@ -23,12 +23,10 @@ import java.util.Set;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 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.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.SchemaConstants;
@@ -58,12 +56,12 @@ public class CassandraNetworkAuthorizer implements INetworkAuthorizer
     @VisibleForTesting
     void process(String query)
     {
-        QueryProcessor.process(query, ConsistencyLevel.LOCAL_ONE);
+        QueryProcessor.process(query, CassandraAuthorizer.authWriteConsistencyLevel());
     }
 
     private Set<String> getAuthorizedDcs(String name)
     {
-        QueryOptions options = QueryOptions.forInternalCalls(ConsistencyLevel.LOCAL_ONE,
+        QueryOptions options = QueryOptions.forInternalCalls(CassandraAuthorizer.authReadConsistencyLevel(),
                                                              Lists.newArrayList(ByteBufferUtil.bytes(name)));
 
         ResultMessage.Rows rows = select(authorizeUserStatement, options);
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index f4c9428..b8421af 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -80,6 +80,13 @@ public class CassandraRoleManager implements IRoleManager
     static final String DEFAULT_SUPERUSER_NAME = "cassandra";
     static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";
 
+    /**
+     * We need to treat the default superuser as a special case since during initial node startup, we may end up with
+     * duplicate creation or deletion + re-creation of this user on different nodes unless we check at quorum to see if
+     * it's already been done.
+     */
+    static final ConsistencyLevel DEFAULT_SUPERUSER_CONSISTENCY_LEVEL = ConsistencyLevel.QUORUM;
+
     // Transform a row in the AuthKeyspace.ROLES to a Role instance
     private static final Function<UntypedResultSet.Row, Role> ROW_TO_ROLE = row ->
     {
@@ -172,7 +179,7 @@ public class CassandraRoleManager implements IRoleManager
                                          escape(role.getRoleName()),
                                          options.getSuperuser().or(false),
                                          options.getLogin().or(false));
-        process(insertCql, consistencyForRole(role.getRoleName()));
+        process(insertCql, consistencyForRoleWrite(role.getRoleName()));
     }
 
     public void dropRole(AuthenticatedUser performer, RoleResource role) throws RequestValidationException, RequestExecutionException
@@ -181,7 +188,7 @@ public class CassandraRoleManager implements IRoleManager
                               SchemaConstants.AUTH_KEYSPACE_NAME,
                               AuthKeyspace.ROLES,
                               escape(role.getRoleName())),
-                consistencyForRole(role.getRoleName()));
+                consistencyForRoleWrite(role.getRoleName()));
         removeAllMembers(role.getRoleName());
     }
 
@@ -197,7 +204,7 @@ public class CassandraRoleManager implements IRoleManager
                                   AuthKeyspace.ROLES,
                                   assignments,
                                   escape(role.getRoleName())),
-                    consistencyForRole(role.getRoleName()));
+                    consistencyForRoleWrite(role.getRoleName()));
         }
     }
 
@@ -219,7 +226,7 @@ public class CassandraRoleManager implements IRoleManager
                               AuthKeyspace.ROLE_MEMBERS,
                               escape(role.getRoleName()),
                               escape(grantee.getRoleName())),
-                consistencyForRole(role.getRoleName()));
+                consistencyForRoleWrite(role.getRoleName()));
     }
 
     public void revokeRole(AuthenticatedUser performer, RoleResource role, RoleResource revokee)
@@ -236,7 +243,7 @@ public class CassandraRoleManager implements IRoleManager
                               AuthKeyspace.ROLE_MEMBERS,
                               escape(role.getRoleName()),
                               escape(revokee.getRoleName())),
-                consistencyForRole(role.getRoleName()));
+                consistencyForRoleWrite(role.getRoleName()));
     }
 
     public Set<RoleResource> getRoles(RoleResource grantee, boolean includeInherited)
@@ -257,6 +264,11 @@ public class CassandraRoleManager implements IRoleManager
                .collect(Collectors.toSet());
     }
 
+    /**
+     * We hard-code this query to Quorum regardless of the role or auth credentials of the queryer given the nature of
+     * this query: we expect to know *all* roles across the entire cluster when we query this, not just local quorum or
+     * on a single node.
+     */
     public Set<RoleResource> getAllRoles() throws RequestValidationException, RequestExecutionException
     {
         ImmutableSet.Builder<RoleResource> builder = ImmutableSet.builder();
@@ -334,7 +346,7 @@ public class CassandraRoleManager implements IRoleManager
                                                      AuthKeyspace.ROLES,
                                                      DEFAULT_SUPERUSER_NAME,
                                                      escape(hashpw(DEFAULT_SUPERUSER_PASSWORD))),
-                                       consistencyForRole(DEFAULT_SUPERUSER_NAME));
+                                       consistencyForRoleWrite(DEFAULT_SUPERUSER_NAME));
                 logger.info("Created default superuser role '{}'", DEFAULT_SUPERUSER_NAME);
             }
         }
@@ -413,7 +425,7 @@ public class CassandraRoleManager implements IRoleManager
      */
     private Role getRole(String name)
     {
-        QueryOptions options = QueryOptions.forInternalCalls(consistencyForRole(name),
+        QueryOptions options = QueryOptions.forInternalCalls(consistencyForRoleRead(name),
                                                              Collections.singletonList(ByteBufferUtil.bytes(name)));
         ResultMessage.Rows rows = select(loadRoleStatement, options);
         if (rows.result.isEmpty())
@@ -435,7 +447,7 @@ public class CassandraRoleManager implements IRoleManager
                               op,
                               escape(role),
                               escape(grantee)),
-                consistencyForRole(grantee));
+                consistencyForRoleWrite(grantee));
     }
 
     /*
@@ -448,7 +460,7 @@ public class CassandraRoleManager implements IRoleManager
                                                       SchemaConstants.AUTH_KEYSPACE_NAME,
                                                       AuthKeyspace.ROLE_MEMBERS,
                                                       escape(role)),
-                                        consistencyForRole(role));
+                                        consistencyForRoleRead(role));
         if (rows.isEmpty())
             return;
 
@@ -461,7 +473,7 @@ public class CassandraRoleManager implements IRoleManager
                               SchemaConstants.AUTH_KEYSPACE_NAME,
                               AuthKeyspace.ROLE_MEMBERS,
                               escape(role)),
-                consistencyForRole(role));
+                consistencyForRoleWrite(role));
     }
 
     /*
@@ -490,14 +502,6 @@ public class CassandraRoleManager implements IRoleManager
                       .collect(Collectors.joining(","));
     }
 
-    protected static ConsistencyLevel consistencyForRole(String role)
-    {
-        if (role.equals(DEFAULT_SUPERUSER_NAME))
-            return ConsistencyLevel.QUORUM;
-        else
-            return ConsistencyLevel.LOCAL_ONE;
-    }
-
     private static String hashpw(String password)
     {
         return BCrypt.hashpw(password, BCrypt.gensalt(GENSALT_LOG2_ROUNDS));
@@ -508,6 +512,21 @@ public class CassandraRoleManager implements IRoleManager
         return StringUtils.replace(name, "'", "''");
     }
 
+    /** Allows selective overriding of the consistency level for specific roles. */
+    protected static ConsistencyLevel consistencyForRoleWrite(String role)
+    {
+        return role.equals(DEFAULT_SUPERUSER_NAME) ?
+               DEFAULT_SUPERUSER_CONSISTENCY_LEVEL :
+               CassandraAuthorizer.authWriteConsistencyLevel();
+    }
+
+    protected static ConsistencyLevel consistencyForRoleRead(String role)
+    {
+        return role.equals(DEFAULT_SUPERUSER_NAME) ?
+               DEFAULT_SUPERUSER_CONSISTENCY_LEVEL :
+               CassandraAuthorizer.authReadConsistencyLevel();
+    }
+
     /**
      * Executes the provided query.
      * This shouldn't be used during setup as this will directly return an error if the manager is not setup yet. Setup tasks
@@ -530,5 +549,4 @@ public class CassandraRoleManager implements IRoleManager
     {
         return statement.execute(QueryState.forInternalCalls(), options, System.nanoTime());
     }
-
-}
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
index 00ccab7..50b86f7 100644
--- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
@@ -44,7 +44,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.mindrot.jbcrypt.BCrypt;
 
-import static org.apache.cassandra.auth.CassandraRoleManager.consistencyForRole;
+import static org.apache.cassandra.auth.CassandraRoleManager.consistencyForRoleRead;
 
 /**
  * PasswordAuthenticator is an IAuthenticator implementation
@@ -104,7 +104,7 @@ public class PasswordAuthenticator implements IAuthenticator
     {
         try
         {
-            QueryOptions options = QueryOptions.forInternalCalls(consistencyForRole(username),
+            QueryOptions options = QueryOptions.forInternalCalls(consistencyForRoleRead(username),
                     Lists.newArrayList(ByteBufferUtil.bytes(username)));
 
             ResultMessage.Rows rows = select(authenticateStatement, options);
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 77607ed..4d4cc11 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -515,6 +515,11 @@ public class Config
      */
     public volatile int validation_preview_purge_head_start_in_sec = 60 * 60;
 
+    // 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";
+    public volatile String auth_write_consistency_level = "EACH_QUORUM";
+
     /**
      * The intial capacity for creating RangeTombstoneList.
      */
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 7a0eeb0..511ef3f 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -3437,6 +3437,26 @@ public class DatabaseDescriptor
         conf.keyspace_count_warn_threshold = value;
     }
 
+    public static ConsistencyLevel getAuthWriteConsistencyLevel()
+    {
+        return ConsistencyLevel.valueOf(conf.auth_write_consistency_level);
+    }
+
+    public static ConsistencyLevel getAuthReadConsistencyLevel()
+    {
+        return ConsistencyLevel.valueOf(conf.auth_read_consistency_level);
+    }
+
+    public static void setAuthWriteConsistencyLevel(ConsistencyLevel cl)
+    {
+        conf.auth_write_consistency_level = cl.toString();
+    }
+
+    public static void setAuthReadConsistencyLevel(ConsistencyLevel cl)
+    {
+        conf.auth_read_consistency_level = cl.toString();
+    }
+
     public static int getConsecutiveMessageErrorsThreshold()
     {
         return conf.consecutive_message_errors_threshold;
diff --git a/test/unit/org/apache/cassandra/auth/AuthPropertiesTest.java b/test/unit/org/apache/cassandra/auth/AuthPropertiesTest.java
new file mode 100644
index 0000000..9a01449
--- /dev/null
+++ b/test/unit/org/apache/cassandra/auth/AuthPropertiesTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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 org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ConsistencyLevel;
+
+public class AuthPropertiesTest
+{
+    @BeforeClass
+    public static void beforeClass()
+    {
+        if (!DatabaseDescriptor.isDaemonInitialized())
+        {
+            DatabaseDescriptor.daemonInitialization();
+        }
+    }
+
+    @Test
+    public void setReadCL_LegitValue()
+    {
+        AuthProperties authProperties = new AuthProperties(ConsistencyLevel.ANY, ConsistencyLevel.THREE, false);
+        authProperties.setReadConsistencyLevel(ConsistencyLevel.LOCAL_QUORUM);
+        Assert.assertEquals(ConsistencyLevel.LOCAL_QUORUM, AuthProperties.instance.getReadConsistencyLevel());
+    }
+
+    @Test (expected = IllegalArgumentException.class)
+    public void setReadCL_BadValue()
+    {
+        AuthProperties authProperties = new AuthProperties(ConsistencyLevel.ANY, ConsistencyLevel.THREE, false);
+        authProperties.setReadConsistencyLevel(ConsistencyLevel.valueOf("ILLEGAL_CL"));
+    }
+}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/auth/AuthTestUtils.java b/test/unit/org/apache/cassandra/auth/AuthTestUtils.java
index a012b62..97c0b09 100644
--- a/test/unit/org/apache/cassandra/auth/AuthTestUtils.java
+++ b/test/unit/org/apache/cassandra/auth/AuthTestUtils.java
@@ -80,7 +80,7 @@ public class AuthTestUtils
             return statement.executeLocally(QueryState.forInternalCalls(), options);
         }
 
-        UntypedResultSet process(String query) throws RequestExecutionException
+        UntypedResultSet process(String query, ConsistencyLevel cl) throws RequestExecutionException
         {
             return QueryProcessor.executeInternal(query);
         }
diff --git a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
index 3dc9d91..f14b253 100644
--- a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
+++ b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java
@@ -43,15 +43,13 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.ClientState;
 
 import static org.apache.cassandra.auth.AuthKeyspace.NETWORK_PERMISSIONS;
-import static org.apache.cassandra.auth.AuthTestUtils.LocalCassandraAuthorizer;
-import static org.apache.cassandra.auth.AuthTestUtils.LocalCassandraNetworkAuthorizer;
 import static org.apache.cassandra.auth.AuthTestUtils.LocalCassandraRoleManager;
 import static org.apache.cassandra.auth.AuthTestUtils.getRolesReadCount;
 import static org.apache.cassandra.schema.SchemaConstants.AUTH_KEYSPACE_NAME;
 
 public class CassandraNetworkAuthorizerTest
 {
-    private static void setupSuperUser()
+    public static void setupSuperUser()
     {
         QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (role, is_superuser, can_login, salted_hash) "
                                                      + "VALUES ('%s', true, true, '%s')",
@@ -67,8 +65,8 @@ public class CassandraNetworkAuthorizerTest
         SchemaLoader.prepareServer();
         SchemaLoader.setupAuth(new LocalCassandraRoleManager(),
                                new PasswordAuthenticator(),
-                               new LocalCassandraAuthorizer(),
-                               new LocalCassandraNetworkAuthorizer());
+                               new AuthTestUtils.LocalCassandraAuthorizer(),
+                               new AuthTestUtils.LocalCassandraNetworkAuthorizer());
         setupSuperUser();
         // not strictly necessary to init the cache here, but better to be explicit
         Roles.initRolesCache(DatabaseDescriptor.getRoleManager(), () -> true);
diff --git a/test/unit/org/apache/cassandra/auth/RolesTest.java b/test/unit/org/apache/cassandra/auth/RolesTest.java
index 0211d46..40b38c9 100644
--- a/test/unit/org/apache/cassandra/auth/RolesTest.java
+++ b/test/unit/org/apache/cassandra/auth/RolesTest.java
@@ -21,10 +21,13 @@ package org.apache.cassandra.auth;
 import java.util.Set;
 
 import com.google.common.collect.Iterables;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.TableMetadata;
@@ -92,4 +95,22 @@ public class RolesTest
         assertEquals(granted, Roles.getRoles(ROLE_A));
         assertEquals(count, getRolesReadCount());
     }
+
+    @Test
+    public void confirmSuperUserConsistency()
+    {
+        // Confirm special treatment of superuser
+        ConsistencyLevel readLevel = CassandraRoleManager.consistencyForRoleRead(CassandraRoleManager.DEFAULT_SUPERUSER_NAME);
+        Assert.assertEquals(CassandraRoleManager.DEFAULT_SUPERUSER_CONSISTENCY_LEVEL, readLevel);
+
+        ConsistencyLevel writeLevel = CassandraRoleManager.consistencyForRoleWrite(CassandraRoleManager.DEFAULT_SUPERUSER_NAME);
+        Assert.assertEquals(CassandraRoleManager.DEFAULT_SUPERUSER_CONSISTENCY_LEVEL, writeLevel);
+
+        // Confirm standard config-based treatment of non
+        ConsistencyLevel nonPrivReadLevel = CassandraRoleManager.consistencyForRoleRead("non-privilaged");
+        Assert.assertEquals(nonPrivReadLevel, DatabaseDescriptor.getAuthReadConsistencyLevel());
+
+        ConsistencyLevel nonPrivWriteLevel = CassandraRoleManager.consistencyForRoleWrite("non-privilaged");
+        Assert.assertEquals(nonPrivWriteLevel, DatabaseDescriptor.getAuthWriteConsistencyLevel());
+    }
 }

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