You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by mc...@apache.org on 2022/10/18 21:25:48 UTC

[cassandra] branch cassandra-3.0 updated: Fix JMX security vulnerabilities

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

mck pushed a commit to branch cassandra-3.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cassandra-3.0 by this push:
     new b2660bcf78 Fix JMX security vulnerabilities
b2660bcf78 is described below

commit b2660bcf78ccf08a3a0ae39a8c9ffb397efef9ff
Author: Abe Ratnofsky <ab...@aber.io>
AuthorDate: Tue Aug 30 15:23:26 2022 -0700

    Fix JMX security vulnerabilities
    
     patch by Abe Ratnofsky; reviewed by Jon Meredith, Mick Semb Wever, Sam Tunnicliffe for CASSANDRA-17921
---
 CHANGES.txt                                        |   1 +
 conf/cassandra-env.sh                              |   4 +
 ide/idea/workspace.xml                             |   2 +-
 .../cassandra/auth/jmx/AuthorizationProxy.java     | 170 +++++++++++++++++++++
 .../apache/cassandra/service/CassandraDaemon.java  |  28 ++++
 5 files changed, 204 insertions(+), 1 deletion(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 7dd28c8a84..799d0b11d9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.28
+ * Harden JMX by resolving beanshooter issues (CASSANDRA-17921)
  * Suppress CVE-2019-2684 (CASSANDRA-17965)
  * Fix auto-completing "WITH" when creating a materialized view (CASSANDRA-17879)
  * Fix scrubber falling into infinite loop when the last partition is broken (CASSANDRA-17862)
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index 016e592a9f..a5ac96c6c5 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -294,6 +294,10 @@ if [ "x$LOCAL_JMX" = "x" ]; then
     LOCAL_JMX=yes
 fi
 
+## Cassandra also ships with a helper for protecting against security gaps in a default JMX configuration. To use it,
+## uncomment line below.
+#JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.authorizer=org.apache.cassandra.auth.jmx.AuthorizationProxy"
+
 # Specifies the default port over which Cassandra will be available for
 # JMX connections.
 # For security reasons, you should not expose this port to the internet.  Firewall it if needed.
diff --git a/ide/idea/workspace.xml b/ide/idea/workspace.xml
index 4a7a5c9028..a3fda6ac5d 100644
--- a/ide/idea/workspace.xml
+++ b/ide/idea/workspace.xml
@@ -188,7 +188,7 @@
     <configuration default="false" name="Cassandra" type="Application" factoryName="Application">
       <extension name="coverage" enabled="false" merge="false" sample_coverage="true" runner="idea" />
       <option name="MAIN_CLASS_NAME" value="org.apache.cassandra.service.CassandraDaemon" />
-      <option name="VM_PARAMETERS" value="-Dcassandra-foreground=yes -Dcassandra.config=file://$PROJECT_DIR$/conf/cassandra.yaml -Dcassandra.storagedir=$PROJECT_DIR$/data -Dlogback.configurationFile=file://$PROJECT_DIR$/conf/logback.xml -Dcassandra.logdir=$PROJECT_DIR$/data/logs -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=7199 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false - [...]
+      <option name="VM_PARAMETERS" value="-Dcassandra-foreground=yes -Dcassandra.config=file://$PROJECT_DIR$/conf/cassandra.yaml -Dcassandra.storagedir=$PROJECT_DIR$/data -Dlogback.configurationFile=file://$PROJECT_DIR$/conf/logback.xml -Dcassandra.logdir=$PROJECT_DIR$/data/logs -Dcassandra.jmx.local.port=7199 -ea -Xmx1G" />
       <option name="PROGRAM_PARAMETERS" value="" />
       <option name="WORKING_DIRECTORY" value="file://$PROJECT_DIR$" />
       <option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
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 0000000000..65f7d20202
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java
@@ -0,0 +1,170 @@
+/*
+ * 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 javax.management.InstanceNotFoundException;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+import javax.security.auth.Subject;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides a proxy interface to the platform's MBeanServer instance to perform minimal authorization and prevent
+ * certain known security issues. In Cassandra 3.11+, this goes even further to include resource-based authorization
+ * controls.
+ *
+ * Certain operations are never allowed for users and these are recorded in a deny list 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);
+
+    private MBeanServer mbs;
+
+    @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");
+
+        // Corresponds to MBeanServer.invoke
+        if (methodName.equals("invoke") && args.length == 4)
+            checkVulnerableMethods(args);
+
+        // 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;
+        }
+
+        return invoke(method, args);
+    }
+
+    /**
+     * 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;
+        }
+    }
+
+    private void checkVulnerableMethods(Object args[])
+    {
+        assert args.length == 4;
+        ObjectName name;
+        String operationName;
+        Object[] params;
+        String[] signature;
+        try
+        {
+            name = (ObjectName) args[0];
+            operationName = (String) args[1];
+            params = (Object[]) args[2];
+            signature = (String[]) args[3];
+        }
+        catch (ClassCastException cce)
+        {
+            logger.warn("Could not interpret arguments to check vulnerable MBean invocations; did the MBeanServer interface change?", cce);
+            return;
+        }
+
+        // When adding compiler directives from a file, most JDKs will log the file contents if invalid, which
+        // leads to an arbitrary file read vulnerability
+        checkCompilerDirectiveAddMethods(name, operationName);
+
+        // Loading arbitrary (JVM and native) libraries from remotes
+        checkJvmtiLoad(name, operationName);
+        checkMLetMethods(name, operationName);
+    }
+
+    private void checkCompilerDirectiveAddMethods(ObjectName name, String operation)
+    {
+        if (name.getCanonicalName().equals("com.sun.management:type=DiagnosticCommand")
+                && operation.equals("compilerDirectivesAdd"))
+            throw new SecurityException("Access is denied!");
+    }
+
+    private void checkJvmtiLoad(ObjectName name, String operation)
+    {
+        if (name.getCanonicalName().equals("com.sun.management:type=DiagnosticCommand")
+                && operation.equals("jvmtiAgentLoad"))
+            throw new SecurityException("Access is denied!");
+    }
+
+    private void checkMLetMethods(ObjectName name, String operation)
+    {
+        // Inspired by MBeanServerAccessController, but that class ignores check if a SecurityManager is installed,
+        // which we don't want
+
+        if (operation == null)
+            return;
+
+        try
+        {
+            if (!mbs.isInstanceOf(name, "javax.management.loading.MLet"))
+                return;
+        } catch (InstanceNotFoundException infe)
+        {
+            return;
+        }
+
+        if (operation.equals("addURL") || operation.equals("getMBeansFromURL"))
+            throw new SecurityException("Access is denied!");
+    }
+}
+
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index ce0b8f3396..1df6161555 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -21,6 +21,8 @@ import java.io.File;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryPoolMXBean;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Proxy;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.rmi.AccessException;
@@ -39,6 +41,7 @@ import javax.management.ObjectName;
 import javax.management.StandardMBean;
 import javax.management.remote.JMXConnectorServer;
 import javax.management.remote.JMXServiceURL;
+import javax.management.remote.MBeanServerForwarder;
 import javax.management.remote.rmi.RMIConnectorServer;
 import javax.management.remote.rmi.RMIJRMPServerImpl;
 
@@ -100,6 +103,7 @@ public class CassandraDaemon
 {
     public static final String MBEAN_NAME = "org.apache.cassandra.db:type=NativeAccess";
     private static JMXConnectorServer jmxServer = null;
+    private static MBeanServerForwarder authzProxy = null;
 
     private static final Logger logger;
 
@@ -137,6 +141,8 @@ public class CassandraDaemon
         if (jmxPort == null)
             return;
 
+        MBeanServerForwarder authzProxy = createAuthzProxy();
+
         System.setProperty("java.rmi.server.hostname", InetAddress.getLoopbackAddress().getHostAddress());
         RMIServerSocketFactory serverFactory = new RMIServerSocketFactoryImpl();
         Map<String, Object> env = new HashMap<>();
@@ -153,6 +159,10 @@ public class CassandraDaemon
                                                              (RMIServerSocketFactory) env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE),
                                                              env);
             jmxServer = new RMIConnectorServer(url, env, server, ManagementFactory.getPlatformMBeanServer());
+
+            if (authzProxy != null)
+                jmxServer.setMBeanServerForwarder(authzProxy);
+
             jmxServer.start();
             ((JmxRegistry)registry).setRemoteServerStub(server.toStub());
         }
@@ -162,6 +172,24 @@ public class CassandraDaemon
         }
     }
 
+    private MBeanServerForwarder createAuthzProxy()
+    {
+        // 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)
+            return 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);
+    }
+
     private static final CassandraDaemon instance = new CassandraDaemon();
 
     private volatile Server thriftServer;


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