You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by sa...@apache.org on 2018/08/01 22:01:18 UTC

[geode] 01/01: GEODE-5338: Geode client to support Trust and Keystore rotation

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

sai_boorlagadda pushed a commit to branch feature/GEODE-5338
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 63f29882f1f241d04708532a4e512080c1520781
Author: Sai Boorlagadda <sb...@pivotal.io>
AuthorDate: Wed Aug 1 14:40:07 2018 -0700

    GEODE-5338: Geode client to support Trust and Keystore rotation
    
       A new ssl property 'ssl-use-default-provider' is added to let clients
       provide custom implementations to manage keystore and truststore and
       custom implementations for KeyManagerFactorySPI and TrustManagerFactorySPI.
    
       Geode assumes the provider is added at the runtime by client applications
       before initializing the cache or initiating a connection.
---
 .../internal/CustomSSLProviderDistributedTest.java | 158 ++++++++++++++++++++
 .../internal/provider/CustomKeyManagerFactory.java | 110 ++++++++++++++
 .../client/internal/provider/CustomProvider.java   |  53 +++++++
 .../provider/CustomTrustManagerFactory.java        | 109 ++++++++++++++
 .../geode/distributed/ConfigurationProperties.java |  13 ++
 .../internal/AbstractDistributionConfig.java       |   2 +
 .../distributed/internal/DistributionConfig.java   |  18 +++
 .../internal/DistributionConfigImpl.java           |  13 ++
 .../org/apache/geode/internal/admin/SSLConfig.java |   9 ++
 .../internal/net/SSLConfigurationFactory.java      |   1 +
 .../apache/geode/internal/net/SocketCreator.java   | 161 +++++++++++----------
 .../internal/DistributionConfigJUnitTest.java      |   4 +-
 .../cache/client/internal/provider/client.keystore | Bin 0 -> 1251 bytes
 .../client/internal/provider/client.truststore     | Bin 0 -> 846 bytes
 14 files changed, 575 insertions(+), 76 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/CustomSSLProviderDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/CustomSSLProviderDistributedTest.java
new file mode 100644
index 0000000..3ac6a6f
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/CustomSSLProviderDistributedTest.java
@@ -0,0 +1,158 @@
+/*
+ * 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.geode.cache.client.internal;
+
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_CIPHERS;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENABLED_COMPONENTS;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE_PASSWORD;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE_TYPE;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_PROTOCOLS;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_REQUIRE_AUTHENTICATION;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_PASSWORD;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_TYPE;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_USE_DEFAULT_PROVIDER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.security.Security;
+import java.util.Properties;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionFactory;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCacheFactory;
+import org.apache.geode.cache.client.ClientRegionFactory;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.cache.client.internal.provider.CustomProvider;
+import org.apache.geode.security.SecurableCommunicationChannels;
+import org.apache.geode.test.dunit.SerializableConsumerIF;
+import org.apache.geode.test.dunit.rules.ClientVM;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.ClientServerTest;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.util.test.TestUtil;
+
+@Category({ClientServerTest.class})
+public class CustomSSLProviderDistributedTest {
+  private static MemberVM locator;
+  private static MemberVM server;
+  private static ClientVM client;
+
+  private static final String SERVER_KEY_STORE = "cacheserver.keystore";
+  private static final String SERVER_TRUST_STORE = "cacheserver.truststore";
+
+  @ClassRule
+  public static ClusterStartupRule cluster = new ClusterStartupRule();
+
+  @ClassRule
+  public static GfshCommandRule gfsh = new GfshCommandRule();
+
+  private static String serverKeystore =
+      TestUtil.getResourcePath(CustomSSLProviderDistributedTest.class, SERVER_KEY_STORE);
+  private static String serverTruststore =
+      TestUtil.getResourcePath(CustomSSLProviderDistributedTest.class, SERVER_TRUST_STORE);
+
+  private static Properties serverSSLProperties = new Properties() {
+    {
+      setProperty(SSL_ENABLED_COMPONENTS, SecurableCommunicationChannels.ALL);
+      setProperty(SSL_KEYSTORE, serverKeystore);
+      setProperty(SSL_KEYSTORE_PASSWORD, "password");
+      setProperty(SSL_KEYSTORE_TYPE, "JKS");
+      setProperty(SSL_TRUSTSTORE, serverTruststore);
+      setProperty(SSL_TRUSTSTORE_PASSWORD, "password");
+      setProperty(SSL_TRUSTSTORE_TYPE, "JKS");
+      setProperty(SSL_CIPHERS, "any");
+      setProperty(SSL_PROTOCOLS, "any");
+      setProperty(SSL_REQUIRE_AUTHENTICATION, String.valueOf("true"));
+    }
+  };
+
+  private static Properties clientSSLProperties = new Properties() {
+    {
+      setProperty(SSL_ENABLED_COMPONENTS, SecurableCommunicationChannels.SERVER);
+      setProperty(SSL_REQUIRE_AUTHENTICATION, String.valueOf("true"));
+      setProperty(SSL_USE_DEFAULT_PROVIDER, String.valueOf("true"));
+    }
+  };
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    // create a cluster
+    locator = cluster.startLocatorVM(0, serverSSLProperties);
+    server = cluster.startServerVM(1, serverSSLProperties, locator.getPort());
+
+    // create region
+    server.invoke(CustomSSLProviderDistributedTest::createServerRegion);
+    locator.waitUntilRegionIsReadyOnExactlyThisManyServers("/region", 1);
+
+    // setup client
+    setupClient(server.getPort(), server.getVM().getHost().getHostName());
+  }
+
+  private static void createServerRegion() {
+    RegionFactory factory =
+        ClusterStartupRule.getCache().createRegionFactory(RegionShortcut.REPLICATE);
+    Region r = factory.create("region");
+    r.put("serverkey", "servervalue");
+  }
+
+  private static void setupClient(int serverPort, String serverHost) throws Exception {
+    SerializableConsumerIF<ClientCacheFactory> clientSetup = cf -> {
+      // add custom provider
+      Security.insertProviderAt(new CustomProvider(), 2);
+      cf.addPoolServer(serverHost, serverPort);
+    };
+
+    client = cluster.startClientVM(2, clientSSLProperties, clientSetup);
+
+    // create a client region
+    client.invoke(CustomSSLProviderDistributedTest::createClientRegion);
+  }
+
+  private static void createClientRegion() {
+    ClientRegionFactory<String, String> regionFactory =
+        ClusterStartupRule.getClientCache().createClientRegionFactory(ClientRegionShortcut.PROXY);
+    Region<String, String> region = regionFactory.create("region");
+    assertThat(region).isNotNull();
+  }
+
+  @Test
+  public void testClientSSLConnection() {
+    client.invoke(CustomSSLProviderDistributedTest::doClientRegionTest);
+    server.invoke(CustomSSLProviderDistributedTest::doServerRegionTest);
+  }
+
+  private static void doClientRegionTest() {
+    Region<String, String> region = ClusterStartupRule.getClientCache().getRegion("region");
+    assertThat("servervalue").isEqualTo(region.get("serverkey"));
+
+    region.put("clientkey", "clientvalue");
+    assertThat("clientvalue").isEqualTo(region.get("clientkey"));
+  }
+
+  private static void doServerRegionTest() {
+    Region<String, String> region = ClusterStartupRule.getCache().getRegion("region");
+    assertThat("servervalue").isEqualTo(region.get("serverkey"));
+    assertThat("clientvalue").isEqualTo(region.get("clientkey"));
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomKeyManagerFactory.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomKeyManagerFactory.java
new file mode 100644
index 0000000..043877b
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomKeyManagerFactory.java
@@ -0,0 +1,110 @@
+/*
+ * 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.geode.cache.client.internal.provider;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.util.logging.Logger;
+
+import javax.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.KeyManagerFactorySpi;
+import javax.net.ssl.ManagerFactoryParameters;
+import javax.net.ssl.X509ExtendedKeyManager;
+
+import org.apache.geode.util.test.TestUtil;
+
+abstract class CustomKeyManagerFactory extends KeyManagerFactorySpi {
+
+  private final Logger logger = Logger.getLogger(this.getClass().getName());
+
+  private final String algorithm;
+  private KeyManagerFactory customKeyManagerFactory;
+  private X509ExtendedKeyManager customKeyManager;
+
+  private CustomKeyManagerFactory(String algorithm) {
+    this.algorithm = algorithm;
+  }
+
+  @Override
+  protected final KeyManager[] engineGetKeyManagers() {
+    X509ExtendedKeyManager systemKeyManager = getCustomKeyManager();
+    return new KeyManager[] {systemKeyManager};
+  }
+
+  @Override
+  protected final void engineInit(ManagerFactoryParameters managerFactoryParameters) {
+    // not supported right now
+    throw new UnsupportedOperationException("use engineInit with keystore");
+  }
+
+  @Override
+  protected final void engineInit(KeyStore keyStore, char[] chars) {
+    // ignore the passed in keystore as it will be null
+    init();
+  }
+
+  private void init() {
+    String CLIENT_KEY_STORE = "client.keystore";
+    String SSL_KEYSTORE_TYPE = "JKS";
+    String SSL_KEYSTORE =
+        TestUtil.getResourcePath(CustomKeyManagerFactory.class, CLIENT_KEY_STORE);
+    String SSL_KEYSTORE_PASSWORD = "password";
+
+    try {
+      FileInputStream fileInputStream = new FileInputStream(SSL_KEYSTORE);
+      KeyStore keyStore = KeyStore.getInstance(SSL_KEYSTORE_TYPE);
+      keyStore.load(fileInputStream, SSL_KEYSTORE_PASSWORD.toCharArray());
+      this.customKeyManagerFactory = KeyManagerFactory.getInstance(this.algorithm, "SunJSSE");
+      this.customKeyManagerFactory.init(keyStore, SSL_KEYSTORE_PASSWORD.toCharArray());
+    } catch (NoSuchAlgorithmException | IOException | CertificateException
+        | UnrecoverableKeyException | KeyStoreException | NoSuchProviderException e) {
+      throw new UndeclaredThrowableException(e);
+    }
+  }
+
+  private X509ExtendedKeyManager getCustomKeyManager() {
+    if (this.customKeyManager == null) {
+      for (KeyManager candidate : this.customKeyManagerFactory.getKeyManagers()) {
+        if (candidate instanceof X509ExtendedKeyManager) {
+          this.logger.info("Adding System Key Manager");
+          this.customKeyManager = (X509ExtendedKeyManager) candidate;
+          break;
+        }
+      }
+    }
+    return this.customKeyManager;
+  }
+
+  public static final class PKIXFactory extends CustomKeyManagerFactory {
+    public PKIXFactory() {
+      super("PKIX");
+    }
+  }
+
+  public static final class SimpleFactory extends CustomKeyManagerFactory {
+    public SimpleFactory() {
+      super("SunX509");
+    }
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomProvider.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomProvider.java
new file mode 100644
index 0000000..6537dce
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomProvider.java
@@ -0,0 +1,53 @@
+/*
+ * 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.geode.cache.client.internal.provider;
+
+import java.security.Provider;
+import java.util.logging.Logger;
+
+
+public final class CustomProvider extends Provider {
+  private static final long serialVersionUID = -2667509590306131953L;
+
+  private final Logger logger = Logger.getLogger(this.getClass().getName());
+
+  public CustomProvider() {
+    super("Custom Provider", 1.0,
+        "KeyManagerFactory and TrustManagerFactory based on a custom factory implementation");
+
+    this.logger.fine("KeyManager enabled");
+
+    put("KeyManagerFactory.SunX509",
+        "org.apache.geode.cache.client.internal.provider.CustomKeyManagerFactory$SimpleFactory");
+    put("KeyManagerFactory.PKIX",
+        "org.apache.geode.cache.client.internal.provider.CustomKeyManagerFactory$PKIXFactory");
+    put("Alg.Alias.KeyManagerFactory.SunPKIX", "PKIX");
+    put("Alg.Alias.KeyManagerFactory.X509", "PKIX");
+    put("Alg.Alias.KeyManagerFactory.X.509", "PKIX");
+
+    this.logger.fine("TrustManager enabled");
+
+    put("TrustManagerFactory.SunX509",
+        "org.apache.geode.cache.client.internal.provider.CustomTrustManagerFactory$SimpleFactory");
+    put("TrustManagerFactory.PKIX",
+        "org.apache.geode.cache.client.internal.provider.CustomTrustManagerFactory$PKIXFactory");
+    put("Alg.Alias.TrustManagerFactory.SunPKIX", "PKIX");
+    put("Alg.Alias.TrustManagerFactory.X509", "PKIX");
+    put("Alg.Alias.TrustManagerFactory.X.509", "PKIX");
+
+    this.logger.fine("Provider loaded");
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomTrustManagerFactory.java b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomTrustManagerFactory.java
new file mode 100644
index 0000000..1d36295
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache/client/internal/provider/CustomTrustManagerFactory.java
@@ -0,0 +1,109 @@
+/*
+ * 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.geode.cache.client.internal.provider;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.cert.CertificateException;
+import java.util.logging.Logger;
+
+import javax.net.ssl.ManagerFactoryParameters;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.TrustManagerFactorySpi;
+import javax.net.ssl.X509ExtendedTrustManager;
+
+import org.apache.geode.util.test.TestUtil;
+
+abstract class CustomTrustManagerFactory extends TrustManagerFactorySpi {
+
+  private final Logger logger = Logger.getLogger(this.getClass().getName());
+
+  private final String algorithm;
+  private TrustManagerFactory customTrustManagerFactory;
+  private X509ExtendedTrustManager customTrustManager;
+
+  private CustomTrustManagerFactory(String algorithm) {
+    this.algorithm = algorithm;
+  }
+
+  @Override
+  protected final TrustManager[] engineGetTrustManagers() {
+    X509ExtendedTrustManager systemTrustManager = getCustomTrustManager();
+    return new TrustManager[] {systemTrustManager};
+  }
+
+  @Override
+  protected final void engineInit(ManagerFactoryParameters managerFactoryParameters) {
+    // not supported right now
+    throw new UnsupportedOperationException("use engineInit with keystore");
+  }
+
+  @Override
+  protected final void engineInit(KeyStore keyStore) {
+    // ignore the passed in keystore as it will be null
+    init();
+  }
+
+  private X509ExtendedTrustManager getCustomTrustManager() {
+    if (this.customTrustManager == null) {
+      for (TrustManager candidate : this.customTrustManagerFactory.getTrustManagers()) {
+        if (candidate instanceof X509ExtendedTrustManager) {
+          this.logger.info("Adding System Trust Manager");
+          this.customTrustManager = (X509ExtendedTrustManager) candidate;
+          break;
+        }
+      }
+    }
+    return this.customTrustManager;
+  }
+
+  private void init() {
+    String trustStoreName = "client.truststore";
+    String trustStorePath =
+        TestUtil.getResourcePath(CustomTrustManagerFactory.class, trustStoreName);
+    String trustStoreType = "JKS";
+    String trustStorePassword = "password";
+
+    try {
+      FileInputStream fileInputStream = new FileInputStream(trustStorePath);
+      KeyStore trustStore = KeyStore.getInstance(trustStoreType);
+      trustStore.load(fileInputStream, trustStorePassword.toCharArray());
+      this.customTrustManagerFactory = TrustManagerFactory.getInstance(this.algorithm, "SunJSSE");
+      this.customTrustManagerFactory.init(trustStore);
+    } catch (NoSuchAlgorithmException | IOException | CertificateException | KeyStoreException
+        | NoSuchProviderException e) {
+      throw new UndeclaredThrowableException(e);
+    }
+  }
+
+  public static final class PKIXFactory extends CustomTrustManagerFactory {
+    public PKIXFactory() {
+      super("PKIX");
+    }
+  }
+
+  public static final class SimpleFactory extends CustomTrustManagerFactory {
+    public SimpleFactory() {
+      super("SunX509");
+    }
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java b/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java
index 858b223..419a857 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java
@@ -1929,6 +1929,19 @@ public interface ConfigurationProperties {
    * Geode 1.0
    */
   String DISTRIBUTED_TRANSACTIONS = "distributed-transactions";
+
+  /**
+   * The static String definition of the <i>"ssl-use-default-provider"</i> property
+   *
+   * <U>Description</U>If true allows clients to provide a custom java.security.Provider. When using
+   *  this clients do not need to configure other ssl properties except ssl-enabled-components and
+   *  ssl-require-authentication.
+   * </p>
+   * <U>Default</U>: "false"
+   * </p>
+   * <U>Since</U>: Geode 1.8
+   */
+  String SSL_USE_DEFAULT_PROVIDER = "ssl-use-default-provider";
   /**
    * The static String definition of the <i>"ssl-enabled-components"</i> property <a
    * name="ssl-enabled-components"/a>
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
index b141a2f..18fd911 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
@@ -1360,6 +1360,8 @@ public abstract class AbstractDistributionConfig extends AbstractConfig
     m.put(SECURITY_POST_PROCESSOR,
         "User defined fully qualified class name implementing PostProcessor interface for integrated security. Defaults to \"{0}\". Legal values can be any \"class name\" implementing PostProcessor that is present in the classpath.");
 
+    m.put(SSL_USE_DEFAULT_PROVIDER, "If true allows clients to provide a custom java.security.Provider. When using this clients do not need to configure other ssl properties except ssl-enabled-components and ssl-require-authentication.");
+
     m.put(SSL_ENABLED_COMPONENTS,
         "A comma delimited list of components that require SSL communications");
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
index f3119e6..ac4dc2e 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
@@ -161,6 +161,7 @@ import static org.apache.geode.distributed.ConfigurationProperties.SSL_SERVER_AL
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_PASSWORD;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_TYPE;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_USE_DEFAULT_PROVIDER;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_WEB_ALIAS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_WEB_SERVICE_REQUIRE_AUTHENTICATION;
 import static org.apache.geode.distributed.ConfigurationProperties.START_DEV_REST_API;
@@ -4910,6 +4911,23 @@ public interface DistributionConfig extends Config, LogConfig {
   SecurableCommunicationChannel[] DEFAULT_SSL_ENABLED_COMPONENTS =
       new SecurableCommunicationChannel[] {};
 
+  boolean DEFAULT_SSL_USE_DEFAULT_PROVIDER = false;
+
+  @ConfigAttribute(type = Boolean.class)
+  String SSL_USE_DEFAULT_PROVIDER_NAME = SSL_USE_DEFAULT_PROVIDER;
+
+  /**
+   * Returns the value of the {@link ConfigurationProperties#SSL_USE_DEFAULT_PROVIDER} property.
+   */
+  @ConfigAttributeGetter(name = SSL_USE_DEFAULT_PROVIDER)
+  boolean getSSLUseDefaultProvider();
+
+  /**
+   * Sets the value of the {@link ConfigurationProperties#SSL_USE_DEFAULT_PROVIDER} property.
+   */
+  @ConfigAttributeSetter(name = SSL_USE_DEFAULT_PROVIDER)
+  void setSSLUseDefaultProvider(boolean useProvider);
+
   /**
    * Returns the value of the {@link ConfigurationProperties#SSL_PROTOCOLS} property.
    */
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfigImpl.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfigImpl.java
index 265a81f..3769fd7 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfigImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfigImpl.java
@@ -606,6 +606,7 @@ public class DistributionConfigImpl extends AbstractDistributionConfig implement
   private SecurableCommunicationChannel[] securableCommunicationChannels =
       DEFAULT_SSL_ENABLED_COMPONENTS;
 
+  private boolean sslUseDefaultProvider = DEFAULT_SSL_USE_DEFAULT_PROVIDER;
   private String sslProtocols = DEFAULT_SSL_PROTOCOLS;
   private String sslCiphers = DEFAULT_SSL_CIPHERS;
   private boolean sslRequireAuthentication = DEFAULT_SSL_REQUIRE_AUTHENTICATION;
@@ -850,6 +851,7 @@ public class DistributionConfigImpl extends AbstractDistributionConfig implement
     this.securableCommunicationChannels =
         ((DistributionConfigImpl) other).securableCommunicationChannels;
 
+    this.sslUseDefaultProvider = other.getSSLUseDefaultProvider();
     this.sslCiphers = other.getSSLCiphers();
     this.sslProtocols = other.getSSLProtocols();
     this.sslRequireAuthentication = other.getSSLRequireAuthentication();
@@ -2769,6 +2771,17 @@ public class DistributionConfigImpl extends AbstractDistributionConfig implement
   }
 
   @Override
+  public boolean getSSLUseDefaultProvider() {
+    return sslUseDefaultProvider;
+  }
+
+  @Override
+  public void setSSLUseDefaultProvider(final boolean useDefaultProvider) {
+    // This conversion is required due to backwards compatibility of the existing protocols code
+    this.sslUseDefaultProvider = useDefaultProvider;
+  }
+
+  @Override
   public String getSSLProtocols() {
     return sslProtocols;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/SSLConfig.java b/geode-core/src/main/java/org/apache/geode/internal/admin/SSLConfig.java
index 7497bff..42bc7fe 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/SSLConfig.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/SSLConfig.java
@@ -32,6 +32,7 @@ import org.apache.geode.management.internal.SSLUtil;
  */
 public class SSLConfig {
 
+  private boolean useDefaultProvider = DistributionConfig.DEFAULT_SSL_USE_DEFAULT_PROVIDER;
   private boolean enabled = DistributionConfig.DEFAULT_SSL_ENABLED;
   private String protocols = DistributionConfig.DEFAULT_SSL_PROTOCOLS;
   private String ciphers = DistributionConfig.DEFAULT_SSL_CIPHERS;
@@ -109,6 +110,14 @@ public class SSLConfig {
     this.enabled = enabled;
   }
 
+  public boolean isUseDefaultProvider() {
+    return this.useDefaultProvider;
+  }
+
+  public void setUseDefaultProvider(boolean useDefaultProvider) {
+    this.useDefaultProvider = useDefaultProvider;
+  }
+
   public String getProtocols() {
     return this.protocols;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/net/SSLConfigurationFactory.java b/geode-core/src/main/java/org/apache/geode/internal/net/SSLConfigurationFactory.java
index 79aafb2..57b4fcf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/net/SSLConfigurationFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/net/SSLConfigurationFactory.java
@@ -178,6 +178,7 @@ public class SSLConfigurationFactory {
     sslConfig.setProtocols(distributionConfig.getSSLProtocols());
     sslConfig.setRequireAuth(distributionConfig.getSSLRequireAuthentication());
     sslConfig.setAlias(distributionConfig.getSSLDefaultAlias());
+    sslConfig.setUseDefaultProvider(distributionConfig.getSSLUseDefaultProvider());
     return sslConfig;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
index 7441fc9..c33ab8a 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
@@ -473,99 +473,112 @@ public class SocketCreator {
 
   private TrustManager[] getTrustManagers()
       throws KeyStoreException, NoSuchAlgorithmException, CertificateException, IOException {
-    TrustManager[] trustManagers = null;
+    if (sslConfig.isUseDefaultProvider()) {
+      TrustManagerFactory tmf =
+          TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init((KeyStore) null);
+      return tmf.getTrustManagers();
+    } else {
+      TrustManager[] trustManagers = null;
 
-    String trustStoreType = sslConfig.getTruststoreType();
-    if (StringUtils.isEmpty(trustStoreType)) {
-      trustStoreType = KeyStore.getDefaultType();
-    }
+      String trustStoreType = sslConfig.getTruststoreType();
+      if (StringUtils.isEmpty(trustStoreType)) {
+        trustStoreType = KeyStore.getDefaultType();
+      }
 
-    KeyStore ts = KeyStore.getInstance(trustStoreType);
-    String trustStorePath = sslConfig.getTruststore();
-    FileInputStream fis = new FileInputStream(trustStorePath);
-    String passwordString = sslConfig.getTruststorePassword();
-    char[] password = null;
-    if (passwordString != null) {
-      if (passwordString.trim().equals("")) {
-        if (!StringUtils.isEmpty(passwordString)) {
-          String toDecrypt = "encrypted(" + passwordString + ")";
-          passwordString = PasswordUtil.decrypt(toDecrypt);
+      KeyStore ts = KeyStore.getInstance(trustStoreType);
+      String trustStorePath = sslConfig.getTruststore();
+      FileInputStream fis = new FileInputStream(trustStorePath);
+      String passwordString = sslConfig.getTruststorePassword();
+      char[] password = null;
+      if (passwordString != null) {
+        if (passwordString.trim().equals("")) {
+          if (!StringUtils.isEmpty(passwordString)) {
+            String toDecrypt = "encrypted(" + passwordString + ")";
+            passwordString = PasswordUtil.decrypt(toDecrypt);
+            password = passwordString.toCharArray();
+          }
+        } else {
           password = passwordString.toCharArray();
         }
-      } else {
-        password = passwordString.toCharArray();
       }
+      ts.load(fis, password);
+
+      // default algorithm can be changed by setting property "ssl.TrustManagerFactory.algorithm" in
+      // security properties
+      TrustManagerFactory tmf =
+          TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+      tmf.init(ts);
+      trustManagers = tmf.getTrustManagers();
+      // follow the security tip in java doc
+      if (password != null) {
+        java.util.Arrays.fill(password, ' ');
+      }
+      return trustManagers;
     }
-    ts.load(fis, password);
-
-    // default algorithm can be changed by setting property "ssl.TrustManagerFactory.algorithm" in
-    // security properties
-    TrustManagerFactory tmf =
-        TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-    tmf.init(ts);
-    trustManagers = tmf.getTrustManagers();
-    // follow the security tip in java doc
-    if (password != null) {
-      java.util.Arrays.fill(password, ' ');
-    }
-
-    return trustManagers;
   }
 
   private KeyManager[] getKeyManagers() throws KeyStoreException, IOException,
       NoSuchAlgorithmException, CertificateException, UnrecoverableKeyException {
-    if (sslConfig.getKeystore() == null) {
-      return null;
-    }
+    if (sslConfig.isUseDefaultProvider()) {
+      KeyManagerFactory keyManagerFactory =
+          KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+      keyManagerFactory.init(null, null);
+      return keyManagerFactory.getKeyManagers();
+    } else {
+      if (sslConfig.getKeystore() == null) {
+        return null;
+      }
 
-    KeyManager[] keyManagers = null;
-    String keyStoreType = sslConfig.getKeystoreType();
-    if (StringUtils.isEmpty(keyStoreType)) {
-      keyStoreType = KeyStore.getDefaultType();
-    }
-    KeyStore keyStore = KeyStore.getInstance(keyStoreType);
-    String keyStoreFilePath = sslConfig.getKeystore();
-    if (StringUtils.isEmpty(keyStoreFilePath)) {
-      keyStoreFilePath =
-          System.getProperty("user.home") + System.getProperty("file.separator") + ".keystore";
-    }
+      KeyManager[] keyManagers = null;
+      String keyStoreType = sslConfig.getKeystoreType();
+      if (StringUtils.isEmpty(keyStoreType)) {
+        keyStoreType = KeyStore.getDefaultType();
+      }
+      KeyStore keyStore = KeyStore.getInstance(keyStoreType);
+      String keyStoreFilePath = sslConfig.getKeystore();
+      if (StringUtils.isEmpty(keyStoreFilePath)) {
+        keyStoreFilePath =
+            System.getProperty("user.home") + System.getProperty("file.separator") + ".keystore";
+      }
 
-    FileInputStream fileInputStream = new FileInputStream(keyStoreFilePath);
-    String passwordString = sslConfig.getKeystorePassword();
-    char[] password = null;
-    if (passwordString != null) {
-      if (passwordString.trim().equals("")) {
-        String encryptedPass = System.getenv("javax.net.ssl.keyStorePassword");
-        if (!StringUtils.isEmpty(encryptedPass)) {
-          String toDecrypt = "encrypted(" + encryptedPass + ")";
-          passwordString = PasswordUtil.decrypt(toDecrypt);
+      FileInputStream fileInputStream = new FileInputStream(keyStoreFilePath);
+      String passwordString = sslConfig.getKeystorePassword();
+      char[] password = null;
+      if (passwordString != null) {
+        if (passwordString.trim().equals("")) {
+          String encryptedPass = System.getenv("javax.net.ssl.keyStorePassword");
+          if (!StringUtils.isEmpty(encryptedPass)) {
+            String toDecrypt = "encrypted(" + encryptedPass + ")";
+            passwordString = PasswordUtil.decrypt(toDecrypt);
+            password = passwordString.toCharArray();
+          }
+        } else {
           password = passwordString.toCharArray();
         }
-      } else {
-        password = passwordString.toCharArray();
       }
-    }
-    keyStore.load(fileInputStream, password);
-    // default algorithm can be changed by setting property "ssl.KeyManagerFactory.algorithm" in
-    // security properties
-    KeyManagerFactory keyManagerFactory =
-        KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-    keyManagerFactory.init(keyStore, password);
-    keyManagers = keyManagerFactory.getKeyManagers();
-    // follow the security tip in java doc
-    if (password != null) {
-      java.util.Arrays.fill(password, ' ');
-    }
+      keyStore.load(fileInputStream, password);
+      // default algorithm can be changed by setting property "ssl.KeyManagerFactory.algorithm" in
+      // security properties
+      KeyManagerFactory keyManagerFactory =
+          KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+      keyManagerFactory.init(keyStore, password);
+      keyManagers = keyManagerFactory.getKeyManagers();
+      // follow the security tip in java doc
+      if (password != null) {
+        java.util.Arrays.fill(password, ' ');
+      }
 
-    KeyManager[] extendedKeyManagers = new KeyManager[keyManagers.length];
+      KeyManager[] extendedKeyManagers = new KeyManager[keyManagers.length];
 
-    for (int i = 0; i < keyManagers.length; i++)
+      for (int i = 0; i < keyManagers.length; i++)
 
-    {
-      extendedKeyManagers[i] = new ExtendedAliasKeyManager(keyManagers[i], sslConfig.getAlias());
-    }
+      {
+        extendedKeyManagers[i] = new ExtendedAliasKeyManager(keyManagers[i], sslConfig.getAlias());
+      }
 
-    return extendedKeyManagers;
+      return extendedKeyManagers;
+    }
   }
 
   public SSLContext getSslContext() {
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionConfigJUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionConfigJUnitTest.java
index 4938e03..a42ab20 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionConfigJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionConfigJUnitTest.java
@@ -98,7 +98,7 @@ public class DistributionConfigJUnitTest {
   @Test
   public void testGetAttributeNames() {
     String[] attNames = AbstractDistributionConfig._getAttNames();
-    assertEquals(attNames.length, 162);
+    assertEquals(attNames.length, 163);
 
     List boolList = new ArrayList();
     List intList = new ArrayList();
@@ -132,7 +132,7 @@ public class DistributionConfigJUnitTest {
 
     // TODO - This makes no sense. One has no idea what the correct expected number of attributes
     // are.
-    assertEquals(31, boolList.size());
+    assertEquals(32, boolList.size());
     assertEquals(35, intList.size());
     assertEquals(87, stringList.size());
     assertEquals(5, fileList.size());
diff --git a/geode-core/src/test/resources/org/apache/geode/cache/client/internal/provider/client.keystore b/geode-core/src/test/resources/org/apache/geode/cache/client/internal/provider/client.keystore
new file mode 100644
index 0000000..38a315d
Binary files /dev/null and b/geode-core/src/test/resources/org/apache/geode/cache/client/internal/provider/client.keystore differ
diff --git a/geode-core/src/test/resources/org/apache/geode/cache/client/internal/provider/client.truststore b/geode-core/src/test/resources/org/apache/geode/cache/client/internal/provider/client.truststore
new file mode 100644
index 0000000..d598b86
Binary files /dev/null and b/geode-core/src/test/resources/org/apache/geode/cache/client/internal/provider/client.truststore differ