You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2015/01/14 20:51:10 UTC

[2/4] cassandra git commit: Introduce role based access control

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 15e7203..33d2bb2 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -19,20 +19,19 @@ package org.apache.cassandra.config;
 
 import java.io.IOException;
 import java.io.StringReader;
-import java.sql.Time;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Sets;
-import org.supercsv.io.CsvListReader;
-import org.supercsv.prefs.CsvPreference;
 
 import org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.NativeAllocator;
 import org.apache.cassandra.utils.FBUtilities;
+import org.supercsv.io.CsvListReader;
+import org.supercsv.prefs.CsvPreference;
 
 /**
  * A class that contains configuration properties for the cassandra node it runs within.
@@ -44,9 +43,11 @@ public class Config
     public String cluster_name = "Test Cluster";
     public String authenticator;
     public String authorizer;
+    public String role_manager;
     public int permissions_validity_in_ms = 2000;
     public int permissions_cache_max_entries = 1000;
     public int permissions_update_interval_in_ms = -1;
+    public int roles_validity_in_ms = 2000;
 
     /* Hashing strategy Random or OPHF */
     public String partitioner;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 9780db7..6d626da 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -81,6 +81,7 @@ public class DatabaseDescriptor
 
     private static IAuthenticator authenticator = new AllowAllAuthenticator();
     private static IAuthorizer authorizer = new AllowAllAuthorizer();
+    private static IRoleManager roleManager = new CassandraRoleManager();
 
     private static IRequestScheduler requestScheduler;
     private static RequestSchedulerId requestSchedulerId;
@@ -184,7 +185,7 @@ public class DatabaseDescriptor
             }
         }
 
-        /* Authentication and authorization backend, implementing IAuthenticator and IAuthorizer */
+        /* Authentication, authorization and role management backend, implementing IAuthenticator, IAuthorizer & IRoleMapper*/
         if (conf.authenticator != null)
             authenticator = FBUtilities.newAuthenticator(conf.authenticator);
 
@@ -194,6 +195,12 @@ public class DatabaseDescriptor
         if (authenticator instanceof AllowAllAuthenticator && !(authorizer instanceof AllowAllAuthorizer))
             throw new ConfigurationException("AllowAllAuthenticator can't be used with " +  conf.authorizer);
 
+        if (conf.role_manager != null)
+            roleManager = FBUtilities.newRoleManager(conf.role_manager);
+
+        if (authenticator instanceof PasswordAuthenticator && !(roleManager instanceof CassandraRoleManager))
+            throw new ConfigurationException("CassandraRoleManager must be used with PasswordAuthenticator");
+
         if (conf.internode_authenticator != null)
             internodeAuthenticator = FBUtilities.construct(conf.internode_authenticator, "internode_authenticator");
         else
@@ -201,6 +208,7 @@ public class DatabaseDescriptor
 
         authenticator.validateConfiguration();
         authorizer.validateConfiguration();
+        roleManager.validateConfiguration();
         internodeAuthenticator.validateConfiguration();
 
         /* Hashing strategy */
@@ -604,6 +612,11 @@ public class DatabaseDescriptor
         return authorizer;
     }
 
+    public static IRoleManager getRoleManager()
+    {
+        return roleManager;
+    }
+
     public static int getPermissionsValidity()
     {
         return conf.permissions_validity_in_ms;
@@ -621,6 +634,11 @@ public class DatabaseDescriptor
              : conf.permissions_update_interval_in_ms;
     }
 
+    public static int getRolesValidity()
+    {
+        return conf.roles_validity_in_ms;
+    }
+
     public static int getThriftFramedTransportSize()
     {
         return conf.thrift_framed_transport_size_in_mb * 1024 * 1024;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index d73dc28..c39da63 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -40,6 +40,7 @@ options {
     import org.apache.cassandra.auth.Permission;
     import org.apache.cassandra.auth.DataResource;
     import org.apache.cassandra.auth.IResource;
+    import org.apache.cassandra.auth.IRoleManager;
     import org.apache.cassandra.cql3.*;
     import org.apache.cassandra.cql3.statements.*;
     import org.apache.cassandra.cql3.selection.*;
@@ -247,6 +248,12 @@ cqlStatement returns [ParsedStatement stmt]
     | st29=dropFunctionStatement       { $stmt = st29; }
     | st30=createAggregateStatement    { $stmt = st30; }
     | st31=dropAggregateStatement      { $stmt = st31; }
+    | st32=createRoleStatement         { $stmt = st32; }
+    | st33=alterRoleStatement          { $stmt = st33; }
+    | st34=dropRoleStatement           { $stmt = st34; }
+    | st35=listRolesStatement          { $stmt = st35; }
+    | st36=grantRoleStatement          { $stmt = st36; }
+    | st37=revokeRoleStatement         { $stmt = st37; }
     ;
 
 /*
@@ -802,7 +809,7 @@ truncateStatement returns [TruncateStatement stmt]
     ;
 
 /**
- * GRANT <permission> ON <resource> TO <username>
+ * GRANT <permission> ON <resource> TO <rolename>
  */
 grantStatement returns [GrantStatement stmt]
     : K_GRANT
@@ -810,12 +817,12 @@ grantStatement returns [GrantStatement stmt]
       K_ON
           resource
       K_TO
-          username
-      { $stmt = new GrantStatement($permissionOrAll.perms, (DataResource) $resource.res, $username.text); }
+          grantee=userOrRoleName
+      { $stmt = new GrantStatement($permissionOrAll.perms, (DataResource) $resource.res, grantee); }
     ;
 
 /**
- * REVOKE <permission> ON <resource> FROM <username>
+ * REVOKE <permission> ON <resource> FROM <rolename>
  */
 revokeStatement returns [RevokeStatement stmt]
     : K_REVOKE
@@ -823,22 +830,44 @@ revokeStatement returns [RevokeStatement stmt]
       K_ON
           resource
       K_FROM
-          username
-      { $stmt = new RevokeStatement($permissionOrAll.perms, (DataResource) $resource.res, $username.text); }
+          revokee=userOrRoleName
+      { $stmt = new RevokeStatement($permissionOrAll.perms, (DataResource) $resource.res, revokee); }
+    ;
+
+/**
+ * GRANT ROLE <rolename> TO <grantee>
+ */
+grantRoleStatement returns [GrantRoleStatement stmt]
+    : K_GRANT
+          role=userOrRoleName
+      K_TO
+          grantee=userOrRoleName
+      { $stmt = new GrantRoleStatement(role, grantee); }
+    ;
+
+/**
+ * REVOKE ROLE <rolename> FROM <revokee>
+ */
+revokeRoleStatement returns [RevokeRoleStatement stmt]
+    : K_REVOKE
+          role=userOrRoleName
+      K_FROM
+          revokee=userOrRoleName
+      { $stmt = new RevokeRoleStatement(role, revokee); }
     ;
 
 listPermissionsStatement returns [ListPermissionsStatement stmt]
     @init {
         IResource resource = null;
-        String username = null;
         boolean recursive = true;
+        RoleName grantee = new RoleName();
     }
     : K_LIST
           permissionOrAll
       ( K_ON resource { resource = $resource.res; } )?
-      ( K_OF username { username = $username.text; } )?
+      ( K_OF roleName[grantee] )?
       ( K_NORECURSIVE { recursive = false; } )?
-      { $stmt = new ListPermissionsStatement($permissionOrAll.perms, (DataResource) resource, username, recursive); }
+      { $stmt = new ListPermissionsStatement($permissionOrAll.perms, (DataResource) resource, grantee, recursive); }
     ;
 
 permission returns [Permission perm]
@@ -859,59 +888,127 @@ 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.columnFamily($cf.name.getKeyspace(), $cf.name.getColumnFamily()); }
+      { $res = DataResource.table($cf.name.getKeyspace(), $cf.name.getColumnFamily()); }
     ;
 
 /**
  * CREATE USER [IF NOT EXISTS] <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
  */
-createUserStatement returns [CreateUserStatement stmt]
+createUserStatement returns [CreateRoleStatement stmt]
     @init {
-        UserOptions opts = new UserOptions();
+        RoleOptions opts = new RoleOptions();
+        opts.put(IRoleManager.Option.LOGIN.name(), true);
         boolean superuser = false;
         boolean ifNotExists = false;
+        RoleName name = new RoleName();
     }
-    : K_CREATE K_USER (K_IF K_NOT K_EXISTS { ifNotExists = true; })? username
-      ( K_WITH userOptions[opts] )?
+    : K_CREATE K_USER (K_IF K_NOT K_EXISTS { ifNotExists = true; })? u=username { name.setName($u.text, false); }
+      ( K_WITH roleOptions[opts] )?
       ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )?
-      { $stmt = new CreateUserStatement($username.text, opts, superuser, ifNotExists); }
+      { opts.put(IRoleManager.Option.SUPERUSER.name(), superuser);
+        $stmt = new CreateRoleStatement(name, opts, ifNotExists); }
     ;
 
 /**
  * ALTER USER <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
  */
-alterUserStatement returns [AlterUserStatement stmt]
+alterUserStatement returns [AlterRoleStatement stmt]
     @init {
-        UserOptions opts = new UserOptions();
-        Boolean superuser = null;
+        RoleOptions opts = new RoleOptions();
+        RoleName name = new RoleName();
     }
-    : K_ALTER K_USER username
-      ( K_WITH userOptions[opts] )?
-      ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )?
-      { $stmt = new AlterUserStatement($username.text, opts, superuser); }
+    : K_ALTER K_USER u=username { name.setName($u.text, false); }
+      ( K_WITH roleOptions[opts] )?
+      ( K_SUPERUSER { opts.put(IRoleManager.Option.SUPERUSER.name(), true); }
+        | K_NOSUPERUSER { opts.put(IRoleManager.Option.SUPERUSER.name(), false); } ) ?
+      {  $stmt = new AlterRoleStatement(name, opts); }
     ;
 
 /**
  * DROP USER [IF EXISTS] <username>
  */
-dropUserStatement returns [DropUserStatement stmt]
-    @init { boolean ifExists = false; }
-    : K_DROP K_USER (K_IF K_EXISTS { ifExists = true; })? username { $stmt = new DropUserStatement($username.text, ifExists); }
+dropUserStatement returns [DropRoleStatement stmt]
+    @init {
+        boolean ifExists = false;
+        RoleName name = new RoleName();
+    }
+    : K_DROP K_USER (K_IF K_EXISTS { ifExists = true; })? u=username { name.setName($u.text, false); $stmt = new DropRoleStatement(name, ifExists); }
     ;
 
 /**
  * LIST USERS
  */
-listUsersStatement returns [ListUsersStatement stmt]
+listUsersStatement returns [ListRolesStatement stmt]
     : K_LIST K_USERS { $stmt = new ListUsersStatement(); }
     ;
 
-userOptions[UserOptions opts]
-    : userOption[opts]
+/**
+ * CREATE ROLE [IF NOT EXISTS] <rolename> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER] [LOGIN|NOLOGIN]
+ */
+createRoleStatement returns [CreateRoleStatement stmt]
+    @init {
+        RoleOptions opts = new RoleOptions();
+        boolean superuser = false;
+        boolean login = false;
+        boolean ifNotExists = false;
+    }
+    : K_CREATE K_ROLE (K_IF K_NOT K_EXISTS { ifNotExists = true; })? name=userOrRoleName
+      ( K_WITH roleOptions[opts] )?
+      ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )?
+      ( K_LOGIN { login = true; } | K_NOLOGIN { login = false; } )?
+      { opts.put(IRoleManager.Option.SUPERUSER.name(), superuser);
+        opts.put(IRoleManager.Option.LOGIN.name(), login);
+        $stmt = new CreateRoleStatement(name, opts, ifNotExists); }
     ;
 
-userOption[UserOptions opts]
-    : k=K_PASSWORD v=STRING_LITERAL { opts.put($k.text, $v.text); }
+/**
+ * ALTER ROLE <rolename> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
+ */
+alterRoleStatement returns [AlterRoleStatement stmt]
+    @init {
+        RoleOptions opts = new RoleOptions();
+    }
+    : K_ALTER K_ROLE name=userOrRoleName
+      ( K_WITH roleOptions[opts] )?
+      ( K_SUPERUSER { opts.put(IRoleManager.Option.SUPERUSER.name(), true); }
+        | K_NOSUPERUSER { opts.put(IRoleManager.Option.SUPERUSER.name(), false); } ) ?
+      ( K_LOGIN { opts.put(IRoleManager.Option.LOGIN.name(), true); }
+        | K_NOLOGIN { opts.put(IRoleManager.Option.LOGIN.name(), false); } )?
+      {  $stmt = new AlterRoleStatement(name, opts); }
+    ;
+
+/**
+ * DROP ROLE [IF EXISTS] <rolename>
+ */
+dropRoleStatement returns [DropRoleStatement stmt]
+    @init {
+        boolean ifExists = false;
+    }
+    : K_DROP K_ROLE (K_IF K_EXISTS { ifExists = true; })? name=userOrRoleName
+      { $stmt = new DropRoleStatement(name, ifExists); }
+    ;
+
+/**
+ * LIST ROLES [OF <rolename>] [NORECURSIVE]
+ */
+listRolesStatement returns [ListRolesStatement stmt]
+    @init {
+        boolean recursive = true;
+        RoleName grantee = new RoleName();
+    }
+    : K_LIST K_ROLES
+      ( K_OF roleName[grantee])?
+      ( K_NORECURSIVE { recursive = false; } )?
+      { $stmt = new ListRolesStatement(grantee, recursive); }
+    ;
+
+roleOptions[RoleOptions opts]
+    : roleOption[opts] (K_AND roleOption[opts])*
+    ;
+
+roleOption[RoleOptions opts]
+    :  k=K_PASSWORD v=STRING_LITERAL { opts.put($k.text, $v.text); }
+    |  k=K_OPTIONS  m=mapLiteral { opts.put(IRoleManager.Option.OPTIONS.name(), convertPropertyMap(m)); }
     ;
 
 /** DEFINITIONS **/
@@ -952,6 +1049,11 @@ userTypeName returns [UTName name]
     : (ks=ident '.')? ut=non_type_ident { return new UTName(ks, ut); }
     ;
 
+userOrRoleName returns [RoleName name]
+    @init { $name = new RoleName(); }
+    : roleName[name] {return $name;}
+    ;
+
 ksName[KeyspaceElementName name]
     : t=IDENT              { $name.setKeyspace($t.text, false);}
     | t=QUOTED_NAME        { $name.setKeyspace($t.text, true);}
@@ -973,6 +1075,13 @@ idxName[IndexName name]
     | QMARK {addRecognitionError("Bind variables cannot be used for index names");}
     ;
 
+roleName[RoleName name]
+    : t=IDENT              { $name.setName($t.text, false); }
+    | t=QUOTED_NAME        { $name.setName($t.text, true); }
+    | k=unreserved_keyword { $name.setName(k, false); }
+    | QMARK {addRecognitionError("Bind variables cannot be used for role names");}
+    ;
+
 constant returns [Constants.Literal constant]
     : t=STRING_LITERAL { $constant = Constants.Literal.string($t.text); }
     | t=INTEGER        { $constant = Constants.Literal.integer($t.text); }
@@ -1326,8 +1435,13 @@ basic_unreserved_keyword returns [String str]
         | K_ALL
         | K_USER
         | K_USERS
+        | K_ROLE
+        | K_ROLES
         | K_SUPERUSER
         | K_NOSUPERUSER
+        | K_LOGIN
+        | K_NOLOGIN
+        | K_OPTIONS
         | K_PASSWORD
         | K_EXISTS
         | K_CUSTOM
@@ -1417,9 +1531,14 @@ K_NORECURSIVE: N O R E C U R S I V E;
 
 K_USER:        U S E R;
 K_USERS:       U S E R S;
+K_ROLE:        R O L E;
+K_ROLES:       R O L E S;
 K_SUPERUSER:   S U P E R U S E R;
 K_NOSUPERUSER: N O S U P E R U S E R;
 K_PASSWORD:    P A S S W O R D;
+K_LOGIN:       L O G I N;
+K_NOLOGIN:     N O L O G I N;
+K_OPTIONS:     O P T I O N S;
 
 K_CLUSTERING:  C L U S T E R I N G;
 K_ASCII:       A S C I I;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/RoleName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/RoleName.java b/src/java/org/apache/cassandra/cql3/RoleName.java
new file mode 100644
index 0000000..9aa3419
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/RoleName.java
@@ -0,0 +1,41 @@
+/*
+ * 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.cql3;
+
+import java.util.Locale;
+
+public class RoleName
+{
+    private String name;
+
+    public void setName(String name, boolean keepCase)
+    {
+        this.name = keepCase ? name : name.toLowerCase(Locale.US);
+    }
+
+    public String getName()
+    {
+        return name;
+    }
+
+    @Override
+    public String toString()
+    {
+        return name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/RoleOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/RoleOptions.java b/src/java/org/apache/cassandra/cql3/RoleOptions.java
new file mode 100644
index 0000000..89f37dd
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/RoleOptions.java
@@ -0,0 +1,62 @@
+/*
+ * 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.cql3;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.cassandra.auth.IRoleManager;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class RoleOptions
+{
+    private final Map<IRoleManager.Option, Object> options = new HashMap<>();
+
+    public void put(String name, Object value)
+    {
+        options.put(IRoleManager.Option.valueOf(name.toUpperCase()), value);
+    }
+
+    public boolean isEmpty()
+    {
+        return options.isEmpty();
+    }
+
+    public Map<IRoleManager.Option, Object> getOptions()
+    {
+        return options;
+    }
+
+    public void validate() throws InvalidRequestException
+    {
+        for (IRoleManager.Option option : options.keySet())
+        {
+            if (!DatabaseDescriptor.getRoleManager().supportedOptions().contains(option))
+                throw new InvalidRequestException(String.format("%s doesn't support %s",
+                                                                DatabaseDescriptor.getRoleManager().getClass().getName(),
+                                                                option));
+        }
+    }
+
+    public String toString()
+    {
+        return FBUtilities.toString(options);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/UserOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UserOptions.java b/src/java/org/apache/cassandra/cql3/UserOptions.java
deleted file mode 100644
index 701debd..0000000
--- a/src/java/org/apache/cassandra/cql3/UserOptions.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.cql3;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.utils.FBUtilities;
-
-public class UserOptions
-{
-    private final Map<IAuthenticator.Option, Object> options = new HashMap<IAuthenticator.Option, Object>();
-
-    public void put(String name, Object value)
-    {
-        options.put(IAuthenticator.Option.valueOf(name.toUpperCase()), value);
-    }
-
-    public boolean isEmpty()
-    {
-        return options.isEmpty();
-    }
-
-    public Map<IAuthenticator.Option, Object> getOptions()
-    {
-        return options;
-    }
-
-    public void validate() throws InvalidRequestException
-    {
-        for (IAuthenticator.Option option : options.keySet())
-        {
-            if (!DatabaseDescriptor.getAuthenticator().supportedOptions().contains(option))
-                throw new InvalidRequestException(String.format("%s doesn't support %s option",
-                                                                DatabaseDescriptor.getAuthenticator().getClass().getName(),
-                                                                option));
-        }
-    }
-
-    public String toString()
-    {
-        return FBUtilities.toString(options);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java
new file mode 100644
index 0000000..8542138
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java
@@ -0,0 +1,84 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.auth.IRoleManager.Option;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.cql3.RoleOptions;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class AlterRoleStatement extends AuthenticationStatement
+{
+    private final String role;
+    private final RoleOptions opts;
+
+    public AlterRoleStatement(RoleName name, RoleOptions opts)
+    {
+        this.role = name.getName();
+        this.opts = opts;
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        opts.validate();
+
+        if (opts.isEmpty())
+            throw new InvalidRequestException("ALTER [ROLE|USER] can't be empty");
+
+        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
+        state.ensureNotAnonymous();
+        if (!DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            throw new InvalidRequestException(String.format("%s doesn't exist", role));
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        AuthenticatedUser user = state.getUser();
+        boolean isSuper = user.isSuper();
+
+        if (opts.getOptions().containsKey(Option.SUPERUSER) && user.getRoles().contains(role))
+            throw new UnauthorizedException("You aren't allowed to alter your own superuser " +
+                                            "status or that of a role granted to you");
+
+        if (opts.getOptions().containsKey(Option.SUPERUSER) && !isSuper)
+            throw new UnauthorizedException("Only superusers are allowed to alter superuser status");
+
+        if (!user.isSuper() && !user.getName().equals(role))
+            throw new UnauthorizedException("You aren't allowed to alter this role");
+
+        if (!isSuper)
+        {
+            for (Option option : opts.getOptions().keySet())
+            {
+                if (!DatabaseDescriptor.getRoleManager().alterableOptions().contains(option))
+                    throw new UnauthorizedException(String.format("You aren't allowed to alter %s", option));
+            }
+        }
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        if (!opts.isEmpty())
+            DatabaseDescriptor.getRoleManager().alterRole(state.getUser(), role, opts.getOptions());
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/AlterUserStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterUserStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterUserStatement.java
deleted file mode 100644
index 7a6e96b..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/AlterUserStatement.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.auth.AuthenticatedUser;
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.UserOptions;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-public class AlterUserStatement extends AuthenticationStatement
-{
-    private final String username;
-    private final UserOptions opts;
-    private final Boolean superuser;
-
-    public AlterUserStatement(String username, UserOptions opts, Boolean superuser)
-    {
-        this.username = username;
-        this.opts = opts;
-        this.superuser = superuser;
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        opts.validate();
-
-        if (superuser == null && opts.isEmpty())
-            throw new InvalidRequestException("ALTER USER can't be empty");
-
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
-        state.ensureNotAnonymous();
-
-        if (!Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        AuthenticatedUser user = state.getUser();
-
-        boolean isSuper = user.isSuper();
-
-        if (superuser != null && user.getName().equals(username))
-            throw new UnauthorizedException("You aren't allowed to alter your own superuser status");
-
-        if (superuser != null && !isSuper)
-            throw new UnauthorizedException("Only superusers are allowed to alter superuser status");
-
-        if (!user.isSuper() && !user.getName().equals(username))
-            throw new UnauthorizedException("You aren't allowed to alter this user");
-
-        if (!isSuper)
-        {
-            for (IAuthenticator.Option option : opts.getOptions().keySet())
-            {
-                if (!DatabaseDescriptor.getAuthenticator().alterableOptions().contains(option))
-                    throw new UnauthorizedException(String.format("You aren't allowed to alter %s option", option));
-            }
-        }
-    }
-
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
-    {
-        if (!opts.isEmpty())
-            DatabaseDescriptor.getAuthenticator().alter(username, opts.getOptions());
-        if (superuser != null)
-            Auth.insertUser(username, superuser.booleanValue());
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
index 2c7f2cb..3dc4510 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
@@ -55,8 +55,8 @@ public abstract class AuthorizationStatement extends ParsedStatement implements
 
     public static DataResource maybeCorrectResource(DataResource resource, ClientState state) throws InvalidRequestException
     {
-        if (resource.isColumnFamilyLevel() && resource.getKeyspace() == null)
-            return DataResource.columnFamily(state.getKeyspace(), resource.getColumnFamily());
+        if (resource.isTableLevel() && resource.getKeyspace() == null)
+            return DataResource.table(state.getKeyspace(), resource.getTable());
         return resource;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
new file mode 100644
index 0000000..64dd9bb
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
@@ -0,0 +1,76 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.auth.IRoleManager.Option;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.cql3.RoleOptions;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class CreateRoleStatement extends AuthorizationStatement
+{
+    private final String role;
+    private final RoleOptions opts;
+    private final boolean ifNotExists;
+
+    public CreateRoleStatement(RoleName name, RoleOptions options, boolean ifNotExists)
+    {
+        this.role = name.getName();
+        this.opts = options;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        if (!state.getUser().isSuper())
+            throw new UnauthorizedException("Only superusers are allowed to perform CREATE [ROLE|USER] queries");
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        opts.validate();
+
+        if (role.isEmpty())
+            throw new InvalidRequestException("Role name can't be an empty string");
+
+        // validate login here before checkAccess to avoid leaking role existence to anonymous users.
+        state.ensureNotAnonymous();
+
+        if (!ifNotExists && DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            throw new InvalidRequestException(String.format("%s already exists", role));
+
+        for (Option option : opts.getOptions().keySet())
+        {
+            if (!DatabaseDescriptor.getRoleManager().supportedOptions().contains(option))
+                throw new UnauthorizedException(String.format("You aren't allowed to alter %s", option));
+        }
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestExecutionException, RequestValidationException
+    {
+        // not rejected in validate()
+        if (ifNotExists && DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            return null;
+
+        DatabaseDescriptor.getRoleManager().createRole(state.getUser(), role, opts.getOptions());
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/CreateUserStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateUserStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateUserStatement.java
deleted file mode 100644
index 0903659..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateUserStatement.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.UserOptions;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-public class CreateUserStatement extends AuthenticationStatement
-{
-    private final String username;
-    private final UserOptions opts;
-    private final boolean superuser;
-    private final boolean ifNotExists;
-
-    public CreateUserStatement(String username, UserOptions opts, boolean superuser, boolean ifNotExists)
-    {
-        this.username = username;
-        this.opts = opts;
-        this.superuser = superuser;
-        this.ifNotExists = ifNotExists;
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        if (username.isEmpty())
-            throw new InvalidRequestException("Username can't be an empty string");
-
-        opts.validate();
-
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
-        state.ensureNotAnonymous();
-
-        if (!ifNotExists && Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s already exists", username));
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        if (!state.getUser().isSuper())
-            throw new UnauthorizedException("Only superusers are allowed to perform CREATE USER queries");
-    }
-
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
-    {
-        // not rejected in validate()
-        if (ifNotExists && Auth.isExistingUser(username))
-            return null;
-
-        DatabaseDescriptor.getAuthenticator().create(username, opts.getOptions());
-        Auth.insertUser(username, superuser);
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java
new file mode 100644
index 0000000..c4beab6
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java
@@ -0,0 +1,68 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class DropRoleStatement extends AuthenticationStatement
+{
+    private final String role;
+    private final boolean ifExists;
+
+    public DropRoleStatement(RoleName name, boolean ifExists)
+    {
+        this.role = name.getName();
+        this.ifExists = ifExists;
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
+        state.ensureNotAnonymous();
+
+        if (!ifExists && !DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            throw new InvalidRequestException(String.format("%s doesn't exist", role));
+
+        AuthenticatedUser user = state.getUser();
+        if (user != null && user.getName().equals(role))
+            throw new InvalidRequestException("Cannot DROP primary role for current login");
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        if (!state.getUser().isSuper())
+            throw new UnauthorizedException("Only superusers are allowed to perform DROP [ROLE|USER] queries" );
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        // not rejected in validate()
+        if (ifExists && !DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            return null;
+
+        // clean up grants and permissions of the dropped role.
+        DatabaseDescriptor.getRoleManager().dropRole(state.getUser(), role);
+        DatabaseDescriptor.getAuthorizer().revokeAll(role);
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/DropUserStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropUserStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropUserStatement.java
deleted file mode 100644
index 9e92b26..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropUserStatement.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.auth.AuthenticatedUser;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-public class DropUserStatement extends AuthenticationStatement
-{
-    private final String username;
-    private final boolean ifExists;
-
-    public DropUserStatement(String username, boolean ifExists)
-    {
-        this.username = username;
-        this.ifExists = ifExists;
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
-        state.ensureNotAnonymous();
-
-        if (!ifExists && !Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
-
-        AuthenticatedUser user = state.getUser();
-        if (user != null && user.getName().equals(username))
-            throw new InvalidRequestException("Users aren't allowed to DROP themselves");
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        if (!state.getUser().isSuper())
-            throw new UnauthorizedException("Only superusers are allowed to perform DROP USER queries");
-    }
-
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
-    {
-        // not rejected in validate()
-        if (ifExists && !Auth.isExistingUser(username))
-            return null;
-
-        // clean up permissions after the dropped user.
-        DatabaseDescriptor.getAuthorizer().revokeAll(username);
-        Auth.deleteUser(username);
-        DatabaseDescriptor.getAuthenticator().drop(username);
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java
new file mode 100644
index 0000000..a22a99a
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java
@@ -0,0 +1,39 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class GrantRoleStatement extends RoleManagementStatement
+{
+    public GrantRoleStatement(RoleName name, RoleName grantee)
+    {
+        super(name, grantee);
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        DatabaseDescriptor.getRoleManager().grantRole(state.getUser(), role, grantee);
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java b/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java
index aabde5d..561fee6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -7,14 +7,13 @@
  * "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
+ *     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.
+ * 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.cql3.statements;
 
@@ -23,6 +22,7 @@ import java.util.Set;
 import org.apache.cassandra.auth.DataResource;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
@@ -30,14 +30,14 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 
 public class GrantStatement extends PermissionAlteringStatement
 {
-    public GrantStatement(Set<Permission> permissions, DataResource resource, String username)
+    public GrantStatement(Set<Permission> permissions, DataResource resource, RoleName grantee)
     {
-        super(permissions, resource, username);
+        super(permissions, resource, grantee);
     }
 
     public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
     {
-        DatabaseDescriptor.getAuthorizer().grant(state.getUser(), permissions, resource, username);
+        DatabaseDescriptor.getAuthorizer().grant(state.getUser(), permissions, resource, grantee);
         return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
index eaae2c6..d5b7e13 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
@@ -21,9 +21,7 @@ import java.util.*;
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.ColumnSpecification;
-import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
@@ -33,7 +31,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 
 public class ListPermissionsStatement extends AuthorizationStatement
 {
-    private static final String KS = Auth.AUTH_KS;
+    private static final String KS = AuthKeyspace.NAME;
     private static final String CF = "permissions"; // virtual cf to use for now.
 
     private static final List<ColumnSpecification> metadata;
@@ -41,23 +39,24 @@ public class ListPermissionsStatement extends AuthorizationStatement
     static
     {
         List<ColumnSpecification> columns = new ArrayList<ColumnSpecification>(4);
+        columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("role", true), UTF8Type.instance));
         columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("username", true), UTF8Type.instance));
         columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("resource", true), UTF8Type.instance));
         columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("permission", true), UTF8Type.instance));
         metadata = Collections.unmodifiableList(columns);
     }
 
-    private final Set<Permission> permissions;
-    private DataResource resource;
-    private final String username;
-    private final boolean recursive;
+    protected final Set<Permission> permissions;
+    protected DataResource resource;
+    protected final boolean recursive;
+    private final String grantee;
 
-    public ListPermissionsStatement(Set<Permission> permissions, DataResource resource, String username, boolean recursive)
+    public ListPermissionsStatement(Set<Permission> permissions, DataResource resource, RoleName grantee, boolean recursive)
     {
         this.permissions = permissions;
         this.resource = resource;
-        this.username = username;
         this.recursive = recursive;
+        this.grantee = grantee.getName();
     }
 
     public void validate(ClientState state) throws RequestValidationException
@@ -65,16 +64,16 @@ public class ListPermissionsStatement extends AuthorizationStatement
         // a check to ensure the existence of the user isn't being leaked by user existence check.
         state.ensureNotAnonymous();
 
-        if (username != null && !Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
-
         if (resource != null)
         {
             resource = maybeCorrectResource(resource, state);
             if (!resource.exists())
                 throw new InvalidRequestException(String.format("%s doesn't exist", resource));
         }
-    }
+
+        if ((grantee != null) && !DatabaseDescriptor.getRoleManager().isExistingRole(grantee))
+            throw new InvalidRequestException(String.format("%s doesn't exist", grantee));
+   }
 
     public void checkAccess(ClientState state)
     {
@@ -100,6 +99,12 @@ public class ListPermissionsStatement extends AuthorizationStatement
         return resultMessage(details);
     }
 
+    private Set<PermissionDetails> list(ClientState state, IResource resource)
+    throws RequestValidationException, RequestExecutionException
+    {
+        return DatabaseDescriptor.getAuthorizer().list(state.getUser(), permissions, resource, grantee);
+    }
+
     private ResultMessage resultMessage(List<PermissionDetails> details)
     {
         if (details.isEmpty())
@@ -108,16 +113,11 @@ public class ListPermissionsStatement extends AuthorizationStatement
         ResultSet result = new ResultSet(metadata);
         for (PermissionDetails pd : details)
         {
-            result.addColumnValue(UTF8Type.instance.decompose(pd.username));
+            result.addColumnValue(UTF8Type.instance.decompose(pd.grantee));
+            result.addColumnValue(UTF8Type.instance.decompose(pd.grantee));
             result.addColumnValue(UTF8Type.instance.decompose(pd.resource.toString()));
             result.addColumnValue(UTF8Type.instance.decompose(pd.permission.toString()));
         }
         return new ResultMessage.Rows(result);
     }
-
-    private Set<PermissionDetails> list(ClientState state, IResource resource)
-    throws RequestValidationException, RequestExecutionException
-    {
-        return DatabaseDescriptor.getAuthorizer().list(state.getUser(), permissions, resource, username);
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
new file mode 100644
index 0000000..4ca9aeb
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.statements;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.auth.AuthKeyspace;
+import org.apache.cassandra.auth.IRoleManager;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class ListRolesStatement extends AuthorizationStatement
+{
+    // pseudo-virtual cf as the actual datasource is dependent on the IRoleManager impl
+    private static final String KS = AuthKeyspace.NAME;
+    private static final String CF = AuthKeyspace.ROLES;
+
+    private static final List<ColumnSpecification> metadata =
+       ImmutableList.of(new ColumnSpecification(KS, CF, new ColumnIdentifier("role", true), UTF8Type.instance),
+                        new ColumnSpecification(KS, CF, new ColumnIdentifier("super", true), BooleanType.instance),
+                        new ColumnSpecification(KS, CF, new ColumnIdentifier("login", true), BooleanType.instance));
+
+    private final String grantee;
+    private final boolean recursive;
+
+    public ListRolesStatement()
+    {
+        this(new RoleName(), false);
+    }
+
+    public ListRolesStatement(RoleName grantee, boolean recursive)
+    {
+        this.grantee = grantee.getName();
+        this.recursive = recursive;
+    }
+
+    public void validate(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        state.ensureNotAnonymous();
+
+        if ((grantee != null) && !DatabaseDescriptor.getRoleManager().isExistingRole(grantee))
+            throw new InvalidRequestException(String.format("%s doesn't exist", grantee));
+    }
+
+    public void checkAccess(ClientState state) throws InvalidRequestException
+    {
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        if (state.getUser().isSuper())
+        {
+            if (grantee == null)
+                return resultMessage(DatabaseDescriptor.getRoleManager().getAllRoles());
+            else
+                return resultMessage(DatabaseDescriptor.getRoleManager().getRoles(grantee, recursive));
+        }
+        else
+        {
+            if (grantee == null)
+                return resultMessage(DatabaseDescriptor.getRoleManager().getRoles(state.getUser().getName(), recursive));
+            if (DatabaseDescriptor.getRoleManager().getRoles(state.getUser().getName(), true).contains(grantee))
+                return resultMessage(DatabaseDescriptor.getRoleManager().getRoles(grantee, recursive));
+            else
+                throw new UnauthorizedException(String.format("You are not authorized to view roles granted to %s ", grantee));
+        }
+    }
+
+    private ResultMessage resultMessage(Set<String> roles)
+    {
+        if (roles.isEmpty())
+            return new ResultMessage.Void();
+
+        List<String> sorted = Lists.newArrayList(roles);
+        Collections.sort(sorted);
+        return formatResults(sorted);
+    }
+
+    // overridden in ListUsersStatement to include legacy metadata
+    protected ResultMessage formatResults(List<String> sortedRoles)
+    {
+        ResultSet result = new ResultSet(metadata);
+
+        IRoleManager roleManager = DatabaseDescriptor.getRoleManager();
+        for (String role : sortedRoles)
+        {
+            result.addColumnValue(UTF8Type.instance.decompose(role));
+            result.addColumnValue(BooleanType.instance.decompose(roleManager.isSuper(role)));
+            result.addColumnValue(BooleanType.instance.decompose(roleManager.canLogin(role)));
+        }
+        return new ResultMessage.Rows(result);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
index 8acbcab..0854166 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
@@ -17,31 +17,43 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.auth.AuthKeyspace;
+import org.apache.cassandra.auth.IRoleManager;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.transport.messages.ResultMessage;
 
-public class ListUsersStatement extends AuthenticationStatement
+public class ListUsersStatement extends ListRolesStatement
 {
-    public void validate(ClientState state)
-    {
-    }
+    // pseudo-virtual cf as the actual datasource is dependent on the IRoleManager impl
+    private static final String KS = AuthKeyspace.NAME;
+    private static final String CF = "users";
 
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        state.ensureNotAnonymous();
-    }
+    private static final List<ColumnSpecification> metadata =
+        ImmutableList.of(new ColumnSpecification(KS, CF, new ColumnIdentifier("name", true), UTF8Type.instance),
+                         new ColumnSpecification(KS, CF, new ColumnIdentifier("super", true), BooleanType.instance));
 
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    @Override
+    protected ResultMessage formatResults(List<String> sortedRoles)
     {
-        return QueryProcessor.process(String.format("SELECT * FROM %s.%s", Auth.AUTH_KS, Auth.USERS_CF),
-                                      ConsistencyLevel.QUORUM,
-                                      QueryState.forInternalCalls());
+        ResultSet result = new ResultSet(metadata);
+
+        IRoleManager roleManager = DatabaseDescriptor.getRoleManager();
+        for (String role : sortedRoles)
+        {
+            if (!roleManager.canLogin(role))
+                continue;
+            result.addColumnValue(UTF8Type.instance.decompose(role));
+            result.addColumnValue(BooleanType.instance.decompose(roleManager.isSuper(role)));
+        }
+        return new ResultMessage.Rows(result);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java b/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java
index 8784930..44c9ea1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java
@@ -19,10 +19,10 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.Set;
 
-import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.auth.DataResource;
-import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
@@ -32,13 +32,13 @@ public abstract class PermissionAlteringStatement extends AuthorizationStatement
 {
     protected final Set<Permission> permissions;
     protected DataResource resource;
-    protected final String username;
+    protected final String grantee;
 
-    protected PermissionAlteringStatement(Set<Permission> permissions, DataResource resource, String username)
+    protected PermissionAlteringStatement(Set<Permission> permissions, DataResource resource, RoleName grantee)
     {
         this.permissions = permissions;
         this.resource = resource;
-        this.username = username;
+        this.grantee = grantee.getName();
     }
 
     public void validate(ClientState state) throws RequestValidationException
@@ -46,13 +46,13 @@ public abstract class PermissionAlteringStatement extends AuthorizationStatement
         // validate login here before checkAccess to avoid leaking user existence to anonymous users.
         state.ensureNotAnonymous();
 
-        if (!Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
+        if (!DatabaseDescriptor.getRoleManager().isExistingRole(grantee))
+            throw new InvalidRequestException(String.format("Role %s doesn't exist", grantee));
 
         // if a keyspace is omitted when GRANT/REVOKE ON TABLE <table>, we need to correct the resource.
         resource = maybeCorrectResource(resource, state);
         if (!resource.exists())
-            throw new InvalidRequestException(String.format("%s doesn't exist", resource));
+            throw new InvalidRequestException(String.format("Resource %s doesn't exist", resource));
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java
new file mode 100644
index 0000000..98c2b4e
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java
@@ -0,0 +1,40 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class RevokeRoleStatement extends RoleManagementStatement
+{
+    public RevokeRoleStatement(RoleName name, RoleName grantee)
+    {
+        super(name, grantee);
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        DatabaseDescriptor.getRoleManager().revokeRole(state.getUser(), role, grantee);
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java b/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java
index 6f8ccd1..7ce5259 100644
--- a/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -7,14 +7,13 @@
  * "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
+ *     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.
+ * 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.cql3.statements;
 
@@ -23,6 +22,7 @@ import java.util.Set;
 import org.apache.cassandra.auth.DataResource;
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
@@ -30,14 +30,14 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 
 public class RevokeStatement extends PermissionAlteringStatement
 {
-    public RevokeStatement(Set<Permission> permissions, DataResource resource, String username)
+    public RevokeStatement(Set<Permission> permissions, DataResource resource, RoleName grantee)
     {
-        super(permissions, resource, username);
+        super(permissions, resource, grantee);
     }
 
     public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
     {
-        DatabaseDescriptor.getAuthorizer().revoke(state.getUser(), permissions, resource, username);
+        DatabaseDescriptor.getAuthorizer().revoke(state.getUser(), permissions, resource, grantee);
         return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java b/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java
new file mode 100644
index 0000000..d67b42c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java
@@ -0,0 +1,54 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.service.ClientState;
+
+public abstract class RoleManagementStatement extends AuthorizationStatement
+{
+    protected final String role;
+    protected final String grantee;
+
+    public RoleManagementStatement(RoleName name, RoleName grantee)
+    {
+        this.role = name.getName();
+        this.grantee = grantee.getName();
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        if (!state.getUser().isSuper())
+            throw new UnauthorizedException("Only superusers are allowed to perform role management queries");
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        state.ensureNotAnonymous();
+
+        if (!DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            throw new InvalidRequestException(String.format("%s doesn't exist", role));
+
+        if (!DatabaseDescriptor.getRoleManager().isExistingRole(grantee))
+            throw new InvalidRequestException(String.format("%s doesn't exist", grantee));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java
index 136c8dc..5ba0a96 100644
--- a/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java
@@ -21,18 +21,13 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.cassandra.auth.IAuthenticator;
+import java.util.*;
+import java.util.concurrent.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.auth.PasswordAuthenticator;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -46,8 +41,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.Progressable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public abstract class AbstractBulkRecordWriter<K, V> extends RecordWriter<K, V>
 implements org.apache.hadoop.mapred.RecordWriter<K, V>
@@ -191,8 +184,8 @@ implements org.apache.hadoop.mapred.RecordWriter<K, V>
                     if (username != null)
                     {
                         Map<String, String> creds = new HashMap<String, String>();
-                        creds.put(IAuthenticator.USERNAME_KEY, username);
-                        creds.put(IAuthenticator.PASSWORD_KEY, password);
+                        creds.put(PasswordAuthenticator.USERNAME_KEY, username);
+                        creds.put(PasswordAuthenticator.PASSWORD_KEY, password);
                         AuthenticationRequest authRequest = new AuthenticationRequest(creds);
                         client.login(authRequest);
                     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
index f4ad40f..6fe2239 100644
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@ -20,36 +20,22 @@ package org.apache.cassandra.hadoop;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.*;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.*;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.auth.PasswordAuthenticator;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.thrift.AuthenticationRequest;
-import org.apache.cassandra.thrift.Cassandra;
-import org.apache.cassandra.thrift.CfSplit;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.KeyRange;
-import org.apache.cassandra.thrift.TokenRange;
-import org.apache.commons.lang3.StringUtils;
+import org.apache.cassandra.thrift.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.*;
 import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -106,8 +92,8 @@ public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<
         if ((ConfigHelper.getInputKeyspaceUserName(conf) != null) && (ConfigHelper.getInputKeyspacePassword(conf) != null))
         {
             Map<String, String> creds = new HashMap<String, String>();
-            creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
-            creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
+            creds.put(PasswordAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
+            creds.put(PasswordAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
             AuthenticationRequest authRequest = new AuthenticationRequest(creds);
             client.login(authRequest);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
index f574641..03d0045 100644
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
@@ -25,8 +25,9 @@ import java.util.Map;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.auth.PasswordAuthenticator;
+import org.apache.cassandra.thrift.AuthenticationRequest;
+import org.apache.cassandra.thrift.Cassandra;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.*;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -134,8 +135,8 @@ public abstract class AbstractColumnFamilyOutputFormat<K, Y> extends OutputForma
     public static void login(String user, String password, Cassandra.Client client) throws Exception
     {
         Map<String, String> creds = new HashMap<String, String>();
-        creds.put(IAuthenticator.USERNAME_KEY, user);
-        creds.put(IAuthenticator.PASSWORD_KEY, password);
+        creds.put(PasswordAuthenticator.USERNAME_KEY, user);
+        creds.put(PasswordAuthenticator.PASSWORD_KEY, password);
         AuthenticationRequest authRequest = new AuthenticationRequest(creds);
         client.login(authRequest);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/879b694d/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
index 0ffd442..447c8ce 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@ -25,27 +25,28 @@ import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.schema.LegacySchemaTables;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.auth.IAuthenticator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.auth.PasswordAuthenticator;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.db.marshal.AbstractCompositeType.CompositeComponent;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.serializers.CollectionSerializer;
-import org.apache.cassandra.hadoop.*;
 import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Hex;
-import org.apache.cassandra.utils.UUIDGen;
-
+import org.apache.cassandra.utils.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.pig.*;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.*;
@@ -54,8 +55,6 @@ import org.apache.thrift.TDeserializer;
 import org.apache.thrift.TException;
 import org.apache.thrift.TSerializer;
 import org.apache.thrift.protocol.TBinaryProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * A LoadStoreFunc for retrieving data from and storing data to Cassandra
@@ -505,8 +504,8 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
                 if (username != null && password != null)
                 {
                     Map<String, String> credentials = new HashMap<String, String>(2);
-                    credentials.put(IAuthenticator.USERNAME_KEY, username);
-                    credentials.put(IAuthenticator.PASSWORD_KEY, password);
+                    credentials.put(PasswordAuthenticator.USERNAME_KEY, username);
+                    credentials.put(PasswordAuthenticator.PASSWORD_KEY, password);
 
                     try
                     {