You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sa...@apache.org on 2016/05/02 13:32:46 UTC

[1/2] cassandra git commit: Integrated JMX Authentication and Authorization

Repository: cassandra
Updated Branches:
  refs/heads/trunk ad7e36b8a -> 7b0c7164a


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index d7cda95..8640b58 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -309,7 +309,7 @@ public class NodeTool
                     nodeClient = new NodeProbe(host, parseInt(port));
                 else
                     nodeClient = new NodeProbe(host, parseInt(port), username, password);
-            } catch (IOException e)
+            } catch (IOException | SecurityException e)
             {
                 Throwable rootCause = Throwables.getRootCause(e);
                 System.err.println(format("nodetool: Failed to connect to '%s:%s' - %s: '%s'.", host, port, rootCause.getClass().getSimpleName(), rootCause.getMessage()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/utils/JMXServerUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/JMXServerUtils.java b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
new file mode 100644
index 0000000..b0e44a2
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
@@ -0,0 +1,299 @@
+/*
+ * 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.utils;
+
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Proxy;
+import java.net.InetAddress;
+import java.rmi.NoSuchObjectException;
+import java.rmi.Remote;
+import java.rmi.RemoteException;
+import java.rmi.registry.LocateRegistry;
+import java.rmi.server.RMIClientSocketFactory;
+import java.rmi.server.RMIServerSocketFactory;
+import java.rmi.server.UnicastRemoteObject;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.management.remote.*;
+import javax.management.remote.rmi.RMIConnectorServer;
+import javax.management.remote.rmi.RMIJRMPServerImpl;
+import javax.rmi.ssl.SslRMIClientSocketFactory;
+import javax.rmi.ssl.SslRMIServerSocketFactory;
+import javax.security.auth.Subject;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.sun.jmx.remote.internal.RMIExporter;
+import com.sun.jmx.remote.security.JMXPluggableAuthenticator;
+import org.apache.cassandra.auth.jmx.AuthenticationProxy;
+import sun.rmi.server.UnicastServerRef2;
+
+public class JMXServerUtils
+{
+    private static final Logger logger = LoggerFactory.getLogger(JMXServerUtils.class);
+
+
+    /**
+     * Creates a server programmatically. This allows us to set parameters which normally are
+     * inaccessable.
+     */
+    public static JMXConnectorServer createJMXServer(int port, boolean local)
+    throws IOException
+    {
+        Map<String, Object> env = new HashMap<>();
+
+        String urlTemplate = "service:jmx:rmi://%1$s/jndi/rmi://%1$s:%2$d/jmxrmi";
+        String url;
+        String host;
+        InetAddress serverAddress;
+        if (local)
+        {
+            serverAddress = InetAddress.getLoopbackAddress();
+            host = serverAddress.getHostAddress();
+            System.setProperty("java.rmi.server.hostname", host);
+        }
+        else
+        {
+            // if the java.rmi.server.hostname property is set, we'll take its value
+            // and use that when creating the RMIServerSocket to which we bind the RMI
+            // registry. This allows us to effectively restrict to a single interface
+            // if required. See http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4880793
+            // for more detail. If the hostname property is not set, the registry will
+            // be bound to the wildcard address
+            host = System.getProperty("java.rmi.server.hostname");
+            serverAddress = host == null ? null : InetAddress.getByName(host);
+        }
+
+        // Configure the RMI client & server socket factories, including SSL config.
+        env.putAll(configureJmxSocketFactories(serverAddress));
+
+        url = String.format(urlTemplate, (host == null ? "0.0.0.0" : serverAddress.getHostAddress()), port);
+        LocateRegistry.createRegistry(port,
+                                     (RMIClientSocketFactory) env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE),
+                                     (RMIServerSocketFactory) env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE));
+
+        // Configure authn, using a JMXAuthenticator which either wraps a set log LoginModules configured
+        // via a JAAS configuration entry, or one which delegates to the standard file based authenticator.
+        // Authn is disabled if com.sun.management.jmxremote.authenticate=false
+        env.putAll(configureJmxAuthentication());
+
+        // Configure authz - if a custom proxy class is specified an instance will be returned.
+        // If not, but a location for the standard access file is set in system properties, the
+        // return value is null, and an entry is added to the env map detailing that location
+        // If neither method is specified, no access control is applied
+        MBeanServerForwarder authzProxy = configureJmxAuthorization(env);
+
+        // Make sure we use our custom exporter so a full GC doesn't get scheduled every
+        // sun.rmi.dgc.server.gcInterval millis (default is 3600000ms/1 hour)
+        env.put(RMIExporter.EXPORTER_ATTRIBUTE, new Exporter());
+
+        JMXConnectorServer jmxServer =
+            JMXConnectorServerFactory.newJMXConnectorServer(new JMXServiceURL(url),
+                                                            env,
+                                                            ManagementFactory.getPlatformMBeanServer());
+
+        // If a custom authz proxy was created, attach it to the server now.
+        if (authzProxy != null)
+            jmxServer.setMBeanServerForwarder(authzProxy);
+
+        logger.info("Configured JMX server at: {}", url);
+        return jmxServer;
+    }
+
+    private static Map<String, Object> configureJmxAuthentication()
+    {
+        Map<String, Object> env = new HashMap<>();
+        if (!Boolean.getBoolean("com.sun.management.jmxremote.authenticate"))
+            return env;
+
+        // If authentication is enabled, initialize the appropriate JMXAuthenticator
+        // and stash it in the environment settings.
+        // A JAAS configuration entry takes precedence. If one is supplied, use
+        // Cassandra's own custom JMXAuthenticator implementation which delegates
+        // auth to the LoginModules specified by the JAAS configuration entry.
+        // If no JAAS entry is found, an instance of the JDK's own
+        // JMXPluggableAuthenticator is created. In that case, the admin may have
+        // set a location for the JMX password file which must be added to env
+        // before creating the authenticator. If no password file has been
+        // explicitly set, it's read from the default location
+        // $JAVA_HOME/lib/management/jmxremote.password
+        String configEntry = System.getProperty("cassandra.jmx.remote.login.config");
+        if (configEntry != null)
+        {
+            env.put(JMXConnectorServer.AUTHENTICATOR, new AuthenticationProxy(configEntry));
+        }
+        else
+        {
+            String passwordFile = System.getProperty("com.sun.management.jmxremote.password.file");
+            if (passwordFile != null)
+            {
+                // stash the password file location where JMXPluggableAuthenticator expects it
+                env.put("jmx.remote.x.password.file", passwordFile);
+            }
+
+            env.put(JMXConnectorServer.AUTHENTICATOR, new JMXPluggableAuthenticatorWrapper(env));
+        }
+
+        return env;
+    }
+
+    private static MBeanServerForwarder configureJmxAuthorization(Map<String, Object> env)
+    {
+        // If a custom authz proxy is supplied (Cassandra ships with AuthorizationProxy, which
+        // delegates to its own role based IAuthorizer), then instantiate and return one which
+        // can be set as the JMXConnectorServer's MBeanServerForwarder.
+        // If no custom proxy is supplied, check system properties for the location of the
+        // standard access file & stash it in env
+        String authzProxyClass = System.getProperty("cassandra.jmx.authorizer");
+        if (authzProxyClass != null)
+        {
+            final InvocationHandler handler = FBUtilities.construct(authzProxyClass, "JMX authz proxy");
+            final Class[] interfaces = { MBeanServerForwarder.class };
+
+            Object proxy = Proxy.newProxyInstance(MBeanServerForwarder.class.getClassLoader(), interfaces, handler);
+            return MBeanServerForwarder.class.cast(proxy);
+        }
+        else
+        {
+            String accessFile = System.getProperty("com.sun.management.jmxremote.access.file");
+            if (accessFile != null)
+            {
+                env.put("jmx.remote.x.access.file", accessFile);
+            }
+            return null;
+        }
+    }
+
+    private static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress)
+    {
+        Map<String, Object> env = new HashMap<>();
+        if (Boolean.getBoolean("com.sun.management.jmxremote.ssl"))
+        {
+            boolean requireClientAuth = Boolean.getBoolean("com.sun.management.jmxremote.ssl.need.client.auth");
+            String[] protocols = null;
+            String protocolList = System.getProperty("com.sun.management.jmxremote.ssl.enabled.protocols");
+            if (protocolList != null)
+            {
+                System.setProperty("javax.rmi.ssl.client.enabledProtocols", protocolList);
+                protocols = StringUtils.split(protocolList, ',');
+            }
+
+            String[] ciphers = null;
+            String cipherList = System.getProperty("com.sun.management.jmxremote.ssl.enabled.cipher.suites");
+            if (cipherList != null)
+            {
+                System.setProperty("javax.rmi.ssl.client.enabledCipherSuites", cipherList);
+                ciphers = StringUtils.split(cipherList, ',');
+            }
+
+            SslRMIClientSocketFactory clientFactory = new SslRMIClientSocketFactory();
+            SslRMIServerSocketFactory serverFactory = new SslRMIServerSocketFactory(ciphers, protocols, requireClientAuth);
+            env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory);
+            env.put(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE, clientFactory);
+            env.put("com.sun.jndi.rmi.factory.socket", clientFactory);
+            logJmxSslConfig(serverFactory);
+        }
+        else
+        {
+            env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE,
+                    new RMIServerSocketFactoryImpl(serverAddress));
+        }
+
+        return env;
+    }
+
+    private static void logJmxSslConfig(SslRMIServerSocketFactory serverFactory)
+    {
+        logger.debug("JMX SSL configuration. { protocols: [{}], cipher_suites: [{}], require_client_auth: {} }",
+                     serverFactory.getEnabledProtocols() == null
+                     ? "'JVM defaults'"
+                     : Arrays.stream(serverFactory.getEnabledProtocols()).collect(Collectors.joining("','", "'", "'")),
+                     serverFactory.getEnabledCipherSuites() == null
+                     ? "'JVM defaults'"
+                     : Arrays.stream(serverFactory.getEnabledCipherSuites()).collect(Collectors.joining("','", "'", "'")),
+                     serverFactory.getNeedClientAuth());
+    }
+
+    private static class JMXPluggableAuthenticatorWrapper implements JMXAuthenticator
+    {
+        final Map<?, ?> env;
+        private JMXPluggableAuthenticatorWrapper(Map<?, ?> env)
+        {
+            this.env = ImmutableMap.copyOf(env);
+        }
+
+        public Subject authenticate(Object credentials)
+        {
+            JMXPluggableAuthenticator authenticator = new JMXPluggableAuthenticator(env);
+            return authenticator.authenticate(credentials);
+        }
+    }
+
+    /**
+     * In the RMI subsystem, the ObjectTable instance holds references to remote
+     * objects for distributed garbage collection purposes. When objects are
+     * added to the ObjectTable (exported), a flag is passed to * indicate the
+     * "permanence" of that object. Exporting as permanent has two effects; the
+     * object is not eligible for distributed garbage collection, and its
+     * existence will not prevent the JVM from exiting after termination of all
+     * non-daemon threads terminate. Neither of these is bad for our case, as we
+     * attach the server exactly once (i.e. at startup, not subsequently using
+     * the Attach API) and don't disconnect it before shutdown. The primary
+     * benefit we gain is that it doesn't trigger the scheduled full GC that
+     * is otherwise incurred by programatically configuring the management server.
+     *
+     * To that end, we use this private implementation of RMIExporter to register
+     * our JMXConnectorServer as a permanent object by adding it to the map of
+     * environment variables under the key RMIExporter.EXPORTER_ATTRIBUTE
+     * (com.sun.jmx.remote.rmi.exporter) prior to calling server.start()
+     *
+     * See also:
+     *  * CASSANDRA-2967 for background
+     *  * https://www.jclarity.com/2015/01/27/rmi-system-gc-unplugged/ for more detail
+     *  * https://bugs.openjdk.java.net/browse/JDK-6760712 for info on setting the exporter
+     *  * sun.management.remote.ConnectorBootstrap to trace how the inbuilt management agent
+     *    sets up the JMXConnectorServer
+     */
+    private static class Exporter implements RMIExporter
+    {
+        public Remote exportObject(Remote obj, int port, RMIClientSocketFactory csf, RMIServerSocketFactory ssf)
+        throws RemoteException
+        {
+            // We should only ever get here by configuring our own JMX Connector server,
+            // so assert some invariants we expect to be true in that case
+            assert ssf != null; // we always configure a custom server socket factory
+
+            // as we always configure a custom server socket factory, either for SSL or to ensure
+            // only loopback addresses, we use a UnicastServerRef2 for exporting
+            return new UnicastServerRef2(port, csf, ssf).exportObject(obj, null, true);
+        }
+
+        public boolean unexportObject(Remote obj, boolean force) throws NoSuchObjectException
+        {
+            return UnicastRemoteObject.unexportObject(obj, force);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/utils/RMIServerSocketFactoryImpl.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/RMIServerSocketFactoryImpl.java b/src/java/org/apache/cassandra/utils/RMIServerSocketFactoryImpl.java
index ec81aa3..e3e901d 100644
--- a/src/java/org/apache/cassandra/utils/RMIServerSocketFactoryImpl.java
+++ b/src/java/org/apache/cassandra/utils/RMIServerSocketFactoryImpl.java
@@ -6,14 +6,19 @@ import java.net.ServerSocket;
 import java.rmi.server.RMIServerSocketFactory;
 import javax.net.ServerSocketFactory;
 
-
 public class RMIServerSocketFactoryImpl implements RMIServerSocketFactory
 {
+    // Address to bind server sockets too, may be null indicating all local interfaces are to be bound
+    private final InetAddress bindAddress;
+
+    public RMIServerSocketFactoryImpl(InetAddress bindAddress)
+    {
+        this.bindAddress = bindAddress;
+    }
 
     public ServerSocket createServerSocket(final int pPort) throws IOException
     {
-        ServerSocket socket = ServerSocketFactory.getDefault()
-                                                 .createServerSocket(pPort, 0, InetAddress.getLoopbackAddress());
+        ServerSocket socket = ServerSocketFactory.getDefault().createServerSocket(pPort, 0, bindAddress);
         socket.setReuseAddress(true);
         return socket;
     }
@@ -37,3 +42,4 @@ public class RMIServerSocketFactoryImpl implements RMIServerSocketFactory
         return RMIServerSocketFactoryImpl.class.hashCode();
     }
 }
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/test/resources/auth/cassandra-test-jaas.conf
----------------------------------------------------------------------
diff --git a/test/resources/auth/cassandra-test-jaas.conf b/test/resources/auth/cassandra-test-jaas.conf
new file mode 100644
index 0000000..ccb8b6a
--- /dev/null
+++ b/test/resources/auth/cassandra-test-jaas.conf
@@ -0,0 +1,4 @@
+// Delegates authentication to a stub login module, hardcoded to authenticate as a particular user - see JMXAuthTest
+TestLogin {
+  org.apache.cassandra.auth.jmx.JMXAuthTest$StubLoginModule REQUIRED role_name=test_role;
+};

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/test/unit/org/apache/cassandra/auth/StubAuthorizer.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/auth/StubAuthorizer.java b/test/unit/org/apache/cassandra/auth/StubAuthorizer.java
new file mode 100644
index 0000000..8e0d141
--- /dev/null
+++ b/test/unit/org/apache/cassandra/auth/StubAuthorizer.java
@@ -0,0 +1,120 @@
+/*
+ * 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 java.util.stream.Collectors;
+
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.utils.Pair;
+
+public class StubAuthorizer implements IAuthorizer
+{
+    Map<Pair<String, IResource>, Set<Permission>> userPermissions = new HashMap<>();
+
+    public void clear()
+    {
+        userPermissions.clear();
+    }
+
+    public Set<Permission> authorize(AuthenticatedUser user, IResource resource)
+    {
+        Pair<String, IResource> key = Pair.create(user.getName(), resource);
+        Set<Permission> perms = userPermissions.get(key);
+        return perms != null ? perms : Collections.emptySet();
+    }
+
+    public void grant(AuthenticatedUser performer,
+                      Set<Permission> permissions,
+                      IResource resource,
+                      RoleResource grantee) throws RequestValidationException, RequestExecutionException
+    {
+        Pair<String, IResource> key = Pair.create(grantee.getRoleName(), resource);
+        Set<Permission> perms = userPermissions.get(key);
+        if (null == perms)
+        {
+            perms = new HashSet<>();
+            userPermissions.put(key, perms);
+        }
+        perms.addAll(permissions);
+    }
+
+    public void revoke(AuthenticatedUser performer,
+                       Set<Permission> permissions,
+                       IResource resource,
+                       RoleResource revokee) throws RequestValidationException, RequestExecutionException
+    {
+        Pair<String, IResource> key = Pair.create(revokee.getRoleName(), resource);
+        Set<Permission> perms = userPermissions.get(key);
+        if (null != perms)
+        {
+            perms.removeAll(permissions);
+            if (perms.isEmpty())
+                userPermissions.remove(key);
+        }
+    }
+
+    public Set<PermissionDetails> list(AuthenticatedUser performer,
+                                       Set<Permission> permissions,
+                                       IResource resource,
+                                       RoleResource grantee) throws RequestValidationException, RequestExecutionException
+    {
+        return userPermissions.entrySet()
+                              .stream()
+                              .filter(entry -> entry.getKey().left.equals(grantee.getRoleName())
+                                               && (resource == null || entry.getKey().right.equals(resource)))
+                              .flatMap(entry -> entry.getValue()
+                                                     .stream()
+                                                     .filter(permissions::contains)
+                                                     .map(p -> new PermissionDetails(entry.getKey().left,
+                                                                                     entry.getKey().right,
+                                                                                     p)))
+                              .collect(Collectors.toSet());
+
+    }
+
+    public void revokeAllFrom(RoleResource revokee)
+    {
+        for (Pair<String, IResource> key : userPermissions.keySet())
+            if (key.left.equals(revokee.getRoleName()))
+                userPermissions.remove(key);
+    }
+
+    public void revokeAllOn(IResource droppedResource)
+    {
+        for (Pair<String, IResource> key : userPermissions.keySet())
+            if (key.right.equals(droppedResource))
+                userPermissions.remove(key);
+    }
+
+    public Set<? extends IResource> protectedResources()
+    {
+        return Collections.emptySet();
+    }
+
+    public void validateConfiguration() throws ConfigurationException
+    {
+    }
+
+    public void setup()
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/test/unit/org/apache/cassandra/auth/jmx/AuthorizationProxyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/auth/jmx/AuthorizationProxyTest.java b/test/unit/org/apache/cassandra/auth/jmx/AuthorizationProxyTest.java
new file mode 100644
index 0000000..9943acb
--- /dev/null
+++ b/test/unit/org/apache/cassandra/auth/jmx/AuthorizationProxyTest.java
@@ -0,0 +1,574 @@
+/*
+ * 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.jmx;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.remote.JMXPrincipal;
+import javax.security.auth.Subject;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.junit.Test;
+
+import org.apache.cassandra.auth.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class AuthorizationProxyTest
+{
+    JMXResource osBean = JMXResource.mbean("java.lang:type=OperatingSystem");
+    JMXResource runtimeBean = JMXResource.mbean("java.lang:type=Runtime");
+    JMXResource threadingBean = JMXResource.mbean("java.lang:type=Threading");
+    JMXResource javaLangWildcard = JMXResource.mbean("java.lang:type=*");
+
+    JMXResource hintsBean = JMXResource.mbean("org.apache.cassandra.hints:type=HintsService");
+    JMXResource batchlogBean = JMXResource.mbean("org.apache.cassandra.db:type=BatchlogManager");
+    JMXResource customBean = JMXResource.mbean("org.apache.cassandra:type=CustomBean,property=foo");
+    Set<ObjectName> allBeans = objectNames(osBean, runtimeBean, threadingBean, hintsBean, batchlogBean, customBean);
+
+    RoleResource role1 = RoleResource.role("r1");
+
+    @Test
+    public void roleHasRequiredPermission() throws Throwable
+    {
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, Collections.singleton(permission(role1, osBean, Permission.SELECT)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .isAuthzRequired(() -> true)
+                                                     .build();
+
+        assertTrue(proxy.authorize(subject(role1.getRoleName()),
+                                   "getAttribute",
+                                   new Object[]{ objectName(osBean), "arch" }));
+    }
+
+    @Test
+    public void roleDoesNotHaveRequiredPermission() throws Throwable
+    {
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, Collections.singleton(permission(role1, osBean, Permission.AUTHORIZE)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .isAuthzRequired(() -> true).build();
+
+        assertFalse(proxy.authorize(subject(role1.getRoleName()),
+                                    "setAttribute",
+                                    new Object[]{ objectName(osBean), "arch" }));
+    }
+
+    @Test
+    public void roleHasRequiredPermissionOnRootResource() throws Throwable
+    {
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, Collections.singleton(permission(role1, JMXResource.root(), Permission.SELECT)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .isAuthzRequired(() -> true)
+                                                     .build();
+
+        assertTrue(proxy.authorize(subject(role1.getRoleName()),
+                                   "getAttribute",
+                                   new Object[]{ objectName(osBean), "arch" }));
+    }
+
+    @Test
+    public void roleHasOtherPermissionOnRootResource() throws Throwable
+    {
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, Collections.singleton(permission(role1, JMXResource.root(), Permission.AUTHORIZE)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .isAuthzRequired(() -> true)
+                                                     .build();
+
+        assertFalse(proxy.authorize(subject(role1.getRoleName()),
+                                    "invoke",
+                                    new Object[]{ objectName(osBean), "bogusMethod" }));
+    }
+
+    @Test
+    public void roleHasNoPermissions() throws Throwable
+    {
+        AuthorizationProxy proxy = new ProxyBuilder().isSuperuser((role) -> false)
+                                                     .getPermissions((role) -> Collections.emptySet())
+                                                     .isAuthzRequired(() -> true)
+                                                     .build();
+
+        assertFalse(proxy.authorize(subject(role1.getRoleName()),
+                                    "getAttribute",
+                                    new Object[]{ objectName(osBean), "arch" }));
+    }
+
+    @Test
+    public void roleHasNoPermissionsButIsSuperuser() throws Throwable
+    {
+        AuthorizationProxy proxy = new ProxyBuilder().isSuperuser((role) -> true)
+                                                     .getPermissions((role) -> Collections.emptySet())
+                                                     .isAuthzRequired(() -> true)
+                                                     .build();
+
+        assertTrue(proxy.authorize(subject(role1.getRoleName()),
+                                   "getAttribute",
+                                   new Object[]{ objectName(osBean), "arch" }));
+    }
+
+    @Test
+    public void roleHasNoPermissionsButAuthzNotRequired() throws Throwable
+    {
+        AuthorizationProxy proxy = new ProxyBuilder().isSuperuser((role) -> false)
+                                                     .getPermissions((role) -> Collections.emptySet())
+                                                     .isAuthzRequired(() -> false)
+                                                     .build();
+
+        assertTrue(proxy.authorize(subject(role1.getRoleName()),
+                                   "getAttribute",
+                                   new Object[]{ objectName(osBean), "arch" }));
+    }
+
+    @Test
+    public void authorizeWhenSubjectIsNull() throws Throwable
+    {
+        // a null subject indicates that the action is being performed by the
+        // connector itself, so we always authorize it
+        // Verify that the superuser status is never tested as the request returns early
+        // due to the null Subject
+        // Also, hardcode the permissions provider to return an empty set, so we know that
+        // can be doubly sure that it's the null Subject which causes the authz to succeed
+        final AtomicBoolean suStatusChecked = new AtomicBoolean(false);
+        AuthorizationProxy proxy = new ProxyBuilder().getPermissions((role) -> Collections.emptySet())
+                                                     .isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) ->
+                                                                  {
+                                                                      suStatusChecked.set(true);
+                                                                      return false;
+                                                                  })
+                                                     .build();
+
+        assertTrue(proxy.authorize(null,
+                                   "getAttribute",
+                                   new Object[]{ objectName(osBean), "arch" }));
+        assertFalse(suStatusChecked.get());
+    }
+
+    @Test
+    public void rejectWhenSubjectNotAuthenticated() throws Throwable
+    {
+        // Access is denied to a Subject without any associated Principals
+        // Verify that the superuser status is never tested as the request is rejected early
+        // due to the Subject
+        final AtomicBoolean suStatusChecked = new AtomicBoolean(false);
+        AuthorizationProxy proxy = new ProxyBuilder().isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) ->
+                                                                  {
+                                                                      suStatusChecked.set(true);
+                                                                      return true;
+                                                                  })
+                                                     .build();
+        assertFalse(proxy.authorize(new Subject(),
+                                    "getAttribute",
+                                    new Object[]{ objectName(osBean), "arch" }));
+        assertFalse(suStatusChecked.get());
+    }
+
+    @Test
+    public void authorizeWhenWildcardGrantCoversExactTarget() throws Throwable
+    {
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, Collections.singleton(permission(role1, javaLangWildcard, Permission.SELECT)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .build();
+
+        assertTrue(proxy.authorize(subject(role1.getRoleName()),
+                                   "getAttribute",
+                                   new Object[]{ objectName(osBean), "arch" }));
+    }
+
+    @Test
+    public void rejectWhenWildcardGrantDoesNotCoverExactTarget() throws Throwable
+    {
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, Collections.singleton(permission(role1, javaLangWildcard, Permission.SELECT)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .build();
+
+        assertFalse(proxy.authorize(subject(role1.getRoleName()),
+                                    "getAttribute",
+                                    new Object[]{ objectName(customBean), "arch" }));
+    }
+
+    @Test
+    public void authorizeWhenWildcardGrantCoversWildcardTarget() throws Throwable
+    {
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, Collections.singleton(permission(role1, javaLangWildcard, Permission.DESCRIBE)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .queryNames(matcher(allBeans))
+                                                     .build();
+
+        assertTrue(proxy.authorize(subject(role1.getRoleName()),
+                                   "queryNames",
+                                   new Object[]{ objectName(javaLangWildcard), null }));
+    }
+
+    @Test
+    public void rejectWhenWildcardGrantIsDisjointWithWildcardTarget() throws Throwable
+    {
+        JMXResource customWildcard = JMXResource.mbean("org.apache.cassandra:*");
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, Collections.singleton(permission(role1, customWildcard, Permission.DESCRIBE)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .queryNames(matcher(allBeans))
+                                                     .build();
+
+        // the grant on org.apache.cassandra:* shouldn't permit us to invoke queryNames with java.lang:*
+        assertFalse(proxy.authorize(subject(role1.getRoleName()),
+                                    "queryNames",
+                                    new Object[]{ objectName(javaLangWildcard), null }));
+    }
+
+    @Test
+    public void rejectWhenWildcardGrantIntersectsWithWildcardTarget() throws Throwable
+    {
+        // in this test, permissions are granted on org.apache.cassandra:type=CustomBean,property=*
+        // and all beans in the org.apache.cassandra.hints domain, but
+        // but the target of the invocation is org.apache.cassandra*:*
+        // i.e. the subject has permissions on all CustomBeans and on the HintsService bean, but is
+        // attempting to query all names in the org.apache.cassandra* domain. The operation should
+        // be rejected as the permissions don't cover all known beans matching that domain, due to
+        // the BatchLogManager bean.
+
+        JMXResource allCustomBeans = JMXResource.mbean("org.apache.cassandra:type=CustomBean,property=*");
+        JMXResource allHintsBeans = JMXResource.mbean("org.apache.cassandra.hints:*");
+        ObjectName allCassandraBeans = ObjectName.getInstance("org.apache.cassandra*:*");
+
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, ImmutableSet.of(permission(role1, allCustomBeans, Permission.DESCRIBE),
+                                                   permission(role1, allHintsBeans, Permission.DESCRIBE)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .queryNames(matcher(allBeans))
+                                                     .build();
+
+        // the grant on org.apache.cassandra:* shouldn't permit us to invoke queryNames with java.lang:*
+        assertFalse(proxy.authorize(subject(role1.getRoleName()),
+                                    "queryNames",
+                                    new Object[]{ allCassandraBeans, null }));
+    }
+
+    @Test
+    public void authorizeOnTargetWildcardWithPermissionOnRoot() throws Throwable
+    {
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+            ImmutableMap.of(role1, Collections.singleton(permission(role1, JMXResource.root(), Permission.SELECT)));
+
+        AuthorizationProxy proxy = new ProxyBuilder().isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .build();
+
+        assertTrue(proxy.authorize(subject(role1.getRoleName()),
+                                   "getAttribute",
+                                   new Object[]{ objectName(javaLangWildcard), "arch" }));
+    }
+
+    @Test
+    public void rejectInvocationOfUnknownMethod() throws Throwable
+    {
+        // Grant ALL permissions on the root resource, so we know that it's
+        // the unknown method that causes the authz rejection. Of course, this
+        // isn't foolproof but it's something.
+        Set<PermissionDetails> allPerms = Permission.ALL.stream()
+                                                        .map(perm -> permission(role1, JMXResource.root(), perm))
+                                                        .collect(Collectors.toSet());
+        Map<RoleResource, Set<PermissionDetails>> permissions = ImmutableMap.of(role1, allPerms);
+        AuthorizationProxy proxy = new ProxyBuilder().isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .build();
+
+        assertFalse(proxy.authorize(subject(role1.getRoleName()),
+                                    "unKnownMethod",
+                                    new Object[] { ObjectName.getInstance(osBean.getObjectName()) }));
+    }
+
+    @Test
+    public void rejectInvocationOfBlacklistedMethods() throws Throwable
+    {
+        String[] methods = { "createMBean",
+                             "deserialize",
+                             "getClassLoader",
+                             "getClassLoaderFor",
+                             "instantiate",
+                             "registerMBean",
+                             "unregisterMBean" };
+
+        // Hardcode the superuser status check to return true, so any allowed method can be invoked.
+        AuthorizationProxy proxy = new ProxyBuilder().isAuthzRequired(() -> true)
+                                                     .isSuperuser((role) -> true)
+                                                     .build();
+
+        for (String method : methods)
+            // the arguments array isn't significant, so it can just be empty
+            assertFalse(proxy.authorize(subject(role1.getRoleName()), method, new Object[0]));
+    }
+
+    @Test
+    public void authorizeMethodsWithoutMBeanArgumentIfPermissionsGranted() throws Throwable
+    {
+        // Certain methods on MBeanServer don't take an ObjectName as their first argument.
+        // These methods are characterised by AuthorizationProxy as being concerned with
+        // the MBeanServer itself, as opposed to a specific managed bean. Of these methods,
+        // only those considered "descriptive" are allowed to be invoked by remote users.
+        // These require the DESCRIBE permission on the root JMXResource.
+        testNonMbeanMethods(true);
+    }
+
+    @Test
+    public void rejectMethodsWithoutMBeanArgumentIfPermissionsNotGranted() throws Throwable
+    {
+        testNonMbeanMethods(false);
+    }
+
+    @Test
+    public void rejectWhenAuthSetupIsNotComplete() throws Throwable
+    {
+        // IAuthorizer & IRoleManager should not be considered ready to use until
+        // we know that auth setup has completed. So, even though the IAuthorizer
+        // would theoretically grant access, the auth proxy should deny it if setup
+        // hasn't finished.
+
+        Map<RoleResource, Set<PermissionDetails>> permissions =
+        ImmutableMap.of(role1, Collections.singleton(permission(role1, osBean, Permission.SELECT)));
+
+        // verify that access is granted when setup is complete
+        AuthorizationProxy proxy = new ProxyBuilder().isSuperuser((role) -> false)
+                                                     .getPermissions(permissions::get)
+                                                     .isAuthzRequired(() -> true)
+                                                     .isAuthSetupComplete(() -> true)
+                                                     .build();
+
+        assertTrue(proxy.authorize(subject(role1.getRoleName()),
+                                   "getAttribute",
+                                   new Object[]{ objectName(osBean), "arch" }));
+
+        // and denied when it isn't
+        proxy = new ProxyBuilder().isSuperuser((role) -> false)
+                                  .getPermissions(permissions::get)
+                                  .isAuthzRequired(() -> true)
+                                  .isAuthSetupComplete(() -> false)
+                                  .build();
+
+        assertFalse(proxy.authorize(subject(role1.getRoleName()),
+                                   "getAttribute",
+                                   new Object[]{ objectName(osBean), "arch" }));
+    }
+
+    private void testNonMbeanMethods(boolean withPermission)
+    {
+        String[] methods = { "getDefaultDomain",
+                             "getDomains",
+                             "getMBeanCount",
+                             "hashCode",
+                             "queryMBeans",
+                             "queryNames",
+                             "toString" };
+
+
+        ProxyBuilder builder = new ProxyBuilder().isAuthzRequired(() -> true).isSuperuser((role) -> false);
+        if (withPermission)
+        {
+            Map<RoleResource, Set<PermissionDetails>> permissions =
+                ImmutableMap.of(role1, ImmutableSet.of(permission(role1, JMXResource.root(), Permission.DESCRIBE)));
+            builder.getPermissions(permissions::get);
+        }
+        else
+        {
+            builder.getPermissions((role) -> Collections.emptySet());
+        }
+        AuthorizationProxy proxy = builder.build();
+
+        for (String method : methods)
+            assertEquals(withPermission, proxy.authorize(subject(role1.getRoleName()), method, new Object[]{ null }));
+
+        // non-whitelisted methods should be rejected regardless.
+        // This isn't exactly comprehensive, but it's better than nothing
+        String[] notAllowed = { "fooMethod", "barMethod", "bazMethod" };
+        for (String method : notAllowed)
+            assertFalse(proxy.authorize(subject(role1.getRoleName()), method, new Object[]{ null }));
+    }
+
+    // provides a simple matching function which can be substituted for the proxy's queryMBeans
+    // utility (which by default just delegates to the MBeanServer)
+    // This function just iterates over a supplied set of ObjectNames and filters out those
+    // to which the target name *doesn't* apply
+    private static Function<ObjectName, Set<ObjectName>> matcher(Set<ObjectName> allBeans)
+    {
+        return (target) -> allBeans.stream()
+                                   .filter(target::apply)
+                                   .collect(Collectors.toSet());
+    }
+
+    private static PermissionDetails permission(RoleResource grantee, IResource resource, Permission permission)
+    {
+        return new PermissionDetails(grantee.getRoleName(), resource, permission);
+    }
+
+    private static Subject subject(String roleName)
+    {
+        Subject subject = new Subject();
+        subject.getPrincipals().add(new CassandraPrincipal(roleName));
+        return subject;
+    }
+
+    private static ObjectName objectName(JMXResource resource) throws MalformedObjectNameException
+    {
+        return ObjectName.getInstance(resource.getObjectName());
+    }
+
+    private static Set<ObjectName> objectNames(JMXResource... resource)
+    {
+        Set<ObjectName> names = new HashSet<>();
+        try
+        {
+            for (JMXResource r : resource)
+                names.add(objectName(r));
+        }
+        catch (MalformedObjectNameException e)
+        {
+            fail("JMXResource returned invalid object name: " + e.getMessage());
+        }
+        return names;
+    }
+
+    public static class ProxyBuilder
+    {
+        Function<RoleResource, Set<PermissionDetails>> getPermissions;
+        Function<ObjectName, Set<ObjectName>> queryNames;
+        Function<RoleResource, Boolean> isSuperuser;
+        Supplier<Boolean> isAuthzRequired;
+        Supplier<Boolean> isAuthSetupComplete = () -> true;
+
+        AuthorizationProxy build()
+        {
+            InjectableAuthProxy proxy = new InjectableAuthProxy();
+
+            if (getPermissions != null)
+                proxy.setGetPermissions(getPermissions);
+
+            if (queryNames != null)
+                proxy.setQueryNames(queryNames);
+
+            if (isSuperuser != null)
+                proxy.setIsSuperuser(isSuperuser);
+
+            if (isAuthzRequired != null)
+                proxy.setIsAuthzRequired(isAuthzRequired);
+
+            proxy.setIsAuthSetupComplete(isAuthSetupComplete);
+
+            return proxy;
+        }
+
+        ProxyBuilder getPermissions(Function<RoleResource, Set<PermissionDetails>> f)
+        {
+            getPermissions = f;
+            return this;
+        }
+
+        ProxyBuilder queryNames(Function<ObjectName, Set<ObjectName>> f)
+        {
+            queryNames = f;
+            return this;
+        }
+
+        ProxyBuilder isSuperuser(Function<RoleResource, Boolean> f)
+        {
+            isSuperuser = f;
+            return this;
+        }
+
+        ProxyBuilder isAuthzRequired(Supplier<Boolean> s)
+        {
+            isAuthzRequired = s;
+            return this;
+        }
+
+        ProxyBuilder isAuthSetupComplete(Supplier<Boolean> s)
+        {
+            isAuthSetupComplete = s;
+            return this;
+        }
+
+        private static class InjectableAuthProxy extends AuthorizationProxy
+        {
+            void setGetPermissions(Function<RoleResource, Set<PermissionDetails>> f)
+            {
+                this.getPermissions = f;
+            }
+
+            void setQueryNames(Function<ObjectName, Set<ObjectName>> f)
+            {
+                this.queryNames = f;
+            }
+
+            void setIsSuperuser(Function<RoleResource, Boolean> f)
+            {
+                this.isSuperuser = f;
+            }
+
+            void setIsAuthzRequired(Supplier<Boolean> s)
+            {
+                this.isAuthzRequired = s;
+            }
+
+            void setIsAuthSetupComplete(Supplier<Boolean> s)
+            {
+                this.isAuthSetupComplete = s;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/test/unit/org/apache/cassandra/auth/jmx/JMXAuthTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/auth/jmx/JMXAuthTest.java b/test/unit/org/apache/cassandra/auth/jmx/JMXAuthTest.java
new file mode 100644
index 0000000..10c871b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/auth/jmx/JMXAuthTest.java
@@ -0,0 +1,279 @@
+/*
+ * 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.jmx;
+
+import java.lang.reflect.Field;
+import java.nio.file.Paths;
+import java.rmi.server.RMISocketFactory;
+import java.util.HashMap;
+import java.util.Map;
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.remote.*;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.login.LoginException;
+import javax.security.auth.spi.LoginModule;
+
+import com.google.common.collect.ImmutableSet;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStoreMBean;
+import org.apache.cassandra.utils.JMXServerUtils;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class JMXAuthTest extends CQLTester
+{
+    private static JMXConnectorServer jmxServer;
+    private static MBeanServerConnection connection;
+    private RoleResource role;
+    private String tableName;
+    private JMXResource tableMBean;
+
+    @FunctionalInterface
+    private interface MBeanAction
+    {
+        void execute();
+    }
+
+    @BeforeClass
+    public static void setupClass() throws Exception
+    {
+        setupAuthorizer();
+        setupJMXServer();
+    }
+
+    private static void setupAuthorizer()
+    {
+        try
+        {
+            IAuthorizer authorizer = new StubAuthorizer();
+            Field authorizerField = DatabaseDescriptor.class.getDeclaredField("authorizer");
+            authorizerField.setAccessible(true);
+            authorizerField.set(null, authorizer);
+            DatabaseDescriptor.setPermissionsValidity(0);
+        }
+        catch (IllegalAccessException | NoSuchFieldException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static void setupJMXServer() throws Exception
+    {
+        String config = Paths.get(ClassLoader.getSystemResource("auth/cassandra-test-jaas.conf").toURI()).toString();
+        System.setProperty("com.sun.management.jmxremote.authenticate", "true");
+        System.setProperty("java.security.auth.login.config", config);
+        System.setProperty("cassandra.jmx.remote.login.config", "TestLogin");
+        System.setProperty("cassandra.jmx.authorizer", NoSuperUserAuthorizationProxy.class.getName());
+        jmxServer = JMXServerUtils.createJMXServer(9999, true);
+        jmxServer.start();
+
+        JMXServiceURL jmxUrl = new JMXServiceURL("service:jmx:rmi:///jndi/rmi://localhost:9999/jmxrmi");
+        Map<String, Object> env = new HashMap<>();
+        env.put("com.sun.jndi.rmi.factory.socket", RMISocketFactory.getDefaultSocketFactory());
+        JMXConnector jmxc = JMXConnectorFactory.connect(jmxUrl, env);
+        connection = jmxc.getMBeanServerConnection();
+    }
+
+    @Before
+    public void setup() throws Throwable
+    {
+        role = RoleResource.role("test_role");
+        clearAllPermissions();
+        tableName = createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k))");
+        tableMBean = JMXResource.mbean(String.format("org.apache.cassandra.db:type=Tables,keyspace=%s,table=%s",
+                                                     KEYSPACE, tableName));
+    }
+
+    @Test
+    public void readAttribute() throws Throwable
+    {
+        ColumnFamilyStoreMBean proxy = JMX.newMBeanProxy(connection,
+                                                         ObjectName.getInstance(tableMBean.getObjectName()),
+                                                         ColumnFamilyStoreMBean.class);
+
+        // grant SELECT on a single specific Table mbean
+        assertPermissionOnResource(Permission.SELECT, tableMBean, proxy::getTableName);
+
+        // grant SELECT on all Table mbeans in named keyspace
+        clearAllPermissions();
+        JMXResource allTablesInKeyspace = JMXResource.mbean(String.format("org.apache.cassandra.db:type=Tables,keyspace=%s,*",
+                                                                          KEYSPACE));
+        assertPermissionOnResource(Permission.SELECT, allTablesInKeyspace, proxy::getTableName);
+
+        // grant SELECT on all Table mbeans
+        clearAllPermissions();
+        JMXResource allTables = JMXResource.mbean("org.apache.cassandra.db:type=Tables,*");
+        assertPermissionOnResource(Permission.SELECT, allTables, proxy::getTableName);
+
+        // grant SELECT ON ALL MBEANS
+        clearAllPermissions();
+        assertPermissionOnResource(Permission.SELECT, JMXResource.root(), proxy::getTableName);
+    }
+
+    @Test
+    public void writeAttribute() throws Throwable
+    {
+        ColumnFamilyStoreMBean proxy = JMX.newMBeanProxy(connection,
+                                                         ObjectName.getInstance(tableMBean.getObjectName()),
+                                                         ColumnFamilyStoreMBean.class);
+        MBeanAction action = () -> proxy.setMinimumCompactionThreshold(4);
+
+        // grant MODIFY on a single specific Table mbean
+        assertPermissionOnResource(Permission.MODIFY, tableMBean, action);
+
+        // grant MODIFY on all Table mbeans in named keyspace
+        clearAllPermissions();
+        JMXResource allTablesInKeyspace = JMXResource.mbean(String.format("org.apache.cassandra.db:type=Tables,keyspace=%s,*",
+                                                                          KEYSPACE));
+        assertPermissionOnResource(Permission.MODIFY, allTablesInKeyspace, action);
+
+        // grant MODIFY on all Table mbeans
+        clearAllPermissions();
+        JMXResource allTables = JMXResource.mbean("org.apache.cassandra.db:type=Tables,*");
+        assertPermissionOnResource(Permission.MODIFY, allTables, action);
+
+        // grant MODIFY ON ALL MBEANS
+        clearAllPermissions();
+        assertPermissionOnResource(Permission.MODIFY, JMXResource.root(), action);
+    }
+
+    @Test
+    public void executeMethod() throws Throwable
+    {
+        ColumnFamilyStoreMBean proxy = JMX.newMBeanProxy(connection,
+                                                         ObjectName.getInstance(tableMBean.getObjectName()),
+                                                         ColumnFamilyStoreMBean.class);
+
+        // grant EXECUTE on a single specific Table mbean
+        assertPermissionOnResource(Permission.EXECUTE, tableMBean, proxy::estimateKeys);
+
+        // grant EXECUTE on all Table mbeans in named keyspace
+        clearAllPermissions();
+        JMXResource allTablesInKeyspace = JMXResource.mbean(String.format("org.apache.cassandra.db:type=Tables,keyspace=%s,*",
+                                                                          KEYSPACE));
+        assertPermissionOnResource(Permission.EXECUTE, allTablesInKeyspace, proxy::estimateKeys);
+
+        // grant EXECUTE on all Table mbeans
+        clearAllPermissions();
+        JMXResource allTables = JMXResource.mbean("org.apache.cassandra.db:type=Tables,*");
+        assertPermissionOnResource(Permission.EXECUTE, allTables, proxy::estimateKeys);
+
+        // grant EXECUTE ON ALL MBEANS
+        clearAllPermissions();
+        assertPermissionOnResource(Permission.EXECUTE, JMXResource.root(), proxy::estimateKeys);
+    }
+
+    private void assertPermissionOnResource(Permission permission,
+                                            JMXResource resource,
+                                            MBeanAction action)
+    {
+        assertUnauthorized(action);
+        grantPermission(permission, resource, role);
+        assertAuthorized(action);
+    }
+
+    private void grantPermission(Permission permission, JMXResource resource, RoleResource role)
+    {
+        DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                 ImmutableSet.of(permission),
+                                                 resource,
+                                                 role);
+    }
+
+    private void assertAuthorized(MBeanAction action)
+    {
+        action.execute();
+    }
+
+    private void assertUnauthorized(MBeanAction action)
+    {
+        try
+        {
+            action.execute();
+            fail("Expected an UnauthorizedException, but none was thrown");
+        }
+        catch (SecurityException e)
+        {
+            assertEquals("Access Denied", e.getLocalizedMessage());
+        }
+    }
+
+    private void clearAllPermissions()
+    {
+        ((StubAuthorizer) DatabaseDescriptor.getAuthorizer()).clear();
+    }
+
+    public static class StubLoginModule implements LoginModule
+    {
+        private CassandraPrincipal principal;
+        private Subject subject;
+
+        public StubLoginModule(){}
+
+        public void initialize(Subject subject, CallbackHandler callbackHandler, Map<String, ?> sharedState, Map<String, ?> options)
+        {
+            this.subject = subject;
+            principal = new CassandraPrincipal((String)options.get("role_name"));
+        }
+
+        public boolean login() throws LoginException
+        {
+            return true;
+        }
+
+        public boolean commit() throws LoginException
+        {
+            if (!subject.getPrincipals().contains(principal))
+                subject.getPrincipals().add(principal);
+            return true;
+        }
+
+        public boolean abort() throws LoginException
+        {
+            return true;
+        }
+
+        public boolean logout() throws LoginException
+        {
+            return true;
+        }
+    }
+
+    // always answers false to isSuperUser and true to isAuthSetup complete - saves us having to initialize
+    // a real IRoleManager and StorageService for the test
+    public static class NoSuperUserAuthorizationProxy extends AuthorizationProxy
+    {
+        public NoSuperUserAuthorizationProxy()
+        {
+            super();
+            this.isSuperuser = (role) -> false;
+            this.isAuthSetupComplete = () -> true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
index 6993bec..d085a9d 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
@@ -29,14 +29,11 @@ import org.junit.Test;
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.Attributes;
-import org.apache.cassandra.cql3.CQLStatement;
-import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
-import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.utils.Pair;
@@ -626,99 +623,4 @@ public class UFAuthTest extends CQLTester
     {
         return String.format("%s(%s)", functionName, Joiner.on(",").join(args));
     }
-
-    static class StubAuthorizer implements IAuthorizer
-    {
-        Map<Pair<String, IResource>, Set<Permission>> userPermissions = new HashMap<>();
-
-        private void clear()
-        {
-            userPermissions.clear();
-        }
-
-        public Set<Permission> authorize(AuthenticatedUser user, IResource resource)
-        {
-            Pair<String, IResource> key = Pair.create(user.getName(), resource);
-            Set<Permission> perms = userPermissions.get(key);
-            return perms != null ? perms : Collections.<Permission>emptySet();
-        }
-
-        public void grant(AuthenticatedUser performer,
-                          Set<Permission> permissions,
-                          IResource resource,
-                          RoleResource grantee) throws RequestValidationException, RequestExecutionException
-        {
-            Pair<String, IResource> key = Pair.create(grantee.getRoleName(), resource);
-            Set<Permission> perms = userPermissions.get(key);
-            if (null == perms)
-            {
-                perms = new HashSet<>();
-                userPermissions.put(key, perms);
-            }
-            perms.addAll(permissions);
-        }
-
-        public void revoke(AuthenticatedUser performer,
-                           Set<Permission> permissions,
-                           IResource resource,
-                           RoleResource revokee) throws RequestValidationException, RequestExecutionException
-        {
-            Pair<String, IResource> key = Pair.create(revokee.getRoleName(), resource);
-            Set<Permission> perms = userPermissions.get(key);
-            if (null != perms)
-                perms.removeAll(permissions);
-            if (perms.isEmpty())
-                userPermissions.remove(key);
-        }
-
-        public Set<PermissionDetails> list(AuthenticatedUser performer,
-                                           Set<Permission> permissions,
-                                           IResource resource,
-                                           RoleResource grantee) throws RequestValidationException, RequestExecutionException
-        {
-            Pair<String, IResource> key = Pair.create(grantee.getRoleName(), resource);
-            Set<Permission> perms = userPermissions.get(key);
-            if (perms == null)
-                return Collections.emptySet();
-
-
-            Set<PermissionDetails> details = new HashSet<>();
-            for (Permission permission : perms)
-            {
-                if (permissions.contains(permission))
-                    details.add(new PermissionDetails(grantee.getRoleName(), resource, permission));
-            }
-            return details;
-        }
-
-        public void revokeAllFrom(RoleResource revokee)
-        {
-            for (Pair<String, IResource> key : userPermissions.keySet())
-                if (key.left.equals(revokee.getRoleName()))
-                    userPermissions.remove(key);
-        }
-
-        public void revokeAllOn(IResource droppedResource)
-        {
-            for (Pair<String, IResource> key : userPermissions.keySet())
-                if (key.right.equals(droppedResource))
-                    userPermissions.remove(key);
-
-        }
-
-        public Set<? extends IResource> protectedResources()
-        {
-            return Collections.emptySet();
-        }
-
-        public void validateConfiguration() throws ConfigurationException
-        {
-
-        }
-
-        public void setup()
-        {
-
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/test/unit/org/apache/cassandra/service/RMIServerSocketFactoryImplTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/RMIServerSocketFactoryImplTest.java b/test/unit/org/apache/cassandra/service/RMIServerSocketFactoryImplTest.java
index 3459ec3..76e2f12 100644
--- a/test/unit/org/apache/cassandra/service/RMIServerSocketFactoryImplTest.java
+++ b/test/unit/org/apache/cassandra/service/RMIServerSocketFactoryImplTest.java
@@ -16,7 +16,7 @@ public class RMIServerSocketFactoryImplTest
     @Test
     public void testReusableAddrSocket() throws IOException
     {
-        RMIServerSocketFactory serverFactory = new RMIServerSocketFactoryImpl();
+        RMIServerSocketFactory serverFactory = new RMIServerSocketFactoryImpl(null);
         ServerSocket socket = serverFactory.createServerSocket(7199);
         assertTrue(socket.getReuseAddress());
     }


[2/2] cassandra git commit: Integrated JMX Authentication and Authorization

Posted by sa...@apache.org.
Integrated JMX Authentication and Authorization

Patch by Jan Karlsson and Sam Tunnicliffe; reviewed by Jake Luciani for
CASSANDRA-10091


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

Branch: refs/heads/trunk
Commit: 7b0c7164aa22c156811a5d1a001c43d099aad8e4
Parents: ad7e36b
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Wed Feb 24 09:31:44 2016 +0000
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Mon May 2 11:44:48 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  10 +-
 conf/cassandra-env.ps1                          |  32 +-
 conf/cassandra-env.sh                           |  43 +-
 conf/cassandra-jaas.config                      |   4 +
 doc/cql3/CQL.textile                            |  30 +-
 pylib/cqlshlib/cql3handling.py                  |   6 +
 src/antlr/Lexer.g                               |   2 +
 src/antlr/Parser.g                              |  38 ++
 .../cassandra/auth/AllowAllAuthorizer.java      |   5 +
 .../cassandra/auth/CassandraAuthorizer.java     |  12 +-
 .../cassandra/auth/CassandraLoginModule.java    | 257 +++++++++
 .../cassandra/auth/CassandraPrincipal.java      | 134 +++++
 .../org/apache/cassandra/auth/JMXResource.java  | 183 ++++++
 .../org/apache/cassandra/auth/Permission.java   |   4 +-
 .../org/apache/cassandra/auth/Resources.java    |   2 +
 .../cassandra/auth/jmx/AuthenticationProxy.java | 157 +++++
 .../cassandra/auth/jmx/AuthorizationProxy.java  | 512 +++++++++++++++++
 .../cassandra/service/CassandraDaemon.java      |  63 +-
 .../apache/cassandra/service/StartupChecks.java |   2 +-
 .../cassandra/service/StorageService.java       |   7 +
 .../org/apache/cassandra/tools/NodeTool.java    |   2 +-
 .../apache/cassandra/utils/JMXServerUtils.java  | 299 ++++++++++
 .../utils/RMIServerSocketFactoryImpl.java       |  12 +-
 test/resources/auth/cassandra-test-jaas.conf    |   4 +
 .../apache/cassandra/auth/StubAuthorizer.java   | 120 ++++
 .../auth/jmx/AuthorizationProxyTest.java        | 574 +++++++++++++++++++
 .../apache/cassandra/auth/jmx/JMXAuthTest.java  | 279 +++++++++
 .../cql3/validation/entities/UFAuthTest.java    | 100 +---
 .../service/RMIServerSocketFactoryImplTest.java |   2 +-
 30 files changed, 2745 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d557846..984ad55 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.6
+ * Integrated JMX authentication and authorization (CASSANDRA-10091)
  * Add units to stress ouput (CASSANDRA-11352)
  * Fix PER PARTITION LIMIT for single and multi partitions queries (CASSANDRA-11603)
  * Add uncompressed chunk cache for RandomAccessReader (CASSANDRA-5863)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 77d3dfd..7a29924 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -18,8 +18,14 @@ using the provided 'sstableupgrade' tool.
 
 New features
 ------------
-   - JSON timestamps are now in UTC and contain the timezone information, see
-     CASSANDRA-11137 for more details.
+   - JMX connections can now use the same auth mechanisms as CQL clients. New options
+     in cassandra-env.(sh|ps1) enable JMX authentication and authorization to be delegated
+     to the IAuthenticator and IAuthorizer configured in cassandra.yaml. The default settings
+     still only expose JMX locally, and use the JVM's own security mechanisms when remote
+     connections are permitted. For more details on how to enable the new options, see the
+     comments in cassandra-env.sh. A new class of IResource, JMXResource, is provided for
+     the purposes of GRANT/REVOKE via CQL. See CASSANDRA-10091 for more details.
+   - JSON timestamps are now in UTC and contain the timezone information, see CASSANDRA-11137 for more details.
    - Collision checks are performed when joining the token ring, regardless of whether
      the node should bootstrap. Additionally, replace_address can legitimately be used
      without bootstrapping to help with recovery of nodes with partially failed disks.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1
index d0a9a24..9373ba6 100644
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@ -446,10 +446,38 @@ Function SetCassandraEnvironment
     # with authentication and ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
     #
     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
-    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
+    #
+    # JMX SSL options
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true"
+    #$env:JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
+    #$env:JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.registry.ssl=true"
+    #$env:JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
+    #$env:JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
+    #$env:JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStore=C:/keystore"
+    #$env:JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
+    #$env:JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStore=C:/truststore"
+    #$env:JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
+    #
+    # JMX auth options
     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
+    ## Basic file based authn & authz
     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.password.file=C:/jmxremote.password"
-    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.access.file=C:/jmxremote.access"
+
+    ## Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities.
+    ## JAAS login modules can be used for authentication by uncommenting these two properties.
+    ## Cassandra ships with a LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule -
+    ## which delegates to the IAuthenticator configured in cassandra.yaml
+    #$env:JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin"
+    #$env:JVM_OPTS="$JVM_OPTS -Djava.security.auth.login.config=C:/cassandra-jaas.config"
+
+    ## Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer,
+    ## uncomment this to use it. Requires one of the two authentication options to be enabled
+    #$env:JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.authorizer=org.apache.cassandra.auth.jmx.AuthorizationProxy"
+
+    # Default JMX setup, bound to local loopback address only
+    $env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT"
 
     $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index 004a0fe..bcbba1e 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -238,23 +238,42 @@ fi
 JMX_PORT="7199"
 
 if [ "$LOCAL_JMX" = "yes" ]; then
-  JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
+  JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=false"
 else
-  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
+  JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.remote.port=$JMX_PORT"
   JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
-  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
   JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
-  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.password.file=/etc/cassandra/jmxremote.password"
-#  JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStore=/path/to/keystore"
-#  JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
-#  JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStore=/path/to/truststore"
-#  JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
-#  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
-#  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.registry.ssl=true"
-#  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
-#  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
 fi
 
+# jmx ssl options
+#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
+#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
+#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.registry.ssl=true"
+#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
+#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
+#JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStore=/path/to/keystore"
+#JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
+#JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStore=/path/to/truststore"
+#JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
+
+# jmx authentication and authorization options. By default, auth is only
+# activated for remote connections but they can also be enabled for local only JMX
+## Basic file based authn & authz
+JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.password.file=/etc/cassandra/jmxremote.password"
+#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.access.file=/etc/cassandra/jmxremote.access"
+## Custom auth settings which can be used as alternatives to JMX's out of the box auth utilities.
+## JAAS login modules can be used for authentication by uncommenting these two properties.
+## Cassandra ships with a LoginModule implementation - org.apache.cassandra.auth.CassandraLoginModule -
+## which delegates to the IAuthenticator configured in cassandra.yaml. See the sample JAAS configuration
+## file cassandra-jaas.config
+#JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin"
+#JVM_OPTS="$JVM_OPTS -Djava.security.auth.login.config=$CASSANDRA_HOME/conf/cassandra-jaas.config"
+
+## Cassandra also ships with a helper for delegating JMX authz calls to the configured IAuthorizer,
+## uncomment this to use it. Requires one of the two authentication options to be enabled
+#JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.authorizer=org.apache.cassandra.auth.jmx.AuthorizationProxy"
+
 # To use mx4j, an HTML interface for JMX, add mx4j-tools.jar to the lib/
 # directory.
 # See http://wiki.apache.org/cassandra/Operations#Monitoring_with_MX4J

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/conf/cassandra-jaas.config
----------------------------------------------------------------------
diff --git a/conf/cassandra-jaas.config b/conf/cassandra-jaas.config
new file mode 100644
index 0000000..f3a9bf7
--- /dev/null
+++ b/conf/cassandra-jaas.config
@@ -0,0 +1,4 @@
+// Delegates authentication to Cassandra's configured IAuthenticator
+CassandraLogin {
+  org.apache.cassandra.auth.CassandraLoginModule REQUIRED;
+};

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 9fd5f85..4afdb4a 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1452,6 +1452,7 @@ Permissions on resources are granted to roles; there are several different types
 * The hierarchy of Data resources, Keyspaces and Tables has the structure @ALL KEYSPACES@ -> @KEYSPACE@ -> @TABLE@
 * Function resources have the structure @ALL FUNCTIONS@ -> @KEYSPACE@ -> @FUNCTION@
 * Resources representing roles have the structure @ALL ROLES@ -> @ROLE@
+* Resources representing JMX ObjectNames, which map to sets of MBeans/MXBeans, have the structure @ALL MBEANS@ -> @MBEAN@
 
 Permissions can be granted at any level of these hierarchies and they flow downwards. So granting a permission on a resource higher up the chain automatically grants that same permission on all resources lower down. For example, granting @SELECT@ on a @KEYSPACE@ automatically grants it on all @TABLES@ in that @KEYSPACE@. Likewise, granting a permission on @ALL FUNCTIONS@ grants it on every defined function, regardless of which keyspace it is scoped in. It is also possible to grant permissions on all functions scoped to a particular keyspace. 
 
@@ -1467,7 +1468,7 @@ The full set of available permissions is:
 * @DESCRIBE@
 * @EXECUTE@
 
-Not all permissions are applicable to every type of resource. For instance, @EXECUTE@ is only relevant in the context of functions; granting @EXECUTE@ on a resource representing a table is nonsensical. Attempting to @GRANT@ a permission on resource to which it cannot be applied results in an error response. The following illustrates which permissions can be granted on which types of resource, and which statements are enabled by that permission.
+Not all permissions are applicable to every type of resource. For instance, @EXECUTE@ is only relevant in the context of functions or mbeans; granting @EXECUTE@ on a resource representing a table is nonsensical. Attempting to @GRANT@ a permission on resource to which it cannot be applied results in an error response. The following illustrates which permissions can be granted on which types of resource, and which statements are enabled by that permission.
 
 |_. permission |_. resource                   |_. operations        |
 | @CREATE@     | @ALL KEYSPACES@              |@CREATE KEYSPACE@ ==<br>== @CREATE TABLE@ in any keyspace|
@@ -1494,9 +1495,15 @@ Not all permissions are applicable to every type of resource. For instance, @EXE
 | @SELECT@     | @ALL KEYSPACES@              |@SELECT@ on any table|
 | @SELECT@     | @KEYSPACE@                   |@SELECT@ on any table in keyspace|
 | @SELECT@     | @TABLE@                      |@SELECT@ on specified table|
+| @SELECT@     | @ALL MBEANS@                 |Call getter methods on any mbean|
+| @SELECT@     | @MBEANS@                     |Call getter methods on any mbean matching a wildcard pattern|
+| @SELECT@     | @MBEAN@                      |Call getter methods on named mbean|
 | @MODIFY@     | @ALL KEYSPACES@              |@INSERT@ on any table ==<br>== @UPDATE@ on any table ==<br>== @DELETE@ on any table ==<br>== @TRUNCATE@ on any table|
-| @MODIFY@     | @KEYSPACE@                  |@INSERT@ on any table in keyspace ==<br>== @UPDATE@ on any table in keyspace ==<br>  == @DELETE@ on any table in keyspace ==<br>== @TRUNCATE@ on any table in keyspace
+| @MODIFY@     | @KEYSPACE@                   |@INSERT@ on any table in keyspace ==<br>== @UPDATE@ on any table in keyspace ==<br>  == @DELETE@ on any table in keyspace ==<br>== @TRUNCATE@ on any table in keyspace
 | @MODIFY@     | @TABLE@                      |@INSERT@ ==<br>== @UPDATE@ ==<br>== @DELETE@ ==<br>== @TRUNCATE@|
+| @MODIFY@     | @ALL MBEANS@                 |Call setter methods on any mbean|
+| @MODIFY@     | @MBEANS@                     |Call setter methods on any mbean matching a wildcard pattern|
+| @MODIFY@     | @MBEAN@                      |Call setter methods on named mbean|
 | @AUTHORIZE@  | @ALL KEYSPACES@              |@GRANT PERMISSION@ on any table ==<br>== @REVOKE PERMISSION@ on any table|
 | @AUTHORIZE@  | @KEYSPACE@                   |@GRANT PERMISSION@ on table in keyspace ==<br>== @REVOKE PERMISSION@ on table in keyspace|
 | @AUTHORIZE@  | @TABLE@                      |@GRANT PERMISSION@ ==<br>== @REVOKE PERMISSION@ |
@@ -1504,12 +1511,21 @@ Not all permissions are applicable to every type of resource. For instance, @EXE
 | @AUTHORIZE@  | @ALL FUNCTIONS IN KEYSPACE@  |@GRANT PERMISSION@ in keyspace ==<br>== @REVOKE PERMISSION@ in keyspace|
 | @AUTHORIZE@  | @ALL FUNCTIONS IN KEYSPACE@  |@GRANT PERMISSION@ in keyspace ==<br>== @REVOKE PERMISSION@ in keyspace|
 | @AUTHORIZE@  | @FUNCTION@                   |@GRANT PERMISSION@ ==<br>== @REVOKE PERMISSION@|
+| @AUTHORIZE@  | @ALL MBEANS@                 |@GRANT PERMISSION@ on any mbean ==<br>== @REVOKE PERMISSION@ on any mbean|
+| @AUTHORIZE@  | @MBEANS@                     |@GRANT PERMISSION@ on any mbean matching a wildcard pattern ==<br>== @REVOKE PERMISSION@ on any mbean matching a wildcard pattern|
+| @AUTHORIZE@  | @MBEAN@                      |@GRANT PERMISSION@ on named mbean ==<br>== @REVOKE PERMISSION@ on named mbean|
 | @AUTHORIZE@  | @ALL ROLES@                  |@GRANT ROLE@ grant any role ==<br>== @REVOKE ROLE@ revoke any role|
 | @AUTHORIZE@  | @ROLES@                      |@GRANT ROLE@ grant role ==<br>== @REVOKE ROLE@ revoke role|
 | @DESCRIBE@   | @ALL ROLES@                  |@LIST ROLES@ all roles or only roles granted to another, specified role|
+| @DESCRIBE@   | @ALL MBEANS                  |Retrieve metadata about any mbean from the platform's MBeanServer|
+| @DESCRIBE@   | @MBEANS                      |Retrieve metadata about any mbean matching a wildcard patter from the platform's MBeanServer|
+| @DESCRIBE@   | @MBEAN                       |Retrieve metadata about a named mbean from the platform's MBeanServer|
 | @EXECUTE@    | @ALL FUNCTIONS@              |@SELECT@, @INSERT@, @UPDATE@ using any function ==<br>== use of any function in @CREATE AGGREGATE@|
 | @EXECUTE@    | @ALL FUNCTIONS IN KEYSPACE@  |@SELECT@, @INSERT@, @UPDATE@ using any function in keyspace ==<br>== use of any function in keyspace in @CREATE AGGREGATE@|
 | @EXECUTE@    | @FUNCTION@                   |@SELECT@, @INSERT@, @UPDATE@ using function ==<br>== use of function in @CREATE AGGREGATE@|
+| @EXECUTE@    | @ALL MBEANS@                 |Execute operations on any mbean|
+| @EXECUTE@    | @MBEANS@                     |Execute operations on any mbean matching a wildcard pattern|
+| @EXECUTE@    | @MBEAN@                      |Execute operations on named mbean|
 
 
 h3(#grantPermissionsStmt). GRANT PERMISSION
@@ -1528,6 +1544,8 @@ bc(syntax)..
              | ROLE <identifier>
              | ALL FUNCTIONS ( IN KEYSPACE <identifier> )?
              | FUNCTION <functionname>
+             | ALL MBEANS
+             | ( MBEAN | MBEANS ) <objectName>
 p. 
 
 __Sample:__ 
@@ -1582,7 +1600,9 @@ bc(syntax)..
              | ROLE <identifier>
              | ALL FUNCTIONS ( IN KEYSPACE <identifier> )?
              | FUNCTION <functionname>
-p. 
+             | ALL MBEANS
+             | ( MBEAN | MBEANS ) <objectName>
+p.
 
 __Sample:__ 
 
@@ -1610,7 +1630,9 @@ bc(syntax)..
              | ROLE <identifier>
              | ALL FUNCTIONS ( IN KEYSPACE <identifier> )?
              | FUNCTION <functionname>
-p. 
+             | ALL MBEANS
+             | ( MBEAN | MBEANS ) <objectName>
+p.
 
 __Sample:__
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 7abed2c..9f1a022 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -1454,6 +1454,7 @@ syntax_rules += r'''
 <resource> ::= <dataResource>
              | <roleResource>
              | <functionResource>
+             | <jmxResource>
              ;
 
 <dataResource> ::= ( "ALL" "KEYSPACES" )
@@ -1472,6 +1473,11 @@ syntax_rules += r'''
                            ")" )
                        )
                      ;
+
+<jmxResource> ::= ( "ALL" "MBEANS")
+                | ( ( "MBEAN" | "MBEANS" ) <stringLiteral> )
+                ;
+
 '''
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/antlr/Lexer.g
----------------------------------------------------------------------
diff --git a/src/antlr/Lexer.g b/src/antlr/Lexer.g
index d93a5eb..16b2ac4 100644
--- a/src/antlr/Lexer.g
+++ b/src/antlr/Lexer.g
@@ -129,6 +129,8 @@ K_AUTHORIZE:   A U T H O R I Z E;
 K_DESCRIBE:    D E S C R I B E;
 K_EXECUTE:     E X E C U T E;
 K_NORECURSIVE: N O R E C U R S I V E;
+K_MBEAN:       M B E A N;
+K_MBEANS:      M B E A N S;
 
 K_USER:        U S E R;
 K_USERS:       U S E R S;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/antlr/Parser.g
----------------------------------------------------------------------
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index cd17475..deaf5d0 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -156,6 +156,31 @@ options {
         return filtered;
     }
 
+    public String canonicalizeObjectName(String s, boolean enforcePattern)
+    {
+        // these two conditions are here because technically they are valid
+        // ObjectNames, but we want to restrict their use without adding unnecessary
+        // work to JMXResource construction as that also happens on hotter code paths
+        if ("".equals(s))
+            addRecognitionError("Empty JMX object name supplied");
+
+        if ("*:*".equals(s))
+            addRecognitionError("Please use ALL MBEANS instead of wildcard pattern");
+
+        try
+        {
+            javax.management.ObjectName objectName = javax.management.ObjectName.getInstance(s);
+            if (enforcePattern && !objectName.isPattern())
+                addRecognitionError("Plural form used, but non-pattern JMX object name specified (" + s + ")");
+            return objectName.getCanonicalName();
+        }
+        catch (javax.management.MalformedObjectNameException e)
+        {
+          addRecognitionError(s + " is not a valid JMX object name");
+          return s;
+        }
+    }
+
 }
 
 /** STATEMENTS **/
@@ -932,6 +957,7 @@ resource returns [IResource res]
     : d=dataResource { $res = $d.res; }
     | r=roleResource { $res = $r.res; }
     | f=functionResource { $res = $f.res; }
+    | j=jmxResource { $res = $j.res; }
     ;
 
 dataResource returns [DataResource res]
@@ -941,6 +967,14 @@ dataResource returns [DataResource res]
       { $res = DataResource.table($cf.name.getKeyspace(), $cf.name.getColumnFamily()); }
     ;
 
+jmxResource returns [JMXResource res]
+    : K_ALL K_MBEANS { $res = JMXResource.root(); }
+    // when a bean name (or pattern) is supplied, validate that it's a legal ObjectName
+    // also, just to be picky, if the "MBEANS" form is used, only allow a pattern style names
+    | K_MBEAN mbean { $res = JMXResource.mbean(canonicalizeObjectName($mbean.text, false)); }
+    | K_MBEANS mbean { $res = JMXResource.mbean(canonicalizeObjectName($mbean.text, true)); }
+    ;
+
 roleResource returns [RoleResource res]
     : K_ALL K_ROLES { $res = RoleResource.root(); }
     | K_ROLE role = userOrRoleName { $res = RoleResource.role($role.name.getName()); }
@@ -1523,6 +1557,10 @@ username
     | QUOTED_NAME { addRecognitionError("Quoted strings are are not supported for user names and USER is deprecated, please use ROLE");}
     ;
 
+mbean
+    : STRING_LITERAL
+    ;
+
 // Basically the same as cident, but we need to exlude existing CQL3 types
 // (which for some reason are not reserved otherwise)
 non_type_ident returns [ColumnIdentifier id]

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/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 3b40979..d054aba 100644
--- a/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java
@@ -68,4 +68,9 @@ public class AllowAllAuthorizer implements IAuthorizer
     public void setup()
     {
     }
+
+    public Set<Permission> authorizeJMX(AuthenticatedUser parUser, IResource parResource)
+    {
+        return Permission.ALL;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index 37e01fc..619ecf8 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -41,6 +41,16 @@ import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.serializers.SetSerializer;
 import org.apache.cassandra.serializers.UTF8Serializer;
 import org.apache.cassandra.service.ClientState;
+import java.lang.management.ManagementFactory;
+
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.UntypedResultSet.Row;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -269,7 +279,7 @@ public class CassandraAuthorizer implements IAuthorizer
                                        RoleResource grantee)
     throws RequestValidationException, RequestExecutionException
     {
-        if (!performer.isSuper() && !performer.getRoles().contains(grantee))
+        if (!(performer.isSuper() || performer.isSystem()) && !performer.getRoles().contains(grantee))
             throw new UnauthorizedException(String.format("You are not authorized to view %s's permissions",
                                                           grantee == null ? "everyone" : grantee.getRoleName()));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/auth/CassandraLoginModule.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraLoginModule.java b/src/java/org/apache/cassandra/auth/CassandraLoginModule.java
new file mode 100644
index 0000000..2ccf962
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/CassandraLoginModule.java
@@ -0,0 +1,257 @@
+/*
+ * 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.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.*;
+import javax.security.auth.login.FailedLoginException;
+import javax.security.auth.login.LoginException;
+import javax.security.auth.spi.LoginModule;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.AuthenticationException;
+import org.apache.cassandra.service.StorageService;
+
+/**
+ * LoginModule which authenticates a user towards the Cassandra database using
+ * the internal authentication mechanism.
+ */
+public class CassandraLoginModule implements LoginModule
+{
+    private static final Logger logger = LoggerFactory.getLogger(CassandraLoginModule.class);
+
+    // initial state
+    private Subject subject;
+    private CallbackHandler callbackHandler;
+
+    // the authentication status
+    private boolean succeeded = false;
+    private boolean commitSucceeded = false;
+
+    // username and password
+    private String username;
+    private char[] password;
+
+    private CassandraPrincipal principal;
+
+    /**
+     * Initialize this {@code}LoginModule{@code}.
+     *
+     * @param subject the {@code}Subject{@code} to be authenticated. <p>
+     * @param callbackHandler a {@code}CallbackHandler{@code} for communicating
+     *        with the end user (prompting for user names and passwords, for example)
+     * @param sharedState shared {@code}LoginModule{@code} state. This param is unused.
+     * @param options options specified in the login {@code}Configuration{@code} for this particular
+     *        {@code}LoginModule{@code}. This param is unused
+     */
+    @Override
+    public void initialize(Subject subject,
+                           CallbackHandler callbackHandler,
+                           Map<java.lang.String, ?> sharedState,
+                           Map<java.lang.String, ?> options)
+    {
+        this.subject = subject;
+        this.callbackHandler = callbackHandler;
+    }
+
+    /**
+     * Authenticate the user, obtaining credentials from the CallbackHandler
+     * supplied in {@code}initialize{@code}. As long as the configured
+     * {@code}IAuthenticator{@code} supports the optional
+     * {@code}legacyAuthenticate{@code} method, it can be used here.
+     *
+     * @return true in all cases since this {@code}LoginModule{@code}
+     *         should not be ignored.
+     * @exception FailedLoginException if the authentication fails.
+     * @exception LoginException if this {@code}LoginModule{@code} is unable to
+     * perform the authentication.
+     */
+    @Override
+    public boolean login() throws LoginException
+    {
+        // prompt for a user name and password
+        if (callbackHandler == null)
+        {
+            logger.info("No CallbackHandler available for authentication");
+            throw new LoginException("Authentication failed");
+        }
+
+        NameCallback nc = new NameCallback("username: ");
+        PasswordCallback pc = new PasswordCallback("password: ", false);
+        try
+        {
+            callbackHandler.handle(new Callback[]{nc, pc});
+            username = nc.getName();
+            char[] tmpPassword = pc.getPassword();
+            if (tmpPassword == null)
+                tmpPassword = new char[0];
+            password = new char[tmpPassword.length];
+            System.arraycopy(tmpPassword, 0, password, 0, tmpPassword.length);
+            pc.clearPassword();
+        }
+        catch (IOException | UnsupportedCallbackException e)
+        {
+            logger.info("Unexpected exception processing authentication callbacks", e);
+            throw new LoginException("Authentication failed");
+        }
+
+        // verify the credentials
+        try
+        {
+            authenticate();
+        }
+        catch (AuthenticationException e)
+        {
+            // authentication failed -- clean up
+            succeeded = false;
+            cleanUpInternalState();
+            throw new FailedLoginException(e.getMessage());
+        }
+
+        succeeded = true;
+        return true;
+    }
+
+    private void authenticate()
+    {
+        if (!StorageService.instance.isAuthSetupComplete())
+            throw new AuthenticationException("Cannot login as server authentication setup is not yet completed");
+
+        IAuthenticator authenticator = DatabaseDescriptor.getAuthenticator();
+        Map<String, String> credentials = new HashMap<>();
+        credentials.put(PasswordAuthenticator.USERNAME_KEY, username);
+        credentials.put(PasswordAuthenticator.PASSWORD_KEY, String.valueOf(password));
+        AuthenticatedUser user = authenticator.legacyAuthenticate(credentials);
+        // Only actual users should be allowed to authenticate for JMX
+        if (user.isAnonymous() || user.isSystem())
+            throw new AuthenticationException("Invalid user");
+
+        // The LOGIN privilege is required to authenticate - c.f. ClientState::login
+        if (!DatabaseDescriptor.getRoleManager().canLogin(user.getPrimaryRole()))
+            throw new AuthenticationException(user.getName() + " is not permitted to log in");
+    }
+
+    /**
+     * This method is called if the LoginContext's overall authentication succeeded
+     * (the relevant REQUIRED, REQUISITE, SUFFICIENT and OPTIONAL LoginModules
+     * succeeded).
+     *
+     * If this LoginModule's own authentication attempt succeeded (checked by
+     * retrieving the private state saved by the {@code}login{@code} method),
+     * then this method associates a {@code}CassandraPrincipal{@code}
+     * with the {@code}Subject{@code}.
+     * If this LoginModule's own authentication attempted failed, then this
+     * method removes any state that was originally saved.
+     *
+     * @return true if this LoginModule's own login and commit attempts succeeded, false otherwise.
+     * @exception LoginException if the commit fails.
+     */
+    @Override
+    public boolean commit() throws LoginException
+    {
+        if (!succeeded)
+        {
+            return false;
+        }
+        else
+        {
+            // add a Principal (authenticated identity)
+            // to the Subject
+            principal = new CassandraPrincipal(username);
+            if (!subject.getPrincipals().contains(principal))
+                subject.getPrincipals().add(principal);
+
+            cleanUpInternalState();
+            commitSucceeded = true;
+            return true;
+        }
+    }
+
+    /**
+     * This method is called if the LoginContext's  overall authentication failed.
+     * (the relevant REQUIRED, REQUISITE, SUFFICIENT and OPTIONAL LoginModules
+     * did not succeed).
+     *
+     * If this LoginModule's own authentication attempt succeeded (checked by
+     * retrieving the private state saved by the {@code}login{@code} and
+     * {@code}commit{@code} methods), then this method cleans up any state that
+     * was originally saved.
+     *
+     * @return false if this LoginModule's own login and/or commit attempts failed, true otherwise.
+     * @throws LoginException if the abort fails.
+     */
+    @Override
+    public boolean abort() throws LoginException
+    {
+        if (!succeeded)
+        {
+            return false;
+        }
+        else if (!commitSucceeded)
+        {
+            // login succeeded but overall authentication failed
+            succeeded = false;
+            cleanUpInternalState();
+            principal = null;
+        }
+        else
+        {
+            // overall authentication succeeded and commit succeeded,
+            // but someone else's commit failed
+            logout();
+        }
+        return true;
+    }
+
+    /**
+     * Logout the user.
+     *
+     * This method removes the principal that was added by the
+     * {@code}commit{@code} method.
+     *
+     * @return true in all cases since this {@code}LoginModule{@code}
+     *         should not be ignored.
+     * @throws LoginException if the logout fails.
+     */
+    @Override
+    public boolean logout() throws LoginException
+    {
+        subject.getPrincipals().remove(principal);
+        succeeded = false;
+        cleanUpInternalState();
+        principal = null;
+        return true;
+    }
+
+    private void cleanUpInternalState()
+    {
+        username = null;
+        if (password != null)
+        {
+            for (int i = 0; i < password.length; i++)
+                password[i] = ' ';
+            password = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/auth/CassandraPrincipal.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraPrincipal.java b/src/java/org/apache/cassandra/auth/CassandraPrincipal.java
new file mode 100644
index 0000000..41de802
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/CassandraPrincipal.java
@@ -0,0 +1,134 @@
+/*
+ * 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.io.Serializable;
+import java.security.Principal;
+
+/**
+ * <p> This class implements the <code>Principal</code> interface
+ * and represents a user.
+ *
+ * <p> Principals such as this <code>CassPrincipal</code>
+ * may be associated with a particular <code>Subject</code>
+ * to augment that <code>Subject</code> with an additional
+ * identity.  Refer to the <code>Subject</code> class for more information
+ * on how to achieve this.  Authorization decisions can then be based upon
+ * the Principals associated with a <code>Subject</code>.
+ *
+ * @see java.security.Principal
+ * @see javax.security.auth.Subject
+ */
+public class CassandraPrincipal implements Principal, Serializable
+{
+
+    /**
+     *
+     */
+    private static final long serialVersionUID = 1L;
+    private final String name;
+
+    /**
+     * Create a CassPrincipal with a username.
+     *
+     * <p>
+     *
+     * @param name the username for this user.
+     *
+     * @exception NullPointerException if the <code>name</code>
+     *                  is <code>null</code>.
+     */
+    public CassandraPrincipal(String name)
+    {
+        if (name == null)
+            throw new NullPointerException("illegal null input");
+
+        this.name = name;
+    }
+
+    /**
+     * Return the username for this <code>CassPrincipal</code>.
+     *
+     * <p>
+     *
+     * @return the username for this <code>CassPrincipal</code>
+     */
+    @Override
+    public String getName()
+    {
+        return name;
+    }
+
+    /**
+     * Return a string representation of this <code>CassPrincipal</code>.
+     *
+     * <p>
+     *
+     * @return a string representation of this <code>CassPrincipal</code>.
+     */
+    @Override
+    public String toString()
+    {
+        return ("CassandraPrincipal:  " + name);
+    }
+
+    /**
+     * Compares the specified Object with this <code>CassPrincipal</code>
+     * for equality.  Returns true if the given object is also a
+     * <code>CassPrincipal</code> and the two CassPrincipals
+     * have the same username.
+     *
+     * <p>
+     *
+     * @param o Object to be compared for equality with this
+     *          <code>CassPrincipal</code>.
+     *
+     * @return true if the specified Object is equal equal to this
+     *          <code>CassPrincipal</code>.
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (o == null)
+            return false;
+
+        if (this == o)
+            return true;
+
+        if (!(o instanceof CassandraPrincipal))
+            return false;
+        CassandraPrincipal that = (CassandraPrincipal) o;
+
+        if (this.getName().equals(that.getName()))
+            return true;
+        return false;
+    }
+
+    /**
+     * Return a hash code for this <code>CassPrincipal</code>.
+     *
+     * <p>
+     *
+     * @return a hash code for this <code>CassPrincipal</code>.
+     */
+    @Override
+    public int hashCode()
+    {
+        return name.hashCode();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/auth/JMXResource.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/JMXResource.java b/src/java/org/apache/cassandra/auth/JMXResource.java
new file mode 100644
index 0000000..cb0ac41
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/JMXResource.java
@@ -0,0 +1,183 @@
+/*
+ * 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.lang.management.ManagementFactory;
+import java.util.Set;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Sets;
+import org.apache.commons.lang3.StringUtils;
+
+public class JMXResource implements IResource
+{
+    enum Level
+    {
+        ROOT, MBEAN
+    }
+
+    private static final String ROOT_NAME = "mbean";
+    private static final JMXResource ROOT_RESOURCE = new JMXResource();
+    private final Level level;
+    private final String name;
+
+    // permissions which may be granted on Mbeans
+    private static final Set<Permission> JMX_PERMISSIONS = Sets.immutableEnumSet(Permission.AUTHORIZE,
+                                                                                 Permission.DESCRIBE,
+                                                                                 Permission.EXECUTE,
+                                                                                 Permission.MODIFY,
+                                                                                 Permission.SELECT);
+
+    private JMXResource()
+    {
+        level = Level.ROOT;
+        name = null;
+    }
+
+    private JMXResource(String name)
+    {
+        this.name = name;
+        level = Level.MBEAN;
+    }
+
+    public static JMXResource mbean(String name)
+    {
+        return new JMXResource(name);
+    }
+
+    /**
+     * Parses a role resource name into a RoleResource instance.
+     *
+     * @param name Name of the data resource.
+     * @return RoleResource instance matching the name.
+     */
+    public static JMXResource fromName(String name)
+    {
+        String[] parts = StringUtils.split(name, '/');
+
+        if (!parts[0].equals(ROOT_NAME) || parts.length > 2)
+            throw new IllegalArgumentException(String.format("%s is not a valid JMX resource name", name));
+
+        if (parts.length == 1)
+            return root();
+
+        return mbean(parts[1]);
+    }
+
+    @Override
+    public String getName()
+    {
+        if (level == Level.ROOT)
+            return ROOT_NAME;
+        else if (level == Level.MBEAN)
+            return String.format("%s/%s", ROOT_NAME, name);
+        throw new AssertionError();
+    }
+
+    /**
+     * @return for a non-root resource, return the short form of the resource name which represents an ObjectName
+     * (which may be of the pattern or exact kind). i.e. not the full "root/name" version returned by getName().
+     * Throws IllegalStateException if called on the root-level resource.
+     */
+    public String getObjectName()
+    {
+        if (level == Level.ROOT)
+            throw new IllegalStateException(String.format("%s JMX resource has no object name", level));
+        return name;
+    }
+
+    /**
+     * @return the root-level resource.
+     */
+    public static JMXResource root()
+    {
+        return ROOT_RESOURCE;
+    }
+
+    @Override
+    public IResource getParent()
+    {
+        if (level == Level.MBEAN)
+            return root();
+        throw new IllegalStateException("Root-level resource can't have a parent");
+    }
+
+    /**
+     * @return Whether or not the resource has a parent in the hierarchy.
+     */
+    @Override
+    public boolean hasParent()
+    {
+        return !level.equals(Level.ROOT);
+    }
+
+    @Override
+    public boolean exists()
+    {
+        if (!hasParent())
+            return true;
+        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+        try
+        {
+            return !(mbs.queryNames(new ObjectName(name), null).isEmpty());
+        }
+        catch (MalformedObjectNameException e)
+        {
+            return false;
+        }
+        catch (NullPointerException e)
+        {
+            return false;
+        }
+    }
+
+    @Override
+    public Set<Permission> applicablePermissions()
+    {
+        return JMX_PERMISSIONS;
+    }
+
+    @Override
+    public String toString()
+    {
+        return level == Level.ROOT ? "<all mbeans>" : String.format("<mbean %s>", name);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof JMXResource))
+            return false;
+
+        JMXResource j = (JMXResource) o;
+
+        return Objects.equal(level, j.level) && Objects.equal(name, j.name);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(level, name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/auth/Permission.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/Permission.java b/src/java/org/apache/cassandra/auth/Permission.java
index d552280..0231270 100644
--- a/src/java/org/apache/cassandra/auth/Permission.java
+++ b/src/java/org/apache/cassandra/auth/Permission.java
@@ -63,7 +63,9 @@ public enum Permission
     // UDF permissions
     EXECUTE;  // required to invoke any user defined function or aggregate
 
+    public static final Set<Permission> ALL_DATA =
+            ImmutableSet.copyOf(EnumSet.range(Permission.CREATE, Permission.EXECUTE));
     public static final Set<Permission> ALL =
-            Sets.immutableEnumSet(EnumSet.range(Permission.CREATE, Permission.EXECUTE));
+            ImmutableSet.copyOf(EnumSet.range(Permission.CREATE, Permission.EXECUTE));
     public static final Set<Permission> NONE = ImmutableSet.of();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/auth/Resources.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/Resources.java b/src/java/org/apache/cassandra/auth/Resources.java
index ebcfc16..653cd46 100644
--- a/src/java/org/apache/cassandra/auth/Resources.java
+++ b/src/java/org/apache/cassandra/auth/Resources.java
@@ -58,6 +58,8 @@ public final class Resources
             return DataResource.fromName(name);
         else if (name.startsWith(FunctionResource.root().getName()))
             return FunctionResource.fromName(name);
+        else if (name.startsWith(JMXResource.root().getName()))
+            return JMXResource.fromName(name);
         else
             throw new IllegalArgumentException(String.format("Name %s is not valid for any resource type", name));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/auth/jmx/AuthenticationProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/jmx/AuthenticationProxy.java b/src/java/org/apache/cassandra/auth/jmx/AuthenticationProxy.java
new file mode 100644
index 0000000..67c9d8c
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/jmx/AuthenticationProxy.java
@@ -0,0 +1,157 @@
+/*
+ * 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.jmx;
+
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+import javax.management.remote.JMXAuthenticator;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.*;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.exceptions.ConfigurationException;
+
+/**
+ * An alternative to the JAAS based implementation of JMXAuthenticator provided
+ * by the JDK (JMXPluggableAuthenticator).
+ *
+ * Authentication is performed via delegation to a LoginModule. The JAAS login
+ * config is specified by passing its identifier in a custom system property:
+ *     cassandra.jmx.remote.login.config
+ *
+ * The location of the JAAS configuration file containing that config is
+ * specified in the standard way, using the java.security.auth.login.config
+ * system property.
+ *
+ * If authentication is successful then a Subject containing one or more
+ * Principals is returned. This Subject may then be used during authorization
+ * if a JMX authorization is enabled.
+ */
+public final class AuthenticationProxy implements JMXAuthenticator
+{
+    private static Logger logger = LoggerFactory.getLogger(AuthenticationProxy.class);
+
+    // Identifier of JAAS configuration to be used for subject authentication
+    private final String loginConfigName;
+
+    /**
+     * Creates an instance of <code>JMXPluggableAuthenticator</code>
+     * and initializes it with a {@link LoginContext}.
+     *
+     * @param loginConfigName name of the specifig JAAS login configuration to
+     *                        use when authenticating JMX connections
+     * @throws SecurityException if the authentication mechanism cannot be
+     *         initialized.
+     */
+    public AuthenticationProxy(String loginConfigName)
+    {
+        if (loginConfigName == null)
+            throw new ConfigurationException("JAAS login configuration missing for JMX authenticator setup");
+
+        this.loginConfigName = loginConfigName;
+    }
+
+    /**
+     * Perform authentication of the client opening the {@code}MBeanServerConnection{@code}
+     *
+     * @param credentials optionally these credentials may be supplied by the JMX user.
+     *                    Out of the box, the JDK's {@code}RMIServerImpl{@code} is capable
+     *                    of supplying a two element String[], containing username & password.
+     *                    If present, these credentials will be made available to configured
+     *                    {@code}LoginModule{@code}s via {@code}JMXCallbackHandler{@code}.
+     *
+     * @return the authenticated subject containing any {@code}Principal{@code}s added by
+     *the {@code}LoginModule{@code}s
+     *
+     * @throws SecurityException if the server cannot authenticate the user
+     *         with the provided credentials.
+     */
+    public Subject authenticate(Object credentials)
+    {
+        // The credentials object is expected to be a string array holding the subject's
+        // username & password. Those values are made accessible to LoginModules via the
+        // JMXCallbackHandler.
+        JMXCallbackHandler callbackHandler = new JMXCallbackHandler(credentials);
+        try
+        {
+            LoginContext loginContext = new LoginContext(loginConfigName, callbackHandler);
+            loginContext.login();
+            final Subject subject = loginContext.getSubject();
+            if (!subject.isReadOnly())
+            {
+                AccessController.doPrivileged((PrivilegedAction<Void>) () -> {
+                    subject.setReadOnly();
+                    return null;
+                });
+            }
+
+            return subject;
+        }
+        catch (LoginException e)
+        {
+            logger.trace("Authentication exception", e);
+            throw new SecurityException("Authentication error", e);
+        }
+    }
+
+    /**
+     * This callback handler supplies the username and password (which was
+     * optionally supplied by the JMX user) to the JAAS login module performing
+     * the authentication, should it require them . No interactive user
+     * prompting is necessary because the credentials are already available to
+     * this class (via its enclosing class).
+     */
+    private final class JMXCallbackHandler implements CallbackHandler
+    {
+        private char[] username;
+        private char[] password;
+        private JMXCallbackHandler(Object credentials)
+        {
+            // if username/password credentials were supplied, store them in
+            // the relevant variables to make them accessible to LoginModules
+            // via JMXCallbackHandler
+            if (credentials instanceof String[])
+            {
+                String[] strings = (String[]) credentials;
+                if (strings[0] != null)
+                    username = strings[0].toCharArray();
+                if (strings[1] != null)
+                    password = strings[1].toCharArray();
+            }
+        }
+
+        public void handle(Callback[] callbacks) throws UnsupportedCallbackException
+        {
+            for (int i = 0; i < callbacks.length; i++)
+            {
+                if (callbacks[i] instanceof NameCallback)
+                    ((NameCallback)callbacks[i]).setName(username == null ? null : new String(username));
+                else if (callbacks[i] instanceof PasswordCallback)
+                    ((PasswordCallback)callbacks[i]).setPassword(password == null ? null : password);
+                else
+                    throw new UnsupportedCallbackException(callbacks[i], "Unrecognized Callback: " + callbacks[i].getClass().getName());
+            }
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
new file mode 100644
index 0000000..3bac1f6
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
@@ -0,0 +1,512 @@
+/*
+ * 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.jmx;
+
+import java.lang.reflect.*;
+import java.security.AccessControlContext;
+import java.security.AccessController;
+import java.security.Principal;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.remote.MBeanServerForwarder;
+import javax.security.auth.Subject;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * Provides a proxy interface to the platform's MBeanServer instance to perform
+ * role-based authorization on method invocation.
+ *
+ * When used in conjunction with a suitable JMXAuthenticator, which attaches a CassandraPrincipal
+ * to authenticated Subjects, this class uses the configured IAuthorizer to verify that the
+ * subject has the required permissions to execute methods on the MBeanServer and the MBeans it
+ * manages.
+ *
+ * Because an ObjectName may contain wildcards, meaning it represents a set of individual MBeans,
+ * JMX resources don't fit well with the hierarchical approach modelled by other IResource
+ * implementations and utilised by ClientState::ensureHasPermission etc. To enable grants to use
+ * pattern-type ObjectNames, this class performs its own custom matching and filtering of resources
+ * rather than pushing that down to the configured IAuthorizer. To that end, during authorization
+ * it pulls back all permissions for the active subject, filtering them to retain only grants on
+ * JMXResources. It then uses ObjectName::apply to assert whether the target MBeans are wholly
+ * represented by the resources with permissions. This means that it cannot use the PermissionsCache
+ * as IAuthorizer can, so it manages its own cache locally.
+ *
+ * Methods are split into 2 categories; those which are to be invoked on the MBeanServer itself
+ * and those which apply to MBean instances. Actually, this is somewhat of a construct as in fact
+ * *all* invocations are performed on the MBeanServer instance, the distinction is made here on
+ * those methods which take an ObjectName as their first argument and those which do not.
+ * Invoking a method of the former type, e.g. MBeanServer::getAttribute(ObjectName name, String attribute),
+ * implies that the caller is concerned with a specific MBean. Conversely, invoking a method such as
+ * MBeanServer::getDomains is primarily a function of the MBeanServer itself. This class makes
+ * such a distinction in order to identify which JMXResource the subject requires permissions on.
+ *
+ * Certain operations are never allowed for users and these are recorded in a blacklist so that we
+ * can short circuit authorization process if one is attempted by a remote subject.
+ *
+ */
+public class AuthorizationProxy implements InvocationHandler
+{
+    private static final Logger logger = LoggerFactory.getLogger(AuthorizationProxy.class);
+
+    /*
+     A whitelist of permitted methods on the MBeanServer interface which *do not* take an ObjectName
+     as their first argument. These methods can be thought of as relating to the MBeanServer itself,
+     rather than to the MBeans it manages. All of the whitelisted methods are essentially descriptive,
+     hence they require the Subject to have the DESCRIBE permission on the root JMX resource.
+     */
+    private static final Set<String> MBEAN_SERVER_METHOD_WHITELIST = ImmutableSet.of("getDefaultDomain",
+                                                                                     "getDomains",
+                                                                                     "getMBeanCount",
+                                                                                     "hashCode",
+                                                                                     "queryMBeans",
+                                                                                     "queryNames",
+                                                                                     "toString");
+
+    /*
+     A blacklist of method names which are never permitted to be executed by a remote user,
+     regardless of privileges they may be granted.
+     */
+    private static final Set<String> METHOD_BLACKLIST = ImmutableSet.of("createMBean",
+                                                                        "deserialize",
+                                                                        "getClassLoader",
+                                                                        "getClassLoaderFor",
+                                                                        "instantiate",
+                                                                        "registerMBean",
+                                                                        "unregisterMBean");
+
+    private static final JMXPermissionsCache permissionsCache = new JMXPermissionsCache();
+    private MBeanServer mbs;
+
+    /*
+     Used to check whether the Role associated with the authenticated Subject has superuser
+     status. By default, just delegates to Roles::hasSuperuserStatus, but can be overridden for testing.
+     */
+    protected Function<RoleResource, Boolean> isSuperuser = Roles::hasSuperuserStatus;
+
+    /*
+     Used to retrieve the set of all permissions granted to a given role. By default, this fetches
+     the permissions from the local cache, which in turn loads them from the configured IAuthorizer
+     but can be overridden for testing.
+     */
+    protected Function<RoleResource, Set<PermissionDetails>> getPermissions = permissionsCache::get;
+
+    /*
+     Used to decide whether authorization is enabled or not, usually this depends on the configured
+     IAuthorizer, but can be overridden for testing.
+     */
+    protected Supplier<Boolean> isAuthzRequired = () -> DatabaseDescriptor.getAuthorizer().requireAuthorization();
+
+    /*
+     Used to find matching MBeans when the invocation target is a pattern type ObjectName.
+     Defaults to querying the MBeanServer but can be overridden for testing. See checkPattern for usage.
+     */
+    protected Function<ObjectName, Set<ObjectName>> queryNames = (name) -> mbs.queryNames(name, null);
+
+    /*
+     Used to determine whether auth setup has completed so we know whether the expect the IAuthorizer
+     to be ready. Can be overridden for testing.
+     */
+    protected Supplier<Boolean> isAuthSetupComplete = () -> StorageService.instance.isAuthSetupComplete();
+
+    @Override
+    public Object invoke(Object proxy, Method method, Object[] args)
+            throws Throwable
+    {
+        String methodName = method.getName();
+
+        if ("getMBeanServer".equals(methodName))
+            throw new SecurityException("Access denied");
+
+        // Retrieve Subject from current AccessControlContext
+        AccessControlContext acc = AccessController.getContext();
+        Subject subject = Subject.getSubject(acc);
+
+        // Allow setMBeanServer iff performed on behalf of the connector server itself
+        if (("setMBeanServer").equals(methodName))
+        {
+            if (subject != null)
+                throw new SecurityException("Access denied");
+
+            if (args[0] == null)
+                throw new IllegalArgumentException("Null MBeanServer");
+
+            if (mbs != null)
+                throw new IllegalArgumentException("MBeanServer already initialized");
+
+            mbs = (MBeanServer) args[0];
+            return null;
+        }
+
+        if (authorize(subject, methodName, args))
+            return invoke(method, args);
+
+        throw new SecurityException("Access Denied");
+    }
+
+    /**
+     * Performs the actual authorization of an identified subject to execute a remote method invocation.
+     * @param subject The principal making the execution request. A null value represents a local invocation
+     *                from the JMX connector itself
+     * @param methodName Name of the method being invoked
+     * @param args Array containing invocation argument. If the first element is an ObjectName instance, for
+     *             authz purposes we consider this an invocation of an MBean method, otherwise it is treated
+     *             as an invocation of a method on the MBeanServer.
+     */
+    @VisibleForTesting
+    boolean authorize(Subject subject, String methodName, Object[] args)
+    {
+        logger.trace("Authorizing JMX method invocation {} for {}",
+                     methodName,
+                     subject == null ? "" :subject.toString().replaceAll("\\n", " "));
+
+        if (!isAuthSetupComplete.get())
+        {
+            logger.trace("Auth setup is not complete, refusing access");
+            return false;
+        }
+
+        // Permissive authorization is enabled
+        if (!isAuthzRequired.get())
+            return true;
+
+        // Allow operations performed locally on behalf of the connector server itself
+        if (subject == null)
+            return true;
+
+        // Restrict access to certain methods by any remote user
+        if (METHOD_BLACKLIST.contains(methodName))
+        {
+            logger.trace("Access denied to blacklisted method {}", methodName);
+            return false;
+        }
+
+        // Reject if the user has not authenticated
+        Set<Principal> principals = subject.getPrincipals();
+        if (principals == null || principals.isEmpty())
+            return false;
+
+        // Currently, we assume that the first Principal returned from the Subject
+        // is the one to use for authorization. It would be good to make this more
+        // robust, but we have no control over which Principals a given LoginModule
+        // might choose to associate with the Subject following successful authentication
+        RoleResource userResource = RoleResource.role(principals.iterator().next().getName());
+        // A role with superuser status can do anything
+        if (isSuperuser.apply(userResource))
+            return true;
+
+        // The method being invoked may be a method on an MBean, or it could belong
+        // to the MBeanServer itself
+        if (args != null && args[0] instanceof ObjectName)
+            return authorizeMBeanMethod(userResource, methodName, args);
+        else
+            return authorizeMBeanServerMethod(userResource, methodName);
+    }
+
+    /**
+     * Authorize execution of a method on the MBeanServer which does not take an MBean ObjectName
+     * as its first argument. The whitelisted methods that match this criteria are generally
+     * descriptive methods concerned with the MBeanServer itself, rather than with any particular
+     * set of MBeans managed by the server and so we check the DESCRIBE permission on the root
+     * JMXResource (representing the MBeanServer)
+     *
+     * @param subject
+     * @param methodName
+     * @return the result of the method invocation, if authorized
+     * @throws Throwable
+     * @throws SecurityException if authorization fails
+     */
+    private boolean authorizeMBeanServerMethod(RoleResource subject, String methodName)
+    {
+        logger.trace("JMX invocation of {} on MBeanServer requires permission {}", methodName, Permission.DESCRIBE);
+        return (MBEAN_SERVER_METHOD_WHITELIST.contains(methodName) &&
+            hasPermission(subject, Permission.DESCRIBE, JMXResource.root()));
+    }
+
+    /**
+     * Authorize execution of a method on an MBean (or set of MBeans) which may be
+     * managed by the MBeanServer. Note that this also includes the queryMBeans and queryNames
+     * methods of MBeanServer as those both take an ObjectName (possibly a pattern containing
+     * wildcards) as their first argument. They both of those methods also accept null arguments,
+     * in which case they will be handled by authorizedMBeanServerMethod
+     *
+     * @param role
+     * @param methodName
+     * @param args
+     * @return the result of the method invocation, if authorized
+     * @throws Throwable
+     * @throws SecurityException if authorization fails
+     */
+    private boolean authorizeMBeanMethod(RoleResource role, String methodName, Object[] args)
+    {
+        ObjectName targetBean = (ObjectName)args[0];
+
+        // work out which permission we need to execute the method being called on the mbean
+        Permission requiredPermission = getRequiredPermission(methodName);
+        if (null == requiredPermission)
+            return false;
+
+        logger.trace("JMX invocation of {} on {} requires permission {}", methodName, targetBean, requiredPermission);
+
+        // find any JMXResources upon which the authenticated subject has been granted the
+        // reqired permission. We'll do ObjectName-specific filtering & matching of resources later
+        Set<JMXResource> permittedResources = getPermittedResources(role, requiredPermission);
+
+        if (permittedResources.isEmpty())
+            return false;
+
+        // finally, check the JMXResource from the grants to see if we have either
+        // an exact match or a wildcard match for the target resource, whichever is
+        // applicable
+        return targetBean.isPattern()
+                ? checkPattern(targetBean, permittedResources)
+                : checkExact(targetBean, permittedResources);
+    }
+
+    /**
+     * Get any grants of the required permission for the authenticated subject, regardless
+     * of the resource the permission applies to as we'll do the filtering & matching in
+     * the calling method
+     * @param subject
+     * @param required
+     * @return the set of JMXResources upon which the subject has been granted the required permission
+     */
+    private Set<JMXResource> getPermittedResources(RoleResource subject, Permission required)
+    {
+        return getPermissions.apply(subject)
+               .stream()
+               .filter(details -> details.permission == required)
+               .map(details -> (JMXResource)details.resource)
+               .collect(Collectors.toSet());
+    }
+
+    /**
+     * Check whether a required permission has been granted to the authenticated subject on a specific resource
+     * @param subject
+     * @param permission
+     * @param resource
+     * @return true if the Subject has been granted the required permission on the specified resource; false otherwise
+     */
+    private boolean hasPermission(RoleResource subject, Permission permission, JMXResource resource)
+    {
+        return getPermissions.apply(subject)
+               .stream()
+               .anyMatch(details -> details.permission == permission && details.resource.equals(resource));
+    }
+
+    /**
+     * Given a set of JMXResources upon which the Subject has been granted a particular permission,
+     * check whether any match the pattern-type ObjectName representing the target of the method
+     * invocation. At this point, we are sure that whatever the required permission, the Subject
+     * has definitely been granted it against this set of JMXResources. The job of this method is
+     * only to verify that the target of the invocation is covered by the members of the set.
+     *
+     * @param target
+     * @param permittedResources
+     * @return true if all registered beans which match the target can also be matched by the
+     *         JMXResources the subject has been granted permissions on; false otherwise
+     */
+    private boolean checkPattern(ObjectName target, Set<JMXResource> permittedResources)
+    {
+        // if the required permission was granted on the root JMX resource, then we're done
+        if (permittedResources.contains(JMXResource.root()))
+            return true;
+
+        // Get the full set of beans which match the target pattern
+        Set<ObjectName> targetNames = queryNames.apply(target);
+
+        // Iterate over the resources the permission has been granted on. Some of these may
+        // be patterns, so query the server to retrieve the full list of matching names and
+        // remove those from the target set. Once the target set is empty (i.e. all required
+        // matches have been satisfied), the requirement is met.
+        // If there are still unsatisfied targets after all the JMXResources have been processed,
+        // there are insufficient grants to permit the operation.
+        for (JMXResource resource : permittedResources)
+        {
+            try
+            {
+                Set<ObjectName> matchingNames = queryNames.apply(ObjectName.getInstance(resource.getObjectName()));
+                targetNames.removeAll(matchingNames);
+                if (targetNames.isEmpty())
+                    return true;
+            }
+            catch (MalformedObjectNameException e)
+            {
+                logger.warn("Permissions for JMX resource contains invalid ObjectName {}", resource.getObjectName());
+            }
+        }
+
+        logger.trace("Subject does not have sufficient permissions on all MBeans matching the target pattern {}", target);
+        return false;
+    }
+
+    /**
+     * Given a set of JMXResources upon which the Subject has been granted a particular permission,
+     * check whether any match the ObjectName representing the target of the method invocation.
+     * At this point, we are sure that whatever the required permission, the Subject has definitely
+     * been granted it against this set of JMXResources. The job of this method is only to verify
+     * that the target of the invocation is matched by a member of the set.
+     *
+     * @param target
+     * @param permittedResources
+     * @return true if at least one of the permitted resources matches the target; false otherwise
+     */
+    private boolean checkExact(ObjectName target, Set<JMXResource> permittedResources)
+    {
+        // if the required permission was granted on the root JMX resource, then we're done
+        if (permittedResources.contains(JMXResource.root()))
+            return true;
+
+        for (JMXResource resource : permittedResources)
+        {
+            try
+            {
+                if (ObjectName.getInstance(resource.getObjectName()).apply(target))
+                    return true;
+            }
+            catch (MalformedObjectNameException e)
+            {
+                logger.warn("Permissions for JMX resource contains invalid ObjectName {}", resource.getObjectName());
+            }
+        }
+
+        logger.trace("Subject does not have sufficient permissions on target MBean {}", target);
+        return false;
+    }
+
+    /**
+     * Mapping between method names and the permission required to invoke them. Note, these
+     * names refer to methods on MBean instances invoked via the MBeanServer.
+     * @param methodName
+     * @return
+     */
+    private static Permission getRequiredPermission(String methodName)
+    {
+        switch (methodName)
+        {
+            case "getAttribute":
+            case "getAttributes":
+                return Permission.SELECT;
+            case "setAttribute":
+            case "setAttributes":
+                return Permission.MODIFY;
+            case "invoke":
+                return Permission.EXECUTE;
+            case "getInstanceOf":
+            case "getMBeanInfo":
+            case "hashCode":
+            case "isInstanceOf":
+            case "isRegistered":
+            case "queryMBeans":
+            case "queryNames":
+                return Permission.DESCRIBE;
+            default:
+                logger.debug("Access denied, method name {} does not map to any defined permission", methodName);
+                return null;
+        }
+    }
+
+    /**
+     * Invoke a method on the MBeanServer instance. This is called when authorization is not required (because
+     * AllowAllAuthorizer is configured, or because the invocation is being performed by the JMXConnector
+     * itself rather than by a connected client), and also when a call from an authenticated subject
+     * has been successfully authorized
+     *
+     * @param method
+     * @param args
+     * @return
+     * @throws Throwable
+     */
+    private Object invoke(Method method, Object[] args) throws Throwable
+    {
+        try
+        {
+            return method.invoke(mbs, args);
+        }
+        catch (InvocationTargetException e) //Catch any exception that might have been thrown by the mbeans
+        {
+            Throwable t = e.getCause(); //Throw the exception that nodetool etc expects
+            throw t;
+        }
+    }
+
+    /**
+     * Query the configured IAuthorizer for the set of all permissions granted on JMXResources to a specific subject
+     * @param subject
+     * @return All permissions granted to the specfied subject (including those transitively inherited from
+     *         any roles the subject has been granted), filtered to include only permissions granted on
+     *         JMXResources
+     */
+    private static Set<PermissionDetails> loadPermissions(RoleResource subject)
+    {
+        // get all permissions for the specified subject. We'll cache them as it's likely
+        // we'll receive multiple lookups for the same subject (but for different resources
+        // and permissions) in quick succession
+        return DatabaseDescriptor.getAuthorizer().list(AuthenticatedUser.SYSTEM_USER, Permission.ALL, null, subject)
+                                                 .stream()
+                                                 .filter(details -> details.resource instanceof JMXResource)
+                                                 .collect(Collectors.toSet());
+    }
+
+    private static final class JMXPermissionsCache extends AuthCache<RoleResource, Set<PermissionDetails>>
+    {
+        protected JMXPermissionsCache()
+        {
+            super("JMXPermissionsCache",
+                  DatabaseDescriptor::setPermissionsValidity,
+                  DatabaseDescriptor::getPermissionsValidity,
+                  DatabaseDescriptor::setPermissionsUpdateInterval,
+                  DatabaseDescriptor::getPermissionsUpdateInterval,
+                  DatabaseDescriptor::setPermissionsCacheMaxEntries,
+                  DatabaseDescriptor::getPermissionsCacheMaxEntries,
+                  AuthorizationProxy::loadPermissions,
+                  () -> true);
+        }
+
+        public Set<PermissionDetails> get(RoleResource roleResource)
+        {
+            try
+            {
+                return super.get(roleResource);
+            }
+            catch (Exception e)
+            {
+                // because the outer class uses this method as Function<RoleResource, Set<PermissionDetails>>,
+                // which can be overridden for testing, it cannot throw checked exceptions. So here we simply
+                // use guava's propagation helper.
+                throw Throwables.propagate(e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/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 0c233a7..c8f8f34 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -24,38 +24,42 @@ import java.lang.management.MemoryPoolMXBean;
 import java.net.InetAddress;
 import java.net.URL;
 import java.net.UnknownHostException;
+import java.rmi.RemoteException;
 import java.rmi.registry.LocateRegistry;
+import java.rmi.server.RMIClientSocketFactory;
 import java.rmi.server.RMIServerSocketFactory;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.TimeUnit;
-
+import java.util.stream.Collectors;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 import javax.management.StandardMBean;
 import javax.management.remote.JMXConnectorServer;
 import javax.management.remote.JMXServiceURL;
 import javax.management.remote.rmi.RMIConnectorServer;
+import javax.rmi.ssl.SslRMIClientSocketFactory;
+import javax.rmi.ssl.SslRMIServerSocketFactory;
 
-import com.addthis.metrics3.reporter.config.ReporterConfig;
-import com.codahale.metrics.Meter;
-import com.codahale.metrics.MetricRegistryListener;
-import com.codahale.metrics.SharedMetricRegistries;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
-
+import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.*;
+import com.addthis.metrics3.reporter.config.ReporterConfig;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricRegistryListener;
+import com.codahale.metrics.SharedMetricRegistries;
+import org.apache.cassandra.auth.jmx.AuthorizationProxy;
+import org.apache.cassandra.batchlog.LegacyBatchlogMigrator;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.functions.ThreadAwareSecurityManager;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.batchlog.LegacyBatchlogMigrator;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.StartupException;
@@ -68,7 +72,6 @@ import org.apache.cassandra.metrics.CassandraMetricsRegistry;
 import org.apache.cassandra.metrics.DefaultNameFactory;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.schema.LegacySchemaMigrator;
-import org.apache.cassandra.cql3.functions.ThreadAwareSecurityManager;
 import org.apache.cassandra.thrift.ThriftServer;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.*;
@@ -82,7 +85,6 @@ import org.apache.cassandra.utils.*;
 public class CassandraDaemon
 {
     public static final String MBEAN_NAME = "org.apache.cassandra.db:type=NativeAccess";
-    private static JMXConnectorServer jmxServer = null;
 
     private static final Logger logger;
     static {
@@ -104,28 +106,40 @@ public class CassandraDaemon
         logger = LoggerFactory.getLogger(CassandraDaemon.class);
     }
 
-    private static void maybeInitJmx()
+    private void maybeInitJmx()
     {
-        if (System.getProperty("com.sun.management.jmxremote.port") != null)
-            return;
+        System.setProperty("java.rmi.server.randomIDs", "true");
+
+        // If a remote port has been specified then use that to set up a JMX
+        // connector server which can be accessed remotely. Otherwise, look
+        // for the local port property and create a server which is bound
+        // only to the loopback address. Auth options are applied to both
+        // remote and local-only servers, but currently SSL is only
+        // available for remote.
+        // If neither is remote nor local port is set in cassandra-env.(sh|ps)
+        // then JMX is effectively  disabled.
+        boolean localOnly = false;
+        String jmxPort = System.getProperty("cassandra.jmx.remote.port");
+
+        if (jmxPort == null)
+        {
+            localOnly = true;
+            jmxPort = System.getProperty("cassandra.jmx.local.port");
+        }
 
-        String jmxPort = System.getProperty("cassandra.jmx.local.port");
         if (jmxPort == null)
             return;
 
-        System.setProperty("java.rmi.server.hostname", InetAddress.getLoopbackAddress().getHostAddress());
-        RMIServerSocketFactory serverFactory = new RMIServerSocketFactoryImpl();
-        Map<String, ?> env = Collections.singletonMap(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory);
         try
         {
-            LocateRegistry.createRegistry(Integer.valueOf(jmxPort), null, serverFactory);
-            JMXServiceURL url = new JMXServiceURL(String.format("service:jmx:rmi://localhost/jndi/rmi://localhost:%s/jmxrmi", jmxPort));
-            jmxServer = new RMIConnectorServer(url, env, ManagementFactory.getPlatformMBeanServer());
+            jmxServer = JMXServerUtils.createJMXServer(Integer.parseInt(jmxPort), localOnly);
+            if (jmxServer == null)
+                return;
             jmxServer.start();
         }
         catch (IOException e)
         {
-            logger.error("Error starting local jmx server: ", e);
+            logger.error("Error starting JMX server: ", e);
         }
     }
 
@@ -133,6 +147,7 @@ public class CassandraDaemon
 
     public Server thriftServer;
     private NativeTransportService nativeTransportService;
+    private JMXConnectorServer jmxServer;
 
     private final boolean runManaged;
     protected final StartupChecks startupChecks;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
index f0ad457..d1c1943 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -145,7 +145,7 @@ public class StartupChecks
     {
         public void execute()
         {
-            String jmxPort = System.getProperty("com.sun.management.jmxremote.port");
+            String jmxPort = System.getProperty("cassandra.jmx.remote.port");
             if (jmxPort == null)
             {
                 logger.warn("JMX is not enabled to receive remote connections. Please see cassandra-env.sh for more info.");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b0c7164/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index d4ad59a..01ae884 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -254,6 +254,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     private volatile boolean initialized = false;
     private volatile boolean joined = false;
     private volatile boolean gossipActive = false;
+    private volatile boolean authSetupComplete = false;
 
     /* the probability for tracing any particular request, 0 disables tracing and 1 enables for all */
     private double traceProbability = 0.0;
@@ -1119,6 +1120,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         DatabaseDescriptor.getAuthenticator().setup();
         DatabaseDescriptor.getAuthorizer().setup();
         MigrationManager.instance.register(new AuthMigrationListener());
+        authSetupComplete = true;
+    }
+
+    public boolean isAuthSetupComplete()
+    {
+        return authSetupComplete;
     }
 
     private void maybeAddKeyspace(KeyspaceMetadata ksm)