You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by je...@apache.org on 2018/04/05 14:20:36 UTC

[geode] branch develop updated: GEODE-4960: Jar deploy fails when SSL is enabled (#1698)

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

jensdeppe pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new d792821  GEODE-4960: Jar deploy fails when SSL is enabled (#1698)
d792821 is described below

commit d79282134c2064a09bfe418afcedaaf4c3741347
Author: Jens Deppe <jd...@pivotal.io>
AuthorDate: Thu Apr 5 07:20:31 2018 -0700

    GEODE-4960: Jar deploy fails when SSL is enabled (#1698)
    
    - Print the deployed jar MD5 hash in hex
---
 .../org/apache/geode/internal/DeployedJar.java     |  10 +-
 .../geode/internal/InternalDataSerializer.java     |   3 +
 .../internal/cache/ClusterConfigurationLoader.java |  17 +--
 .../ContextAwareSSLRMIClientSocketFactory.java     |  59 +++++++++
 .../geode/management/internal/ManagementAgent.java |   3 +-
 .../sanctioned-geode-core-serializables.txt        |   1 +
 .../configuration/DeployJarWithSSLDUnitTest.java   | 143 +++++++++++++++++++++
 .../cli/commands/ConnectCommandWithSSLTest.java    |   4 -
 8 files changed, 222 insertions(+), 18 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java b/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
index 328f1f8..808d4af 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DeployedJar.java
@@ -431,9 +431,17 @@ public class DeployedJar {
     sb.append('@').append(System.identityHashCode(this)).append('{');
     sb.append("jarName=").append(this.jarName);
     sb.append(",file=").append(this.file.getAbsolutePath());
-    sb.append(",md5hash=").append(Arrays.toString(this.md5hash));
+    sb.append(",md5hash=").append(toHex(this.md5hash));
     sb.append(",version=").append(this.getVersion());
     sb.append('}');
     return sb.toString();
   }
+
+  private String toHex(byte[] data) {
+    StringBuilder result = new StringBuilder();
+    for (byte b : data) {
+      result.append(String.format("%02x", b));
+    }
+    return result.toString();
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
index 133c592..78768c5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
@@ -186,6 +186,9 @@ public abstract class InternalDataSerializer extends DataSerializer implements D
 
           // jar deployment
           + ";com.sun.proxy.$Proxy*" + ";com.healthmarketscience.rmiio.RemoteInputStream"
+          + ";javax.rmi.ssl.SslRMIClientSocketFactory" + ";javax.net.ssl.SSLHandshakeException"
+          + ";javax.net.ssl.SSLException;sun.security.validator.ValidatorException"
+          + ";sun.security.provider.certpath.SunCertPathBuilderException"
 
           // geode-modules
           + ";org.apache.geode.modules.util.SessionCustomExpiry" + ";";
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
index 7cc56cd..0423964 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
@@ -39,6 +39,8 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import com.healthmarketscience.rmiio.RemoteInputStream;
+import com.healthmarketscience.rmiio.RemoteInputStreamClient;
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.Logger;
 
@@ -137,23 +139,16 @@ public class ClusterConfigurationLoader {
             Collections.singleton(locator));
 
     List<RemoteInputStream> result = rc.getResult();
-    RemoteInputStream jarStream = result.get(0);
 
     Path tempDir = FileUploader.createSecuredTempDirectory("deploy-");
     Path tempJar = Paths.get(tempDir.toString(), jarName);
     FileOutputStream fos = new FileOutputStream(tempJar.toString());
 
-    int packetId = 0;
-    while (true) {
-      byte[] data = jarStream.readPacket(packetId);
-      if (data == null) {
-        break;
-      }
-      fos.write(data);
-      packetId++;
-    }
+    InputStream jarStream = RemoteInputStreamClient.wrap(result.get(0));
+    IOUtils.copyLarge(jarStream, fos);
+
     fos.close();
-    jarStream.close(true);
+    jarStream.close();
 
     return tempJar.toFile();
   }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/ContextAwareSSLRMIClientSocketFactory.java b/geode-core/src/main/java/org/apache/geode/management/internal/ContextAwareSSLRMIClientSocketFactory.java
new file mode 100644
index 0000000..a81ba81
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/ContextAwareSSLRMIClientSocketFactory.java
@@ -0,0 +1,59 @@
+/*
+ * 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.management.internal;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.Socket;
+import java.rmi.server.RMIClientSocketFactory;
+
+import javax.rmi.ssl.SslRMIClientSocketFactory;
+
+import org.apache.geode.internal.net.SocketCreator;
+import org.apache.geode.internal.net.SocketCreatorFactory;
+import org.apache.geode.internal.security.SecurableCommunicationChannel;
+
+/**
+ * What's going on here? For jar deployment we're using the RMI-IO library which uses RMI to
+ * allow streaming files. When using SSL with RMI, the weird thing is that the
+ * RMIClientSocketFactory is defined on the server where the relevant object is being exported.
+ * This factory is serialized to the client to be used when making calls back to the server. Thus
+ * the factory needs to be able to create sockets configured according to the remote configuration.
+ */
+public class ContextAwareSSLRMIClientSocketFactory implements RMIClientSocketFactory, Serializable {
+
+  private static final long serialVersionUID = 8159615071011918570L;
+
+  private static final SslRMIClientSocketFactory defaultFactory = new SslRMIClientSocketFactory();
+
+  @Override
+  public Socket createSocket(String host, int port) throws IOException {
+    SocketCreator socketCreator;
+    try {
+      socketCreator =
+          SocketCreatorFactory.getSocketCreatorForComponent(SecurableCommunicationChannel.JMX);
+    } catch (Exception exception) {
+      /*
+       * In the context of a gfsh VM the javax.net.ssl properties are used to configure SSL
+       * appropriately - see the constructor for JMXOperationInvoker.
+       */
+      return defaultFactory.createSocket(host, port);
+    }
+
+    return socketCreator.connectForClient(host, port, 0);
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java b/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java
index 3f47a25..4532652 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java
@@ -40,7 +40,6 @@ import javax.management.remote.JMXServiceURL;
 import javax.management.remote.rmi.RMIConnectorServer;
 import javax.management.remote.rmi.RMIJRMPServerImpl;
 import javax.management.remote.rmi.RMIServerImpl;
-import javax.rmi.ssl.SslRMIClientSocketFactory;
 
 import com.healthmarketscience.rmiio.exporter.RemoteStreamExporter;
 import org.apache.commons.lang.StringUtils;
@@ -402,7 +401,7 @@ public class ManagementAgent {
       logger.debug("Starting jmx manager agent on port {}{}", port,
           (bindAddr != null ? (" bound to " + bindAddr) : "") + (ssl ? " using SSL" : ""));
     }
-    rmiClientSocketFactory = ssl ? new SslRMIClientSocketFactory() : null;
+    rmiClientSocketFactory = ssl ? new ContextAwareSSLRMIClientSocketFactory() : null;
     rmiServerSocketFactory = new GemFireRMIServerSocketFactory(socketCreator, bindAddr);
 
     // Following is done to prevent rmi causing stop the world gcs
diff --git a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
index d296f77..98fa997 100644
--- a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
+++ b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
@@ -471,6 +471,7 @@ org/apache/geode/management/cli/CommandProcessingException,true,-139877952163957
 org/apache/geode/management/cli/CommandServiceException,true,7316102209844678329
 org/apache/geode/management/cli/Result$Status,false,code:int
 org/apache/geode/management/internal/BackupStatusImpl,true,3704172840296221840,backedUpDiskStores:java/util/Map,offlineDiskStores:java/util/Set
+org/apache/geode/management/internal/ContextAwareSSLRMIClientSocketFactory,true,8159615071011918570
 org/apache/geode/management/internal/JmxManagerLocator$StartJmxManagerFunction,true,-2860286061903069789
 org/apache/geode/management/internal/ManagementAgent$GemFireRMIServerSocketFactory,true,-811909050641332716,bindAddr:java/net/InetAddress
 org/apache/geode/management/internal/ManagementFunction,true,1,mbeanServer:javax/management/MBeanServer,notificationHub:org/apache/geode/management/internal/NotificationHub
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/DeployJarWithSSLDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/DeployJarWithSSLDUnitTest.java
new file mode 100644
index 0000000..0d93d2f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/DeployJarWithSSLDUnitTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.management.internal.configuration;
+
+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_TRUSTSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_PASSWORD;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_TYPE;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.security.SecurableCommunicationChannels;
+import org.apache.geode.test.compiler.ClassBuilder;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+
+@Category(DistributedTest.class)
+public class DeployJarWithSSLDUnitTest {
+
+  private static File jks;
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public ClusterStartupRule lsRule = new ClusterStartupRule();
+
+  @Rule
+  public GfshCommandRule gfsh = new GfshCommandRule();
+
+  private static MemberVM locator;
+
+  private File sslConfigFile = null;
+
+  static {
+    try {
+      /*
+       * This file was generated with the following command:
+       * keytool -genkey -dname "CN=localhost" -alias self -validity 3650 -keyalg EC \
+       * -keystore trusted.keystore -keypass password -storepass password \
+       * -ext san=ip:127.0.0.1 -storetype jks
+       */
+      jks = new File(DeployJarWithSSLDUnitTest.class.getClassLoader()
+          .getResource("ssl/trusted.keystore").toURI());
+    } catch (URISyntaxException e) {
+      e.printStackTrace();
+    }
+  }
+
+  private static Properties sslProperties = new Properties() {
+    {
+      setProperty(SSL_ENABLED_COMPONENTS, SecurableCommunicationChannels.ALL);
+      setProperty(SSL_KEYSTORE, jks.getAbsolutePath());
+      setProperty(SSL_KEYSTORE_PASSWORD, "password");
+      setProperty(SSL_KEYSTORE_TYPE, "JKS");
+      setProperty(SSL_TRUSTSTORE, jks.getAbsolutePath());
+      setProperty(SSL_TRUSTSTORE_PASSWORD, "password");
+      setProperty(SSL_TRUSTSTORE_TYPE, "JKS");
+      setProperty(SSL_CIPHERS, "any");
+      setProperty(SSL_PROTOCOLS, "any");
+    }
+  };
+
+  @Before
+  public void before() throws Exception {
+    locator = lsRule.startLocatorVM(0, sslProperties);
+
+    sslConfigFile = temporaryFolder.newFile("ssl.properties");
+    FileOutputStream out = new FileOutputStream(sslConfigFile);
+    sslProperties.store(out, null);
+  }
+
+  @Test
+  public void deployJarToCluster() throws Exception {
+    lsRule.startServerVM(1, sslProperties, locator.getPort());
+
+    gfsh.connectAndVerify(locator.getPort(), GfshCommandRule.PortType.locator,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+
+    String clusterJar = createJarFileWithClass("Cluster", "cluster.jar", temporaryFolder.getRoot());
+    gfsh.executeAndAssertThat("deploy --jar=" + clusterJar).statusIsSuccess();
+  }
+
+  @Test
+  public void startServerAfterDeployJar() throws Exception {
+    gfsh.connectAndVerify(locator.getPort(), GfshCommandRule.PortType.locator,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+
+    String clusterJar = createJarFileWithClass("Cluster", "cluster.jar", temporaryFolder.getRoot());
+    gfsh.executeAndAssertThat("deploy --jar=" + clusterJar).statusIsSuccess();
+
+    lsRule.startServerVM(1, sslProperties, locator.getPort());
+  }
+
+  @Test
+  public void deployJarWithMultipleLocators() throws Exception {
+    MemberVM locator2 = lsRule.startLocatorVM(1, sslProperties, locator.getPort());
+    lsRule.startServerVM(2, sslProperties, locator2.getPort());
+
+    gfsh.connectAndVerify(locator.getPort(), GfshCommandRule.PortType.locator,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+
+    String clusterJar = createJarFileWithClass("Cluster", "cluster.jar", temporaryFolder.getRoot());
+    gfsh.executeAndAssertThat("deploy --jar=" + clusterJar).statusIsSuccess();
+  }
+
+  protected String createJarFileWithClass(String className, String jarName, File dir)
+      throws IOException {
+    File jarFile = new File(dir, jarName);
+    new ClassBuilder().writeJarFromName(className, jarFile);
+    return jarFile.getCanonicalPath();
+  }
+}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSSLTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSSLTest.java
index 5efe22d..537bf84 100644
--- a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSSLTest.java
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSSLTest.java
@@ -60,7 +60,6 @@ import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 
@@ -159,9 +158,6 @@ public class ConnectCommandWithSSLTest {
   @Rule
   public ClusterStartupRule lsRule = new ClusterStartupRule();
 
-  @Rule
-  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-
   private static MemberVM locator;
   private OutputStream out = null;
   private File sslConfigFile = null;

-- 
To stop receiving notification emails like this one, please contact
jensdeppe@apache.org.