You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2018/02/09 13:34:30 UTC

cassandra git commit: Add hot reloading of SSL Certificates

Repository: cassandra
Updated Branches:
  refs/heads/trunk 714703a08 -> 473e8dfd7


Add hot reloading of SSL Certificates

patch by Dinesh Joshi; reviewed by jasobrown for CASSANDRA-14222


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

Branch: refs/heads/trunk
Commit: 473e8dfd7be95815ee10502f021bd7deb8734fba
Parents: 714703a
Author: Dinesh Joshi <di...@yahoo.com>
Authored: Wed Feb 7 15:08:01 2018 -0800
Committer: Jason Brown <ja...@gmail.com>
Committed: Fri Feb 9 05:32:21 2018 -0800

----------------------------------------------------------------------
 doc/source/operating/security.rst               |  10 ++
 .../cassandra/config/DatabaseDescriptor.java    |   8 +
 .../apache/cassandra/net/MessagingService.java  |   7 +
 .../cassandra/net/MessagingServiceMBean.java    |   2 +
 .../apache/cassandra/security/SSLFactory.java   | 148 ++++++++++++++++++-
 .../org/apache/cassandra/tools/NodeProbe.java   |   5 +
 .../org/apache/cassandra/tools/NodeTool.java    |   3 +-
 .../cassandra/tools/ReloadSslCertificates.java  |  30 ++++
 .../cassandra/security/SSLFactoryTest.java      |  38 ++++-
 9 files changed, 239 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/473e8dfd/doc/source/operating/security.rst
----------------------------------------------------------------------
diff --git a/doc/source/operating/security.rst b/doc/source/operating/security.rst
index 21245fd..1859dbc 100644
--- a/doc/source/operating/security.rst
+++ b/doc/source/operating/security.rst
@@ -56,6 +56,16 @@ for more details.
 For information on generating the keystore and truststore files used in SSL communications, see the
 `java documentation on creating keystores <http://download.oracle.com/javase/6/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore>`__
 
+SSL Certificate Hot Reloading
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+Beginning with Cassandra 4, Cassandra supports hot reloading of SSL Certificates. If SSL/TLS support is enabled in Cassandra,
+the node periodically polls the Trust and Key Stores specified in cassandra.yaml. When the files are updated, Cassandra will
+reload them and use them for subsequent connections. Please note that the Trust & Key Store passwords are part of the yaml so
+the updated files should also use the same passwords. The default polling interval is 10 minutes.
+
+Certificate Hot reloading may also be triggered using the ``nodetool reloadssl`` command. Use this if you want to Cassandra to
+immediately notice the changed certificates.
+
 Inter-node Encryption
 ~~~~~~~~~~~~~~~~~~~~~
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/473e8dfd/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 8e831cf..0714245 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -60,6 +60,7 @@ import org.apache.cassandra.locator.SeedProvider;
 import org.apache.cassandra.net.BackPressureStrategy;
 import org.apache.cassandra.net.RateBasedBackPressure;
 import org.apache.cassandra.security.EncryptionContext;
+import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.service.CacheService.CacheType;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -322,6 +323,8 @@ public class DatabaseDescriptor
         applySeedProvider();
 
         applyEncryptionContext();
+
+        applySslContextHotReload();
     }
 
     private static void applySimpleConfig()
@@ -865,6 +868,11 @@ public class DatabaseDescriptor
         encryptionContext = new EncryptionContext(conf.transparent_data_encryption_options);
     }
 
+    public static void applySslContextHotReload()
+    {
+        SSLFactory.initHotReloading(conf.server_encryption_options, conf.client_encryption_options, false);
+    }
+
     public static void applySeedProvider()
     {
         // load the seeds for node contact points

http://git-wip-us.apache.org/repos/asf/cassandra/blob/473e8dfd/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 9f00d27..8fdb395 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -101,6 +101,7 @@ import org.apache.cassandra.net.async.NettyFactory.InboundInitializer;
 import org.apache.cassandra.repair.messages.RepairMessage;
 import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.service.AbstractWriteResponseHandler;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.StorageService;
@@ -1664,4 +1665,10 @@ public final class MessagingService implements MessagingServiceMBean
         }
         return true;
     }
+
+    @Override
+    public void reloadSslCertificates()
+    {
+        SSLFactory.checkCertFilesForHotReloading();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/473e8dfd/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingServiceMBean.java b/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
index f4a0c43..6adb891 100644
--- a/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
+++ b/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
@@ -129,4 +129,6 @@ public interface MessagingServiceMBean
     public boolean isBackPressureEnabled();
 
     public int getVersion(String address) throws UnknownHostException;
+
+    void reloadSslCertificates();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/473e8dfd/src/java/org/apache/cassandra/security/SSLFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/security/SSLFactory.java b/src/java/org/apache/cassandra/security/SSLFactory.java
index a931f5f..0bf769c 100644
--- a/src/java/org/apache/cassandra/security/SSLFactory.java
+++ b/src/java/org/apache/cassandra/security/SSLFactory.java
@@ -18,16 +18,20 @@
 package org.apache.cassandra.security;
 
 
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.security.KeyStore;
 import java.security.cert.X509Certificate;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Date;
 import java.util.Enumeration;
 import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
@@ -37,6 +41,7 @@ import javax.net.ssl.TrustManagerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
@@ -49,7 +54,7 @@ import io.netty.handler.ssl.SslContext;
 import io.netty.handler.ssl.SslContextBuilder;
 import io.netty.handler.ssl.SslProvider;
 import io.netty.handler.ssl.SupportedCipherSuiteFilter;
-import io.netty.util.ReferenceCountUtil;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.EncryptionOptions;
 
 /**
@@ -78,6 +83,67 @@ public final class SSLFactory
     private static final AtomicReference<SslContext> serverSslContext = new AtomicReference<>();
 
     /**
+     * List of files that trigger hot reloading of SSL certificates
+     */
+    private static volatile List<HotReloadableFile> hotReloadableFiles = ImmutableList.of();
+
+    /**
+     * Default initial delay for hot reloading
+     */
+    public static final int DEFAULT_HOT_RELOAD_INITIAL_DELAY_SEC = 600;
+
+    /**
+     * Default periodic check delay for hot reloading
+     */
+    public static final int DEFAULT_HOT_RELOAD_PERIOD_SEC = 600;
+
+    /**
+     * State variable to maintain initialization invariant
+     */
+    private static boolean isHotReloadingInitialized = false;
+
+    /**
+     * Helper class for hot reloading SSL Contexts
+     */
+    private static class HotReloadableFile
+    {
+        enum Type
+        {
+            SERVER,
+            CLIENT
+        }
+
+        private final File file;
+        private volatile long lastModTime;
+        private final Type certType;
+
+        HotReloadableFile(String path, Type type)
+        {
+            file = new File(path);
+            lastModTime = file.lastModified();
+            certType = type;
+        }
+
+        boolean shouldReload()
+        {
+            long curModTime = file.lastModified();
+            boolean result = curModTime != lastModTime;
+            lastModTime = curModTime;
+            return result;
+        }
+
+        public boolean isServer()
+        {
+            return certType == Type.SERVER;
+        }
+
+        public boolean isClient()
+        {
+            return certType == Type.CLIENT;
+        }
+    }
+
+    /**
      * Create a JSSE {@link SSLContext}.
      */
     @SuppressWarnings("resource")
@@ -176,10 +242,14 @@ public final class SSLFactory
     @VisibleForTesting
     static SslContext getSslContext(EncryptionOptions options, boolean buildTruststore, boolean forServer, boolean useOpenSsl) throws IOException
     {
-        if (forServer && serverSslContext.get() != null)
-            return serverSslContext.get();
-        if (!forServer && clientSslContext.get() != null)
-            return clientSslContext.get();
+
+        SslContext sslContext;
+
+        if (forServer && (sslContext = serverSslContext.get()) != null)
+            return sslContext;
+
+        if (!forServer && (sslContext = clientSslContext.get()) != null)
+            return sslContext;
 
         /*
             There is a case where the netty/openssl combo might not support using KeyManagerFactory. specifically,
@@ -219,7 +289,73 @@ public final class SSLFactory
         if (ref.compareAndSet(null, ctx))
             return ctx;
 
-        ReferenceCountUtil.release(ctx);
         return ref.get();
     }
+
+    /**
+     * Performs a lightweight check whether the certificate files have been refreshed.
+     *
+     * @throws IllegalStateException if {@link #initHotReloading(EncryptionOptions.ServerEncryptionOptions, EncryptionOptions, boolean)}
+     * is not called first
+     */
+    public static void checkCertFilesForHotReloading()
+    {
+        if (!isHotReloadingInitialized)
+            throw new IllegalStateException("Hot reloading functionality has not been initialized.");
+
+        logger.trace("Checking whether certificates have been updated");
+
+        if (hotReloadableFiles.stream().anyMatch(f -> f.isServer() && f.shouldReload()))
+        {
+            logger.info("Server ssl certificates have been updated. Reseting the context for new peer connections.");
+            serverSslContext.set(null);
+        }
+
+        if (hotReloadableFiles.stream().anyMatch(f -> f.isClient() && f.shouldReload()))
+        {
+            logger.info("Client ssl certificates have been updated. Reseting the context for new client connections.");
+            clientSslContext.set(null);
+        }
+    }
+
+    /**
+     * Determines whether to hot reload certificates and schedules a periodic task for it.
+     *
+     * @param serverEncryptionOptions
+     * @param clientEncryptionOptions
+     */
+    public static synchronized void initHotReloading(EncryptionOptions.ServerEncryptionOptions serverEncryptionOptions,
+                                                     EncryptionOptions clientEncryptionOptions,
+                                                     boolean force)
+    {
+        if (isHotReloadingInitialized && !force)
+            return;
+
+        logger.debug("Initializing hot reloading SSLContext");
+
+        List<HotReloadableFile> fileList = new ArrayList<>();
+
+        if (serverEncryptionOptions.enabled)
+        {
+            fileList.add(new HotReloadableFile(serverEncryptionOptions.keystore, HotReloadableFile.Type.SERVER));
+            fileList.add(new HotReloadableFile(serverEncryptionOptions.truststore, HotReloadableFile.Type.SERVER));
+        }
+
+        if (clientEncryptionOptions.enabled)
+        {
+            fileList.add(new HotReloadableFile(clientEncryptionOptions.keystore, HotReloadableFile.Type.CLIENT));
+            fileList.add(new HotReloadableFile(clientEncryptionOptions.truststore, HotReloadableFile.Type.CLIENT));
+        }
+
+        hotReloadableFiles = ImmutableList.copyOf(fileList);
+
+        if (!isHotReloadingInitialized)
+        {
+            ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(SSLFactory::checkCertFilesForHotReloading,
+                                                                     DEFAULT_HOT_RELOAD_INITIAL_DELAY_SEC,
+                                                                     DEFAULT_HOT_RELOAD_PERIOD_SEC, TimeUnit.SECONDS);
+        }
+
+        isHotReloadingInitialized = true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/473e8dfd/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 69b64ab..7ce5341 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -1647,6 +1647,11 @@ public class NodeProbe implements AutoCloseable
     {
         return arsProxy;
     }
+
+    public void reloadSslCerts()
+    {
+        msProxy.reloadSslCertificates();
+    }
 }
 
 class ColumnFamilyStoreMBeanIterator implements Iterator<Map.Entry<String, ColumnFamilyStoreMBean>>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/473e8dfd/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 81f2023..2b6fabf 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -156,7 +156,8 @@ public class NodeTool
                 RefreshSizeEstimates.class,
                 RelocateSSTables.class,
                 ViewBuildStatus.class,
-                HandoffWindow.class
+                HandoffWindow.class,
+                ReloadSslCertificates.class
         );
 
         Cli.CliBuilder<Runnable> builder = Cli.builder("nodetool");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/473e8dfd/src/java/org/apache/cassandra/tools/ReloadSslCertificates.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/ReloadSslCertificates.java b/src/java/org/apache/cassandra/tools/ReloadSslCertificates.java
new file mode 100644
index 0000000..f38b8c0
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/ReloadSslCertificates.java
@@ -0,0 +1,30 @@
+/*
+ * 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.tools;
+
+import io.airlift.airline.Command;
+
+@Command(name = "reloadssl", description = "Signals Cassandra to reload SSL certificates")
+public class ReloadSslCertificates extends NodeTool.NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.reloadSslCerts();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/473e8dfd/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/security/SSLFactoryTest.java b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
index 61933a5..5153a11 100644
--- a/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
+++ b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
@@ -18,16 +18,12 @@
 */
 package org.apache.cassandra.security;
 
+import java.io.File;
 import java.io.IOException;
-import java.net.InetAddress;
 import java.security.cert.CertificateException;
 import java.util.Arrays;
-import javax.net.ssl.SSLServerSocket;
 import javax.net.ssl.TrustManagerFactory;
 
-import com.google.common.base.Predicates;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -162,4 +158,36 @@ public class SSLFactoryTest
         SSLFactory.buildKeyManagerFactory(options);
         Assert.assertTrue(SSLFactory.checkedExpiry);
     }
+
+    @Test
+    public void testSslContextReload_HappyPath() throws IOException, InterruptedException
+    {
+        try
+        {
+            EncryptionOptions options = addKeystoreOptions(encryptionOptions);
+            options.enabled = true;
+
+            SSLFactory.initHotReloading((ServerEncryptionOptions) options, options, true);
+
+            SslContext oldCtx = SSLFactory.getSslContext(options, true, true, OpenSsl.isAvailable());
+            File keystoreFile = new File(options.keystore);
+
+            SSLFactory.checkCertFilesForHotReloading();
+            Thread.sleep(5000);
+            keystoreFile.setLastModified(System.currentTimeMillis());
+
+            SSLFactory.checkCertFilesForHotReloading();
+            SslContext newCtx = SSLFactory.getSslContext(options, true, true, OpenSsl.isAvailable());
+
+            Assert.assertNotSame(oldCtx, newCtx);
+        }
+        catch (Exception e)
+        {
+            throw e;
+        }
+        finally
+        {
+            DatabaseDescriptor.loadConfig();
+        }
+    }
 }


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