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 2012/12/08 16:27:46 UTC

[1/2] git commit: Update IAuthenticator to match the new IAuthorizer; patch by Aleksey Yeschenko, reviewed by Jonathan Ellis for CASSANDRA-5003

Updated Branches:
  refs/heads/cassandra-1.2.0 fbf9f5533 -> bddfa9e12


Update IAuthenticator to match the new IAuthorizer;
patch by Aleksey Yeschenko, reviewed by Jonathan Ellis for CASSANDRA-5003


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

Branch: refs/heads/cassandra-1.2.0
Commit: bddfa9e120bdafc204bde97bb51de1f86bf9695b
Parents: fbf9f55
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Sat Dec 8 18:18:22 2012 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Sat Dec 8 18:21:14 2012 +0300

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 NEWS.txt                                           |   14 ++
 doc/native_protocol.spec                           |    2 +
 .../simple_authentication/conf/passwd.properties   |    5 +-
 .../apache/cassandra/auth/SimpleAuthenticator.java |   24 +--
 pylib/cqlshlib/cql3handling.py                     |   42 ++++-
 .../cassandra/auth/AllowAllAuthenticator.java      |   50 +++++-
 .../apache/cassandra/auth/AllowAllAuthorizer.java  |   18 +-
 src/java/org/apache/cassandra/auth/Auth.java       |  131 +++++++++++++++
 .../apache/cassandra/auth/AuthenticatedUser.java   |   38 +++-
 .../org/apache/cassandra/auth/IAuthenticator.java  |   96 ++++++++++-
 .../org/apache/cassandra/auth/IAuthorizer.java     |   18 +-
 src/java/org/apache/cassandra/auth/IResource.java  |    8 +-
 .../apache/cassandra/auth/LegacyAuthenticator.java |   92 ++++++++++
 .../apache/cassandra/auth/LegacyAuthorizer.java    |   15 +-
 .../org/apache/cassandra/config/CFMetaData.java    |    6 +
 .../cassandra/config/DatabaseDescriptor.java       |    4 +-
 .../org/apache/cassandra/config/KSMetaData.java    |    7 +
 src/java/org/apache/cassandra/config/Schema.java   |    5 +-
 src/java/org/apache/cassandra/cql3/Cql.g           |   88 +++++++++--
 .../org/apache/cassandra/cql3/QueryProcessor.java  |   21 +++
 .../org/apache/cassandra/cql3/UserOptions.java     |   62 +++++++
 .../cql3/statements/AlterUserStatement.java        |   86 ++++++++++
 .../cql3/statements/AuthenticationStatement.java   |   57 +++++++
 .../cql3/statements/AuthorizationStatement.java    |   10 +-
 .../cql3/statements/CreateUserStatement.java       |   64 +++++++
 .../cql3/statements/DropUserStatement.java         |   62 +++++++
 .../cassandra/cql3/statements/GrantStatement.java  |    7 +-
 .../cql3/statements/ListPermissionsStatement.java  |   25 ++-
 .../cql3/statements/ListUsersStatement.java        |   48 ++++++
 .../statements/PermissionAlteringStatement.java    |    7 +-
 .../cassandra/cql3/statements/RevokeStatement.java |    7 +-
 .../exceptions/AuthenticationException.java        |   26 +++
 .../apache/cassandra/exceptions/ExceptionCode.java |    2 +
 .../apache/cassandra/service/CassandraDaemon.java  |    8 +-
 .../org/apache/cassandra/service/ClientState.java  |   60 +++----
 .../apache/cassandra/thrift/CassandraServer.java   |    9 +-
 .../apache/cassandra/thrift/ThriftConversion.java  |    5 +
 .../transport/messages/CredentialsMessage.java     |    3 +-
 .../cassandra/transport/messages/ErrorMessage.java |    3 +
 .../transport/messages/StartupMessage.java         |    6 +-
 41 files changed, 1079 insertions(+), 163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 77a488c..48ebb87 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -6,6 +6,7 @@
  * Fix preparing updates with collections (CASSANDRA-5017)
  * Don't generate UUID based on other node address (CASSANDRA-5002)
  * Fix message when trying to alter a clustering key type (CASSANDRA-5012)
+ * Update IAuthenticator to match the new IAuthorizer (CASSANDRA-5003)
 Merged from 1.1
  * Improve schema propagation performance (CASSANDRA-5025)
  * Fix for IndexHelper.IndexFor throws OOB Exception (CASSANDRA-5030)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index bc361c5..b834923 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -14,6 +14,15 @@ by version X, but the inverse is not necessarily the case.)
 
 Upgrading
 ---------
+    - IAuthenticator interface has been updated to support dynamic
+      user creation, modification and removal. Users, even when stored
+      externally, now have to be explicitly created using
+      CREATE USER query first. AllowAllAuthenticator and SimpleAuthenticator
+      have been updated for the new interface, but you'll have to update
+      your old IAuthenticator implementations for 1.2. To ease this process,
+      a new abstract LegacyAuthenticator class has been added - subclass it
+      in your old IAuthenticator implementaion and everything should just work
+      (this only affects users who implemented custom authenticators).
     - IAuthority interface has been deprecated in favor of IAuthorizer.
       AllowAllAuthority and SimpleAuthority have been renamed to
       AllowAllAuthorizer and SimpleAuthorizer, respectively. In order to
@@ -107,6 +116,11 @@ Features
       GRANT, REVOKE, LIST PERMISSIONS. A native implementation storing
       the permissions in Cassandra is being worked on and we expect to
       include it in 1.2.1 or 1.2.2.
+    - IAuthenticator interface has been updated to support dynamic user
+      creation, modification and removal via new CQL3 statements:
+      CREATE USER, ALTER USER, DROP USER, LIST USERS. A native implementation
+      that stores users in Cassandra itself is being worked on and is expected to
+      become part of 1.2.1 or 1.2.2.
 
 
 1.1.5

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/doc/native_protocol.spec
----------------------------------------------------------------------
diff --git a/doc/native_protocol.spec b/doc/native_protocol.spec
index ab44b21..7d0d07b 100644
--- a/doc/native_protocol.spec
+++ b/doc/native_protocol.spec
@@ -550,6 +550,8 @@ Table of Contents
     0x000A    Protocol error: some client message triggered a protocol
               violation (for instance a QUERY message is sent before a STARTUP
               one has been sent)
+    0x0100    Bad credentials: CREDENTIALS request failed because Cassandra
+              did not accept the provided credentials.
 
     0x1000    Unavailable exception. The rest of the ERROR message body will be
                 <cl><required><alive>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/examples/simple_authentication/conf/passwd.properties
----------------------------------------------------------------------
diff --git a/examples/simple_authentication/conf/passwd.properties b/examples/simple_authentication/conf/passwd.properties
index 96a8ec9..3099ba6 100644
--- a/examples/simple_authentication/conf/passwd.properties
+++ b/examples/simple_authentication/conf/passwd.properties
@@ -17,7 +17,8 @@
 # This is a sample password file for SimpleAuthenticator. The format of
 # this file is username=password. If -Dpasswd.mode=MD5 then the password
 # is represented as an md5 digest, otherwise it is cleartext (keep this
-# in mind when setting file mode and ownership).
+# in mind when setting file mode and ownership). 'cassandra' is the default
+# superuser and can be removed later.
+cassandra=cassandra
 jsmith=havebadpass
-Elvis\ Presley=graceland4evar
 dilbert=nomoovertime

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/examples/simple_authentication/src/org/apache/cassandra/auth/SimpleAuthenticator.java
----------------------------------------------------------------------
diff --git a/examples/simple_authentication/src/org/apache/cassandra/auth/SimpleAuthenticator.java b/examples/simple_authentication/src/org/apache/cassandra/auth/SimpleAuthenticator.java
index 357a465..0248244 100644
--- a/examples/simple_authentication/src/org/apache/cassandra/auth/SimpleAuthenticator.java
+++ b/examples/simple_authentication/src/org/apache/cassandra/auth/SimpleAuthenticator.java
@@ -29,23 +29,21 @@ import java.security.MessageDigest;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.thrift.AuthenticationException;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Hex;
 
-public class SimpleAuthenticator implements IAuthenticator
+public class SimpleAuthenticator extends LegacyAuthenticator
 {
     public final static String PASSWD_FILENAME_PROPERTY        = "passwd.properties";
     public final static String PMODE_PROPERTY                  = "passwd.mode";
-    public static final String USERNAME_KEY                    = "username";
-    public static final String PASSWORD_KEY                    = "password";
 
     public enum PasswordMode
     {
         PLAIN, MD5,
-    };
+    }
 
     public AuthenticatedUser defaultUser()
     {
@@ -53,7 +51,7 @@ public class SimpleAuthenticator implements IAuthenticator
         return null;
     }
 
-    public AuthenticatedUser authenticate(Map<? extends CharSequence,? extends CharSequence> credentials) throws AuthenticationException
+    public AuthenticatedUser authenticate(Map<String, String> credentials) throws AuthenticationException
     {
         String pmode_plain = System.getProperty(PMODE_PROPERTY);
         PasswordMode mode = PasswordMode.PLAIN;
@@ -78,19 +76,13 @@ public class SimpleAuthenticator implements IAuthenticator
 
         String pfilename = System.getProperty(PASSWD_FILENAME_PROPERTY);
 
-        String username = null;
-        CharSequence user = credentials.get(USERNAME_KEY);
-        if (user == null)
+        String username = credentials.get(USERNAME_KEY);
+        if (username == null)
             throw new AuthenticationException("Authentication request was missing the required key '" + USERNAME_KEY + "'");
-        else
-            username = user.toString();
 
-        String password = null;
-        CharSequence pass = credentials.get(PASSWORD_KEY);
-        if (pass == null)
+        String password = credentials.get(PASSWORD_KEY);
+        if (password == null)
             throw new AuthenticationException("Authentication request was missing the required key '" + PASSWORD_KEY + "'");
-        else
-            password = pass.toString();
 
         boolean authenticated = false;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index cbb2700..3c24114 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -257,6 +257,7 @@ JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
                   | <selectStatement>
                   | <dataChangeStatement>
                   | <schemaChangeStatement>
+                  | <authenticationStatement>
                   | <authorizationStatement>
                   ;
 
@@ -277,10 +278,16 @@ JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
                           | <alterKeyspaceStatement>
                           ;
 
-<authorizationStatement> ::= | <grantStatement>
-                             | <revokeStatement>
-                             | <listPermissionsStatement>
-                             ;
+<authenticationStatement> ::= <createUserStatement>
+                            | <alterUserStatement>
+                            | <dropUserStatement>
+                            | <listUsersStatement>
+                            ;
+
+<authorizationStatement> ::= <grantStatement>
+                           | <revokeStatement>
+                           | <listPermissionsStatement>
+                           ;
 
 # timestamp is included here, since it's also a keyword
 <simpleStorageType> ::= typename=( <identifier> | <stringLiteral> | <K_TIMESTAMP> ) ;
@@ -1240,6 +1247,27 @@ syntax_rules += r'''
 '''
 
 syntax_rules += r'''
+<username> ::= user=( <identifier> | <stringLiteral> )
+             ;
+
+<createUserStatement> ::= "CREATE" "USER" <username>
+                              ( "WITH" "PASSWORD" <stringLiteral> )?
+                              ( "SUPERUSER" | "NOSUPERUSER" )?
+                        ;
+
+<alterUserStatement> ::= "ALTER" "USER" <username>
+                              ( "WITH" "PASSWORD" <stringLiteral> )?
+                              ( "SUPERUSER" | "NOSUPERUSER" )?
+                       ;
+
+<dropUserStatement> ::= "DROP" "USER" <username>
+                      ;
+
+<listUsersStatement> ::= "LIST" "USERS"
+                       ;
+'''
+
+syntax_rules += r'''
 <grantStatement> ::= "GRANT" <permissionExpr> "ON" <resource> "TO" <username>
                    ;
 
@@ -1269,14 +1297,12 @@ syntax_rules += r'''
                  | ( "KEYSPACE" <nonSystemKeyspaceName> )
                  | ( "TABLE"? <columnFamilyName> )
                  ;
-
-<username> ::= user=( <identifier> | <stringLiteral> )
-             ;
 '''
 
+
 @completer_for('username', 'user')
 def username_user_completer(ctxt, cass):
-    # TODO: implement user autocompletion
+    # TODO: implement user autocompletion for grant/revoke/list/drop user/alter user
     # with I could see a way to do this usefully, but I don't. I don't know
     # how any Authorities other than AllowAllAuthorizer work :/
     return [Hint('<username>')]

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/auth/AllowAllAuthenticator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AllowAllAuthenticator.java b/src/java/org/apache/cassandra/auth/AllowAllAuthenticator.java
index cd93dd0..def6045 100644
--- a/src/java/org/apache/cassandra/auth/AllowAllAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/AllowAllAuthenticator.java
@@ -17,31 +17,61 @@
  */
 package org.apache.cassandra.auth;
 
+import java.util.Collections;
 import java.util.Map;
+import java.util.Set;
 
+import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 
 public class AllowAllAuthenticator implements IAuthenticator
 {
-    private final static AuthenticatedUser DEFAULT_USER = new AuthenticatedUser("nobody");
+    public boolean requireAuthentication()
+    {
+        return false;
+    }
+
+    public Set<Option> supportedOptions()
+    {
+        return Collections.emptySet();
+    }
 
-    public AuthenticatedUser defaultUser()
+    public Set<Option> alterableOptions()
     {
-        return DEFAULT_USER;
+        return Collections.emptySet();
     }
 
-    public AuthenticatedUser authenticate(Map<? extends CharSequence,? extends CharSequence> credentials) throws AuthenticationException
+    public AuthenticatedUser authenticate(Map<String, String> credentials) throws AuthenticationException
     {
+        return AuthenticatedUser.ANONYMOUS_USER;
+    }
 
-        CharSequence username = credentials.get(IAuthenticator.USERNAME_KEY);
-        if (username == null)
-            return DEFAULT_USER;
-        return new AuthenticatedUser((String)username);
+    public void create(String username, Map<Option, Object> options) throws InvalidRequestException
+    {
+        throw new InvalidRequestException("CREATE USER operation is not supported by AllowAllAuthenticator");
+    }
+
+    public void alter(String username, Map<Option, Object> options) throws InvalidRequestException
+    {
+        throw new InvalidRequestException("ALTER USER operation is not supported by AllowAllAuthenticator");
+    }
+
+    public void drop(String username) throws InvalidRequestException
+    {
+        throw new InvalidRequestException("DROP USER operation is not supported by AllowAllAuthenticator");
+    }
+
+    public Set<IResource> protectedResources()
+    {
+        return Collections.emptySet();
     }
 
     public void validateConfiguration() throws ConfigurationException
     {
-        // pass
+    }
+
+    public void setup()
+    {
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java b/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java
index 8b839ea..04b4b52 100644
--- a/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java
@@ -29,14 +29,6 @@ public class AllowAllAuthorizer implements IAuthorizer
         return Permission.ALL;
     }
 
-    public void validateConfiguration()
-    {
-    }
-
-    public void setup()
-    {
-    }
-
     public void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String to)
     throws InvalidRequestException
     {
@@ -57,7 +49,7 @@ public class AllowAllAuthorizer implements IAuthorizer
     {
     }
 
-    public Set<PermissionDetails> listPermissions(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
+    public Set<PermissionDetails> list(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
     throws InvalidRequestException
     {
         throw new InvalidRequestException("LIST PERMISSIONS operation is not supported by AllowAllAuthorizer");
@@ -67,4 +59,12 @@ public class AllowAllAuthorizer implements IAuthorizer
     {
         return Collections.emptySet();
     }
+
+    public void validateConfiguration()
+    {
+    }
+
+    public void setup()
+    {
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/auth/Auth.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/Auth.java b/src/java/org/apache/cassandra/auth/Auth.java
new file mode 100644
index 0000000..95312fe
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/Auth.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.auth;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang.StringUtils;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.WrappedRunnable;
+
+public class Auth
+{
+    public static final String DEFAULT_SUPERUSER_NAME = "cassandra";
+
+    public static final String AUTH_KS = "system_auth";
+    public static final String USERS_CF = "users";
+
+    /**
+     * Checks if the username is stored in AUTH_KS.USERS_CF.
+     *
+     * @param username Username to query.
+     * @return whether or not Cassandra knows about the user.
+     */
+    public static boolean isExistingUser(String username)
+    {
+        String query = String.format("SELECT * FROM %s.%s WHERE name = '%s'", AUTH_KS, USERS_CF, escape(username));
+        return !QueryProcessor.process(query).isEmpty();
+    }
+
+    /**
+     * Checks if the user is a known superuser.
+     *
+     * @param username Username to query.
+     * @return true is the user is a superuser, false if they aren't or don't exist at all.
+     */
+    public static boolean isSuperuser(String username)
+    {
+        String query = String.format("SELECT super FROM %s.%s WHERE name = '%s'", AUTH_KS, USERS_CF, escape(username));
+        UntypedResultSet result = QueryProcessor.process(query);
+        return !result.isEmpty() && result.one().getBoolean("super");
+    }
+
+    /**
+     * Inserts the user into AUTH_KS.USERS_CF (or overwrites their superuser status as a result of an ALTER USER query).
+     *
+     * @param username Username to insert.
+     * @param isSuper User's new status.
+     */
+    public static void insertUser(String username, boolean isSuper)
+    {
+        QueryProcessor.process(String.format("INSERT INTO %s.%s (name, super) VALUES ('%s', %s)",
+                                             AUTH_KS,
+                                             USERS_CF,
+                                             escape(username),
+                                             isSuper));
+    }
+
+    /**
+     * Deletes the user from AUTH_KS.USERS_CF.
+     *
+     * @param username Username to delete.
+     */
+    public static void deleteUser(String username)
+    {
+        QueryProcessor.process(String.format("DELETE FROM %s.%s WHERE name = '%s'",
+                                             AUTH_KS,
+                                             USERS_CF,
+                                             escape(username)));
+    }
+
+    /**
+     * Sets up Authenticator and Authorizer.
+     */
+    public static void setup()
+    {
+        authenticator().setup();
+        authorizer().setup();
+
+        // register a custom MigrationListener for permissions cleanup after dropped keyspaces/cfs.
+        MigrationManager.instance.register(new MigrationListener());
+
+        // schedule seeding a superuser in RING_DELAY milliseconds.
+        Runnable runnable = new WrappedRunnable()
+        {
+            public void runMayThrow() throws InvalidRequestException
+            {
+                // insert a default superuser if AUTH_KS.USERS_CF is empty.
+                if (QueryProcessor.process(String.format("SELECT * FROM %s.%s", AUTH_KS, USERS_CF)).isEmpty())
+                    insertUser(DEFAULT_SUPERUSER_NAME, true);
+            }
+        };
+        StorageService.tasks.schedule(runnable, StorageService.RING_DELAY, TimeUnit.MILLISECONDS);
+    }
+
+    // we only worry about one character ('). Make sure it's properly escaped.
+    private static String escape(String name)
+    {
+        return StringUtils.replace(name, "'", "''");
+    }
+
+    private static IAuthenticator authenticator()
+    {
+        return DatabaseDescriptor.getAuthenticator();
+    }
+
+    private static IAuthorizer authorizer()
+    {
+        return DatabaseDescriptor.getAuthorizer();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
index e65446d..cf208b8 100644
--- a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
+++ b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
@@ -17,30 +17,48 @@
  */
 package org.apache.cassandra.auth;
 
+/**
+ * Returned from IAuthenticator#authenticate(), represents an authenticated user everywhere internally.
+ */
 public class AuthenticatedUser
 {
-    public final String username;
-    private final boolean isSuperUser;
+    public static final String ANONYMOUS_USERNAME = "anonymous";
+    public static final AuthenticatedUser ANONYMOUS_USER = new AuthenticatedUser(ANONYMOUS_USERNAME);
+
+    private final String name;
+
+    public AuthenticatedUser(String name)
+    {
+        this.name = name;
+    }
 
-    public AuthenticatedUser(String username)
+    public String getName()
     {
-        this(username, false);
+        return name;
     }
 
-    public AuthenticatedUser(String username, boolean isSuperUser)
+    /**
+     * Checks the user's superuser status.
+     * Only a superuser is allowed to perform CREATE USER and DROP USER queries.
+     * Im most cased, though not necessarily, a superuser will have Permission.ALL on every resource
+     * (depends on IAuthorizer implementation).
+     */
+    public boolean isSuper()
     {
-        this.username = username;
-        this.isSuperUser = isSuperUser;
+        return !isAnonymous() && Auth.isSuperuser(name);
     }
 
-    public boolean isSuperUser()
+    /**
+     * If IAuthenticator doesn't require authentication, this method may return true.
+     */
+    public boolean isAnonymous()
     {
-        return isSuperUser;
+        return this == ANONYMOUS_USER;
     }
 
     @Override
     public String toString()
     {
-        return String.format("#<User %s super=%s>", username, isSuperUser);
+        return String.format("#<User %s>", name);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/auth/IAuthenticator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/IAuthenticator.java b/src/java/org/apache/cassandra/auth/IAuthenticator.java
index 4f50383..69a9e83 100644
--- a/src/java/org/apache/cassandra/auth/IAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/IAuthenticator.java
@@ -18,25 +18,103 @@
 package org.apache.cassandra.auth;
 
 import java.util.Map;
+import java.util.Set;
 
+import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 
 public interface IAuthenticator
 {
-    public static final String USERNAME_KEY = "username";
-    public static final String PASSWORD_KEY = "password";
+    static final String USERNAME_KEY = "username";
+    static final String PASSWORD_KEY = "password";
 
     /**
-     * @return The user that a connection is initialized with, or 'null' if a user must call login().
+     * Supported CREATE USER/ALTER USER options.
+     * Currently only PASSWORD is available.
      */
-    public AuthenticatedUser defaultUser();
+    enum Option
+    {
+        PASSWORD
+    }
 
     /**
-     * @param credentials An implementation specific collection of identifying information.
-     * @return A successfully authenticated user: should throw AuthenticationException rather than ever returning null.
+     * Whether or not the authenticator requires explicit login.
+     * If false will instantiate user with AuthenticatedUser.ANONYMOUS_USER.
      */
-    public AuthenticatedUser authenticate(Map<? extends CharSequence,? extends CharSequence> credentials) throws AuthenticationException;
+    boolean requireAuthentication();
 
-    public void validateConfiguration() throws ConfigurationException;
+    /**
+     * Set of options supported by CREATE USER and ALTER USER queries.
+     * Should never return null - always return an empty set instead.
+     */
+    Set<Option> supportedOptions();
+
+    /**
+     * Subset of supportedOptions that users are allowed to alter when performing ALTER USER [themselves].
+     * Should never return null - always return an empty set instead.
+     */
+    Set<Option> alterableOptions();
+
+    /**
+     * Authenticates a user given a Map<String, String> of credentials.
+     * Should never return null - always throw AuthenticationException instead.
+     * Returning AuthenticatedUser.ANONYMOUS_USER is an option as well if authentication is not required.
+     *
+     * @throws AuthenticationException if credentials don't match any known user.
+     */
+    AuthenticatedUser authenticate(Map<String, String> credentials) throws AuthenticationException;
+
+    /**
+     * Called during execution of CREATE USER query (also may be called on startup, see seedSuperuserOptions method).
+     * If authenticator is static then the body of the method should be left blank, but don't throw an exception.
+     * options are guaranteed to be a subset of supportedOptions().
+     *
+     * @param username Username of the user to create.
+     * @param options Options the user will be created with.
+     * @throws InvalidRequestException
+     */
+    void create(String username, Map<Option, Object> options) throws InvalidRequestException;
+
+    /**
+     * Called during execution of ALTER USER query.
+     * options are always guaranteed to be a subset of supportedOptions(). Furthermore, if the user performing the query
+     * is not a superuser and is altering himself, then options are guaranteed to be a subset of alterableOptions().
+     * Keep the body of the method blank if your implementation doesn't support any options.
+     *
+     * @param username Username of the user that will be altered.
+     * @param options Options to alter.
+     * @throws InvalidRequestException
+     */
+    void alter(String username, Map<Option, Object> options) throws InvalidRequestException;
+
+
+    /**
+     * Called during execution of DROP USER query.
+     *
+     * @param username Username of the user that will be dropped.
+     * @throws InvalidRequestException
+     */
+    void drop(String username) throws InvalidRequestException;
+
+     /**
+     * Set of resources that should be made inaccessible to users and only accessible internally.
+     *
+     * @return Keyspaces, column families that will be unreadable and unmodifiable by users; other resources.
+     */
+    Set<? extends IResource> protectedResources();
+
+    /**
+     * Validates configuration of IAuthenticator implementation (if configurable).
+     *
+     * @throws ConfigurationException when there is a configuration error.
+     */
+    void validateConfiguration() throws ConfigurationException;
+
+    /**
+     * Setup is called once upon system startup to initialize the IAuthenticator.
+     *
+     * For example, use this method to create any required keyspaces/column families.
+     */
+    void setup();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/auth/IAuthorizer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/IAuthorizer.java b/src/java/org/apache/cassandra/auth/IAuthorizer.java
index 916878b..de1b393 100644
--- a/src/java/org/apache/cassandra/auth/IAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/IAuthorizer.java
@@ -35,7 +35,7 @@ public interface IAuthorizer
      * @param resource Resource for which the authorization is being requested. @see DataResource.
      * @return Set of permissions of the user on the resource. Should never return null. Use Permission.NONE instead.
      */
-    public Set<Permission> authorize(AuthenticatedUser user, IResource resource);
+    Set<Permission> authorize(AuthenticatedUser user, IResource resource);
 
     /**
      * Grants a set of permissions on a resource to a user.
@@ -49,7 +49,7 @@ public interface IAuthorizer
      * @throws UnauthorizedException if the granting user isn't allowed to grant (and revoke) the permissions on the resource.
      * @throws InvalidRequestException upon parameter misconfiguration or internal error.
      */
-    public void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String to)
+    void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String to)
     throws UnauthorizedException, InvalidRequestException;
 
     /**
@@ -64,7 +64,7 @@ public interface IAuthorizer
      * @throws UnauthorizedException if the revoking user isn't allowed to revoke the permissions on the resource.
      * @throws InvalidRequestException upon parameter misconfiguration or internal error.
      */
-    public void revoke(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String from)
+    void revoke(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String from)
     throws UnauthorizedException, InvalidRequestException;
 
     /**
@@ -81,7 +81,7 @@ public interface IAuthorizer
      * @throws UnauthorizedException if the user isn't allowed to view the requested permissions.
      * @throws InvalidRequestException upon parameter misconfiguration or internal error.
      */
-    public Set<PermissionDetails> listPermissions(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
+    Set<PermissionDetails> list(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
     throws UnauthorizedException, InvalidRequestException;
 
     /**
@@ -90,33 +90,33 @@ public interface IAuthorizer
      *
      * @param droppedUser The user to revoke all permissions from.
      */
-    public void revokeAll(String droppedUser);
+    void revokeAll(String droppedUser);
 
     /**
      * This method is called after a resource is removed (i.e. keyspace or a table is dropped).
      *
      * @param droppedResource The resource to revoke all permissions on.
      */
-    public void revokeAll(IResource droppedResource);
+    void revokeAll(IResource droppedResource);
 
     /**
      * Set of resources that should be made inaccessible to users and only accessible internally.
      *
      * @return Keyspaces, column families that will be unreadable and unmodifiable by users; other resources.
      */
-    public Set<? extends IResource> protectedResources();
+    Set<? extends IResource> protectedResources();
 
     /**
      * Validates configuration of IAuthorizer implementation (if configurable).
      *
      * @throws ConfigurationException when there is a configuration error.
      */
-    public void validateConfiguration() throws ConfigurationException;
+    void validateConfiguration() throws ConfigurationException;
 
     /**
      * Setup is called once upon system startup to initialize the IAuthorizer.
      *
      * For example, use this method to create any required keyspaces/column families.
      */
-    public void setup();
+    void setup();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/auth/IResource.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/IResource.java b/src/java/org/apache/cassandra/auth/IResource.java
index 635b0a6..75f8d2a 100644
--- a/src/java/org/apache/cassandra/auth/IResource.java
+++ b/src/java/org/apache/cassandra/auth/IResource.java
@@ -29,14 +29,14 @@ public interface IResource
     /**
      * @return printable name of the resource.
      */
-    public String getName();
+    String getName();
 
     /**
      * Gets next resource in the hierarchy. Call hasParent first to make sure there is one.
      *
      * @return Resource parent (or IllegalStateException if there is none). Never a null.
      */
-    public IResource getParent();
+    IResource getParent();
 
     /**
      * Indicates whether or not this resource has a parent in the hierarchy.
@@ -44,10 +44,10 @@ public interface IResource
      * Please perform this check before calling getParent() method.
      * @return Whether or not the resource has a parent.
      */
-    public boolean hasParent();
+    boolean hasParent();
 
     /**
      * @return Whether or not this resource exists in Cassandra.
      */
-    public boolean exists();
+    boolean exists();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/auth/LegacyAuthenticator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/LegacyAuthenticator.java b/src/java/org/apache/cassandra/auth/LegacyAuthenticator.java
new file mode 100644
index 0000000..5210af5
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/LegacyAuthenticator.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.auth;
+
+import java.util.*;
+
+import org.apache.cassandra.exceptions.AuthenticationException;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Provides a transitional IAuthenticator implementation for old-style (pre-1.2) authenticators.
+ *
+ * Comes with default implementation for the all of the new methods.
+ * Subclass LegacyAuthenticator instead of implementing the old IAuthenticator and your old IAuthenticator
+ * implementation should continue to work.
+ */
+public abstract class LegacyAuthenticator implements IAuthenticator
+{
+    /**
+     * @return The user that a connection is initialized with, or 'null' if a user must call login().
+     */
+    public abstract AuthenticatedUser defaultUser();
+
+    /**
+     * @param credentials An implementation specific collection of identifying information.
+     * @return A successfully authenticated user: should throw AuthenticationException rather than ever returning null.
+     */
+    public abstract AuthenticatedUser authenticate(Map<String, String> credentials) throws AuthenticationException;
+
+    public abstract void validateConfiguration() throws ConfigurationException;
+
+    @Override
+    public boolean requireAuthentication()
+    {
+        return defaultUser() == null;
+    }
+
+    @Override
+    public Set<Option> supportedOptions()
+    {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public Set<Option> alterableOptions()
+    {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public void create(String username, Map<Option, Object> options) throws InvalidRequestException
+    {
+    }
+
+    @Override
+    public void alter(String username, Map<Option, Object> options) throws InvalidRequestException
+    {
+        throw new InvalidRequestException("ALTER USER operation is not supported by LegacyAuthenticator");
+    }
+
+    @Override
+    public void drop(String username) throws InvalidRequestException
+    {
+    }
+
+    @Override
+    public Set<IResource> protectedResources()
+    {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public void setup()
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/auth/LegacyAuthorizer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/LegacyAuthorizer.java b/src/java/org/apache/cassandra/auth/LegacyAuthorizer.java
index 99e2ba8..5948d34 100644
--- a/src/java/org/apache/cassandra/auth/LegacyAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/LegacyAuthorizer.java
@@ -19,13 +19,14 @@ package org.apache.cassandra.auth;
 
 import java.util.*;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 
 /**
  * Provides a transitional IAuthorizer implementation for old-style (pre-1.2) authorizers.
  *
- * Translates old-style authorze() calls to the new-style, expands Permission.READ and Permission.WRITE
+ * Translates old-style authorize() calls to the new-style, expands Permission.READ and Permission.WRITE
  * into the new Permission values, translates the new resource hierarchy into the old hierarchy.
  * Stubs the rest of the new methods.
  * Subclass LegacyAuthorizer instead of implementing the old IAuthority and your old IAuthority implementation should
@@ -40,10 +41,7 @@ public abstract class LegacyAuthorizer implements IAuthorizer
      */
     public abstract EnumSet<Permission> authorize(AuthenticatedUser user, List<Object> resource);
 
-    @Override
-    public void setup()
-    {
-    }
+    public abstract void validateConfiguration() throws ConfigurationException;
 
     /**
      * Translates new-style authorize() method call to the old-style (including permissions and the hierarchy).
@@ -97,7 +95,7 @@ public abstract class LegacyAuthorizer implements IAuthorizer
     }
 
     @Override
-    public Set<PermissionDetails> listPermissions(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
+    public Set<PermissionDetails> list(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
     throws InvalidRequestException, UnauthorizedException
     {
         throw new InvalidRequestException("LIST PERMISSIONS operation is not supported by LegacyAuthorizer");
@@ -108,4 +106,9 @@ public abstract class LegacyAuthorizer implements IAuthorizer
     {
         return Collections.emptySet();
     }
+
+    @Override
+    public void setup()
+    {
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index 7ad2dc4..9d2e013 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -32,6 +32,7 @@ import org.apache.commons.lang.builder.ToStringBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.cql3.CFDefinition;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -218,6 +219,11 @@ public final class CFMetaData
                                                               + "requested_at timestamp"
                                                               + ") WITH COMMENT='ranges requested for transfer here'");
 
+    public static final CFMetaData AuthUsersCf = compile(18, "CREATE TABLE " + Auth.USERS_CF + " ("
+                                                             + "name text PRIMARY KEY,"
+                                                             + "super boolean"
+                                                             + ");", Auth.AUTH_KS);
+
     public enum Caching
     {
         ALL, KEYS_ONLY, ROWS_ONLY, NONE;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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 f6095af..0a8d67c 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -458,7 +458,9 @@ public class DatabaseDescriptor
             }
 
             // Hardcoded system tables
-            List<KSMetaData> systemKeyspaces = Arrays.asList(KSMetaData.systemKeyspace(), KSMetaData.traceKeyspace());
+            List<KSMetaData> systemKeyspaces = Arrays.asList(KSMetaData.systemKeyspace(),
+                                                             KSMetaData.traceKeyspace(),
+                                                             KSMetaData.authKeyspace());
             assert systemKeyspaces.size() == Schema.systemKeyspaceNames.size();
             for (KSMetaData ksmd : systemKeyspaces)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/config/KSMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/KSMetaData.java b/src/java/org/apache/cassandra/config/KSMetaData.java
index 2d1b709..e5b349a 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap;
 import org.apache.commons.lang.ObjectUtils;
 import org.apache.commons.lang.StringUtils;
 
+import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.*;
@@ -101,6 +102,12 @@ public final class KSMetaData
         return new KSMetaData(Tracing.TRACE_KS, SimpleStrategy.class, ImmutableMap.of("replication_factor", "1"), true, cfDefs);
     }
 
+    public static KSMetaData authKeyspace()
+    {
+        List<CFMetaData> cfDefs = Arrays.asList(CFMetaData.AuthUsersCf);
+        return new KSMetaData(Auth.AUTH_KS, SimpleStrategy.class, ImmutableMap.of("replication_factor", "1"), true, cfDefs);
+    }
+
     public static KSMetaData testMetadata(String name, Class<? extends AbstractReplicationStrategy> strategyClass, Map<String, String> strategyOptions, CFMetaData... cfDefs)
     {
         return new KSMetaData(name, strategyClass, strategyOptions, true, Arrays.asList(cfDefs));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index 2f24d39..5076f47 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -24,16 +24,17 @@ import java.security.NoSuchAlgorithmException;
 import java.util.*;
 
 import com.google.common.collect.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.Table;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
@@ -66,7 +67,7 @@ public class Schema
 
     // 59adb24e-f3cd-3e02-97f0-5b395827453f
     public static final UUID emptyVersion;
-    public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Table.SYSTEM_KS, Tracing.TRACE_KS);
+    public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Table.SYSTEM_KS, Tracing.TRACE_KS, Auth.AUTH_KS);
 
     static
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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 acd30cc..98585c8 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -169,10 +169,14 @@ cqlStatement returns [ParsedStatement stmt]
     | st12=dropColumnFamilyStatement   { $stmt = st12; }
     | st13=dropIndexStatement          { $stmt = st13; }
     | st14=alterTableStatement         { $stmt = st14; }
-    | st15=grantStatement              { $stmt = st15; }
-    | st16=revokeStatement             { $stmt = st16; }
-    | st17=listPermissionsStatement    { $stmt = st17; }
-    | st18=alterKeyspaceStatement      { $stmt = st18; }
+    | st15=alterKeyspaceStatement      { $stmt = st15; }
+    | st16=grantStatement              { $stmt = st16; }
+    | st17=revokeStatement             { $stmt = st17; }
+    | st18=listPermissionsStatement    { $stmt = st18; }
+    | st19=createUserStatement         { $stmt = st19; }
+    | st20=alterUserStatement          { $stmt = st20; }
+    | st21=dropUserStatement           { $stmt = st21; }
+    | st22=listUsersStatement          { $stmt = st22; }
     ;
 
 /*
@@ -545,11 +549,6 @@ permissionOrAll returns [Set<Permission> perms]
     | p=permission ( K_PERMISSION )? { $perms = EnumSet.of($p.perm); }
     ;
 
-username
-    : IDENT
-    | STRING_LITERAL
-    ;
-
 resource returns [IResource res]
     : r=dataResource { $res = $r.res; }
     ;
@@ -561,6 +560,56 @@ dataResource returns [DataResource res]
       { $res = DataResource.columnFamily($cf.name.getKeyspace(), $cf.name.getColumnFamily()); }
     ;
 
+/**
+ * CREATE USER <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
+ */
+createUserStatement returns [CreateUserStatement stmt]
+    @init {
+        UserOptions opts = new UserOptions();
+        boolean superuser = false;
+    }
+    : K_CREATE K_USER username
+      ( K_WITH userOptions[opts] )?
+      ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )?
+      { $stmt = new CreateUserStatement($username.text, opts, superuser); }
+    ;
+
+/**
+ * ALTER USER <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
+ */
+alterUserStatement returns [AlterUserStatement stmt]
+    @init {
+        UserOptions opts = new UserOptions();
+        Boolean superuser = null;
+    }
+    : K_ALTER K_USER username
+      ( K_WITH userOptions[opts] )?
+      ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )?
+      { $stmt = new AlterUserStatement($username.text, opts, superuser); }
+    ;
+
+/**
+ * DROP USER <username>
+ */
+dropUserStatement returns [DropUserStatement stmt]
+    : K_DROP K_USER username { $stmt = new DropUserStatement($username.text); }
+    ;
+
+/**
+ * LIST USERS
+ */
+listUsersStatement returns [ListUsersStatement stmt]
+    : K_LIST K_USERS { $stmt = new ListUsersStatement(); }
+    ;
+
+userOptions[UserOptions opts]
+    : userOption[opts]
+    ;
+
+userOption[UserOptions opts]
+    : k=K_PASSWORD v=STRING_LITERAL { opts.put($k.text, $v.text); }
+    ;
+
 /** DEFINITIONS **/
 
 // Column Identifiers
@@ -767,6 +816,11 @@ collection_type returns [ParsedType pt]
         { try { $pt = ParsedType.Collection.set(t); } catch (InvalidRequestException e) { addRecognitionError(e.getMessage()); } }
     ;
 
+username
+    : IDENT
+    | STRING_LITERAL
+    ;
+
 unreserved_keyword returns [String str]
     : k=( K_KEY
         | K_CLUSTERING
@@ -784,6 +838,11 @@ unreserved_keyword returns [String str]
         | K_PERMISSIONS
         | K_KEYSPACES
         | K_ALL
+        | K_USER
+        | K_USERS
+        | K_SUPERUSER
+        | K_NOSUPERUSER
+        | K_PASSWORD
         ) { $str = $k.text; }
     | t=native_type { $str = t.toString(); }
     ;
@@ -835,18 +894,25 @@ K_ORDER:       O R D E R;
 K_BY:          B Y;
 K_ASC:         A S C;
 K_DESC:        D E S C;
+K_ALLOW:       A L L O W;
+K_FILTERING:   F I L T E R I N G;
+
 K_GRANT:       G R A N T;
 K_ALL:         A L L;
 K_PERMISSION:  P E R M I S S I O N;
 K_PERMISSIONS: P E R M I S S I O N S;
 K_OF:          O F;
 K_REVOKE:      R E V O K E;
-K_ALLOW:       A L L O W;
-K_FILTERING:   F I L T E R I N G;
 K_MODIFY:      M O D I F Y;
 K_AUTHORIZE:   A U T H O R I Z E;
 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_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_CLUSTERING:  C L U S T E R I N G;
 K_ASCII:       A S C I I;
 K_BIGINT:      B I G I N T;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 1d4ddfa..b68f3c2 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -140,6 +140,27 @@ public class QueryProcessor
         return processStatement(getStatement(queryString, queryState.getClientState()).statement, cl, queryState, Collections.<ByteBuffer>emptyList());
     }
 
+    public static UntypedResultSet process(String query)
+    {
+        try
+        {
+            QueryState state = new QueryState(new ClientState(true));
+            ResultMessage result = process(query, ConsistencyLevel.ONE, state);
+            if (result instanceof ResultMessage.Rows)
+                return new UntypedResultSet(((ResultMessage.Rows)result).result);
+            else
+                return null;
+        }
+        catch (RequestExecutionException e)
+        {
+            throw new RuntimeException(e);
+        }
+        catch (RequestValidationException e)
+        {
+            throw new AssertionError(e);
+        }
+    }
+
     public static UntypedResultSet processInternal(String query)
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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
new file mode 100644
index 0000000..701debd
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/UserOptions.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.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/bddfa9e1/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
new file mode 100644
index 0000000..61655db
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterUserStatement.java
@@ -0,0 +1,86 @@
+/*
+ * 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.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 InvalidRequestException
+    {
+        opts.validate();
+
+        if (superuser == null && opts.isEmpty())
+            throw new InvalidRequestException("ALTER USER can't be empty");
+
+        if (!Auth.isExistingUser(username))
+            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        state.validateLogin();
+        AuthenticatedUser user = state.getUser();
+
+        if (superuser != null && user.getName().equals(username))
+            throw new UnauthorizedException("You aren't allowed to alter your own superuser status");
+
+        if (superuser != null && !user.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 (!user.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 InvalidRequestException
+    {
+        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/bddfa9e1/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
new file mode 100644
index 0000000..64468af
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
@@ -0,0 +1,57 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public abstract class AuthenticationStatement extends ParsedStatement implements CQLStatement
+{
+    @Override
+    public Prepared prepare()
+    {
+        return new Prepared(this);
+    }
+
+    public int getBoundsTerms()
+    {
+        return 0;
+    }
+
+    public ResultMessage execute(ConsistencyLevel cl, QueryState state, List<ByteBuffer> variables)
+    throws RequestExecutionException, RequestValidationException
+    {
+        return execute(state.getClientState());
+    }
+
+    public abstract ResultMessage execute(ClientState state) throws RequestExecutionException, RequestValidationException;
+
+    public ResultMessage executeInternal(QueryState state)
+    {
+        // executeInternal is for local query only, thus altering users doesn't make sense and is not supported
+        throw new UnsupportedOperationException();
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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 fefcd95..af1bd17 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
@@ -41,15 +41,13 @@ public abstract class AuthorizationStatement extends ParsedStatement implements
         return 0;
     }
 
-    public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
-    {}
-
-    public ResultMessage execute(ConsistencyLevel cl, QueryState state, List<ByteBuffer> variables) throws UnauthorizedException, InvalidRequestException
+    public ResultMessage execute(ConsistencyLevel cl, QueryState state, List<ByteBuffer> variables)
+    throws RequestValidationException, RequestExecutionException
     {
-        return execute(state.getClientState(), variables);
+        return execute(state.getClientState());
     }
 
-    public abstract ResultMessage execute(ClientState state, List<ByteBuffer> variables) throws UnauthorizedException, InvalidRequestException;
+    public abstract ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException;
 
     public ResultMessage executeInternal(QueryState state)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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
new file mode 100644
index 0000000..6dd1b93
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateUserStatement.java
@@ -0,0 +1,64 @@
+/*
+ * 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.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;
+
+    public CreateUserStatement(String username, UserOptions opts, boolean superuser)
+    {
+        this.username = username;
+        this.opts = opts;
+        this.superuser = superuser;
+    }
+
+    public void validate(ClientState state) throws InvalidRequestException
+    {
+        if (username.isEmpty())
+            throw new InvalidRequestException("Username can't be an empty string");
+        opts.validate();
+        if (Auth.isExistingUser(username))
+            throw new InvalidRequestException(String.format("User %s already exists", username));
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        state.validateLogin();
+
+        if (!state.getUser().isSuper())
+            throw new UnauthorizedException("Only superusers are allowed to perfrom CREATE USER queries");
+    }
+
+    public ResultMessage execute(ClientState state) throws InvalidRequestException
+    {
+        DatabaseDescriptor.getAuthenticator().create(username, opts.getOptions());
+        Auth.insertUser(username, superuser);
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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
new file mode 100644
index 0000000..af92fda
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/DropUserStatement.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.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.UnauthorizedException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class DropUserStatement extends AuthenticationStatement
+{
+    private final String username;
+
+    public DropUserStatement(String username)
+    {
+        this.username = username;
+    }
+
+    public void validate(ClientState state) throws InvalidRequestException
+    {
+        if (!Auth.isExistingUser(username))
+            throw new InvalidRequestException(String.format("User %s doesn't exists", 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
+    {
+        state.validateLogin();
+        if (!state.getUser().isSuper())
+            throw new UnauthorizedException("Only superusers are allowed to perfrom DROP USER queries");
+    }
+
+    public ResultMessage execute(ClientState state) throws InvalidRequestException
+    {
+        // 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/bddfa9e1/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 1264def..f4f35da 100644
--- a/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java
@@ -18,12 +18,11 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import java.nio.ByteBuffer;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.service.ClientState;
@@ -36,9 +35,9 @@ public class GrantStatement extends PermissionAlteringStatement
         super(permissions, resource, username);
     }
 
-    public ResultMessage execute(ClientState state, List<ByteBuffer> variables) throws UnauthorizedException, InvalidRequestException
+    public ResultMessage execute(ClientState state) throws UnauthorizedException, InvalidRequestException
     {
-        state.grantPermission(permissions, resource, username);
+        DatabaseDescriptor.getAuthorizer().grant(state.getUser(), permissions, resource, username);
         return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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 7b5d34a..2b9624d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
@@ -17,10 +17,10 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import java.nio.ByteBuffer;
 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;
@@ -32,7 +32,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 
 public class ListPermissionsStatement extends AuthorizationStatement
 {
-    private static final String KS = "auth"; // virtual keyspace to use for now.
+    private static final String KS = Auth.AUTH_KS;
     private static final String CF = "permissions"; // virtual cf to use for now.
 
     private static final List<ColumnSpecification> metadata;
@@ -59,9 +59,12 @@ public class ListPermissionsStatement extends AuthorizationStatement
         this.recursive = recursive;
     }
 
-    // TODO: user existence check (when IAuthenticator rewrite is done)
     public void validate(ClientState state) throws InvalidRequestException
     {
+        // a check to ensure the existence of the user isn't being leaked by user existence check.
+        if (username != null && !Auth.isExistingUser(username))
+            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
+
         if (resource != null)
         {
             resource = maybeCorrectResource(resource, state);
@@ -70,19 +73,24 @@ public class ListPermissionsStatement extends AuthorizationStatement
         }
     }
 
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        state.ensureNotAnonymous();
+    }
+
     // TODO: Create a new ResultMessage type (?). Rows will do for now.
-    public ResultMessage execute(ClientState state, List<ByteBuffer> variables) throws UnauthorizedException, InvalidRequestException
+    public ResultMessage execute(ClientState state) throws UnauthorizedException, InvalidRequestException
     {
         List<PermissionDetails> details = new ArrayList<PermissionDetails>();
 
         if (resource != null && recursive)
         {
             for (IResource r : Resources.chain(resource))
-                details.addAll(state.listPermissions(permissions, r, username));
+                details.addAll(list(state, r));
         }
         else
         {
-            details.addAll(state.listPermissions(permissions, resource, username));
+            details.addAll(list(state, resource));
         }
 
         Collections.sort(details);
@@ -103,4 +111,9 @@ public class ListPermissionsStatement extends AuthorizationStatement
         }
         return new ResultMessage.Rows(result);
     }
+
+    private Set<PermissionDetails> list(ClientState state, IResource resource) throws UnauthorizedException, InvalidRequestException
+    {
+        return DatabaseDescriptor.getAuthorizer().list(state.getUser(), permissions, resource, username);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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
new file mode 100644
index 0000000..da6cb54
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
@@ -0,0 +1,48 @@
+/*
+ * 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.cql3.QueryProcessor;
+import org.apache.cassandra.db.ConsistencyLevel;
+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.service.QueryState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class ListUsersStatement extends AuthenticationStatement
+{
+    public void validate(ClientState state)
+    {
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        state.ensureNotAnonymous();
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        return QueryProcessor.process(String.format("SELECT * FROM %s.%s", Auth.AUTH_KS, Auth.USERS_CF),
+                                      ConsistencyLevel.ONE,
+                                      new QueryState(new ClientState(true)));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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 6e7af77..b7d899d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java
@@ -19,6 +19,7 @@ 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;
@@ -39,7 +40,7 @@ public abstract class PermissionAlteringStatement extends AuthorizationStatement
         this.username = username;
     }
 
-    public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
+    public void checkAccess(ClientState state) throws UnauthorizedException
     {
         // check that the user has AUTHORIZE permission on the resource or its parents, otherwise reject GRANT/REVOKE.
         state.ensureHasPermission(Permission.AUTHORIZE, resource);
@@ -48,9 +49,11 @@ public abstract class PermissionAlteringStatement extends AuthorizationStatement
             state.ensureHasPermission(p, resource);
     }
 
-    // TODO: user existence check (when IAuthenticator rewrite is done)
     public void validate(ClientState state) throws InvalidRequestException
     {
+        if (!Auth.isExistingUser(username))
+            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
+
         // if a keyspace is omitted when GRANT/REVOKE ON TABLE <table>, we need to correct the resource.
         resource = maybeCorrectResource(resource, state);
         if (!resource.exists())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/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 23d7d55..2b10342 100644
--- a/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java
@@ -18,12 +18,11 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import java.nio.ByteBuffer;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.service.ClientState;
@@ -36,9 +35,9 @@ public class RevokeStatement extends PermissionAlteringStatement
         super(permissions, resource, username);
     }
 
-    public ResultMessage execute(ClientState state, List<ByteBuffer> variables) throws UnauthorizedException, InvalidRequestException
+    public ResultMessage execute(ClientState state) throws UnauthorizedException, InvalidRequestException
     {
-        state.revokePermission(permissions, resource, username);
+        DatabaseDescriptor.getAuthorizer().revoke(state.getUser(), permissions, resource, username);
         return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/exceptions/AuthenticationException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/AuthenticationException.java b/src/java/org/apache/cassandra/exceptions/AuthenticationException.java
new file mode 100644
index 0000000..ce6cb2c
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/AuthenticationException.java
@@ -0,0 +1,26 @@
+/*
+ * 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.exceptions;
+
+public class AuthenticationException extends RequestValidationException
+{
+    public AuthenticationException(String msg)
+    {
+        super(ExceptionCode.BAD_CREDENTIALS, msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/ExceptionCode.java b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
index e8dfb4e..ce082a7 100644
--- a/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
+++ b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
@@ -30,6 +30,8 @@ public enum ExceptionCode
     SERVER_ERROR    (0x0000),
     PROTOCOL_ERROR  (0x000A),
 
+    BAD_CREDENTIALS (0x0100),
+
     // 1xx: problem during request execution
     UNAVAILABLE     (0x1000),
     OVERLOADED      (0x1001),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bddfa9e1/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 1d66d9a..8aeffd8 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -31,6 +31,7 @@ import org.apache.log4j.PropertyConfigurator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.auth.Auth;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -219,11 +220,8 @@ public class CassandraDaemon
             System.exit(100);
         }
 
-        // TODO: setup authenticator
-        // setup Authorizer.
-        DatabaseDescriptor.getAuthorizer().setup();
-        // register a custom MigrationListener for permissions cleanup after dropped keyspaces/cfs.
-        MigrationManager.instance.register(new org.apache.cassandra.auth.MigrationListener());
+        // setup Authenticator and Authorizer.
+        Auth.setup();
 
         // clean up debris in the rest of the tables
         for (String table : Schema.instance.getTables())