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

[cassandra] branch trunk updated: Allow to grant permission for all tables in a keyspace

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

blerer 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 df6ecaa  Allow to grant permission for all tables in a keyspace
df6ecaa is described below

commit df6ecaae1c3ac3a05c8cd0c6a9c3da58d4e86a98
Author: Benjamin Lerer <b....@gmail.com>
AuthorDate: Fri Oct 8 17:32:15 2021 +0200

    Allow to grant permission for all tables in a keyspace
    
    Patch by Benjamin Lerer; Review by Andres de la Peña and Ekaterina Dimitrova for CASSANDRA-17027
    
    In some cases it is useful to prevent users to alter or drop a keyspace
    while allowing them to create new tables.
    This patch add support for a new DataResource below KEYSPACE but above
    TABLE. The syntax to grant permission at this level in ALL TABLES IN
    KEYSPACE.
---
 CHANGES.txt                                        |   1 +
 NEWS.txt                                           |   2 +
 pylib/cqlshlib/cql3handling.py                     |   1 +
 pylib/cqlshlib/test/test_cqlsh_completion.py       |  46 ++++
 src/antlr/Parser.g                                 |   4 +-
 .../cassandra/auth/AuthSchemaChangeListener.java   |   1 +
 .../cassandra/auth/CassandraRoleManager.java       |  33 ++-
 .../org/apache/cassandra/auth/DataResource.java    |  42 ++-
 .../cql3/statements/CreateRoleStatement.java       |   1 +
 .../cql3/statements/schema/AlterTypeStatement.java |   2 +-
 .../statements/schema/CreateTableStatement.java    |   2 +-
 .../statements/schema/CreateTypeStatement.java     |   2 +-
 .../cql3/statements/schema/DropTypeStatement.java  |   2 +-
 .../org/apache/cassandra/service/ClientState.java  |   5 +
 .../apache/cassandra/auth/GrantAndRevokeTest.java  | 302 +++++++++++++++++++++
 test/unit/org/apache/cassandra/cql3/CQLTester.java | 151 ++++++++++-
 16 files changed, 569 insertions(+), 28 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 21118eb..0bf931d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1
+ * Allow to grant permission for all tables in a keyspace (CASSANDRA-17027)
  * Log time spent writing keys during compaction (CASSANDRA-17037)
  * Make nodetool compactionstats and sstable_tasks consistent (CASSANDRA-16976)
  * Add metrics and logging around index summary redistribution (CASSANDRA-17036)
diff --git a/NEWS.txt b/NEWS.txt
index 0bec628..162241c 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -38,6 +38,8 @@ using the provided 'sstableupgrade' tool.
 
 New features
 ------------
+    - A new ALL TABLES IN KEYSPACE resource has been added. It allows to grant permissions for all tables and user types
+      in a keyspace while preventing the user to use those permissions on the keyspace itself.
     - Added support for type casting in the WHERE clause components and in the values of INSERT and UPDATE statements.
     - Warn/abort thresholds added to read queries notifying clients when these thresholds trigger (by
       emitting a client warning or aborting the query).  This feature is disabled by default, scheduled
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 68484f5..a99e779 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -1524,6 +1524,7 @@ syntax_rules += r'''
 
 <dataResource> ::= ( "ALL" "KEYSPACES" )
                  | ( "KEYSPACE" <keyspaceName> )
+                 | ( "ALL" "TABLES" "IN" "KEYSPACE" <keyspaceName> )
                  | ( "TABLE"? <columnFamilyName> )
                  ;
 
diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py
index c898cbe..bc82033 100644
--- a/pylib/cqlshlib/test/test_cqlsh_completion.py
+++ b/pylib/cqlshlib/test/test_cqlsh_completion.py
@@ -826,3 +826,49 @@ class TestCqlshCompletion(CqlshCompletionCase):
         self.trycompletions('ALTER KEYSPACE system_trac', "es WITH replication = {'class': '")
         self.trycompletions("ALTER KEYSPACE system_traces WITH replication = {'class': '", '',
                             choices=['NetworkTopologyStrategy', 'SimpleStrategy'])
+
+    def test_complete_in_grant(self):
+        self.trycompletions("GR",
+                            immediate='ANT ')
+        self.trycompletions("GRANT ",
+                            choices=['ALL', 'ALTER', 'AUTHORIZE', 'CREATE', 'DESCRIBE', 'DROP', 'EXECUTE', 'MODIFY', 'SELECT'],
+                            other_choices_ok=True)
+        self.trycompletions("GRANT MODIFY ",
+                            choices=['ON', 'PERMISSION'])
+        self.trycompletions("GRANT MODIFY P",
+                            immediate='ERMISSION ON ')
+        self.trycompletions("GRANT MODIFY PERMISSION O",
+                            immediate='N ')
+        self.trycompletions("GRANT MODIFY ON ",
+                            choices=['ALL', 'KEYSPACE', 'MBEANS', 'ROLE', 'FUNCTION', 'MBEAN', 'TABLE'],
+                            other_choices_ok=True)
+        self.trycompletions("GRANT MODIFY ON ALL ",
+                            choices=['KEYSPACES', 'TABLES'],
+                            other_choices_ok=True)
+        self.trycompletions("GRANT MODIFY PERMISSION ON KEY",
+                            immediate='SPACE ')
+        self.trycompletions("GRANT MODIFY PERMISSION ON KEYSPACE system_tr",
+                            immediate='aces TO ')
+
+    def test_complete_in_revoke(self):
+        self.trycompletions("RE",
+                            immediate='VOKE ')
+        self.trycompletions("REVOKE ",
+                            choices=['ALL', 'ALTER', 'AUTHORIZE', 'CREATE', 'DESCRIBE', 'DROP', 'EXECUTE', 'MODIFY', 'SELECT'],
+                            other_choices_ok=True)
+        self.trycompletions("REVOKE MODIFY ",
+                            choices=['ON', 'PERMISSION'])
+        self.trycompletions("REVOKE MODIFY P",
+                            immediate='ERMISSION ON ')
+        self.trycompletions("REVOKE MODIFY PERMISSION O",
+                            immediate='N ')
+        self.trycompletions("REVOKE MODIFY PERMISSION ON ",
+                            choices=['ALL', 'KEYSPACE', 'MBEANS', 'ROLE', 'FUNCTION', 'MBEAN', 'TABLE'],
+                            other_choices_ok=True)
+        self.trycompletions("REVOKE MODIFY ON ALL ",
+                            choices=['KEYSPACES', 'TABLES'],
+                            other_choices_ok=True)
+        self.trycompletions("REVOKE MODIFY PERMISSION ON KEY",
+                            immediate='SPACE ')
+        self.trycompletions("REVOKE MODIFY PERMISSION ON KEYSPACE system_tr",
+                            immediate='aces FROM ')
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index 39b3d9f..caeedde 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -1118,8 +1118,8 @@ resource returns [IResource res]
 dataResource returns [DataResource res]
     : K_ALL K_KEYSPACES { $res = DataResource.root(); }
     | K_KEYSPACE ks = keyspaceName { $res = DataResource.keyspace($ks.id); }
-    | ( K_COLUMNFAMILY )? cf = columnFamilyName
-      { $res = DataResource.table($cf.name.getKeyspace(), $cf.name.getName()); }
+    | ( K_COLUMNFAMILY )? cf = columnFamilyName { $res = DataResource.table($cf.name.getKeyspace(), $cf.name.getName()); }
+    | K_ALL K_TABLES K_IN K_KEYSPACE ks = keyspaceName { $res = DataResource.allTables($ks.id); }
     ;
 
 jmxResource returns [JMXResource res]
diff --git a/src/java/org/apache/cassandra/auth/AuthSchemaChangeListener.java b/src/java/org/apache/cassandra/auth/AuthSchemaChangeListener.java
index 6c21d7b..217eb1c 100644
--- a/src/java/org/apache/cassandra/auth/AuthSchemaChangeListener.java
+++ b/src/java/org/apache/cassandra/auth/AuthSchemaChangeListener.java
@@ -31,6 +31,7 @@ public class AuthSchemaChangeListener extends SchemaChangeListener
     public void onDropKeyspace(String ksName)
     {
         DatabaseDescriptor.getAuthorizer().revokeAllOn(DataResource.keyspace(ksName));
+        DatabaseDescriptor.getAuthorizer().revokeAllOn(DataResource.allTables(ksName));
         DatabaseDescriptor.getAuthorizer().revokeAllOn(FunctionResource.keyspace(ksName));
     }
 
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index a79fb12..0e49056 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -137,25 +137,32 @@ public class CassandraRoleManager implements IRoleManager
 
     public CassandraRoleManager()
     {
-        supportedOptions = DatabaseDescriptor.getAuthenticator().getClass() == PasswordAuthenticator.class
+        supportedOptions = DatabaseDescriptor.getAuthenticator() instanceof PasswordAuthenticator
                          ? ImmutableSet.of(Option.LOGIN, Option.SUPERUSER, Option.PASSWORD)
                          : ImmutableSet.of(Option.LOGIN, Option.SUPERUSER);
-        alterableOptions = DatabaseDescriptor.getAuthenticator().getClass().equals(PasswordAuthenticator.class)
+        alterableOptions = DatabaseDescriptor.getAuthenticator() instanceof PasswordAuthenticator
                          ? ImmutableSet.of(Option.PASSWORD)
                          : ImmutableSet.<Option>of();
     }
 
+    @Override
     public void setup()
     {
-        loadRoleStatement = (SelectStatement) prepare("SELECT * from %s.%s WHERE role = ?",
-                                                      SchemaConstants.AUTH_KEYSPACE_NAME,
-                                                      AuthKeyspace.ROLES);
+        loadRoleStatement();
+
         scheduleSetupTask(() -> {
             setupDefaultRole();
             return null;
         });
     }
 
+    protected final void loadRoleStatement()
+    {
+        loadRoleStatement = (SelectStatement) prepare("SELECT * from %s.%s WHERE role = ?",
+                                                      SchemaConstants.AUTH_KEYSPACE_NAME,
+                                                      AuthKeyspace.ROLES);
+    }
+
     public Set<Option> supportedOptions()
     {
         return supportedOptions;
@@ -344,12 +351,7 @@ public class CassandraRoleManager implements IRoleManager
         {
             if (!hasExistingRoles())
             {
-                QueryProcessor.process(String.format("INSERT INTO %s.%s (role, is_superuser, can_login, salted_hash) " +
-                                                     "VALUES ('%s', true, true, '%s')",
-                                                     SchemaConstants.AUTH_KEYSPACE_NAME,
-                                                     AuthKeyspace.ROLES,
-                                                     DEFAULT_SUPERUSER_NAME,
-                                                     escape(hashpw(DEFAULT_SUPERUSER_PASSWORD))),
+                QueryProcessor.process(createDefaultRoleQuery(),
                                        consistencyForRoleWrite(DEFAULT_SUPERUSER_NAME));
                 logger.info("Created default superuser role '{}'", DEFAULT_SUPERUSER_NAME);
             }
@@ -361,6 +363,15 @@ public class CassandraRoleManager implements IRoleManager
         }
     }
 
+    protected static String createDefaultRoleQuery()
+    {
+        return String.format("INSERT INTO %s.%s (role, is_superuser, can_login, salted_hash) VALUES ('%s', true, true, '%s')",
+                             SchemaConstants.AUTH_KEYSPACE_NAME,
+                             AuthKeyspace.ROLES,
+                             DEFAULT_SUPERUSER_NAME,
+                             escape(hashpw(DEFAULT_SUPERUSER_PASSWORD)));
+    }
+
     private static boolean hasExistingRoles() throws RequestExecutionException
     {
         // Try looking up the 'cassandra' default role first, to avoid the range query if possible.
diff --git a/src/java/org/apache/cassandra/auth/DataResource.java b/src/java/org/apache/cassandra/auth/DataResource.java
index c3f5b32..2421930 100644
--- a/src/java/org/apache/cassandra/auth/DataResource.java
+++ b/src/java/org/apache/cassandra/auth/DataResource.java
@@ -31,13 +31,14 @@ import org.apache.cassandra.schema.Schema;
  * Used to represent a table or a keyspace or the root level "data" resource.
  * "data"                                 - the root level data resource.
  * "data/keyspace_name"                   - keyspace-level data resource.
+ * "data/keyspace_name/*"                 - all tables-level data resource.
  * "data/keyspace_name/table_name"        - table-level data resource.
  */
 public class DataResource implements IResource
 {
     enum Level
     {
-        ROOT, KEYSPACE, TABLE
+        ROOT, KEYSPACE, ALL_TABLES, TABLE
     }
 
     // permissions which may be granted on tables
@@ -46,6 +47,15 @@ public class DataResource implements IResource
                                                                                          Permission.SELECT,
                                                                                          Permission.MODIFY,
                                                                                          Permission.AUTHORIZE);
+
+    // permissions which may be granted on all tables of a given keyspace
+    private static final Set<Permission> ALL_TABLES_LEVEL_PERMISSIONS = Sets.immutableEnumSet(Permission.CREATE,
+                                                                                              Permission.ALTER,
+                                                                                              Permission.DROP,
+                                                                                              Permission.SELECT,
+                                                                                              Permission.MODIFY,
+                                                                                              Permission.AUTHORIZE);
+
     // permissions which may be granted on one or all keyspaces
     private static final Set<Permission> KEYSPACE_LEVEL_PERMISSIONS = Sets.immutableEnumSet(Permission.CREATE,
                                                                                             Permission.ALTER,
@@ -92,6 +102,17 @@ public class DataResource implements IResource
     }
 
     /**
+     * Creates a DataResource representing all tables of a keyspace.
+     *
+     * @param keyspace Name of the keyspace.
+     * @return DataResource instance representing the keyspace.
+     */
+    public static DataResource allTables(String keyspace)
+    {
+        return new DataResource(Level.ALL_TABLES, keyspace, null);
+    }
+
+    /**
      * Creates a DataResource instance representing a table.
      *
      * @param keyspace Name of the keyspace.
@@ -122,6 +143,9 @@ public class DataResource implements IResource
         if (parts.length == 2)
             return keyspace(parts[1]);
 
+        if ("*".equals(parts[2]))
+            return allTables(parts[1]);
+
         return table(parts[1], parts[2]);
     }
 
@@ -136,6 +160,8 @@ public class DataResource implements IResource
                 return ROOT_NAME;
             case KEYSPACE:
                 return String.format("%s/%s", ROOT_NAME, keyspace);
+            case ALL_TABLES:
+                return String.format("%s/%s/*", ROOT_NAME, keyspace);
             case TABLE:
                 return String.format("%s/%s/%s", ROOT_NAME, keyspace, table);
         }
@@ -151,8 +177,10 @@ public class DataResource implements IResource
         {
             case KEYSPACE:
                 return root();
-            case TABLE:
+            case ALL_TABLES:
                 return keyspace(keyspace);
+            case TABLE:
+                return allTables(keyspace);
         }
         throw new IllegalStateException("Root-level resource can't have a parent");
     }
@@ -167,6 +195,11 @@ public class DataResource implements IResource
         return level == Level.KEYSPACE;
     }
 
+    public boolean isAllTablesLevel()
+    {
+        return level == Level.ALL_TABLES;
+    }
+
     public boolean isTableLevel()
     {
         return level == Level.TABLE;
@@ -209,6 +242,7 @@ public class DataResource implements IResource
             case ROOT:
                 return true;
             case KEYSPACE:
+            case ALL_TABLES:
                 return Schema.instance.getKeyspaces().contains(keyspace);
             case TABLE:
                 return Schema.instance.getTableMetadata(keyspace, table) != null;
@@ -223,6 +257,8 @@ public class DataResource implements IResource
             case ROOT:
             case KEYSPACE:
                 return KEYSPACE_LEVEL_PERMISSIONS;
+            case ALL_TABLES:
+                return ALL_TABLES_LEVEL_PERMISSIONS;
             case TABLE:
                 return TABLE_LEVEL_PERMISSIONS;
         }
@@ -238,6 +274,8 @@ public class DataResource implements IResource
                 return "<all keyspaces>";
             case KEYSPACE:
                 return String.format("<keyspace %s>", keyspace);
+            case ALL_TABLES:
+                return String.format("<all tables in %s>", keyspace);
             case TABLE:
                 return String.format("<table %s.%s>", keyspace, table);
         }
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
index 574d661..1d49114 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
@@ -84,6 +84,7 @@ public class CreateRoleStatement extends AuthenticationStatement
             DatabaseDescriptor.getNetworkAuthorizer().setRoleDatacenters(role, dcPermissions);
         }
         grantPermissionsToCreator(state);
+
         return null;
     }
 
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java
index a9887c4..5099d33 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTypeStatement.java
@@ -58,7 +58,7 @@ public abstract class AlterTypeStatement extends AlterSchemaStatement
 
     public void authorize(ClientState client)
     {
-        client.ensureKeyspacePermission(keyspaceName, Permission.ALTER);
+        client.ensureAllTablesPermission(keyspaceName, Permission.ALTER);
     }
 
     SchemaChange schemaChangeEvent(Keyspaces.KeyspacesDiff diff)
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
index 1339ba3..b2a8e96 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTableStatement.java
@@ -125,7 +125,7 @@ public final class CreateTableStatement extends AlterSchemaStatement
 
     public void authorize(ClientState client)
     {
-        client.ensureKeyspacePermission(keyspaceName, Permission.CREATE);
+        client.ensureAllTablesPermission(keyspaceName, Permission.CREATE);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java
index 7c1717e..3d506cc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/CreateTypeStatement.java
@@ -106,7 +106,7 @@ public final class CreateTypeStatement extends AlterSchemaStatement
 
     public void authorize(ClientState client)
     {
-        client.ensureKeyspacePermission(keyspaceName, Permission.CREATE);
+        client.ensureAllTablesPermission(keyspaceName, Permission.CREATE);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/DropTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/DropTypeStatement.java
index 6cda7ba..d188bdb 100644
--- a/src/java/org/apache/cassandra/cql3/statements/schema/DropTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/schema/DropTypeStatement.java
@@ -120,7 +120,7 @@ public final class DropTypeStatement extends AlterSchemaStatement
 
     public void authorize(ClientState client)
     {
-        client.ensureKeyspacePermission(keyspaceName, Permission.DROP);
+        client.ensureAllTablesPermission(keyspaceName, Permission.DROP);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index 4333473..69ee28f 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -366,6 +366,11 @@ public class ClientState
         ensurePermission(keyspace, perm, DataResource.keyspace(keyspace));
     }
 
+    public void ensureAllTablesPermission(String keyspace, Permission perm)
+    {
+        ensurePermission(keyspace, perm, DataResource.allTables(keyspace));
+    }
+
     public void ensureTablePermission(String keyspace, String table, Permission perm)
     {
         ensurePermission(keyspace, perm, DataResource.table(keyspace, table));
diff --git a/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java
new file mode 100644
index 0000000..181a039
--- /dev/null
+++ b/test/unit/org/apache/cassandra/auth/GrantAndRevokeTest.java
@@ -0,0 +1,302 @@
+/*
+ * 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.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+
+public class GrantAndRevokeTest extends CQLTester
+{
+    private static final String user = "user";
+    private static final String pass = "12345";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        DatabaseDescriptor.setPermissionsValidity(0);
+        CQLTester.setUpClass();
+        requireAuthentication();
+        requireNetwork();
+    }
+
+    @After
+    public void tearDown() throws Throwable
+    {
+        useSuperUser();
+        executeNet("DROP ROLE " + user);
+    }
+
+    @Test
+    public void testGrantedKeyspace() throws Throwable
+    {
+        useSuperUser();
+
+        executeNet(String.format("CREATE ROLE %s WITH LOGIN = TRUE AND password='%s'", user, pass));
+        executeNet("GRANT CREATE ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+        String table = KEYSPACE_PER_TEST + '.' + createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))");
+        String index = KEYSPACE_PER_TEST + '.' + createIndex(KEYSPACE_PER_TEST, "CREATE INDEX ON %s (val_2)");
+        String type = KEYSPACE_PER_TEST + '.' + createType(KEYSPACE_PER_TEST, "CREATE TYPE %s (a int, b text)");
+        String mv = KEYSPACE_PER_TEST + ".ks_mv_01";
+        executeNet("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)");
+
+        useUser(user, pass);
+
+        // ALTER and DROP tables created by somebody else
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')"));
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1"));
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2"));
+        assertUnauthorizedQuery("User user has no SELECT permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1"));
+        assertUnauthorizedQuery("User user has no SELECT permission on <table " + table + "> or any of its parents",
+                                "SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1 AND ck = 1");
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s"));
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int"));
+        assertUnauthorizedQuery("User user has no DROP permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s"));
+        assertUnauthorizedQuery("User user has no ALTER permission on <all tables in " + KEYSPACE_PER_TEST + "> or any of its parents",
+                                "ALTER TYPE " + type + " ADD c bigint");
+        assertUnauthorizedQuery("User user has no DROP permission on <all tables in " + KEYSPACE_PER_TEST + "> or any of its parents",
+                                "DROP TYPE " + type);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                "DROP MATERIALIZED VIEW " + mv);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                "DROP INDEX " + index);
+
+
+        useSuperUser();
+
+        executeNet("GRANT ALTER ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+        executeNet("GRANT DROP ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+        executeNet("GRANT SELECT ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+        executeNet("GRANT MODIFY ON KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+
+        useUser(user, pass);
+
+        executeNet("ALTER KEYSPACE " + KEYSPACE_PER_TEST + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
+
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2"));
+        assertRowsNet(executeNet(formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")), row(1, 1, 1, "1"));
+        assertRowsNet(executeNet("SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1"), row(1, 1, 1, "1"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int"));
+        executeNet("DROP MATERIALIZED VIEW " + mv);
+        executeNet("DROP INDEX " + index);
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s"));
+        executeNet("ALTER TYPE " + type + " ADD c bigint");
+        executeNet("DROP TYPE " + type);
+
+        // calling creatTableName to create a new table name that will be used by the formatQuery
+        table = createTableName();
+        type = KEYSPACE_PER_TEST + "." + createTypeName();
+        mv = KEYSPACE_PER_TEST + ".ks_mv_02";
+        executeNet("CREATE TYPE " + type + " (a int, b text)");
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))"));
+        executeNet("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)");
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2"));
+        assertRowsNet(executeNet(formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")), row(1, 1, 1, "1"));
+        assertRowsNet(executeNet("SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1"), row(1, 1, 1, "1"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int"));
+        executeNet("DROP MATERIALIZED VIEW " + mv);
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s"));
+        executeNet("ALTER TYPE " + type + " ADD c bigint");
+        executeNet("DROP TYPE " + type);
+
+        useSuperUser();
+
+        executeNet("REVOKE ALTER ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user);
+        executeNet("REVOKE DROP ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user);
+        executeNet("REVOKE SELECT ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user);
+        executeNet("REVOKE MODIFY ON KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user);
+
+        table = KEYSPACE_PER_TEST + "." + createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))");
+        type = KEYSPACE_PER_TEST + "." + createType(KEYSPACE_PER_TEST, "CREATE TYPE %s (a int, b text)");
+        index = KEYSPACE_PER_TEST + '.' + createIndex(KEYSPACE_PER_TEST, "CREATE INDEX ON %s (val_2)");
+        mv = KEYSPACE_PER_TEST + ".ks_mv_03";
+        executeNet("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)");
+
+        useUser(user, pass);
+
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                "INSERT INTO " + table + " (pk, ck, val, val_2) VALUES (1, 1, 1, '1')");
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                "UPDATE " + table + " SET val = 1 WHERE pk = 1 AND ck = 1");
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                "DELETE FROM " + table + " WHERE pk = 1 AND ck = 2");
+        assertUnauthorizedQuery("User user has no SELECT permission on <table " + table + "> or any of its parents",
+                                "SELECT * FROM " + table + " WHERE pk = 1 AND ck = 1");
+        assertUnauthorizedQuery("User user has no SELECT permission on <table " + table + "> or any of its parents",
+                                "SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1 AND ck = 1");
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                "TRUNCATE TABLE " + table);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE " + table + " ADD val_3 int"));
+        assertUnauthorizedQuery("User user has no DROP permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "DROP TABLE " + table));
+        assertUnauthorizedQuery("User user has no ALTER permission on <all tables in " + KEYSPACE_PER_TEST + "> or any of its parents",
+                                "ALTER TYPE " + type + " ADD c bigint");
+        assertUnauthorizedQuery("User user has no DROP permission on <all tables in " + KEYSPACE_PER_TEST + "> or any of its parents",
+                                "DROP TYPE " + type);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                "DROP MATERIALIZED VIEW " + mv);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                "DROP INDEX " + index);
+
+    }
+
+    @Test
+    public void testGrantedAllTables() throws Throwable
+    {
+        useSuperUser();
+
+        executeNet(String.format("CREATE ROLE %s WITH LOGIN = TRUE AND password='%s'", user, pass));
+        executeNet("GRANT CREATE ON ALL TABLES IN KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+        String table = KEYSPACE_PER_TEST + "." + createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))");
+        String index = KEYSPACE_PER_TEST + '.' + createIndex(KEYSPACE_PER_TEST, "CREATE INDEX ON %s (val_2)");
+        String type = KEYSPACE_PER_TEST + "." + createType(KEYSPACE_PER_TEST, "CREATE TYPE %s (a int, b text)");
+        String mv = KEYSPACE_PER_TEST + ".alltables_mv_01";
+        executeNet("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)");
+
+        useUser(user, pass);
+
+        // ALTER and DROP tables created by somebody else
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')"));
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1"));
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2"));
+        assertUnauthorizedQuery("User user has no SELECT permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1"));
+        assertUnauthorizedQuery("User user has no SELECT permission on <table " + table + "> or any of its parents",
+                                "SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1 AND ck = 1");
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s"));
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int"));
+        assertUnauthorizedQuery("User user has no DROP permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s"));
+        assertUnauthorizedQuery("User user has no ALTER permission on <all tables in " + KEYSPACE_PER_TEST + "> or any of its parents",
+                                "ALTER TYPE " + type + " ADD c bigint");
+        assertUnauthorizedQuery("User user has no DROP permission on <all tables in " + KEYSPACE_PER_TEST + "> or any of its parents",
+                                "DROP TYPE " + type);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                "DROP MATERIALIZED VIEW " + mv);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                "DROP INDEX " + index);
+
+        useSuperUser();
+
+        executeNet("GRANT ALTER ON ALL TABLES IN KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+        executeNet("GRANT DROP ON ALL TABLES IN KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+        executeNet("GRANT SELECT ON ALL TABLES IN KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+        executeNet("GRANT MODIFY ON ALL TABLES IN KEYSPACE " + KEYSPACE_PER_TEST + " TO " + user);
+
+        useUser(user, pass);
+
+        assertUnauthorizedQuery("User user has no ALTER permission on <keyspace " + KEYSPACE_PER_TEST + "> or any of its parents",
+                                "ALTER KEYSPACE " + KEYSPACE_PER_TEST + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
+
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2"));
+        assertRowsNet(executeNet(formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")), row(1, 1, 1, "1"));
+        assertRowsNet(executeNet("SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1"), row(1, 1, 1, "1"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int"));
+        executeNet("DROP MATERIALIZED VIEW " + mv);
+        executeNet("DROP INDEX " + index);
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s"));
+        executeNet("ALTER TYPE " + type + " ADD c bigint");
+        executeNet("DROP TYPE " + type);
+
+        // calling creatTableName to create a new table name that will be used by the formatQuery
+        table = createTableName();
+        type = KEYSPACE_PER_TEST + "." + createTypeName();
+        mv = KEYSPACE_PER_TEST + ".alltables_mv_02";
+        executeNet("CREATE TYPE " + type + " (a int, b text)");
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))"));
+        index = KEYSPACE_PER_TEST + '.' + createIndex(KEYSPACE_PER_TEST, "CREATE INDEX ON %s (val_2)");
+        executeNet("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)");
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "INSERT INTO %s (pk, ck, val, val_2) VALUES (1, 1, 1, '1')"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "UPDATE %s SET val = 1 WHERE pk = 1 AND ck = 1"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "DELETE FROM %s WHERE pk = 1 AND ck = 2"));
+        assertRowsNet(executeNet(formatQuery(KEYSPACE_PER_TEST, "SELECT * FROM %s WHERE pk = 1 AND ck = 1")), row(1, 1, 1, "1"));
+        assertRowsNet(executeNet("SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1"), row(1, 1, 1, "1"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "TRUNCATE TABLE %s"));
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE %s ADD val_3 int"));
+        executeNet("DROP MATERIALIZED VIEW " + mv);
+        executeNet("DROP INDEX " + index);
+        executeNet(formatQuery(KEYSPACE_PER_TEST, "DROP TABLE %s"));
+        executeNet("ALTER TYPE " + type + " ADD c bigint");
+        executeNet("DROP TYPE " + type);
+
+        useSuperUser();
+
+        executeNet("REVOKE ALTER ON ALL TABLES IN KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user);
+        executeNet("REVOKE DROP ON ALL TABLES IN KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user);
+        executeNet("REVOKE SELECT ON ALL TABLES IN KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user);
+        executeNet("REVOKE MODIFY ON ALL TABLES IN KEYSPACE " + KEYSPACE_PER_TEST + " FROM " + user);
+
+        table = KEYSPACE_PER_TEST + "." + createTable(KEYSPACE_PER_TEST, "CREATE TABLE %s (pk int, ck int, val int, val_2 text, PRIMARY KEY (pk, ck))");
+        index = KEYSPACE_PER_TEST + '.' + createIndex(KEYSPACE_PER_TEST, "CREATE INDEX ON %s (val_2)");
+        type = KEYSPACE_PER_TEST + "." + createType(KEYSPACE_PER_TEST, "CREATE TYPE %s (a int, b text)");
+        mv = KEYSPACE_PER_TEST + ".alltables_mv_03";
+        executeNet("CREATE MATERIALIZED VIEW " + mv + " AS SELECT * FROM " + table + " WHERE val IS NOT NULL AND pk IS NOT NULL AND ck IS NOT NULL PRIMARY KEY (val, pk, ck)");
+
+        useUser(user, pass);
+
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                "INSERT INTO " + table + " (pk, ck, val, val_2) VALUES (1, 1, 1, '1')");
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                "UPDATE " + table + " SET val = 1 WHERE pk = 1 AND ck = 1");
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                "DELETE FROM " + table + " WHERE pk = 1 AND ck = 2");
+        assertUnauthorizedQuery("User user has no SELECT permission on <table " + table + "> or any of its parents",
+                                "SELECT * FROM " + table + " WHERE pk = 1 AND ck = 1");
+        assertUnauthorizedQuery("User user has no SELECT permission on <table " + table + "> or any of its parents",
+                                "SELECT * FROM " + mv + " WHERE val = 1 AND pk = 1 AND ck = 1");
+        assertUnauthorizedQuery("User user has no MODIFY permission on <table " + table + "> or any of its parents",
+                                "TRUNCATE TABLE " + table);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "ALTER TABLE " + table + " ADD val_3 int"));
+        assertUnauthorizedQuery("User user has no DROP permission on <table " + table + "> or any of its parents",
+                                formatQuery(KEYSPACE_PER_TEST, "DROP TABLE " + table));
+        assertUnauthorizedQuery("User user has no ALTER permission on <all tables in " + KEYSPACE_PER_TEST + "> or any of its parents",
+                                "ALTER TYPE " + type + " ADD c bigint");
+        assertUnauthorizedQuery("User user has no DROP permission on <all tables in " + KEYSPACE_PER_TEST + "> or any of its parents",
+                                "DROP TYPE " + type);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                "DROP MATERIALIZED VIEW " + mv);
+        assertUnauthorizedQuery("User user has no ALTER permission on <table " + table + "> or any of its parents",
+                                "DROP INDEX " + index);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 167ad57..049e5ab 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -48,7 +48,9 @@ import com.google.common.collect.ImmutableSet;
 
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.util.File;
+
 import org.junit.*;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -56,9 +58,14 @@ import com.codahale.metrics.Gauge;
 import com.datastax.driver.core.*;
 import com.datastax.driver.core.DataType;
 import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.exceptions.UnauthorizedException;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.ServerTestUtils;
+import org.apache.cassandra.auth.AuthKeyspace;
+import org.apache.cassandra.auth.AuthSchemaChangeListener;
+import org.apache.cassandra.auth.AuthTestUtils;
+import org.apache.cassandra.auth.IRoleManager;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry;
 import org.apache.cassandra.db.virtual.VirtualSchemaKeyspace;
@@ -92,6 +99,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JMXServerUtils;
 import org.assertj.core.api.Assertions;
+import org.apache.cassandra.utils.Pair;
 
 import static com.datastax.driver.core.SocketOptions.DEFAULT_CONNECT_TIMEOUT_MILLIS;
 import static com.datastax.driver.core.SocketOptions.DEFAULT_READ_TIMEOUT_MILLIS;
@@ -105,6 +113,11 @@ import static org.junit.Assert.fail;
  */
 public abstract class CQLTester
 {
+    /**
+     * The super user
+     */
+    private static final User SUPER_USER = new User("cassandra", "cassandra");
+
     protected static final Logger logger = LoggerFactory.getLogger(CQLTester.class);
 
     public static final String KEYSPACE = "cql_test_keyspace";
@@ -127,8 +140,8 @@ public abstract class CQLTester
     protected static final int nativePort;
     protected static final InetAddress nativeAddr;
     protected static final Set<InetAddressAndPort> remoteAddrs = new HashSet<>();
-    private static final Map<ProtocolVersion, Cluster> clusters = new HashMap<>();
-    private static final Map<ProtocolVersion, Session> sessions = new HashMap<>();
+    private static final Map<Pair<User, ProtocolVersion>, Cluster> clusters = new HashMap<>();
+    private static final Map<Pair<User, ProtocolVersion>, Session> sessions = new HashMap<>();
 
     private static Consumer<Cluster.Builder> clusterBuilderConfigurator;
 
@@ -170,6 +183,8 @@ public abstract class CQLTester
     private List<String> functions = new ArrayList<>();
     private List<String> aggregates = new ArrayList<>();
 
+    private User user;
+
     // We don't use USE_PREPARED_VALUES in the code below so some test can foce value preparation (if the result
     // is not expected to be the same without preparation)
     private boolean usePrepared = USE_PREPARED_VALUES;
@@ -181,6 +196,24 @@ public abstract class CQLTester
     }
 
     /**
+     * Use the specified user for executing the queries over the network.
+     * @param username the user name
+     * @param password the user password
+     */
+    public void useUser(String username, String password)
+    {
+        this.user = new User(username, password);
+    }
+
+    /**
+     * Use the super user for executing the queries over the network.
+     */
+    public void useSuperUser()
+    {
+        this.user = SUPER_USER;
+    }
+
+    /**
      * Returns a port number that is automatically allocated,
      * typically from an ephemeral port range.
      *
@@ -332,6 +365,7 @@ public abstract class CQLTester
         types = null;
         functions = null;
         aggregates = null;
+        user = null;
 
         // We want to clean up after the test, but dropping a table is rather long so just do that asynchronously
         ScheduledExecutors.optionalTasks.execute(new Runnable()
@@ -416,6 +450,32 @@ public abstract class CQLTester
         return allArgs;
     }
 
+    protected static void requireAuthentication()
+    {
+        DatabaseDescriptor.setAuthenticator(new AuthTestUtils.LocalPasswordAuthenticator());
+        DatabaseDescriptor.setAuthorizer(new AuthTestUtils.LocalCassandraAuthorizer());
+        DatabaseDescriptor.setNetworkAuthorizer(new AuthTestUtils.LocalCassandraNetworkAuthorizer());
+
+        // The CassandraRoleManager constructor set the supported and alterable options based on
+        // DatabaseDescriptor authenticator type so it needs to be created only after the authenticator is set.
+        IRoleManager roleManager =  new AuthTestUtils.LocalCassandraRoleManager()
+        {
+            public void setup()
+            {
+                loadRoleStatement();
+                QueryProcessor.executeInternal(createDefaultRoleQuery());
+            }
+        };
+
+        DatabaseDescriptor.setRoleManager(roleManager);
+        MigrationManager.announceNewKeyspace(AuthKeyspace.metadata(), true);
+        DatabaseDescriptor.getRoleManager().setup();
+        DatabaseDescriptor.getAuthenticator().setup();
+        DatabaseDescriptor.getAuthorizer().setup();
+        DatabaseDescriptor.getNetworkAuthorizer().setup();
+        Schema.instance.registerListener(new AuthSchemaChangeListener());
+    }
+
     /**
      *  Initialize Native Transport for test that need it.
      */
@@ -483,7 +543,7 @@ public abstract class CQLTester
         server.start();
     }
 
-    private static Cluster initClientCluster(ProtocolVersion version)
+    private static Cluster initClientCluster(User user, ProtocolVersion version)
     {
         SocketOptions socketOptions =
                 new SocketOptions().setConnectTimeoutMillis(Integer.getInteger("cassandra.test.driver.connection_timeout_ms",
@@ -499,6 +559,8 @@ public abstract class CQLTester
                                          .withClusterName("Test Cluster")
                                          .withPort(nativePort)
                                          .withSocketOptions(socketOptions);
+        if (user != null)
+            builder.withCredentials(user.username, user.password);
 
         if (version.isBeta())
             builder = builder.allowBetaProtocolVersion();
@@ -671,14 +733,25 @@ public abstract class CQLTester
 
     protected String createType(String query)
     {
-        String typeName = String.format("type_%02d", seqNumber.getAndIncrement());
-        String fullQuery = String.format(query, KEYSPACE + "." + typeName);
-        types.add(typeName);
+        return createType(KEYSPACE, query);
+    }
+
+    protected String createType(String keyspace, String query)
+    {
+        String typeName = createTypeName();
+        String fullQuery = String.format(query, keyspace + "." + typeName);
         logger.info(fullQuery);
         schemaChange(fullQuery);
         return typeName;
     }
 
+    protected String createTypeName()
+    {
+        String typeName = String.format("type_%02d", seqNumber.getAndIncrement());
+        types.add(typeName);
+        return typeName;
+    }
+
     protected String createFunctionName(String keyspace)
     {
         return String.format("%s.function_%02d", keyspace, seqNumber.getAndIncrement());
@@ -815,7 +888,12 @@ public abstract class CQLTester
 
     protected String createIndex(String query)
     {
-        String formattedQuery = formatQuery(query);
+        return createIndex(KEYSPACE, query);
+    }
+
+    protected String createIndex(String keyspace, String query)
+    {
+        String formattedQuery = formatQuery(keyspace, query);
         return createFormattedIndex(formattedQuery);
     }
 
@@ -1031,12 +1109,12 @@ public abstract class CQLTester
     private Session getSession(ProtocolVersion protocolVersion)
     {
         Cluster cluster = getCluster(protocolVersion);
-        return sessions.computeIfAbsent(protocolVersion, userProto -> cluster.connect());
+        return sessions.computeIfAbsent(Pair.create(user, protocolVersion), userProto -> cluster.connect());
     }
 
     private Cluster getCluster(ProtocolVersion protocolVersion)
     {
-        return clusters.computeIfAbsent(protocolVersion, userProto -> initClientCluster(protocolVersion));
+        return clusters.computeIfAbsent(Pair.create(user, protocolVersion), userProto -> initClientCluster(userProto.left, userProto.right));
     }
 
     protected SimpleClient newSimpleClient(ProtocolVersion version) throws IOException
@@ -1550,6 +1628,21 @@ public abstract class CQLTester
                 e.getMessage().contains(text));
     }
 
+    /**
+     * Checks that the specified query is not authorized for the current user.
+     * @param errorMessage The expected error message
+     * @param query the query
+     * @param values the query parameters
+     */
+    protected void assertUnauthorizedQuery(String errorMessage, String query, Object... values) throws Throwable
+    {
+        assertInvalidThrowMessage(Optional.of(ProtocolVersion.CURRENT),
+                                  errorMessage,
+                                  UnauthorizedException.class,
+                                  query,
+                                  values);
+    }
+
     @FunctionalInterface
     public interface CheckedFunction {
         void apply() throws Throwable;
@@ -2018,4 +2111,44 @@ public abstract class CQLTester
             return "UserTypeValue" + toCQLString();
         }
     }
+
+    private static class User
+    {
+        /**
+         * The user name
+         */
+        public final String username;
+
+        /**
+         * The user password
+         */
+        public final String password;
+
+        public User(String username, String password)
+        {
+            this.username = username;
+            this.password = password;
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hashCode(username, password);
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o)
+                return true;
+
+            if (!(o instanceof User))
+                return false;
+
+            User u = (User) o;
+
+            return Objects.equal(username, u.username)
+                && Objects.equal(password, u.password);
+        }
+    }
 }

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