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/07/20 12:11:47 UTC

[1/3] cassandra git commit: Use custom RMI registry to avoid issues with JMX and SSL

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.9 8df6d4d4c -> 9eae8d340
  refs/heads/trunk c193c2c43 -> a41fb5075


Use custom RMI registry to avoid issues with JMX and SSL

Patch by Sam Tunnicliffe; reviewed by Jake Luciani for CASSANDRA-12109


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

Branch: refs/heads/cassandra-3.9
Commit: 9eae8d340cb00fcabcab9b1c8c0d85943eac061b
Parents: 8df6d4d
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 30 16:47:10 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 20 13:06:02 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 conf/cassandra-env.ps1                          |  21 ++--
 conf/cassandra-env.sh                           |  25 ++--
 .../cassandra/service/CassandraDaemon.java      |   1 -
 .../apache/cassandra/utils/JMXServerUtils.java  | 126 ++++++++++++++++---
 5 files changed, 132 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index dac46ab..c9dce65 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,9 +1,11 @@
 3.9
+ * Fix SSL JMX requiring truststore containing server cert (CASSANDRA-12109)
 Merged from 3.0:
  * Fix problem with undeleteable rows on upgrade to new sstable format (CASSANDRA-12144)
 Merged from 2.2:
  * Fixed cqlshlib.test.remove_test_db (CASSANDRA-12214)
 
+
 3.8
  * Fix hdr logging for single operation workloads (CASSANDRA-12145)
  * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1
index 9373ba6..d7a4867 100644
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@ -450,14 +450,13 @@ Function SetCassandraEnvironment
     #
     # 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>"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStore=C:/keystore"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStore=C:/truststore"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
     #
     # JMX auth options
     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
@@ -469,12 +468,12 @@ Function SetCassandraEnvironment
     ## 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"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin"
+    #$env:JVM_OPTS="$env: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"
+    #$env:JVM_OPTS="$env: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"

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index 93434c9..5a02f79 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -242,20 +242,23 @@ if [ "$LOCAL_JMX" = "yes" ]; then
   JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=false"
 else
   JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.remote.port=$JMX_PORT"
+  # if ssl is enabled the same port cannot be used for both jmx and rmi so either
+  # pick another value for this property or comment out to use a random port (though see CASSANDRA-7087 for origins)
   JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
+
+  # turn on JMX authentication. See below for further options
   JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
-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 ssl options
+  #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true"
+  #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=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>"
+fi
 
 # jmx authentication and authorization options. By default, auth is only
 # activated for remote connections but they can also be enabled for local only JMX

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/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 88b3c88..0151208 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -137,7 +137,6 @@ public class CassandraDaemon
             jmxServer = JMXServerUtils.createJMXServer(Integer.parseInt(jmxPort), localOnly);
             if (jmxServer == null)
                 return;
-            jmxServer.start();
         }
         catch (IOException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/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
index ad87efd..dad757e 100644
--- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java
+++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
@@ -23,10 +23,7 @@ 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.*;
 import java.rmi.server.RMIClientSocketFactory;
 import java.rmi.server.RMIServerSocketFactory;
 import java.rmi.server.UnicastRemoteObject;
@@ -36,7 +33,6 @@ 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;
@@ -49,12 +45,15 @@ 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 org.apache.cassandra.exceptions.ConfigurationException;
+import sun.rmi.registry.RegistryImpl;
 import sun.rmi.server.UnicastServerRef2;
 
 public class JMXServerUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(JMXServerUtils.class);
 
+    private static java.rmi.registry.Registry registry;
 
     /**
      * Creates a server programmatically. This allows us to set parameters which normally are
@@ -74,12 +73,8 @@ public class JMXServerUtils
         }
 
         // Configure the RMI client & server socket factories, including SSL config.
-        env.putAll(configureJmxSocketFactories(serverAddress));
+        env.putAll(configureJmxSocketFactories(serverAddress, local));
 
-        String url = String.format(urlTemplate, (serverAddress != null ? serverAddress.getHostAddress() : "0.0.0.0"), 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.
@@ -96,8 +91,11 @@ public class JMXServerUtils
         // sun.rmi.dgc.server.gcInterval millis (default is 3600000ms/1 hour)
         env.put(RMIExporter.EXPORTER_ATTRIBUTE, new Exporter());
 
+        String url = String.format(urlTemplate, (serverAddress != null ? serverAddress.getHostAddress() : "0.0.0.0"), port);
+
+        int rmiPort = Integer.getInteger("com.sun.management.jmxremote.rmi.port", 0);
         JMXConnectorServer jmxServer =
-            JMXConnectorServerFactory.newJMXConnectorServer(new JMXServiceURL(url),
+            JMXConnectorServerFactory.newJMXConnectorServer(new JMXServiceURL("rmi", null, rmiPort),
                                                             env,
                                                             ManagementFactory.getPlatformMBeanServer());
 
@@ -105,10 +103,33 @@ public class JMXServerUtils
         if (authzProxy != null)
             jmxServer.setMBeanServerForwarder(authzProxy);
 
+        jmxServer.start();
+
+        // use a custom Registry to avoid having to interact with it internally using the remoting interface
+        configureRMIRegistry(port, env);
+
         logger.info("Configured JMX server at: {}", url);
         return jmxServer;
     }
 
+    private static void configureRMIRegistry(int port, Map<String, Object> env) throws RemoteException
+    {
+        Exporter exporter = (Exporter)env.get(RMIExporter.EXPORTER_ATTRIBUTE);
+        // If ssl is enabled, make sure it's also in place for the RMI registry
+        // by using the SSL socket factories already created and stashed in env
+        if (Boolean.getBoolean("com.sun.management.jmxremote.ssl"))
+        {
+            registry = new Registry(port,
+                                   (RMIClientSocketFactory)env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE),
+                                   (RMIServerSocketFactory)env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE),
+                                   exporter.connectorServer);
+        }
+        else
+        {
+            registry = new Registry(port, exporter.connectorServer);
+        }
+    }
+
     private static Map<String, Object> configureJmxAuthentication()
     {
         Map<String, Object> env = new HashMap<>();
@@ -173,7 +194,7 @@ public class JMXServerUtils
         }
     }
 
-    private static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress)
+    private static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress, boolean localOnly)
     {
         Map<String, Object> env = new HashMap<>();
         if (Boolean.getBoolean("com.sun.management.jmxremote.ssl"))
@@ -202,8 +223,7 @@ public class JMXServerUtils
             env.put("com.sun.jndi.rmi.factory.socket", clientFactory);
             logJmxSslConfig(serverFactory);
         }
-        else
-        {
+        else if (localOnly){
             env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE,
                     new RMIServerSocketFactoryImpl(serverAddress));
         }
@@ -265,16 +285,20 @@ public class JMXServerUtils
      */
     private static class Exporter implements RMIExporter
     {
+        // the first object to be exported by this instance is *always* the JMXConnectorServer
+        // instance created by createJMXServer. Keep a handle to it, as it needs to be supplied
+        // to our custom Registry too.
+        private Remote connectorServer;
+
         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
+            Remote remote = new UnicastServerRef2(port, csf, ssf).exportObject(obj, null, true);
+            // Keep a reference to the first object exported, the JMXConnectorServer
+            if (connectorServer == null)
+                connectorServer = remote;
 
-            // 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);
+            return remote;
         }
 
         public boolean unexportObject(Remote obj, boolean force) throws NoSuchObjectException
@@ -282,4 +306,66 @@ public class JMXServerUtils
             return UnicastRemoteObject.unexportObject(obj, force);
         }
     }
+
+    /**
+     * Using this class avoids the necessity to interact with the registry via its
+     * remoting interface. This is necessary because when SSL is enabled for the registry,
+     * that remote interaction is treated just the same as one from an external client.
+     * That is problematic when binding the JMXConnectorServer to the Registry as it requires
+     * the client, which in this case is our own internal code, to connect like any other SSL
+     * client, meaning we need a truststore containing our own certificate.
+     * This bypasses the binding API completely, which emulates the behaviour of
+     * ConnectorBootstrap when the subsystem is initialized by the JVM Agent directly.
+     *
+     * See CASSANDRA-12109.
+     */
+    private static class Registry extends RegistryImpl
+    {
+        private final static String KEY = "jmxrmi";
+        private final Remote connectorServer;
+
+        private Registry(int port, Remote connectorServer) throws RemoteException
+        {
+            super(port);
+            this.connectorServer = connectorServer;
+        }
+
+        private Registry(int port,
+                         RMIClientSocketFactory csf,
+                         RMIServerSocketFactory ssf,
+                         Remote connectorServer) throws RemoteException
+        {
+            super(port, csf, ssf);
+            this.connectorServer = connectorServer;
+        }
+
+        public Remote lookup(String name) throws RemoteException, NotBoundException
+        {
+            if (name.equals(KEY))
+                return connectorServer;
+
+            throw new NotBoundException(String.format("Only the JMX Connector Server named %s " +
+                                                      "is bound in this registry", KEY));
+        }
+
+        public void bind(String name, Remote obj) throws RemoteException, AlreadyBoundException
+        {
+            throw new UnsupportedOperationException("Unsupported");
+        }
+
+        public void unbind(String name) throws RemoteException, NotBoundException
+        {
+            throw new UnsupportedOperationException("Unsupported");
+        }
+
+        public void rebind(String name, Remote obj) throws RemoteException
+        {
+            throw new UnsupportedOperationException("Unsupported");
+        }
+
+        public String[] list() throws RemoteException
+        {
+            return new String[] {KEY};
+        }
+    }
 }


[3/3] cassandra git commit: Merge branch 'cassandra-3.9' into trunk

Posted by sa...@apache.org.
Merge branch 'cassandra-3.9' into trunk


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

Branch: refs/heads/trunk
Commit: a41fb50750f0ff034daf1734f5a90c274b098f3b
Parents: c193c2c 9eae8d3
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Wed Jul 20 13:06:25 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 20 13:06:25 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 conf/cassandra-env.ps1                          |  21 ++--
 conf/cassandra-env.sh                           |  25 ++--
 .../cassandra/service/CassandraDaemon.java      |   1 -
 .../apache/cassandra/utils/JMXServerUtils.java  | 125 ++++++++++++++++---
 5 files changed, 132 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a41fb507/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c34a76f,c9dce65..679f60f
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,20 -1,5 +1,21 @@@
 +3.10
 + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424)
 + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035)
 + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635)
 + * Prepend snapshot name with "truncated" or "dropped" when a snapshot
 +   is taken before truncating or dropping a table (CASSANDRA-12178)
 + * Optimize RestrictionSet (CASSANDRA-12153)
 + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150)
 + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613)
 + * Create a system table to expose prepared statements (CASSANDRA-8831)
 + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970)
 + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580)
 + * Add supplied username to authentication error messages (CASSANDRA-12076)
 + * Remove pre-startup check for open JMX port (CASSANDRA-12074)
 +
 +
  3.9
+  * Fix SSL JMX requiring truststore containing server cert (CASSANDRA-12109)
  Merged from 3.0:
   * Fix problem with undeleteable rows on upgrade to new sstable format (CASSANDRA-12144)
  Merged from 2.2:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a41fb507/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------


[2/3] cassandra git commit: Use custom RMI registry to avoid issues with JMX and SSL

Posted by sa...@apache.org.
Use custom RMI registry to avoid issues with JMX and SSL

Patch by Sam Tunnicliffe; reviewed by Jake Luciani for CASSANDRA-12109


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

Branch: refs/heads/trunk
Commit: 9eae8d340cb00fcabcab9b1c8c0d85943eac061b
Parents: 8df6d4d
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Thu Jun 30 16:47:10 2016 +0100
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Wed Jul 20 13:06:02 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 conf/cassandra-env.ps1                          |  21 ++--
 conf/cassandra-env.sh                           |  25 ++--
 .../cassandra/service/CassandraDaemon.java      |   1 -
 .../apache/cassandra/utils/JMXServerUtils.java  | 126 ++++++++++++++++---
 5 files changed, 132 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index dac46ab..c9dce65 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,9 +1,11 @@
 3.9
+ * Fix SSL JMX requiring truststore containing server cert (CASSANDRA-12109)
 Merged from 3.0:
  * Fix problem with undeleteable rows on upgrade to new sstable format (CASSANDRA-12144)
 Merged from 2.2:
  * Fixed cqlshlib.test.remove_test_db (CASSANDRA-12214)
 
+
 3.8
  * Fix hdr logging for single operation workloads (CASSANDRA-12145)
  * Fix SASI PREFIX search in CONTAINS mode with partial terms (CASSANDRA-12073)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/conf/cassandra-env.ps1
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1
index 9373ba6..d7a4867 100644
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@ -450,14 +450,13 @@ Function SetCassandraEnvironment
     #
     # 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>"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStore=C:/keystore"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStore=C:/truststore"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
     #
     # JMX auth options
     #$env:JVM_OPTS="$env:JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
@@ -469,12 +468,12 @@ Function SetCassandraEnvironment
     ## 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"
+    #$env:JVM_OPTS="$env:JVM_OPTS -Dcassandra.jmx.remote.login.config=CassandraLogin"
+    #$env:JVM_OPTS="$env: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"
+    #$env:JVM_OPTS="$env: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"

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index 93434c9..5a02f79 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -242,20 +242,23 @@ if [ "$LOCAL_JMX" = "yes" ]; then
   JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=false"
 else
   JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.remote.port=$JMX_PORT"
+  # if ssl is enabled the same port cannot be used for both jmx and rmi so either
+  # pick another value for this property or comment out to use a random port (though see CASSANDRA-7087 for origins)
   JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
+
+  # turn on JMX authentication. See below for further options
   JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
-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 ssl options
+  #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=true"
+  #JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=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>"
+fi
 
 # jmx authentication and authorization options. By default, auth is only
 # activated for remote connections but they can also be enabled for local only JMX

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/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 88b3c88..0151208 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -137,7 +137,6 @@ public class CassandraDaemon
             jmxServer = JMXServerUtils.createJMXServer(Integer.parseInt(jmxPort), localOnly);
             if (jmxServer == null)
                 return;
-            jmxServer.start();
         }
         catch (IOException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9eae8d34/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
index ad87efd..dad757e 100644
--- a/src/java/org/apache/cassandra/utils/JMXServerUtils.java
+++ b/src/java/org/apache/cassandra/utils/JMXServerUtils.java
@@ -23,10 +23,7 @@ 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.*;
 import java.rmi.server.RMIClientSocketFactory;
 import java.rmi.server.RMIServerSocketFactory;
 import java.rmi.server.UnicastRemoteObject;
@@ -36,7 +33,6 @@ 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;
@@ -49,12 +45,15 @@ 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 org.apache.cassandra.exceptions.ConfigurationException;
+import sun.rmi.registry.RegistryImpl;
 import sun.rmi.server.UnicastServerRef2;
 
 public class JMXServerUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(JMXServerUtils.class);
 
+    private static java.rmi.registry.Registry registry;
 
     /**
      * Creates a server programmatically. This allows us to set parameters which normally are
@@ -74,12 +73,8 @@ public class JMXServerUtils
         }
 
         // Configure the RMI client & server socket factories, including SSL config.
-        env.putAll(configureJmxSocketFactories(serverAddress));
+        env.putAll(configureJmxSocketFactories(serverAddress, local));
 
-        String url = String.format(urlTemplate, (serverAddress != null ? serverAddress.getHostAddress() : "0.0.0.0"), 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.
@@ -96,8 +91,11 @@ public class JMXServerUtils
         // sun.rmi.dgc.server.gcInterval millis (default is 3600000ms/1 hour)
         env.put(RMIExporter.EXPORTER_ATTRIBUTE, new Exporter());
 
+        String url = String.format(urlTemplate, (serverAddress != null ? serverAddress.getHostAddress() : "0.0.0.0"), port);
+
+        int rmiPort = Integer.getInteger("com.sun.management.jmxremote.rmi.port", 0);
         JMXConnectorServer jmxServer =
-            JMXConnectorServerFactory.newJMXConnectorServer(new JMXServiceURL(url),
+            JMXConnectorServerFactory.newJMXConnectorServer(new JMXServiceURL("rmi", null, rmiPort),
                                                             env,
                                                             ManagementFactory.getPlatformMBeanServer());
 
@@ -105,10 +103,33 @@ public class JMXServerUtils
         if (authzProxy != null)
             jmxServer.setMBeanServerForwarder(authzProxy);
 
+        jmxServer.start();
+
+        // use a custom Registry to avoid having to interact with it internally using the remoting interface
+        configureRMIRegistry(port, env);
+
         logger.info("Configured JMX server at: {}", url);
         return jmxServer;
     }
 
+    private static void configureRMIRegistry(int port, Map<String, Object> env) throws RemoteException
+    {
+        Exporter exporter = (Exporter)env.get(RMIExporter.EXPORTER_ATTRIBUTE);
+        // If ssl is enabled, make sure it's also in place for the RMI registry
+        // by using the SSL socket factories already created and stashed in env
+        if (Boolean.getBoolean("com.sun.management.jmxremote.ssl"))
+        {
+            registry = new Registry(port,
+                                   (RMIClientSocketFactory)env.get(RMIConnectorServer.RMI_CLIENT_SOCKET_FACTORY_ATTRIBUTE),
+                                   (RMIServerSocketFactory)env.get(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE),
+                                   exporter.connectorServer);
+        }
+        else
+        {
+            registry = new Registry(port, exporter.connectorServer);
+        }
+    }
+
     private static Map<String, Object> configureJmxAuthentication()
     {
         Map<String, Object> env = new HashMap<>();
@@ -173,7 +194,7 @@ public class JMXServerUtils
         }
     }
 
-    private static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress)
+    private static Map<String, Object> configureJmxSocketFactories(InetAddress serverAddress, boolean localOnly)
     {
         Map<String, Object> env = new HashMap<>();
         if (Boolean.getBoolean("com.sun.management.jmxremote.ssl"))
@@ -202,8 +223,7 @@ public class JMXServerUtils
             env.put("com.sun.jndi.rmi.factory.socket", clientFactory);
             logJmxSslConfig(serverFactory);
         }
-        else
-        {
+        else if (localOnly){
             env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE,
                     new RMIServerSocketFactoryImpl(serverAddress));
         }
@@ -265,16 +285,20 @@ public class JMXServerUtils
      */
     private static class Exporter implements RMIExporter
     {
+        // the first object to be exported by this instance is *always* the JMXConnectorServer
+        // instance created by createJMXServer. Keep a handle to it, as it needs to be supplied
+        // to our custom Registry too.
+        private Remote connectorServer;
+
         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
+            Remote remote = new UnicastServerRef2(port, csf, ssf).exportObject(obj, null, true);
+            // Keep a reference to the first object exported, the JMXConnectorServer
+            if (connectorServer == null)
+                connectorServer = remote;
 
-            // 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);
+            return remote;
         }
 
         public boolean unexportObject(Remote obj, boolean force) throws NoSuchObjectException
@@ -282,4 +306,66 @@ public class JMXServerUtils
             return UnicastRemoteObject.unexportObject(obj, force);
         }
     }
+
+    /**
+     * Using this class avoids the necessity to interact with the registry via its
+     * remoting interface. This is necessary because when SSL is enabled for the registry,
+     * that remote interaction is treated just the same as one from an external client.
+     * That is problematic when binding the JMXConnectorServer to the Registry as it requires
+     * the client, which in this case is our own internal code, to connect like any other SSL
+     * client, meaning we need a truststore containing our own certificate.
+     * This bypasses the binding API completely, which emulates the behaviour of
+     * ConnectorBootstrap when the subsystem is initialized by the JVM Agent directly.
+     *
+     * See CASSANDRA-12109.
+     */
+    private static class Registry extends RegistryImpl
+    {
+        private final static String KEY = "jmxrmi";
+        private final Remote connectorServer;
+
+        private Registry(int port, Remote connectorServer) throws RemoteException
+        {
+            super(port);
+            this.connectorServer = connectorServer;
+        }
+
+        private Registry(int port,
+                         RMIClientSocketFactory csf,
+                         RMIServerSocketFactory ssf,
+                         Remote connectorServer) throws RemoteException
+        {
+            super(port, csf, ssf);
+            this.connectorServer = connectorServer;
+        }
+
+        public Remote lookup(String name) throws RemoteException, NotBoundException
+        {
+            if (name.equals(KEY))
+                return connectorServer;
+
+            throw new NotBoundException(String.format("Only the JMX Connector Server named %s " +
+                                                      "is bound in this registry", KEY));
+        }
+
+        public void bind(String name, Remote obj) throws RemoteException, AlreadyBoundException
+        {
+            throw new UnsupportedOperationException("Unsupported");
+        }
+
+        public void unbind(String name) throws RemoteException, NotBoundException
+        {
+            throw new UnsupportedOperationException("Unsupported");
+        }
+
+        public void rebind(String name, Remote obj) throws RemoteException
+        {
+            throw new UnsupportedOperationException("Unsupported");
+        }
+
+        public String[] list() throws RemoteException
+        {
+            return new String[] {KEY};
+        }
+    }
 }