You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ud...@apache.org on 2017/08/11 16:47:00 UTC

[01/10] geode git commit: GEODE-2226: SessionReplicationIntegrationTests do not run on windows [Forced Update!]

Repository: geode
Updated Branches:
  refs/heads/feature/GEODE-3416 6d899bdb4 -> 5d567b1f3 (forced update)


GEODE-2226: SessionReplicationIntegrationTests do not run on windows

- moved Assume(not windows) until tester variable set to avoid NPE on tearDown


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

Branch: refs/heads/feature/GEODE-3416
Commit: 1efbf58f3a254a6e27bd63c2b676f216c35c9d9f
Parents: b06f69f
Author: Lynn Hughes-Godfrey <lh...@pivotal.io>
Authored: Wed Aug 9 14:59:34 2017 -0700
Committer: Lynn Hughes-Godfrey <lh...@pivotal.io>
Committed: Wed Aug 9 14:59:34 2017 -0700

----------------------------------------------------------------------
 .../internal/filter/SessionReplicationIntegrationJUnitTest.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/1efbf58f/extensions/geode-modules-session/src/test/java/org/apache/geode/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-session/src/test/java/org/apache/geode/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java b/extensions/geode-modules-session/src/test/java/org/apache/geode/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
index 5a0add2..2679ecc 100644
--- a/extensions/geode-modules-session/src/test/java/org/apache/geode/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
+++ b/extensions/geode-modules-session/src/test/java/org/apache/geode/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
@@ -85,8 +85,6 @@ public class SessionReplicationIntegrationJUnitTest {
   @Before
   public void setUp() throws Exception {
 
-    Assume.assumeFalse(System.getProperty("os.name").toLowerCase().contains("win"));
-
     File gemfireLogFile = new File(tmpdir.newFolder(), "gemfire_modules.log");
 
     request = HttpTester.newRequest();
@@ -94,6 +92,8 @@ public class SessionReplicationIntegrationJUnitTest {
     tester = new MyServletTester();
     tester.setContextPath("/test");
 
+    Assume.assumeFalse(System.getProperty("os.name").toLowerCase().contains("win"));
+
     filterHolder =
         tester.addFilter(SessionCachingFilter.class, "/*", EnumSet.of(DispatcherType.REQUEST));
     filterHolder.setInitParameter(DistributionConfig.GEMFIRE_PREFIX + "property.mcast-port", "0");


[06/10] geode git commit: GEODE-3328: refactor ConnectCommand

Posted by ud...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSSLTest.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7c4fb44
--- /dev/null
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSSLTest.java
@@ -0,0 +1,303 @@
+/*
+ * 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.cli.commands;
+
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_CIPHERS;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_ENABLED;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_KEYSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_KEYSTORE_PASSWORD;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_KEYSTORE_TYPE;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_PROTOCOLS;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_TRUSTSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_TRUSTSTORE_PASSWORD;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_CIPHERS;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_ENABLED;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_KEYSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_KEYSTORE_TYPE;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_PROTOCOLS;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_TRUSTSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_CIPHERS;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_ENABLED;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_KEYSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_KEYSTORE_PASSWORD;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_KEYSTORE_TYPE;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_PROTOCOLS;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_TRUSTSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_TRUSTSTORE_PASSWORD;
+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 static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStream;
+import java.net.URISyntaxException;
+import java.util.Properties;
+
+import javax.net.ssl.HttpsURLConnection;
+
+import org.junit.Before;
+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;
+
+import org.apache.geode.security.SecurableCommunicationChannels;
+import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class ConnectCommandWithSSLTest {
+
+  private static File jks;
+
+  static {
+    try {
+      jks = new File(ConnectCommandWithSSLTest.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");
+    }
+  };
+
+  private static Properties jmxSslProperties = new Properties() {
+    {
+      setProperty(JMX_MANAGER_SSL_ENABLED, "true");
+      setProperty(JMX_MANAGER_SSL_KEYSTORE, jks.getAbsolutePath());
+      setProperty(JMX_MANAGER_SSL_KEYSTORE_PASSWORD, "password");
+      setProperty(JMX_MANAGER_SSL_KEYSTORE_TYPE, "JKS");
+      setProperty(JMX_MANAGER_SSL_TRUSTSTORE, jks.getAbsolutePath());
+      setProperty(JMX_MANAGER_SSL_TRUSTSTORE_PASSWORD, "password");
+      setProperty(JMX_MANAGER_SSL_CIPHERS, "any");
+      setProperty(JMX_MANAGER_SSL_PROTOCOLS, "any");
+    }
+  };
+
+  private static Properties clusterSslProperties = new Properties() {
+    {
+      setProperty(CLUSTER_SSL_ENABLED, "true");
+      setProperty(CLUSTER_SSL_KEYSTORE, jks.getAbsolutePath());
+      setProperty(CLUSTER_SSL_KEYSTORE_PASSWORD, "password");
+      setProperty(CLUSTER_SSL_KEYSTORE_TYPE, "JKS");
+      setProperty(CLUSTER_SSL_TRUSTSTORE, jks.getAbsolutePath());
+      setProperty(CLUSTER_SSL_TRUSTSTORE_PASSWORD, "password");
+      setProperty(CLUSTER_SSL_CIPHERS, "any");
+      setProperty(CLUSTER_SSL_PROTOCOLS, "any");
+    }
+  };
+
+  private static Properties httpSslProperties = new Properties() {
+    {
+      setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
+      setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getAbsolutePath());
+      setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
+      setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE, "JKS");
+      setProperty(HTTP_SERVICE_SSL_TRUSTSTORE, jks.getAbsolutePath());
+      setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD, "password");
+      setProperty(HTTP_SERVICE_SSL_CIPHERS, "any");
+      setProperty(HTTP_SERVICE_SSL_PROTOCOLS, "any");
+    }
+  };
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  private static MemberVM locator;
+  private OutputStream out = null;
+  private File sslConfigFile = null;
+
+  @Before
+  public void before() throws Exception {
+    locator = lsRule.startLocatorVM(0, sslProperties);
+    HttpsURLConnection.setDefaultHostnameVerifier((hostname, session) -> true);
+    IgnoredException.addIgnoredException("javax.net.ssl.SSLException: Unrecognized SSL message");
+    sslConfigFile = temporaryFolder.newFile("ssl.properties");
+    out = new FileOutputStream(sslConfigFile);
+  }
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @Test
+  public void connectWithNoSSL() throws Exception {
+    gfsh.connect(locator.getPort(), GfshShellConnectionRule.PortType.locator);
+    assertThat(gfsh.isConnected()).isFalse();
+    // should fail at connecting to locator stage
+    assertThat(gfsh.getGfshOutput()).doesNotContain("Connecting to Manager at");
+    assertThat(gfsh.getGfshOutput())
+        .contains("trying to connect a non-SSL-enabled client to an SSL-enabled locator");
+
+    gfsh.connect(locator.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger);
+    assertThat(gfsh.isConnected()).isFalse();
+    assertThat(gfsh.getGfshOutput()).contains("non-JRMP server at remote endpoint");
+
+    gfsh.connect(locator.getHttpPort(), GfshShellConnectionRule.PortType.http);
+    assertThat(gfsh.isConnected()).isFalse();
+    assertThat(gfsh.getGfshOutput()).contains("Unexpected end of file from server");
+  }
+
+  @Test
+  public void connectWithSSL() throws Exception {
+    sslProperties.store(out, null);
+
+    gfsh.connect(locator.getPort(), GfshShellConnectionRule.PortType.locator,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+    gfsh.disconnect();
+
+    gfsh.connect(locator.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+    gfsh.disconnect();
+
+    gfsh.connect(locator.getHttpPort(), GfshShellConnectionRule.PortType.http,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+  }
+
+  @Test
+  public void connectWithJmxSSL() throws Exception {
+    jmxSslProperties.store(out, null);
+    // can't connect locator
+    gfsh.connect(locator.getPort(), GfshShellConnectionRule.PortType.locator,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isFalse();
+    assertThat(gfsh.getGfshOutput()).doesNotContain("Connecting to Manager at");
+    assertThat(gfsh.getGfshOutput())
+        .contains("trying to connect a non-SSL-enabled client to an SSL-enabled locator");
+
+    // can connect to jmx
+    gfsh.connect(locator.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+    gfsh.disconnect();
+
+    // cannot conect to http
+    gfsh.connect(locator.getHttpPort(), GfshShellConnectionRule.PortType.http,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isFalse();
+  }
+
+  @Test
+  /*
+   * apparently cluster-ssl-* configurations are copied to jmx-ssl-* and http-server-ssl-*, so
+   * connection to other channels will succeed as well. see DistributionConfigImpl around line 986:
+   * if (!isConnected) { copySSLPropsToServerSSLProps(); copySSLPropsToJMXSSLProps();
+   * copyClusterSSLPropsToGatewaySSLProps(); copySSLPropsToHTTPSSLProps(); }
+   */
+  public void connectWithClusterSSL() throws Exception {
+    clusterSslProperties.store(out, null);
+    // can connect to locator and jmx
+    gfsh.connect(locator.getPort(), GfshShellConnectionRule.PortType.locator,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+    gfsh.disconnect();
+
+    // can connect to jmx
+    gfsh.connect(locator.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+    gfsh.disconnect();
+
+    // can conect to http
+    gfsh.connect(locator.getHttpPort(), GfshShellConnectionRule.PortType.http,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+  }
+
+  @Test
+  public void connectWithHttpSSL() throws Exception {
+    httpSslProperties.store(out, null);
+    // can connect to locator and jmx
+    gfsh.connect(locator.getPort(), GfshShellConnectionRule.PortType.locator,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isFalse();
+
+    // can connect to jmx
+    gfsh.connect(locator.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isFalse();
+
+    // cannot conect to http
+    gfsh.connect(locator.getHttpPort(), GfshShellConnectionRule.PortType.http,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+  }
+
+  @Test
+  public void connectWithClusterAndJmxSSL() throws Exception {
+    Properties combined = new Properties();
+    combined.putAll(jmxSslProperties);
+    combined.putAll(clusterSslProperties);
+    combined.store(out, null);
+
+    // can connect to both locator and jmx
+    gfsh.connect(locator.getPort(), GfshShellConnectionRule.PortType.locator,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+  }
+
+
+  @Test
+  public void connectWithSSLAndThenWithNoSSL() throws Exception {
+    sslProperties.store(out, null);
+
+    // can connect to both locator and jmx
+    gfsh.connect(locator.getPort(), GfshShellConnectionRule.PortType.locator,
+        "security-properties-file", sslConfigFile.getAbsolutePath());
+    assertThat(gfsh.isConnected()).isTrue();
+    gfsh.disconnect();
+
+    // reconnect again with no SSL should fail
+    gfsh.connect(locator.getPort(), GfshShellConnectionRule.PortType.locator);
+    assertThat(gfsh.isConnected()).isFalse();
+    // it should fail at connecting to locator, not connecting to manager
+    assertThat(gfsh.getGfshOutput()).doesNotContain("Connecting to Manager at");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSecurityTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSecurityTest.java
new file mode 100644
index 0000000..838924a
--- /dev/null
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSecurityTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.cli.commands;
+
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.security.SimpleTestSecurityManager;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class ConnectCommandWithSecurityTest {
+
+  @ClassRule
+  public static LocatorStarterRule locator =
+      new LocatorStarterRule().withSecurityManager(SimpleTestSecurityManager.class).withAutoStart();
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @Test
+  public void connectToLocator() throws Exception {
+    gfsh.secureConnectAndVerify(locator.getPort(), GfshShellConnectionRule.PortType.locator, "test",
+        "test");
+  }
+
+  @Test
+  public void connectOverJmx() throws Exception {
+    gfsh.secureConnectAndVerify(locator.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger,
+        "test", "test");
+  }
+
+  @Test
+  public void connectOverHttp() throws Exception {
+    gfsh.secureConnectAndVerify(locator.getHttpPort(), GfshShellConnectionRule.PortType.http,
+        "test", "test");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectToLocatorSSLOverHttpTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectToLocatorSSLOverHttpTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectToLocatorSSLOverHttpTest.java
deleted file mode 100644
index e5b8d25..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectToLocatorSSLOverHttpTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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.cli.commands;
-
-import javax.net.ssl.HttpsURLConnection;
-
-import org.apache.geode.management.ConnectToLocatorSSLDUnitTest;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
-
-public class ConnectToLocatorSSLOverHttpTest extends ConnectToLocatorSSLDUnitTest {
-
-  protected void connect() throws Exception {
-    final int httpPort = locator.getHttpPort();
-    final String securityPropsFilePath = securityPropsFile.getCanonicalPath();
-    Host.getHost(0).getVM(1).invoke(() -> {
-      // Our SSL certificate used for tests does not match the hostname "localhost"
-      HttpsURLConnection.setDefaultHostnameVerifier((hostname, session) -> true);
-
-      GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
-      gfshConnector.connectAndVerify(httpPort, GfshShellConnectionRule.PortType.http,
-          CliStrings.CONNECT__SECURITY_PROPERTIES, securityPropsFilePath,
-          CliStrings.CONNECT__USE_SSL, "true");
-      gfshConnector.executeAndVerifyCommand("list members");
-      gfshConnector.close();
-    });
-  }
-}
-


[08/10] geode git commit: GEODE-3328: refactor ConnectCommand

Posted by ud...@apache.org.
GEODE-3328: refactor ConnectCommand

* connect command will prompt for missing ssl configs if ssl is indicated by any options
* command ssl options will override the properties loaded in the file
* reworked connect with ssl so that the configuration won't get cached accidentally.
* have auto connect also prompt for missing ssl configs, not in the socket creator
* change the properties file type in the start server and start locator command to be File instead of String


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

Branch: refs/heads/feature/GEODE-3416
Commit: 7352fcc7e485984b57d72373b468440d09556b5a
Parents: 1bd15f8
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Sat Jul 29 21:32:50 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Aug 10 13:50:31 2017 -0700

----------------------------------------------------------------------
 .../distributed/ConfigurationProperties.java    |   4 +
 .../internal/tcpserver/TcpClient.java           |  12 +-
 .../internal/net/SSLConfigurationFactory.java   | 202 ++++----
 .../internal/JmxManagerLocatorRequest.java      |  47 +-
 .../internal/beans/BeanUtilFuncs.java           |   1 +
 .../cli/AbstractCliAroundInterceptor.java       |  18 +-
 .../internal/cli/commands/ConnectCommand.java   | 482 ++++++++-----------
 .../internal/cli/commands/ShellCommands.java    |  77 ++-
 .../cli/commands/StartLocatorCommand.java       | 148 ++----
 .../internal/cli/commands/StartMemberUtils.java |  13 +-
 .../cli/commands/StartServerCommand.java        |  34 +-
 .../cli/commands/UserInputProperty.java         | 120 +++++
 .../internal/cli/i18n/CliStrings.java           |   4 +-
 .../management/internal/cli/shell/Gfsh.java     |  20 +-
 .../internal/cli/shell/JmxOperationInvoker.java | 147 ++----
 .../internal/cli/util/GfshConsoleReader.java    |  16 +-
 .../ClusterConfigurationStatusRetriever.java    |  17 +-
 .../net/SSLConfigurationFactoryJUnitTest.java   |  37 +-
 .../ConnectToLocatorSSLDUnitTest.java           | 104 ----
 .../ConnectToLocatorWithLegacySSLDUnitTest.java | 118 -----
 .../internal/cli/GfshParserConverterTest.java   |  25 +-
 .../cli/commands/ConnectCommandTest.java        | 300 ++++++++++++
 .../cli/commands/GfshCommandJUnitTest.java      |  41 +-
 .../cli/commands/UserInputPropertyTest.java     | 102 ++++
 .../CacheServerManagementDUnitTest.java         |  58 ++-
 .../commands/ConnectCommandIntegrationTest.java |  50 ++
 .../cli/commands/ConnectCommandWithSSLTest.java | 303 ++++++++++++
 .../ConnectCommandWithSecurityTest.java         |  55 +++
 .../ConnectToLocatorSSLOverHttpTest.java        |  43 --
 29 files changed, 1551 insertions(+), 1047 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/distributed/ConfigurationProperties.java
----------------------------------------------------------------------
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 91a6443..aae7d8a 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
@@ -687,6 +687,10 @@ public interface ConfigurationProperties {
    */
   String SSL_WEB_ALIAS = "ssl-web-alias";
 
+
+  @Deprecated
+  String HTTP_SERVICE_SSL_PREFIX = "http-service-ssl-";
+
   /**
    * The static String definition of the <i>"http-service-ssl-ciphers"</i> property <a
    * name="http-service-ssl-ciphers"/a>

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpClient.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpClient.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpClient.java
index d406891..933b25d 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpClient.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/tcpserver/TcpClient.java
@@ -24,9 +24,9 @@ import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Properties;
 
 import javax.net.ssl.SSLException;
-import javax.net.ssl.SSLHandshakeException;
 
 import org.apache.logging.log4j.Logger;
 
@@ -36,7 +36,9 @@ import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.VersionedDataInputStream;
 import org.apache.geode.internal.VersionedDataOutputStream;
+import org.apache.geode.internal.admin.SSLConfig;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.net.SSLConfigurationFactory;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.net.SocketCreatorFactory;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
@@ -72,6 +74,12 @@ public class TcpClient {
     this(SocketCreatorFactory.getSocketCreatorForComponent(SecurableCommunicationChannel.LOCATOR));
   }
 
+  public TcpClient(Properties properties) {
+    SSLConfig sslConfig = SSLConfigurationFactory.getSSLConfigForComponent(properties,
+        SecurableCommunicationChannel.LOCATOR);
+    this.socketCreator = new SocketCreator(sslConfig);
+  }
+
   /**
    * Constructs a new TcpClient
    * 
@@ -310,7 +318,7 @@ public class TcpClient {
         Object readObject = DataSerializer.readObject(in);
         if (!(readObject instanceof VersionResponse)) {
           throw new LocatorCancelException(
-              "Unrecognisable response received: object is null. This could be the result of trying to connect a non-SSL-enabled locator to an SSL-enabled locator.");
+              "Unrecognisable response received: This could be the result of trying to connect a non-SSL-enabled client to an SSL-enabled locator.");
         }
         VersionResponse response = (VersionResponse) readObject;
         if (response != null) {

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/internal/net/SSLConfigurationFactory.java
----------------------------------------------------------------------
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 4b98617..00ccb74 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
@@ -24,6 +24,7 @@ import org.apache.commons.lang.StringUtils;
 
 import org.apache.geode.GemFireConfigException;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.internal.admin.SSLConfig;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
 
@@ -63,6 +64,11 @@ public class SSLConfigurationFactory {
     getInstance().distributionConfig = distributionConfig;
   }
 
+  /**
+   * @deprecated since GEODE 1.3, use #{getSSLConfigForComponent({@link DistributionConfig} ,
+   *             {@link SecurableCommunicationChannel})} instead
+   */
+  @Deprecated
   public static SSLConfig getSSLConfigForComponent(
       SecurableCommunicationChannel sslEnabledComponent) {
     SSLConfig sslConfig = getInstance().getRegisteredSSLConfigForComponent(sslEnabledComponent);
@@ -80,64 +86,68 @@ public class SSLConfigurationFactory {
 
   private SSLConfig createSSLConfigForComponent(
       final SecurableCommunicationChannel sslEnabledComponent) {
-    SSLConfig sslConfig = createSSLConfig(sslEnabledComponent);
+    return createSSLConfigForComponent(getDistributionConfig(), sslEnabledComponent);
+  }
+
+  private SSLConfig createSSLConfigForComponent(final DistributionConfig distributionConfig,
+      final SecurableCommunicationChannel sslEnabledComponent) {
+    SSLConfig sslConfig = createSSLConfig(distributionConfig, sslEnabledComponent);
     SecurableCommunicationChannel[] sslEnabledComponents =
-        getDistributionConfig().getSecurableCommunicationChannels();
+        distributionConfig.getSecurableCommunicationChannels();
     if (sslEnabledComponents.length == 0) {
-      sslConfig = configureLegacyClusterSSL(sslConfig);
+      sslConfig = configureLegacyClusterSSL(distributionConfig, sslConfig);
     }
     sslConfig.setSecurableCommunicationChannel(sslEnabledComponent);
     switch (sslEnabledComponent) {
       case ALL: {
         // Create a SSLConfig separate for HTTP Service. As the require-authentication might differ
-        createSSLConfigForComponent(SecurableCommunicationChannel.WEB);
+        createSSLConfigForComponent(distributionConfig, SecurableCommunicationChannel.WEB);
         break;
       }
       case CLUSTER: {
         if (sslEnabledComponents.length > 0) {
-          sslConfig = setAliasForComponent(sslConfig, getDistributionConfig().getClusterSSLAlias());
+          sslConfig = setAliasForComponent(sslConfig, distributionConfig.getClusterSSLAlias());
         } else {
-          sslConfig = configureLegacyClusterSSL(sslConfig);
+          sslConfig = configureLegacyClusterSSL(distributionConfig, sslConfig);
         }
         break;
       }
       case LOCATOR: {
         if (sslEnabledComponents.length > 0) {
-          sslConfig = setAliasForComponent(sslConfig, getDistributionConfig().getLocatorSSLAlias());
+          sslConfig = setAliasForComponent(sslConfig, distributionConfig.getLocatorSSLAlias());
         }
         break;
       }
       case SERVER: {
         if (sslEnabledComponents.length > 0) {
-          sslConfig = setAliasForComponent(sslConfig, getDistributionConfig().getServerSSLAlias());
+          sslConfig = setAliasForComponent(sslConfig, distributionConfig.getServerSSLAlias());
         } else {
-          sslConfig = configureLegacyServerSSL(sslConfig);
+          sslConfig = configureLegacyServerSSL(distributionConfig, sslConfig);
         }
         break;
       }
       case GATEWAY: {
         if (sslEnabledComponents.length > 0) {
-          sslConfig = setAliasForComponent(sslConfig, getDistributionConfig().getGatewaySSLAlias());
+          sslConfig = setAliasForComponent(sslConfig, distributionConfig.getGatewaySSLAlias());
         } else {
-          sslConfig = configureLegacyGatewaySSL(sslConfig);
+          sslConfig = configureLegacyGatewaySSL(distributionConfig, sslConfig);
         }
         break;
       }
       case WEB: {
         if (sslEnabledComponents.length > 0) {
-          sslConfig =
-              setAliasForComponent(sslConfig, getDistributionConfig().getHTTPServiceSSLAlias());
-          sslConfig.setRequireAuth(getDistributionConfig().getSSLWebRequireAuthentication());
+          sslConfig = setAliasForComponent(sslConfig, distributionConfig.getHTTPServiceSSLAlias());
+          sslConfig.setRequireAuth(distributionConfig.getSSLWebRequireAuthentication());
         } else {
-          sslConfig = configureLegacyHttpServiceSSL(sslConfig);
+          sslConfig = configureLegacyHttpServiceSSL(distributionConfig, sslConfig);
         }
         break;
       }
       case JMX: {
         if (sslEnabledComponents.length > 0) {
-          sslConfig = setAliasForComponent(sslConfig, getDistributionConfig().getJMXSSLAlias());
+          sslConfig = setAliasForComponent(sslConfig, distributionConfig.getJMXSSLAlias());
         } else {
-          sslConfig = configureLegacyJMXSSL(sslConfig);
+          sslConfig = configureLegacyJMXSSL(distributionConfig, sslConfig);
         }
         break;
       }
@@ -153,33 +163,35 @@ public class SSLConfigurationFactory {
     return sslConfig;
   }
 
-  private SSLConfig createSSLConfig(final SecurableCommunicationChannel sslEnabledComponent) {
+  private SSLConfig createSSLConfig(final DistributionConfig distributionConfig,
+      final SecurableCommunicationChannel sslEnabledComponent) {
     SSLConfig sslConfig = new SSLConfig();
-    sslConfig.setCiphers(getDistributionConfig().getSSLCiphers());
-    sslConfig.setEnabled(determineIfSSLEnabledForSSLComponent(sslEnabledComponent));
-    sslConfig.setKeystore(getDistributionConfig().getSSLKeyStore());
-    sslConfig.setKeystorePassword(getDistributionConfig().getSSLKeyStorePassword());
-    sslConfig.setKeystoreType(getDistributionConfig().getSSLKeyStoreType());
-    sslConfig.setTruststore(getDistributionConfig().getSSLTrustStore());
-    sslConfig.setTruststorePassword(getDistributionConfig().getSSLTrustStorePassword());
-    sslConfig.setTruststoreType(getDistributionConfig().getSSLTrustStoreType());
-    sslConfig.setProtocols(getDistributionConfig().getSSLProtocols());
-    sslConfig.setRequireAuth(getDistributionConfig().getSSLRequireAuthentication());
-    sslConfig.setAlias(getDistributionConfig().getSSLDefaultAlias());
+    sslConfig.setCiphers(distributionConfig.getSSLCiphers());
+    sslConfig
+        .setEnabled(determineIfSSLEnabledForSSLComponent(distributionConfig, sslEnabledComponent));
+    sslConfig.setKeystore(distributionConfig.getSSLKeyStore());
+    sslConfig.setKeystorePassword(distributionConfig.getSSLKeyStorePassword());
+    sslConfig.setKeystoreType(distributionConfig.getSSLKeyStoreType());
+    sslConfig.setTruststore(distributionConfig.getSSLTrustStore());
+    sslConfig.setTruststorePassword(distributionConfig.getSSLTrustStorePassword());
+    sslConfig.setTruststoreType(distributionConfig.getSSLTrustStoreType());
+    sslConfig.setProtocols(distributionConfig.getSSLProtocols());
+    sslConfig.setRequireAuth(distributionConfig.getSSLRequireAuthentication());
+    sslConfig.setAlias(distributionConfig.getSSLDefaultAlias());
     return sslConfig;
   }
 
-  private boolean determineIfSSLEnabledForSSLComponent(
+  private boolean determineIfSSLEnabledForSSLComponent(final DistributionConfig distributionConfig,
       final SecurableCommunicationChannel sslEnabledComponent) {
-    if (ArrayUtils.contains(getDistributionConfig().getSecurableCommunicationChannels(),
+    if (ArrayUtils.contains(distributionConfig.getSecurableCommunicationChannels(),
         SecurableCommunicationChannel.NONE)) {
       return false;
     }
-    if (ArrayUtils.contains(getDistributionConfig().getSecurableCommunicationChannels(),
+    if (ArrayUtils.contains(distributionConfig.getSecurableCommunicationChannels(),
         SecurableCommunicationChannel.ALL)) {
       return true;
     }
-    return ArrayUtils.contains(getDistributionConfig().getSecurableCommunicationChannels(),
+    return ArrayUtils.contains(distributionConfig.getSecurableCommunicationChannels(),
         sslEnabledComponent) ? true : false;
   }
 
@@ -189,17 +201,18 @@ public class SSLConfigurationFactory {
    * @return A sslConfig object describing the ssl config for the server component
    * @deprecated as of Geode 1.0
    */
-  private SSLConfig configureLegacyClusterSSL(SSLConfig sslConfig) {
-    sslConfig.setCiphers(getDistributionConfig().getClusterSSLCiphers());
-    sslConfig.setEnabled(getDistributionConfig().getClusterSSLEnabled());
-    sslConfig.setKeystore(getDistributionConfig().getClusterSSLKeyStore());
-    sslConfig.setKeystorePassword(getDistributionConfig().getClusterSSLKeyStorePassword());
-    sslConfig.setKeystoreType(getDistributionConfig().getClusterSSLKeyStoreType());
-    sslConfig.setTruststore(getDistributionConfig().getClusterSSLTrustStore());
-    sslConfig.setTruststorePassword(getDistributionConfig().getClusterSSLTrustStorePassword());
-    sslConfig.setTruststoreType(getDistributionConfig().getClusterSSLKeyStoreType());
-    sslConfig.setProtocols(getDistributionConfig().getClusterSSLProtocols());
-    sslConfig.setRequireAuth(getDistributionConfig().getClusterSSLRequireAuthentication());
+  private SSLConfig configureLegacyClusterSSL(final DistributionConfig distributionConfig,
+      final SSLConfig sslConfig) {
+    sslConfig.setCiphers(distributionConfig.getClusterSSLCiphers());
+    sslConfig.setEnabled(distributionConfig.getClusterSSLEnabled());
+    sslConfig.setKeystore(distributionConfig.getClusterSSLKeyStore());
+    sslConfig.setKeystorePassword(distributionConfig.getClusterSSLKeyStorePassword());
+    sslConfig.setKeystoreType(distributionConfig.getClusterSSLKeyStoreType());
+    sslConfig.setTruststore(distributionConfig.getClusterSSLTrustStore());
+    sslConfig.setTruststorePassword(distributionConfig.getClusterSSLTrustStorePassword());
+    sslConfig.setTruststoreType(distributionConfig.getClusterSSLKeyStoreType());
+    sslConfig.setProtocols(distributionConfig.getClusterSSLProtocols());
+    sslConfig.setRequireAuth(distributionConfig.getClusterSSLRequireAuthentication());
     return sslConfig;
   }
 
@@ -209,17 +222,18 @@ public class SSLConfigurationFactory {
    * @return A sslConfig object describing the ssl config for the server component
    * @deprecated as of Geode 1.0
    */
-  private SSLConfig configureLegacyServerSSL(SSLConfig sslConfig) {
-    sslConfig.setCiphers(getDistributionConfig().getServerSSLCiphers());
-    sslConfig.setEnabled(getDistributionConfig().getServerSSLEnabled());
-    sslConfig.setKeystore(getDistributionConfig().getServerSSLKeyStore());
-    sslConfig.setKeystorePassword(getDistributionConfig().getServerSSLKeyStorePassword());
-    sslConfig.setKeystoreType(getDistributionConfig().getServerSSLKeyStoreType());
-    sslConfig.setTruststore(getDistributionConfig().getServerSSLTrustStore());
-    sslConfig.setTruststorePassword(getDistributionConfig().getServerSSLTrustStorePassword());
-    sslConfig.setTruststoreType(getDistributionConfig().getServerSSLKeyStoreType());
-    sslConfig.setProtocols(getDistributionConfig().getServerSSLProtocols());
-    sslConfig.setRequireAuth(getDistributionConfig().getServerSSLRequireAuthentication());
+  private SSLConfig configureLegacyServerSSL(final DistributionConfig distributionConfig,
+      final SSLConfig sslConfig) {
+    sslConfig.setCiphers(distributionConfig.getServerSSLCiphers());
+    sslConfig.setEnabled(distributionConfig.getServerSSLEnabled());
+    sslConfig.setKeystore(distributionConfig.getServerSSLKeyStore());
+    sslConfig.setKeystorePassword(distributionConfig.getServerSSLKeyStorePassword());
+    sslConfig.setKeystoreType(distributionConfig.getServerSSLKeyStoreType());
+    sslConfig.setTruststore(distributionConfig.getServerSSLTrustStore());
+    sslConfig.setTruststorePassword(distributionConfig.getServerSSLTrustStorePassword());
+    sslConfig.setTruststoreType(distributionConfig.getServerSSLKeyStoreType());
+    sslConfig.setProtocols(distributionConfig.getServerSSLProtocols());
+    sslConfig.setRequireAuth(distributionConfig.getServerSSLRequireAuthentication());
     return sslConfig;
   }
 
@@ -229,17 +243,18 @@ public class SSLConfigurationFactory {
    * @return A sslConfig object describing the ssl config for the jmx component
    * @deprecated as of Geode 1.0
    */
-  private SSLConfig configureLegacyJMXSSL(SSLConfig sslConfig) {
-    sslConfig.setCiphers(getDistributionConfig().getJmxManagerSSLCiphers());
-    sslConfig.setEnabled(getDistributionConfig().getJmxManagerSSLEnabled());
-    sslConfig.setKeystore(getDistributionConfig().getJmxManagerSSLKeyStore());
-    sslConfig.setKeystorePassword(getDistributionConfig().getJmxManagerSSLKeyStorePassword());
-    sslConfig.setKeystoreType(getDistributionConfig().getJmxManagerSSLKeyStoreType());
-    sslConfig.setTruststore(getDistributionConfig().getJmxManagerSSLTrustStore());
-    sslConfig.setTruststorePassword(getDistributionConfig().getJmxManagerSSLTrustStorePassword());
-    sslConfig.setTruststoreType(getDistributionConfig().getJmxManagerSSLKeyStoreType());
-    sslConfig.setProtocols(getDistributionConfig().getJmxManagerSSLProtocols());
-    sslConfig.setRequireAuth(getDistributionConfig().getJmxManagerSSLRequireAuthentication());
+  private SSLConfig configureLegacyJMXSSL(final DistributionConfig distributionConfig,
+      final SSLConfig sslConfig) {
+    sslConfig.setCiphers(distributionConfig.getJmxManagerSSLCiphers());
+    sslConfig.setEnabled(distributionConfig.getJmxManagerSSLEnabled());
+    sslConfig.setKeystore(distributionConfig.getJmxManagerSSLKeyStore());
+    sslConfig.setKeystorePassword(distributionConfig.getJmxManagerSSLKeyStorePassword());
+    sslConfig.setKeystoreType(distributionConfig.getJmxManagerSSLKeyStoreType());
+    sslConfig.setTruststore(distributionConfig.getJmxManagerSSLTrustStore());
+    sslConfig.setTruststorePassword(distributionConfig.getJmxManagerSSLTrustStorePassword());
+    sslConfig.setTruststoreType(distributionConfig.getJmxManagerSSLKeyStoreType());
+    sslConfig.setProtocols(distributionConfig.getJmxManagerSSLProtocols());
+    sslConfig.setRequireAuth(distributionConfig.getJmxManagerSSLRequireAuthentication());
     return sslConfig;
   }
 
@@ -249,16 +264,17 @@ public class SSLConfigurationFactory {
    * @return A sslConfig object describing the ssl config for the gateway component
    * @deprecated as of Geode 1.0
    */
-  private SSLConfig configureLegacyGatewaySSL(SSLConfig sslConfig) {
-    sslConfig.setCiphers(getDistributionConfig().getGatewaySSLCiphers());
-    sslConfig.setEnabled(getDistributionConfig().getGatewaySSLEnabled());
-    sslConfig.setKeystore(getDistributionConfig().getGatewaySSLKeyStore());
-    sslConfig.setKeystorePassword(getDistributionConfig().getGatewaySSLKeyStorePassword());
-    sslConfig.setKeystoreType(getDistributionConfig().getGatewaySSLKeyStoreType());
-    sslConfig.setTruststore(getDistributionConfig().getGatewaySSLTrustStore());
-    sslConfig.setTruststorePassword(getDistributionConfig().getGatewaySSLTrustStorePassword());
-    sslConfig.setProtocols(getDistributionConfig().getGatewaySSLProtocols());
-    sslConfig.setRequireAuth(getDistributionConfig().getGatewaySSLRequireAuthentication());
+  private SSLConfig configureLegacyGatewaySSL(final DistributionConfig distributionConfig,
+      final SSLConfig sslConfig) {
+    sslConfig.setCiphers(distributionConfig.getGatewaySSLCiphers());
+    sslConfig.setEnabled(distributionConfig.getGatewaySSLEnabled());
+    sslConfig.setKeystore(distributionConfig.getGatewaySSLKeyStore());
+    sslConfig.setKeystorePassword(distributionConfig.getGatewaySSLKeyStorePassword());
+    sslConfig.setKeystoreType(distributionConfig.getGatewaySSLKeyStoreType());
+    sslConfig.setTruststore(distributionConfig.getGatewaySSLTrustStore());
+    sslConfig.setTruststorePassword(distributionConfig.getGatewaySSLTrustStorePassword());
+    sslConfig.setProtocols(distributionConfig.getGatewaySSLProtocols());
+    sslConfig.setRequireAuth(distributionConfig.getGatewaySSLRequireAuthentication());
     return sslConfig;
   }
 
@@ -268,17 +284,18 @@ public class SSLConfigurationFactory {
    * @return A sslConfig object describing the ssl config for the http service component
    * @deprecated as of Geode 1.0
    */
-  private SSLConfig configureLegacyHttpServiceSSL(SSLConfig sslConfig) {
-    sslConfig.setCiphers(getDistributionConfig().getHttpServiceSSLCiphers());
-    sslConfig.setEnabled(getDistributionConfig().getHttpServiceSSLEnabled());
-    sslConfig.setKeystore(getDistributionConfig().getHttpServiceSSLKeyStore());
-    sslConfig.setKeystorePassword(getDistributionConfig().getHttpServiceSSLKeyStorePassword());
-    sslConfig.setKeystoreType(getDistributionConfig().getHttpServiceSSLKeyStoreType());
-    sslConfig.setTruststore(getDistributionConfig().getHttpServiceSSLTrustStore());
-    sslConfig.setTruststorePassword(getDistributionConfig().getHttpServiceSSLTrustStorePassword());
-    sslConfig.setTruststoreType(getDistributionConfig().getHttpServiceSSLKeyStoreType());
-    sslConfig.setProtocols(getDistributionConfig().getHttpServiceSSLProtocols());
-    sslConfig.setRequireAuth(getDistributionConfig().getHttpServiceSSLRequireAuthentication());
+  private SSLConfig configureLegacyHttpServiceSSL(final DistributionConfig distributionConfig,
+      final SSLConfig sslConfig) {
+    sslConfig.setCiphers(distributionConfig.getHttpServiceSSLCiphers());
+    sslConfig.setEnabled(distributionConfig.getHttpServiceSSLEnabled());
+    sslConfig.setKeystore(distributionConfig.getHttpServiceSSLKeyStore());
+    sslConfig.setKeystorePassword(distributionConfig.getHttpServiceSSLKeyStorePassword());
+    sslConfig.setKeystoreType(distributionConfig.getHttpServiceSSLKeyStoreType());
+    sslConfig.setTruststore(distributionConfig.getHttpServiceSSLTrustStore());
+    sslConfig.setTruststorePassword(distributionConfig.getHttpServiceSSLTrustStorePassword());
+    sslConfig.setTruststoreType(distributionConfig.getHttpServiceSSLKeyStoreType());
+    sslConfig.setProtocols(distributionConfig.getHttpServiceSSLProtocols());
+    sslConfig.setRequireAuth(distributionConfig.getHttpServiceSSLRequireAuthentication());
     return sslConfig;
   }
 
@@ -357,4 +374,15 @@ public class SSLConfigurationFactory {
 
     return sslConfig;
   }
+
+  public static SSLConfig getSSLConfigForComponent(DistributionConfig distributionConfig,
+      SecurableCommunicationChannel sslEnabledComponent) {
+    return getInstance().createSSLConfigForComponent(distributionConfig, sslEnabledComponent);
+  }
+
+  public static SSLConfig getSSLConfigForComponent(Properties properties,
+      SecurableCommunicationChannel sslEnabledComponent) {
+    return getInstance().createSSLConfigForComponent(new DistributionConfigImpl(properties),
+        sslEnabledComponent);
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/JmxManagerLocatorRequest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/JmxManagerLocatorRequest.java b/geode-core/src/main/java/org/apache/geode/management/internal/JmxManagerLocatorRequest.java
index eb71d38..dcf9b22 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/JmxManagerLocatorRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/JmxManagerLocatorRequest.java
@@ -19,14 +19,16 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
-import java.util.Map;
 import java.util.Properties;
 
-import org.apache.geode.distributed.internal.DistributionConfigImpl;
+import org.apache.geode.distributed.internal.tcpserver.LocatorCancelException;
 import org.apache.geode.distributed.internal.tcpserver.TcpClient;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.net.*;
+import org.apache.geode.internal.admin.SSLConfig;
+import org.apache.geode.internal.net.SSLConfigurationFactory;
+import org.apache.geode.internal.net.SocketCreator;
+import org.apache.geode.internal.security.SecurableCommunicationChannel;
 
 /**
  * Sent to a locator to request it to find (and possibly start) a jmx manager for us. It returns a
@@ -37,10 +39,6 @@ import org.apache.geode.internal.net.*;
  */
 public class JmxManagerLocatorRequest implements DataSerializableFixedID {
 
-  public JmxManagerLocatorRequest() {
-    super();
-  }
-
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {}
 
   public void toData(DataOutput out) throws IOException {}
@@ -69,38 +67,25 @@ public class JmxManagerLocatorRequest implements DataSerializableFixedID {
    *         have trouble communicating with it.
    */
   public static JmxManagerLocatorResponse send(String locatorHost, int locatorPort, int msTimeout,
-      Map<String, String> sslConfigProps) throws IOException {
-    Properties distributionConfigProps = new Properties();
+      Properties sslConfigProps) throws IOException, ClassNotFoundException {
     InetAddress networkAddress = InetAddress.getByName(locatorHost);
     InetSocketAddress inetSockAddr = new InetSocketAddress(networkAddress, locatorPort);
 
-    try {
-      if (sslConfigProps != null) {
-        distributionConfigProps.putAll(sslConfigProps);
-      }
-
-      TcpClient client = new TcpClient(new DistributionConfigImpl(distributionConfigProps));
-      Object responseFromServer = client.requestToServer(inetSockAddr, SINGLETON, msTimeout, true);
+    // simply need to turn sslConfigProps into sslConfig for locator
+    SSLConfig sslConfig = SSLConfigurationFactory.getSSLConfigForComponent(sslConfigProps,
+        SecurableCommunicationChannel.LOCATOR);
+    SocketCreator socketCreator = new SocketCreator(sslConfig);
+    TcpClient client = new TcpClient(socketCreator);
+    Object responseFromServer = client.requestToServer(inetSockAddr, SINGLETON, msTimeout, true);
 
+    if (responseFromServer instanceof JmxManagerLocatorResponse)
       return (JmxManagerLocatorResponse) responseFromServer;
-    } catch (ClassNotFoundException unexpected) {
-      throw new IllegalStateException(unexpected);
-    } catch (ClassCastException unexpected) {
-      // FIXME - Abhishek: object read is type "int" instead of
-      // JmxManagerLocatorResponse when the Locator is using SSL & the request
-      // didn't use SSL -> this causes ClassCastException. Not sure how to make
-      // locator meaningful message
-      throw new IllegalStateException(unexpected);
-    } finally {
-      distributionConfigProps.clear();
+    else {
+      throw new LocatorCancelException(
+          "Unrecognisable response received: This could be the result of trying to connect a non-SSL-enabled client to an SSL-enabled locator.");
     }
   }
 
-  public static JmxManagerLocatorResponse send(String locatorHost, int locatorPort, int msTimeout)
-      throws IOException {
-    return send(locatorHost, locatorPort, msTimeout, null);
-  }
-
   @Override
   public Version[] getSerializationVersions() {
     // TODO Auto-generated method stub

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/beans/BeanUtilFuncs.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/BeanUtilFuncs.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/BeanUtilFuncs.java
index 16d45bc..31a198b 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/BeanUtilFuncs.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/BeanUtilFuncs.java
@@ -308,6 +308,7 @@ public class BeanUtilFuncs {
     gemFirePropertyData.setSSLKeyStore(config.getSSLKeyStore());
     gemFirePropertyData.setSSLKeyStoreType(config.getSSLKeyStoreType());
     gemFirePropertyData.setSSLKeyStorePassword(config.getSSLKeyStorePassword());
+
     gemFirePropertyData.setSSLTrustStore(config.getSSLTrustStore());
     gemFirePropertyData.setSSLTrustStorePassword(config.getSSLTrustStorePassword());
     gemFirePropertyData.setSSLTrustStoreType(config.getSSLTrustStoreType());

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/AbstractCliAroundInterceptor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/AbstractCliAroundInterceptor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/AbstractCliAroundInterceptor.java
index 3e1357d..1b84649 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/AbstractCliAroundInterceptor.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/AbstractCliAroundInterceptor.java
@@ -16,8 +16,6 @@ package org.apache.geode.management.internal.cli;
 
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 
-import java.io.IOException;
-
 /**
  * Semi-complete implementation of {@link CliAroundInterceptor} for convenience for implementors.
  * 
@@ -66,7 +64,7 @@ public abstract class AbstractCliAroundInterceptor implements CliAroundIntercept
     return gfsh != null && !gfsh.isQuietMode() && !gfsh.isHeadlessMode();
   }
 
-  protected String interact(String message) throws IOException {
+  protected String interact(String message) {
     return Gfsh.getCurrentInstance().interact(message);
   }
 
@@ -83,18 +81,12 @@ public abstract class AbstractCliAroundInterceptor implements CliAroundIntercept
 
     Response response = null;
     do {
-      try {
-        String userInput = interact(message);
-
-        if (isNullOrEmpty(userInput)) {
-          return defaultResponse;
-        }
-        response = Response.fromString(userInput);
+      String userInput = interact(message);
 
-      } catch (IOException ioex) {
-        severe("Could not read user response", ioex);
-        // What can you do except try again???
+      if (isNullOrEmpty(userInput)) {
+        return defaultResponse;
       }
+      response = Response.fromString(userInput);
 
     } while (response == null);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
index be556a4..274f61c 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
@@ -15,26 +15,19 @@
 
 package org.apache.geode.management.internal.cli.commands;
 
-import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.SSL_CIPHERS;
-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_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.management.internal.cli.shell.Gfsh.SSL_ENABLED_CIPHERS;
-import static org.apache.geode.management.internal.cli.shell.Gfsh.SSL_ENABLED_PROTOCOLS;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_PREFIX;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_PREFIX;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_PREFIX;
 
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
-import java.net.URL;
+import java.net.MalformedURLException;
 import java.security.KeyStore;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
+import java.util.Arrays;
 import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
 
 import javax.net.ssl.HttpsURLConnection;
 import javax.net.ssl.KeyManagerFactory;
@@ -46,16 +39,16 @@ import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
 import org.apache.geode.internal.DSFIDFactory;
+import org.apache.geode.internal.admin.SSLConfig;
 import org.apache.geode.internal.lang.Initializer;
-import org.apache.geode.internal.util.IOUtils;
-import org.apache.geode.internal.util.PasswordUtil;
+import org.apache.geode.internal.net.SSLConfigurationFactory;
+import org.apache.geode.internal.security.SecurableCommunicationChannel;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.ConverterHint;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.JmxManagerLocatorRequest;
 import org.apache.geode.management.internal.JmxManagerLocatorResponse;
 import org.apache.geode.management.internal.SSLUtil;
-import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.LogWrapper;
 import org.apache.geode.management.internal.cli.converters.ConnectionEndpointConverter;
 import org.apache.geode.management.internal.cli.domain.ConnectToLocatorResult;
@@ -65,6 +58,7 @@ import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.cli.shell.JmxOperationInvoker;
 import org.apache.geode.management.internal.cli.util.ConnectionEndpoint;
+import org.apache.geode.management.internal.security.ResourceConstants;
 import org.apache.geode.management.internal.web.domain.LinkIndex;
 import org.apache.geode.management.internal.web.http.support.SimpleHttpRequester;
 import org.apache.geode.management.internal.web.shell.HttpOperationInvoker;
@@ -75,6 +69,12 @@ public class ConnectCommand implements GfshCommand {
   // millis that connect --locator will wait for a response from the locator.
   public final static int CONNECT_LOCATOR_TIMEOUT_MS = 60000; // see bug 45971
 
+  static UserInputProperty[] USERINPUTPROPERTIES =
+      {UserInputProperty.KEYSTORE, UserInputProperty.KEYSTORE_PASSWORD,
+          UserInputProperty.KEYSTORE_TYPE, UserInputProperty.TRUSTSTORE,
+          UserInputProperty.TRUSTSTORE_PASSWORD, UserInputProperty.TRUSTSTORE_TYPE,
+          UserInputProperty.CIPHERS, UserInputProperty.PROTOCOL, UserInputProperty.COMPONENT};
+
   @CliCommand(value = {CliStrings.CONNECT}, help = CliStrings.CONNECT__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_JMX,
       CliStrings.TOPIC_GEODE_MANAGER})
@@ -82,118 +82,178 @@ public class ConnectCommand implements GfshCommand {
       @CliOption(key = {CliStrings.CONNECT__LOCATOR},
           unspecifiedDefaultValue = ConnectionEndpointConverter.DEFAULT_LOCATOR_ENDPOINTS,
           optionContext = ConnectionEndpoint.LOCATOR_OPTION_CONTEXT,
-          help = CliStrings.CONNECT__LOCATOR__HELP) ConnectionEndpoint locatorTcpHostPort,
+          help = CliStrings.CONNECT__LOCATOR__HELP) ConnectionEndpoint locatorEndPoint,
       @CliOption(key = {CliStrings.CONNECT__JMX_MANAGER},
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           optionContext = ConnectionEndpoint.JMXMANAGER_OPTION_CONTEXT,
-          help = CliStrings.CONNECT__JMX_MANAGER__HELP) ConnectionEndpoint memberRmiHostPort,
-      @CliOption(key = {CliStrings.CONNECT__USE_HTTP}, mandatory = false,
-          specifiedDefaultValue = "true", unspecifiedDefaultValue = "false",
+          help = CliStrings.CONNECT__JMX_MANAGER__HELP) ConnectionEndpoint jmxManagerEndPoint,
+      @CliOption(key = {CliStrings.CONNECT__USE_HTTP}, specifiedDefaultValue = "true",
+          unspecifiedDefaultValue = "false",
           help = CliStrings.CONNECT__USE_HTTP__HELP) boolean useHttp,
-      @CliOption(key = {CliStrings.CONNECT__URL}, mandatory = false,
+      @CliOption(key = {CliStrings.CONNECT__URL},
           unspecifiedDefaultValue = CliStrings.CONNECT__DEFAULT_BASE_URL,
           help = CliStrings.CONNECT__URL__HELP) String url,
       @CliOption(key = {CliStrings.CONNECT__USERNAME},
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONNECT__USERNAME__HELP) String userName,
       @CliOption(key = {CliStrings.CONNECT__PASSWORD},
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONNECT__PASSWORD__HELP) String password,
       @CliOption(key = {CliStrings.CONNECT__KEY_STORE},
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONNECT__KEY_STORE__HELP) String keystore,
       @CliOption(key = {CliStrings.CONNECT__KEY_STORE_PASSWORD},
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONNECT__KEY_STORE_PASSWORD__HELP) String keystorePassword,
       @CliOption(key = {CliStrings.CONNECT__TRUST_STORE},
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONNECT__TRUST_STORE__HELP) String truststore,
       @CliOption(key = {CliStrings.CONNECT__TRUST_STORE_PASSWORD},
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONNECT__TRUST_STORE_PASSWORD__HELP) String truststorePassword,
       @CliOption(key = {CliStrings.CONNECT__SSL_CIPHERS},
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONNECT__SSL_CIPHERS__HELP) String sslCiphers,
       @CliOption(key = {CliStrings.CONNECT__SSL_PROTOCOLS},
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONNECT__SSL_PROTOCOLS__HELP) String sslProtocols,
-      @CliOption(key = CliStrings.CONNECT__SECURITY_PROPERTIES,
-          optionContext = ConverterHint.FILE_PATH,
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.CONNECT__SECURITY_PROPERTIES__HELP) final String gfSecurityPropertiesPath,
+      @CliOption(key = CliStrings.CONNECT__SECURITY_PROPERTIES, optionContext = ConverterHint.FILE,
+          help = CliStrings.CONNECT__SECURITY_PROPERTIES__HELP) final File gfSecurityPropertiesFile,
       @CliOption(key = {CliStrings.CONNECT__USE_SSL}, specifiedDefaultValue = "true",
           unspecifiedDefaultValue = "false",
-          help = CliStrings.CONNECT__USE_SSL__HELP) final boolean useSsl) {
-    Result result;
-    String passwordToUse = PasswordUtil.decrypt(password);
-    String keystoreToUse = keystore;
-    String keystorePasswordToUse = keystorePassword;
-    String truststoreToUse = truststore;
-    String truststorePasswordToUse = truststorePassword;
-    String sslCiphersToUse = sslCiphers;
-    String sslProtocolsToUse = sslProtocols;
+          help = CliStrings.CONNECT__USE_SSL__HELP) boolean useSsl)
+      throws MalformedURLException {
 
+    Result result;
     Gfsh gfsh = getGfsh();
+
+    // bail out if gfsh is already connected.
     if (gfsh != null && gfsh.isConnectedAndReady()) {
       return ResultBuilder
           .createInfoResult("Already connected to: " + getGfsh().getOperationInvoker().toString());
     }
 
-    Map<String, String> sslConfigProps = null;
-    try {
-      if (userName != null && userName.length() > 0) {
-        if (passwordToUse == null || passwordToUse.length() == 0) {
-          passwordToUse = gfsh.readPassword(CliStrings.CONNECT__PASSWORD + ": ");
-        }
-        if (passwordToUse == null || passwordToUse.length() == 0) {
-          return ResultBuilder
-              .createConnectionErrorResult(CliStrings.CONNECT__MSG__JMX_PASSWORD_MUST_BE_SPECIFIED);
-        }
-      }
+    // ssl options are passed in in the order defined in USERINPUTPROPERTIES, note the two types
+    // are null, because we don't have connect command options for them yet
+    Properties gfProperties = resolveSslProperties(gfsh, useSsl, null, gfSecurityPropertiesFile,
+        keystore, keystorePassword, null, truststore, truststorePassword, null, sslCiphers,
+        sslProtocols, null);
 
-      sslConfigProps = this.readSSLConfiguration(useSsl, keystoreToUse, keystorePasswordToUse,
-          truststoreToUse, truststorePasswordToUse, sslCiphersToUse, sslProtocolsToUse,
-          gfSecurityPropertiesPath);
-    } catch (IOException e) {
-      return handleExcpetion(e, null);
+    if (containsSSLConfig(gfProperties) || containsLegacySSLConfig(gfProperties)) {
+      useSsl = true;
+    }
+
+    // if username is specified in the option but password is not, prompt for the password
+    // note if gfProperties has username but no password, we would not prompt for password yet,
+    // because we may not need username/password combination to connect.
+    if (userName != null) {
+      gfProperties.setProperty(ResourceConstants.USER_NAME, userName);
+      if (password == null) {
+        password = UserInputProperty.PASSWORD.promptForAcceptableValue(gfsh);
+      }
+      gfProperties.setProperty(UserInputProperty.PASSWORD.getKey(), password);
     }
 
+    // TODO: refactor this to be more readable, like
+    /*
+     * if(useHttp) connectOverHttp else if(jmxManagerEndPoint==null) connectToLocator to get the
+     * jmxManagerEndPoint else connectTo jmxManagerEndPoint
+     */
     if (useHttp) {
-      result = httpConnect(sslConfigProps, useSsl, url, userName, passwordToUse);
+      result = httpConnect(gfProperties, url);
     } else {
-      result = jmxConnect(sslConfigProps, memberRmiHostPort, locatorTcpHostPort, useSsl, userName,
-          passwordToUse, gfSecurityPropertiesPath, false);
+      result = jmxConnect(gfProperties, useSsl, jmxManagerEndPoint, locatorEndPoint, false);
     }
 
     return result;
   }
 
+  /**
+   *
+   * @param gfsh
+   * @param useSsl if true, and no files/options passed, we would still insist on prompting for ssl
+   *        config (considered only when the last three parameters are null)
+   * @param gfPropertiesFile gemfire properties file, can be null
+   * @param gfSecurityPropertiesFile gemfire security properties file, can be null
+   * @param sslOptionValues an array of 9 in this order, as defined in USERINPUTPROPERTIES
+   * @return the properties
+   */
+  Properties resolveSslProperties(Gfsh gfsh, boolean useSsl, File gfPropertiesFile,
+      File gfSecurityPropertiesFile, String... sslOptionValues) {
 
-  private Result httpConnect(Map<String, String> sslConfigProps, boolean useSsl, String url,
-      String userName, String passwordToUse) {
-    Gfsh gfsh = getGfsh();
-    try {
-      Map<String, String> securityProperties = new HashMap<String, String>();
+    // first trying to load the sslProperties from the file
+    Properties gfProperties = loadProperties(gfPropertiesFile, gfSecurityPropertiesFile);
 
-      // at this point, if userName is not empty, password should not be empty either
-      if (userName != null && userName.length() > 0) {
-        securityProperties.put("security-username", userName);
-        securityProperties.put("security-password", passwordToUse);
+    // if the security file is a legacy ssl security file, then the rest of the command options, if
+    // any, are ignored. Because we are not trying to add/replace the legacy ssl values using the
+    // command line values. all command line ssl values updates the ssl-* options.
+    if (containsLegacySSLConfig(gfProperties)) {
+      return gfProperties;
+    }
+
+    // if nothing indicates we should prompt for missing ssl config info, return immediately
+    if (!(useSsl || containsSSLConfig(gfProperties) || isSslImpliedBySslOptions(sslOptionValues))) {
+      return gfProperties;
+    }
+
+    // if use ssl is implied by any of the options, then command option will add to/update the
+    // properties loaded from file. If the ssl config is not specified anywhere, prompt user for it.
+    for (int i = 0; i < USERINPUTPROPERTIES.length; i++) {
+      UserInputProperty userInputProperty = USERINPUTPROPERTIES[i];
+      String sslOptionValue = null;
+      if (sslOptionValues != null && sslOptionValues.length > i) {
+        sslOptionValue = sslOptionValues[i];
       }
+      String sslConfigValue = gfProperties.getProperty(userInputProperty.getKey());
 
-      if (useSsl) {
-        configureHttpsURLConnection(sslConfigProps);
+      // if this option is specified, always use this value
+      if (sslOptionValue != null) {
+        gfProperties.setProperty(userInputProperty.getKey(), sslOptionValue);
+      }
+      // if option is not specified and not present in the original properties, prompt for it
+      else if (sslConfigValue == null) {
+        gfProperties.setProperty(userInputProperty.getKey(),
+            userInputProperty.promptForAcceptableValue(gfsh));
+      }
+    }
+
+    return gfProperties;
+  }
+
+  boolean isSslImpliedBySslOptions(String... sslOptions) {
+    if (sslOptions == null) {
+      return false;
+    }
+    return Arrays.stream(sslOptions).anyMatch(Objects::nonNull);
+  }
+
+  Properties loadProperties(File... files) {
+    Properties properties = new Properties();
+    if (files == null) {
+      return properties;
+    }
+    for (File file : files) {
+      if (file != null) {
+        properties.putAll(ShellCommands.loadProperties(file));
+      }
+    }
+    return properties;
+  }
+
+  static boolean containsLegacySSLConfig(Properties properties) {
+    return properties.stringPropertyNames().stream()
+        .anyMatch(key -> key.startsWith(CLUSTER_SSL_PREFIX)
+            || key.startsWith(JMX_MANAGER_SSL_PREFIX) || key.startsWith(HTTP_SERVICE_SSL_PREFIX));
+  }
+
+  static boolean containsSSLConfig(Properties properties) {
+    return properties.stringPropertyNames().stream().anyMatch(key -> key.startsWith("ssl-"));
+  }
+
+
+  Result httpConnect(Properties gfProperties, String url) {
+    Gfsh gfsh = getGfsh();
+    try {
+      SSLConfig sslConfig = SSLConfigurationFactory.getSSLConfigForComponent(gfProperties,
+          SecurableCommunicationChannel.WEB);
+      if (sslConfig.isEnabled()) {
+        configureHttpsURLConnection(sslConfig);
         if (url.startsWith("http:")) {
           url = url.replace("http:", "https:");
         }
       }
 
-      Iterator<String> it = sslConfigProps.keySet().iterator();
-      while (it.hasNext()) {
-        String secKey = it.next();
-        securityProperties.put(secKey, sslConfigProps.get(secKey));
-      }
-
       // This is so that SSL termination results in https URLs being returned
       String query = (url.startsWith("https")) ? "?scheme=https" : "";
 
@@ -201,14 +261,14 @@ public class ConnectCommand implements GfshCommand {
           "Sending HTTP request for Link Index at (%1$s)...", url.concat("/index").concat(query)));
 
       LinkIndex linkIndex =
-          new SimpleHttpRequester(gfsh, CONNECT_LOCATOR_TIMEOUT_MS, securityProperties)
+          new SimpleHttpRequester(gfsh, CONNECT_LOCATOR_TIMEOUT_MS, (Map) gfProperties)
               .exchange(url.concat("/index").concat(query), LinkIndex.class);
 
       LogWrapper.getInstance()
           .warning(String.format("Received Link Index (%1$s)", linkIndex.toString()));
 
       HttpOperationInvoker operationInvoker =
-          new RestHttpOperationInvoker(linkIndex, gfsh, url, securityProperties);
+          new RestHttpOperationInvoker(linkIndex, gfsh, url, (Map) gfProperties);
 
       Initializer.init(operationInvoker);
       gfsh.setOperationInvoker(operationInvoker);
@@ -226,244 +286,103 @@ public class ConnectCommand implements GfshCommand {
 
       // if it's security exception, and we already sent in username and password, still retuns the
       // connection error
-      if (userName != null) {
+      if (gfProperties.containsKey(ResourceConstants.USER_NAME)) {
         return handleExcpetion(e, null);
       }
 
       // otherwise, prompt for username and password and retry the conenction
-      try {
-        userName = gfsh.readText(CliStrings.CONNECT__USERNAME + ": ");
-        passwordToUse = gfsh.readPassword(CliStrings.CONNECT__PASSWORD + ": ");
-        return httpConnect(sslConfigProps, useSsl, url, userName, passwordToUse);
-      } catch (IOException ioe) {
-        return handleExcpetion(ioe, null);
-      }
+      gfProperties.setProperty(UserInputProperty.USERNAME.getKey(),
+          UserInputProperty.USERNAME.promptForAcceptableValue(gfsh));
+      gfProperties.setProperty(UserInputProperty.PASSWORD.getKey(),
+          UserInputProperty.PASSWORD.promptForAcceptableValue(gfsh));
+      return httpConnect(gfProperties, url);
+
     } finally {
       Gfsh.redirectInternalJavaLoggers();
     }
   }
 
-  private Result jmxConnect(Map<String, String> sslConfigProps,
-      ConnectionEndpoint memberRmiHostPort, ConnectionEndpoint locatorTcpHostPort, boolean useSsl,
-      String userName, String passwordToUse, String gfSecurityPropertiesPath, boolean retry) {
-    ConnectionEndpoint hostPortToConnect = null;
+  Result jmxConnect(Properties gfProperties, boolean useSsl, ConnectionEndpoint memberRmiHostPort,
+      ConnectionEndpoint locatorTcpHostPort, boolean retry) {
+    ConnectionEndpoint jmxHostPortToConnect = null;
     Gfsh gfsh = getGfsh();
 
     try {
-
-      // trying to find the hostPortToConnect, if rmi host port exists, use that, otherwise, use
+      // trying to find the rmi host and port, if rmi host port exists, use that, otherwise, use
       // locator to find the rmi host port
       if (memberRmiHostPort != null) {
-        hostPortToConnect = memberRmiHostPort;
+        jmxHostPortToConnect = memberRmiHostPort;
       } else {
-        // Props required to configure a SocketCreator with SSL.
-        // Used for gfsh->locator connection & not needed for gfsh->manager connection
-        if (useSsl || !sslConfigProps.isEmpty()) {
-          sslConfigProps.put(MCAST_PORT, String.valueOf(0));
-          sslConfigProps.put(LOCATORS, "");
-
-          String sslInfoLogMsg = "Connecting to Locator via SSL.";
-          if (useSsl) {
-            sslInfoLogMsg = CliStrings.CONNECT__USE_SSL + " is set to true. " + sslInfoLogMsg;
-          }
-          gfsh.logToFile(sslInfoLogMsg, null);
+        if (useSsl) {
+          gfsh.logToFile(
+              CliStrings.CONNECT__USE_SSL + " is set to true. Connecting to Locator via SSL.",
+              null);
         }
 
         Gfsh.println(CliStrings.format(CliStrings.CONNECT__MSG__CONNECTING_TO_LOCATOR_AT_0,
             new Object[] {locatorTcpHostPort.toString(false)}));
         ConnectToLocatorResult connectToLocatorResult =
             connectToLocator(locatorTcpHostPort.getHost(), locatorTcpHostPort.getPort(),
-                CONNECT_LOCATOR_TIMEOUT_MS, sslConfigProps);
-        hostPortToConnect = connectToLocatorResult.getMemberEndpoint();
+                CONNECT_LOCATOR_TIMEOUT_MS, gfProperties);
+        jmxHostPortToConnect = connectToLocatorResult.getMemberEndpoint();
 
         // when locator is configured to use SSL (ssl-enabled=true) but manager is not
         // (jmx-manager-ssl=false)
-        if ((useSsl || !sslConfigProps.isEmpty())
-            && !connectToLocatorResult.isJmxManagerSslEnabled()) {
+        if (useSsl && !connectToLocatorResult.isJmxManagerSslEnabled()) {
           gfsh.logInfo(
               CliStrings.CONNECT__USE_SSL
                   + " is set to true. But JMX Manager doesn't support SSL, connecting without SSL.",
               null);
-          sslConfigProps.clear();
+          useSsl = false;
         }
       }
 
-      if (!sslConfigProps.isEmpty()) {
+      if (useSsl) {
         gfsh.logToFile("Connecting to manager via SSL.", null);
       }
 
       // print out the connecting endpoint
       if (!retry) {
         Gfsh.println(CliStrings.format(CliStrings.CONNECT__MSG__CONNECTING_TO_MANAGER_AT_0,
-            new Object[] {hostPortToConnect.toString(false)}));
+            new Object[] {jmxHostPortToConnect.toString(false)}));
       }
 
       InfoResultData infoResultData = ResultBuilder.createInfoResultData();
-      JmxOperationInvoker operationInvoker =
-          new JmxOperationInvoker(hostPortToConnect.getHost(), hostPortToConnect.getPort(),
-              userName, passwordToUse, sslConfigProps, gfSecurityPropertiesPath);
+      JmxOperationInvoker operationInvoker = new JmxOperationInvoker(jmxHostPortToConnect.getHost(),
+          jmxHostPortToConnect.getPort(), gfProperties);
 
       gfsh.setOperationInvoker(operationInvoker);
-      infoResultData.addLine(
-          CliStrings.format(CliStrings.CONNECT__MSG__SUCCESS, hostPortToConnect.toString(false)));
-      LogWrapper.getInstance().info(
-          CliStrings.format(CliStrings.CONNECT__MSG__SUCCESS, hostPortToConnect.toString(false)));
+      infoResultData.addLine(CliStrings.format(CliStrings.CONNECT__MSG__SUCCESS,
+          jmxHostPortToConnect.toString(false)));
+      LogWrapper.getInstance().info(CliStrings.format(CliStrings.CONNECT__MSG__SUCCESS,
+          jmxHostPortToConnect.toString(false)));
       return ResultBuilder.buildResult(infoResultData);
     } catch (Exception e) {
       // all other exceptions, just logs it and returns a connection error
       if (!(e instanceof SecurityException) && !(e instanceof AuthenticationFailedException)) {
-        return handleExcpetion(e, hostPortToConnect);
+        return handleExcpetion(e, jmxHostPortToConnect);
       }
 
       // if it's security exception, and we already sent in username and password, still returns the
       // connection error
-      if (userName != null) {
-        return handleExcpetion(e, hostPortToConnect);
+      if (gfProperties.containsKey(ResourceConstants.USER_NAME)) {
+        return handleExcpetion(e, jmxHostPortToConnect);
       }
 
       // otherwise, prompt for username and password and retry the conenction
-      try {
-        userName = gfsh.readText(CliStrings.CONNECT__USERNAME + ": ");
-        passwordToUse = gfsh.readPassword(CliStrings.CONNECT__PASSWORD + ": ");
-        // GEODE-2250 If no value for both username and password, at this point we need to error to
-        // avoid a stack overflow.
-        if (userName == null && passwordToUse == null)
-          return handleExcpetion(e, hostPortToConnect);
-        return jmxConnect(sslConfigProps, hostPortToConnect, null, useSsl, userName, passwordToUse,
-            gfSecurityPropertiesPath, true);
-      } catch (IOException ioe) {
-        return handleExcpetion(ioe, hostPortToConnect);
-      }
+      gfProperties.setProperty(UserInputProperty.USERNAME.getKey(),
+          UserInputProperty.USERNAME.promptForAcceptableValue(gfsh));
+      gfProperties.setProperty(UserInputProperty.PASSWORD.getKey(),
+          UserInputProperty.PASSWORD.promptForAcceptableValue(gfsh));
+      return jmxConnect(gfProperties, useSsl, jmxHostPortToConnect, null, true);
+
     } finally {
       Gfsh.redirectInternalJavaLoggers();
     }
   }
 
-  /**
-   * Common code to read SSL information. Used by JMX, Locator & HTTP mode connect
-   */
-  private Map<String, String> readSSLConfiguration(boolean useSsl, String keystoreToUse,
-      String keystorePasswordToUse, String truststoreToUse, String truststorePasswordToUse,
-      String sslCiphersToUse, String sslProtocolsToUse, String gfSecurityPropertiesPath)
-      throws IOException {
-
-    Gfsh gfshInstance = getGfsh();
-    final Map<String, String> sslConfigProps = new LinkedHashMap<String, String>();
-
-    // JMX SSL Config 1:
-    // First from gfsecurity properties file if it's specified OR
-    // if the default gfsecurity.properties exists useSsl==true
-    if (useSsl || gfSecurityPropertiesPath != null) {
-      // reference to hold resolved gfSecurityPropertiesPath
-      String gfSecurityPropertiesPathToUse = CliUtil.resolvePathname(gfSecurityPropertiesPath);
-      URL gfSecurityPropertiesUrl = null;
-
-      // Case 1: User has specified gfSecurity properties file
-      if (StringUtils.isNotBlank(gfSecurityPropertiesPathToUse)) {
-        // User specified gfSecurity properties doesn't exist
-        if (!IOUtils.isExistingPathname(gfSecurityPropertiesPathToUse)) {
-          gfshInstance
-              .printAsSevere(CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
-                  "Security ", gfSecurityPropertiesPathToUse));
-        } else {
-          gfSecurityPropertiesUrl = new File(gfSecurityPropertiesPathToUse).toURI().toURL();
-        }
-      } else if (useSsl && gfSecurityPropertiesPath == null) {
-        // Case 2: User has specified to useSsl but hasn't specified
-        // gfSecurity properties file. Use default "gfsecurity.properties"
-        // in current dir, user's home or classpath
-        gfSecurityPropertiesUrl = ShellCommands.getFileUrl("gfsecurity.properties");
-      }
-      // if 'gfSecurityPropertiesPath' OR gfsecurity.properties has resolvable path
-      if (gfSecurityPropertiesUrl != null) {
-        gfshInstance.logToFile("Using security properties file : "
-            + CliUtil.decodeWithDefaultCharSet(gfSecurityPropertiesUrl.getPath()), null);
-        Map<String, String> gfsecurityProps =
-            ShellCommands.loadPropertiesFromURL(gfSecurityPropertiesUrl);
-        // command line options (if any) would override props in gfsecurity.properties
-        sslConfigProps.putAll(gfsecurityProps);
-      }
-    }
-
-    int numTimesPrompted = 0;
-    /*
-     * Using do-while here for a case when --use-ssl=true is specified but no SSL options were
-     * specified & there was no gfsecurity properties specified or readable in default gfsh
-     * directory.
-     *
-     * NOTE: 2nd round of prompting is done only when sslConfigProps map is empty & useSsl is true -
-     * so we won't over-write any previous values.
-     */
-    do {
-      // JMX SSL Config 2: Now read the options
-      if (numTimesPrompted > 0) {
-        Gfsh.println("Please specify these SSL Configuration properties: ");
-      }
-
-      if (numTimesPrompted > 0) {
-        // NOTE: sslConfigProps map was empty
-        keystoreToUse = gfshInstance.readText(CliStrings.CONNECT__KEY_STORE + ": ");
-      }
-      if (keystoreToUse != null && keystoreToUse.length() > 0) {
-        if (keystorePasswordToUse == null || keystorePasswordToUse.length() == 0) {
-          // Check whether specified in gfsecurity props earlier
-          keystorePasswordToUse = sslConfigProps.get(SSL_KEYSTORE_PASSWORD);
-          if (keystorePasswordToUse == null || keystorePasswordToUse.length() == 0) {
-            // not even in properties file, prompt user for it
-            keystorePasswordToUse =
-                gfshInstance.readPassword(CliStrings.CONNECT__KEY_STORE_PASSWORD + ": ");
-            sslConfigProps.put(SSL_KEYSTORE_PASSWORD, keystorePasswordToUse);
-          }
-        } else {// For cases where password is already part of command option
-          sslConfigProps.put(SSL_KEYSTORE_PASSWORD, keystorePasswordToUse);
-        }
-        sslConfigProps.put(SSL_KEYSTORE, keystoreToUse);
-      }
-
-      if (numTimesPrompted > 0) {
-        truststoreToUse = gfshInstance.readText(CliStrings.CONNECT__TRUST_STORE + ": ");
-      }
-      if (truststoreToUse != null && truststoreToUse.length() > 0) {
-        if (truststorePasswordToUse == null || truststorePasswordToUse.length() == 0) {
-          // Check whether specified in gfsecurity props earlier?
-          truststorePasswordToUse = sslConfigProps.get(SSL_TRUSTSTORE_PASSWORD);
-          if (truststorePasswordToUse == null || truststorePasswordToUse.length() == 0) {
-            // not even in properties file, prompt user for it
-            truststorePasswordToUse =
-                gfshInstance.readPassword(CliStrings.CONNECT__TRUST_STORE_PASSWORD + ": ");
-            sslConfigProps.put(SSL_TRUSTSTORE_PASSWORD, truststorePasswordToUse);
-          }
-        } else {// For cases where password is already part of command option
-          sslConfigProps.put(SSL_TRUSTSTORE_PASSWORD, truststorePasswordToUse);
-        }
-        sslConfigProps.put(SSL_TRUSTSTORE, truststoreToUse);
-      }
-
-      if (numTimesPrompted > 0) {
-        sslCiphersToUse = gfshInstance.readText(CliStrings.CONNECT__SSL_CIPHERS + ": ");
-      }
-      if (sslCiphersToUse != null && sslCiphersToUse.length() > 0) {
-        // sslConfigProps.put(DistributionConfig.CLUSTER_SSL_CIPHERS_NAME, sslCiphersToUse);
-        sslConfigProps.put(SSL_ENABLED_CIPHERS, sslCiphersToUse);
-      }
-
-      if (numTimesPrompted > 0) {
-        sslProtocolsToUse = gfshInstance.readText(CliStrings.CONNECT__SSL_PROTOCOLS + ": ");
-      }
-      if (sslProtocolsToUse != null && sslProtocolsToUse.length() > 0) {
-        // sslConfigProps.put(DistributionConfig.CLUSTER_SSL_PROTOCOLS_NAME, sslProtocolsToUse);
-        sslConfigProps.put(SSL_ENABLED_PROTOCOLS, sslProtocolsToUse);
-      }
-
-      // SSL is required to be used but no SSL config found
-    } while (useSsl && sslConfigProps.isEmpty() && (0 == numTimesPrompted++)
-        && !gfshInstance.isQuietMode());
-    return sslConfigProps;
-  }
-
-
   public static ConnectToLocatorResult connectToLocator(String host, int port, int timeout,
-      Map<String, String> props) throws IOException {
+      Properties props) throws IOException, ClassNotFoundException {
     // register DSFID types first; invoked explicitly so that all message type
     // initializations do not happen in first deserialization on a possibly
     // "precious" thread
@@ -497,48 +416,34 @@ public class ConnectCommand implements GfshCommand {
         locatorResponse.isJmxManagerSslEnabled());
   }
 
-  private void configureHttpsURLConnection(Map<String, String> sslConfigProps) throws Exception {
-    String keystoreToUse = sslConfigProps.get(SSL_KEYSTORE);
-    String keystorePasswordToUse = sslConfigProps.get(SSL_KEYSTORE_PASSWORD);
-    String truststoreToUse = sslConfigProps.get(SSL_TRUSTSTORE);
-    String truststorePasswordToUse = sslConfigProps.get(SSL_TRUSTSTORE_PASSWORD);
-    // Ciphers are not passed to HttpsURLConnection. Could not find a clean way
-    // to pass this attribute to socket layer (see #51645)
-    String sslCiphersToUse = sslConfigProps.get(SSL_CIPHERS);
-    String sslProtocolsToUse = sslConfigProps.get(SSL_PROTOCOLS);
-
-    // Commenting the code to set cipher suites in GFSH rest connect (see #51645)
-    /*
-     * if(sslCiphersToUse != null){ System.setProperty("https.cipherSuites", sslCiphersToUse); }
-     */
+  private void configureHttpsURLConnection(SSLConfig sslConfig) throws Exception {
     FileInputStream keyStoreStream = null;
     FileInputStream trustStoreStream = null;
     try {
-
       KeyManagerFactory keyManagerFactory = null;
-      if (StringUtils.isNotBlank(keystoreToUse)) {
-        KeyStore clientKeys = KeyStore.getInstance("JKS");
-        keyStoreStream = new FileInputStream(keystoreToUse);
-        clientKeys.load(keyStoreStream, keystorePasswordToUse.toCharArray());
+      if (StringUtils.isNotBlank(sslConfig.getKeystore())) {
+        KeyStore clientKeys = KeyStore.getInstance(sslConfig.getKeystoreType());
+        keyStoreStream = new FileInputStream(sslConfig.getKeystore());
+        clientKeys.load(keyStoreStream, sslConfig.getKeystorePassword().toCharArray());
 
         keyManagerFactory =
             KeyManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-        keyManagerFactory.init(clientKeys, keystorePasswordToUse.toCharArray());
+        keyManagerFactory.init(clientKeys, sslConfig.getKeystorePassword().toCharArray());
       }
 
       // load server public key
       TrustManagerFactory trustManagerFactory = null;
-      if (StringUtils.isNotBlank(truststoreToUse)) {
-        KeyStore serverPub = KeyStore.getInstance("JKS");
-        trustStoreStream = new FileInputStream(truststoreToUse);
-        serverPub.load(trustStoreStream, truststorePasswordToUse.toCharArray());
+      if (StringUtils.isNotBlank(sslConfig.getTruststore())) {
+        KeyStore serverPub = KeyStore.getInstance(sslConfig.getTruststoreType());
+        trustStoreStream = new FileInputStream(sslConfig.getTruststore());
+        serverPub.load(trustStoreStream, sslConfig.getTruststorePassword().toCharArray());
         trustManagerFactory =
             TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
         trustManagerFactory.init(serverPub);
       }
 
       SSLContext ssl =
-          SSLContext.getInstance(SSLUtil.getSSLAlgo(SSLUtil.readArray(sslProtocolsToUse)));
+          SSLContext.getInstance(SSLUtil.getSSLAlgo(SSLUtil.readArray(sslConfig.getProtocols())));
 
       ssl.init(keyManagerFactory != null ? keyManagerFactory.getKeyManagers() : null,
           trustManagerFactory != null ? trustManagerFactory.getTrustManagers() : null,
@@ -552,13 +457,9 @@ public class ConnectCommand implements GfshCommand {
       if (trustStoreStream != null) {
         trustStoreStream.close();
       }
-
     }
-
-
   }
 
-
   private Result handleExcpetion(Exception e, ConnectionEndpoint hostPortToConnect) {
     String errorMessage = e.getMessage();
     if (hostPortToConnect != null) {
@@ -568,5 +469,4 @@ public class ConnectCommand implements GfshCommand {
     LogWrapper.getInstance().severe(errorMessage, e);
     return ResultBuilder.createConnectionErrorResult(errorMessage);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
index 1aea253..84ee5db 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
@@ -25,10 +25,7 @@ import java.io.InputStreamReader;
 import java.io.Writer;
 import java.net.MalformedURLException;
 import java.net.URL;
-import java.util.Collections;
-import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
@@ -38,7 +35,6 @@ import org.springframework.shell.core.ExitShellRequest;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
-import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.lang.SystemUtils;
 import org.apache.geode.internal.util.IOUtils;
@@ -62,52 +58,43 @@ import org.apache.geode.management.internal.cli.shell.jline.GfshHistory;
  * @since GemFire 7.0
  */
 public class ShellCommands implements GfshCommand {
+  static Properties loadProperties(URL url) {
+    Properties properties = new Properties();
 
-  /* package-private */
-  static Map<String, String> loadPropertiesFromURL(URL gfSecurityPropertiesUrl) {
-    Map<String, String> propsMap = Collections.emptyMap();
+    if (url == null) {
+      return properties;
+    }
 
-    if (gfSecurityPropertiesUrl != null) {
-      InputStream inputStream = null;
-      try {
-        Properties props = new Properties();
-        inputStream = gfSecurityPropertiesUrl.openStream();
-        props.load(inputStream);
-        if (!props.isEmpty()) {
-          Set<String> jmxSpecificProps = new HashSet<String>();
-          propsMap = new LinkedHashMap<String, String>();
-          Set<Entry<Object, Object>> entrySet = props.entrySet();
-          for (Entry<Object, Object> entry : entrySet) {
-
-            String key = (String) entry.getKey();
-            if (key.endsWith(DistributionConfig.JMX_SSL_PROPS_SUFFIX)) {
-              key =
-                  key.substring(0, key.length() - DistributionConfig.JMX_SSL_PROPS_SUFFIX.length());
-              jmxSpecificProps.add(key);
-
-              propsMap.put(key, (String) entry.getValue());
-            } else if (!jmxSpecificProps.contains(key)) {// Prefer properties ending with "-jmx"
-              // over default SSL props.
-              propsMap.put(key, (String) entry.getValue());
-            }
-          }
-          props.clear();
-          jmxSpecificProps.clear();
-        }
-      } catch (IOException io) {
-        throw new RuntimeException(
-            CliStrings.format(CliStrings.CONNECT__MSG__COULD_NOT_READ_CONFIG_FROM_0,
-                CliUtil.decodeWithDefaultCharSet(gfSecurityPropertiesUrl.getPath())),
-            io);
-      } finally {
-        IOUtils.close(inputStream);
-      }
+    try (InputStream inputStream = url.openStream()) {
+      properties.load(inputStream);
+    } catch (IOException io) {
+      throw new RuntimeException(
+          CliStrings.format(CliStrings.CONNECT__MSG__COULD_NOT_READ_CONFIG_FROM_0,
+              CliUtil.decodeWithDefaultCharSet(url.getPath())),
+          io);
+    }
+
+    return properties;
+  }
+
+  static Properties loadProperties(File propertyFile) {
+    try {
+      return loadProperties(propertyFile.toURI().toURL());
+    } catch (MalformedURLException e) {
+      throw new RuntimeException(
+          CliStrings.format("Failed to load configuration properties from pathname (%1$s)!",
+              propertyFile.getAbsolutePath()),
+          e);
     }
-    return propsMap;
   }
 
-  // Copied from DistributedSystem.java
-  public static URL getFileUrl(String fileName) {
+  /**
+   * try to find the file in the current dir or the user home or the classpath in this order
+   * 
+   * @param fileName the name of the file
+   * @return URL if the file is found, otherwise null
+   */
+  public static URL searchFile(String fileName) {
     File file = new File(fileName);
 
     if (file.exists()) {

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartLocatorCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartLocatorCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartLocatorCommand.java
index 5b289a5..72ccfbb 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartLocatorCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartLocatorCommand.java
@@ -17,17 +17,13 @@ package org.apache.geode.management.internal.cli.commands;
 
 import java.io.File;
 import java.net.InetAddress;
-import java.net.MalformedURLException;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 import javax.management.MalformedObjectNameException;
 import javax.net.ssl.SSLException;
-import javax.net.ssl.SSLHandshakeException;
 
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
@@ -47,9 +43,7 @@ import org.apache.geode.internal.util.IOUtils;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.ConverterHint;
 import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.LogWrapper;
 import org.apache.geode.management.internal.cli.domain.ConnectToLocatorResult;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.InfoResultData;
@@ -100,12 +94,11 @@ public class StartLocatorCommand implements GfshCommand {
           help = CliStrings.START_LOCATOR__PORT__HELP) final Integer port,
       @CliOption(key = CliStrings.START_LOCATOR__DIR,
           help = CliStrings.START_LOCATOR__DIR__HELP) String workingDirectory,
-      @CliOption(key = CliStrings.START_LOCATOR__PROPERTIES,
-          optionContext = ConverterHint.FILE_PATH,
-          help = CliStrings.START_LOCATOR__PROPERTIES__HELP) String gemfirePropertiesPathname,
+      @CliOption(key = CliStrings.START_LOCATOR__PROPERTIES, optionContext = ConverterHint.FILE,
+          help = CliStrings.START_LOCATOR__PROPERTIES__HELP) File gemfirePropertiesFile,
       @CliOption(key = CliStrings.START_LOCATOR__SECURITY_PROPERTIES,
-          optionContext = ConverterHint.FILE_PATH,
-          help = CliStrings.START_LOCATOR__SECURITY_PROPERTIES__HELP) String gemfireSecurityPropertiesPathname,
+          optionContext = ConverterHint.FILE,
+          help = CliStrings.START_LOCATOR__SECURITY_PROPERTIES__HELP) File gemfireSecurityPropertiesFile,
       @CliOption(key = CliStrings.START_LOCATOR__INITIALHEAP,
           help = CliStrings.START_LOCATOR__INITIALHEAP__HELP) final String initialHeap,
       @CliOption(key = CliStrings.START_LOCATOR__MAXHEAP,
@@ -135,23 +128,16 @@ public class StartLocatorCommand implements GfshCommand {
 
       workingDirectory = StartMemberUtils.resolveWorkingDir(workingDirectory, memberName);
 
-      gemfirePropertiesPathname = CliUtil.resolvePathname(gemfirePropertiesPathname);
-
-      if (StringUtils.isNotBlank(gemfirePropertiesPathname)
-          && !IOUtils.isExistingPathname(gemfirePropertiesPathname)) {
+      if (gemfirePropertiesFile != null && !gemfirePropertiesFile.exists()) {
         return ResultBuilder.createUserErrorResult(
             CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, StringUtils.EMPTY,
-                gemfirePropertiesPathname));
+                gemfirePropertiesFile.getAbsolutePath()));
       }
 
-      gemfireSecurityPropertiesPathname =
-          CliUtil.resolvePathname(gemfireSecurityPropertiesPathname);
-
-      if (StringUtils.isNotBlank(gemfireSecurityPropertiesPathname)
-          && !IOUtils.isExistingPathname(gemfireSecurityPropertiesPathname)) {
+      if (gemfireSecurityPropertiesFile != null && !gemfireSecurityPropertiesFile.exists()) {
         return ResultBuilder.createUserErrorResult(
             CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ",
-                gemfireSecurityPropertiesPathname));
+                gemfireSecurityPropertiesFile.getAbsolutePath()));
       }
 
       File locatorPidFile = new File(workingDirectory, ProcessType.LOCATOR.getPidFileName());
@@ -200,8 +186,8 @@ public class StartLocatorCommand implements GfshCommand {
       LocatorLauncher locatorLauncher = locatorLauncherBuilder.build();
 
       String[] locatorCommandLine = createStartLocatorCommandLine(locatorLauncher,
-          gemfirePropertiesPathname, gemfireSecurityPropertiesPathname, gemfireProperties,
-          classpath, includeSystemClasspath, jvmArgsOpts, initialHeap, maxHeap);
+          gemfirePropertiesFile, gemfireSecurityPropertiesFile, gemfireProperties, classpath,
+          includeSystemClasspath, jvmArgsOpts, initialHeap, maxHeap);
 
       final Process locatorProcess = new ProcessBuilder(locatorCommandLine)
           .directory(new File(locatorLauncher.getWorkingDirectory())).start();
@@ -240,7 +226,7 @@ public class StartLocatorCommand implements GfshCommand {
                 new File(locatorLauncher.getWorkingDirectory()))),
             null);
 
-        locatorState = LocatorLauncher.LocatorState.fromDirectory(workingDirectory, memberName);
+        LocatorLauncher.LocatorState.fromDirectory(workingDirectory, memberName);
         do {
           if (locatorProcess.isAlive()) {
             Gfsh.print(".");
@@ -289,33 +275,39 @@ public class StartLocatorCommand implements GfshCommand {
       if (asyncStart) {
         infoResultData.addLine(
             String.format(CliStrings.ASYNC_PROCESS_LAUNCH_MESSAGE, CliStrings.LOCATOR_TERM_NAME));
+        return ResultBuilder.buildResult(infoResultData);
+      }
+
+      infoResultData.addLine(locatorState.toString());
+      String locatorHostName;
+      InetAddress bindAddr = locatorLauncher.getBindAddress();
+      if (bindAddr != null) {
+        locatorHostName = bindAddr.getCanonicalHostName();
       } else {
-        infoResultData.addLine(locatorState.toString());
-
-        String locatorHostName;
-        InetAddress bindAddr = locatorLauncher.getBindAddress();
-        if (bindAddr != null) {
-          locatorHostName = bindAddr.getCanonicalHostName();
-        } else {
-          locatorHostName = StringUtils.defaultIfBlank(locatorLauncher.getHostnameForClients(),
-              HostUtils.getLocalHost());
-        }
+        locatorHostName = StringUtils.defaultIfBlank(locatorLauncher.getHostnameForClients(),
+            HostUtils.getLocalHost());
+      }
 
-        int locatorPort = Integer.parseInt(locatorState.getPort());
+      int locatorPort = Integer.parseInt(locatorState.getPort());
 
-        // AUTO-CONNECT
-        // If the connect succeeds add the connected message to the result,
-        // Else, ask the user to use the "connect" command to connect to the Locator.
-        if (shouldAutoConnect(connect)) {
-          doAutoConnect(locatorHostName, locatorPort, gemfirePropertiesPathname,
-              gemfireSecurityPropertiesPathname, infoResultData);
-        }
-        // Report on the state of the Shared Configuration service if enabled...
-        if (enableSharedConfiguration) {
-          infoResultData.addLine(
-              ClusterConfigurationStatusRetriever.fromLocator(locatorHostName, locatorPort));
-        }
+
+      ConnectCommand connectCommand = new ConnectCommand();
+      Properties configProperties = connectCommand.resolveSslProperties(getGfsh(), false,
+          gemfirePropertiesFile, gemfireSecurityPropertiesFile);
+
+      // AUTO-CONNECT
+      // If the connect succeeds add the connected message to the result,
+      // Else, ask the user to use the "connect" command to connect to the Locator.
+      if (shouldAutoConnect(connect)) {
+        doAutoConnect(locatorHostName, locatorPort, configProperties, infoResultData);
       }
+
+      // Report on the state of the Shared Configuration service if enabled...
+      if (enableSharedConfiguration) {
+        infoResultData.addLine(ClusterConfigurationStatusRetriever.fromLocator(locatorHostName,
+            locatorPort, configProperties));
+      }
+
       return ResultBuilder.buildResult(infoResultData);
     } catch (IllegalArgumentException e) {
       String message = e.getMessage();
@@ -348,38 +340,29 @@ public class StartLocatorCommand implements GfshCommand {
   // With execute option (-e), there could be multiple commands which might presume that a prior
   // "start locator" has formed the connection.
   private boolean shouldAutoConnect(final boolean connect) {
-    return (connect && !(getGfsh() == null || isConnectedAndReady()));
+    return (connect && !isConnectedAndReady());
   }
 
   private void doAutoConnect(final String locatorHostname, final int locatorPort,
-      final String gemfirePropertiesPathname, final String gemfireSecurityPropertiesPathname,
-      final InfoResultData infoResultData) {
+      final Properties configurationProperties, final InfoResultData infoResultData) {
     boolean connectSuccess = false;
     boolean jmxManagerAuthEnabled = false;
     boolean jmxManagerSslEnabled = false;
 
-    Map<String, String> configurationProperties = loadConfigurationProperties(
-        gemfireSecurityPropertiesPathname, loadConfigurationProperties(gemfirePropertiesPathname));
-    Map<String, String> locatorConfigurationProperties = new HashMap<>(configurationProperties);
-
     String responseFailureMessage = null;
 
     for (int attempts = 0; (attempts < 10 && !connectSuccess); attempts++) {
       try {
         ConnectToLocatorResult connectToLocatorResult =
             ConnectCommand.connectToLocator(locatorHostname, locatorPort,
-                ConnectCommand.CONNECT_LOCATOR_TIMEOUT_MS / 4, locatorConfigurationProperties);
+                ConnectCommand.CONNECT_LOCATOR_TIMEOUT_MS / 4, configurationProperties);
 
         ConnectionEndpoint memberEndpoint = connectToLocatorResult.getMemberEndpoint();
 
         jmxManagerSslEnabled = connectToLocatorResult.isJmxManagerSslEnabled();
 
-        if (!jmxManagerSslEnabled) {
-          configurationProperties.clear();
-        }
-
         getGfsh().setOperationInvoker(new JmxOperationInvoker(memberEndpoint.getHost(),
-            memberEndpoint.getPort(), null, null, configurationProperties, null));
+            memberEndpoint.getPort(), configurationProperties));
 
         String shellAndLogMessage = CliStrings.format(CliStrings.CONNECT__MSG__SUCCESS,
             "JMX Manager " + memberEndpoint.toString(false));
@@ -403,16 +386,9 @@ public class StartLocatorCommand implements GfshCommand {
         jmxManagerAuthEnabled = true;
         break; // no need to continue after AuthenticationFailedException
       } catch (SSLException ignore) {
-        if (ignore instanceof SSLHandshakeException) {
-          // try to connect again without SSL since the SSL handshake failed implying a plain text
-          // connection...
-          locatorConfigurationProperties.clear();
-        } else {
-          // another type of SSL error occurred (possibly a configuration issue); pass the buck...
-          getGfsh().logToFile(ignore.getMessage(), ignore);
-          responseFailureMessage = "Check your SSL configuration and try again.";
-          break;
-        }
+        // another type of SSL error occurred (possibly a configuration issue); pass the buck...
+        getGfsh().logToFile(ignore.getMessage(), ignore);
+        responseFailureMessage = "Check your SSL configuration and try again.";
       } catch (Exception ignore) {
         getGfsh().logToFile(ignore.getMessage(), ignore);
         responseFailureMessage = "Failed to connect; unknown cause: " + ignore.getMessage();
@@ -431,30 +407,6 @@ public class StartLocatorCommand implements GfshCommand {
 
   }
 
-  private Map<String, String> loadConfigurationProperties(
-      final String configurationPropertiesPathname) {
-    return loadConfigurationProperties(configurationPropertiesPathname, null);
-  }
-
-  private Map<String, String> loadConfigurationProperties(
-      final String configurationPropertiesPathname, Map<String, String> configurationProperties) {
-    configurationProperties =
-        (configurationProperties != null ? configurationProperties : new HashMap<>());
-
-    if (IOUtils.isExistingPathname(configurationPropertiesPathname)) {
-      try {
-        configurationProperties.putAll(ShellCommands
-            .loadPropertiesFromURL(new File(configurationPropertiesPathname).toURI().toURL()));
-      } catch (MalformedURLException ignore) {
-        LogWrapper.getInstance()
-            .warning(String.format(
-                "Failed to load GemFire configuration properties from pathname (%1$s)!",
-                configurationPropertiesPathname), ignore);
-      }
-    }
-    return configurationProperties;
-  }
-
   private void doOnConnectionFailure(final String locatorHostName, final int locatorPort,
       final boolean jmxManagerAuthEnabled, final boolean jmxManagerSslEnabled,
       final InfoResultData infoResultData) {
@@ -481,7 +433,7 @@ public class StartLocatorCommand implements GfshCommand {
   }
 
   String[] createStartLocatorCommandLine(final LocatorLauncher launcher,
-      final String gemfirePropertiesPathname, final String gemfireSecurityPropertiesPathname,
+      final File gemfirePropertiesFile, final File gemfireSecurityPropertiesFile,
       final Properties gemfireProperties, final String userClasspath,
       final Boolean includeSystemClasspath, final String[] jvmArgsOpts, final String initialHeap,
       final String maxHeap) throws MalformedObjectNameException {
@@ -494,8 +446,8 @@ public class StartLocatorCommand implements GfshCommand {
         .add(getLocatorClasspath(Boolean.TRUE.equals(includeSystemClasspath), userClasspath));
 
     StartMemberUtils.addCurrentLocators(this, commandLine, gemfireProperties);
-    StartMemberUtils.addGemFirePropertyFile(commandLine, gemfirePropertiesPathname);
-    StartMemberUtils.addGemFireSecurityPropertyFile(commandLine, gemfireSecurityPropertiesPathname);
+    StartMemberUtils.addGemFirePropertyFile(commandLine, gemfirePropertiesFile);
+    StartMemberUtils.addGemFireSecurityPropertyFile(commandLine, gemfireSecurityPropertiesFile);
     StartMemberUtils.addGemFireSystemProperties(commandLine, gemfireProperties);
     StartMemberUtils.addJvmArgumentsAndOptions(commandLine, jvmArgsOpts);
     StartMemberUtils.addInitialHeap(commandLine, initialHeap);


[02/10] geode git commit: GEODE-3313: Test utility supports building jar files with multiple classes

Posted by ud...@apache.org.
GEODE-3313: Test utility supports building jar files with multiple classes


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

Branch: refs/heads/feature/GEODE-3416
Commit: 06b839c2b458c47eb26f42de65ca82947b8988c2
Parents: 1efbf58
Author: Jared Stewart <js...@pivotal.io>
Authored: Tue Aug 8 10:11:07 2017 -0700
Committer: Jared Stewart <js...@pivotal.io>
Committed: Thu Aug 10 09:27:39 2017 -0700

----------------------------------------------------------------------
 geode-junit/build.gradle                        |   6 +-
 .../geode/test/compiler/ClassNameExtractor.java |  33 +++++
 .../geode/test/compiler/CompiledSourceCode.java |  35 +++++
 .../apache/geode/test/compiler/JarBuilder.java  | 122 ++++++++++++++++++
 .../geode/test/compiler/JavaCompiler.java       | 123 ++++++++++++++++++
 .../test/compiler/UncompiledSourceCode.java     |  71 +++++++++++
 .../test/compiler/ClassNameExtractorTest.java   |  54 ++++++++
 .../geode/test/compiler/JarBuilderTest.java     | 127 +++++++++++++++++++
 .../geode/test/compiler/JavaCompilerTest.java   |  79 ++++++++++++
 .../test/compiler/UncompiledSourceCodeTest.java |  43 +++++++
 .../geode/test/compiler/AbstractClass.java      |  18 +++
 .../geode/test/compiler/ConcreteClass.java      |  19 +++
 12 files changed, 729 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/build.gradle
----------------------------------------------------------------------
diff --git a/geode-junit/build.gradle b/geode-junit/build.gradle
index e47095f..7c533ad 100755
--- a/geode-junit/build.gradle
+++ b/geode-junit/build.gradle
@@ -19,7 +19,11 @@ dependencies {
   compile 'com.jayway.jsonpath:json-path:' + project.'json-path.version'
   testCompile 'commons-lang:commons-lang:' + project.'commons-lang.version'
   testCompile 'com.google.guava:guava:' + project.'guava.version'
-  testCompile 'org.assertj:assertj-core:' + project.'assertj-core.version'
+  compile 'org.assertj:assertj-core:' + project.'assertj-core.version'
+  compile 'commons-io:commons-io:' + project.'commons-io.version'
+  compile 'commons-lang:commons-lang:' + project.'commons-lang.version'
+  compile 'com.google.guava:guava:' + project.'guava.version'
+
 
   compile('junit:junit:' + project.'junit.version') {
     exclude module: 'hamcrest-core'

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/main/java/org/apache/geode/test/compiler/ClassNameExtractor.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/org/apache/geode/test/compiler/ClassNameExtractor.java b/geode-junit/src/main/java/org/apache/geode/test/compiler/ClassNameExtractor.java
new file mode 100644
index 0000000..c48224c
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/compiler/ClassNameExtractor.java
@@ -0,0 +1,33 @@
+/*
+ * 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.test.compiler;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ClassNameExtractor {
+  private static final Pattern EXTRACT_CLASS_NAME_REGEX =
+      Pattern.compile("(?:public|private|protected)* *(?:abstract)* *(?:class|interface) +(\\w+)");
+
+  public String extractFromSourceCode(String sourceCode) {
+    Matcher m = EXTRACT_CLASS_NAME_REGEX.matcher(sourceCode);
+    if (m.find()) {
+      return m.group(1);
+    } else {
+      throw new IllegalArgumentException(
+          String.format("Unable to parse class or interface name from: \n'%s'", sourceCode));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/main/java/org/apache/geode/test/compiler/CompiledSourceCode.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/org/apache/geode/test/compiler/CompiledSourceCode.java b/geode-junit/src/main/java/org/apache/geode/test/compiler/CompiledSourceCode.java
new file mode 100644
index 0000000..c1bfa3d
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/compiler/CompiledSourceCode.java
@@ -0,0 +1,35 @@
+/*
+ * 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.test.compiler;
+
+public class CompiledSourceCode {
+
+  /**
+   * Fully qualified classname in a format suitable for Class.forName
+   */
+  public String className;
+
+  public byte[] compiledBytecode;
+
+  public CompiledSourceCode(String className, byte[] aBytes) {
+    this.className = className.replace('/', '.');
+    this.compiledBytecode = aBytes;
+  }
+
+  @Override
+  public String toString() {
+    return className;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/main/java/org/apache/geode/test/compiler/JarBuilder.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/org/apache/geode/test/compiler/JarBuilder.java b/geode-junit/src/main/java/org/apache/geode/test/compiler/JarBuilder.java
new file mode 100644
index 0000000..beea476
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/compiler/JarBuilder.java
@@ -0,0 +1,122 @@
+/*
+ * 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.test.compiler;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.jar.JarEntry;
+import java.util.jar.JarOutputStream;
+
+import org.assertj.core.api.Assertions;
+
+
+/**
+ * This class accepts java source code in the format of .java source files or strings containing the
+ * contents of .java source files, and compiles the given source code into a jar file specified by
+ * the user.
+ *
+ * <p>
+ * Example of usage:
+ *
+ * <pre>
+ *  &#064;Rule
+ *  public TemporaryFolder temporaryFolder= new TemporaryFolder();
+ *
+ *  &#064;Test
+ *  public void buildJarUsingStrings() {
+ *  File tempDir = temporaryFolder.getRoot()
+ *  JarBuilder jarBuilder = new JarBuilder(tempDir);
+ *  File outputJar = new File("output.jar");
+ *
+ *  String classInFooBarPackage = &quot;package foo.bar; public class ClassA {int n = 10;}&quot;;
+ *  String classInDefaultPackage = &quot;public class ClassB {}&quot;;
+ *  jarBuilder.buildJar(outputJar, classInFooBarPackage, classInDefaultPackage);
+ *     }
+ *
+ *  &#064;Test
+ *  public void buildJarUsingFiles() {
+ *  File tempDir = temporaryFolder.getRoot()
+ *  JarBuilder jarBuilder = new JarBuilder(tempDir);
+ *  File outputJar = new File("output.jar");
+ *
+ *  File sourceFileOne = new File("ClassA.java");
+ *  File sourceFileTwo = new File("ClassB.java");
+ *  jarBuilder.buildJar(outputJar, sourceFileOne, sourceFileTwo);
+ *     }
+ *
+ *  &#064;Test
+ *  public void buildJarUsingClassNames() {
+ *  File tempDir = temporaryFolder.getRoot()
+ *  JarBuilder jarBuilder = new JarBuilder(tempDir);
+ *  File outputJar = new File("output.jar");
+ *
+ *  String classInFooBarPackage = "foo.bar.ClassInFooBarPackage";
+ *  String classInDefaultPackage = "ClassInDefaultPackage";
+ *  jarBuilder.buildJar(outputJar, classInFooBarPackage, classInDefaultPackage);
+ *     }
+ * </pre>
+ **/
+public class JarBuilder {
+  private final JavaCompiler javaCompiler = new JavaCompiler();
+
+  public void buildJarFromClassNames(File outputJarFile, String... classNames) throws IOException {
+    UncompiledSourceCode[] uncompiledSourceCodes = Arrays.stream(classNames)
+        .map(UncompiledSourceCode::fromClassName).toArray(UncompiledSourceCode[]::new);
+
+    List<CompiledSourceCode> compiledSourceCodes = javaCompiler.compile(uncompiledSourceCodes);
+
+    buildJar(outputJarFile, compiledSourceCodes);
+  }
+
+  public void buildJar(File outputJarFile, String... sourceFileContents) throws IOException {
+    List<CompiledSourceCode> compiledSourceCodes = javaCompiler.compile(sourceFileContents);
+
+    buildJar(outputJarFile, compiledSourceCodes);
+  }
+
+  public void buildJar(File outputJarFile, File... sourceFiles) throws IOException {
+    List<CompiledSourceCode> compiledSourceCodes = javaCompiler.compile(sourceFiles);
+
+    buildJar(outputJarFile, compiledSourceCodes);
+  }
+
+  private void buildJar(File outputJarFile, List<CompiledSourceCode> compiledSourceCodes)
+      throws IOException {
+    assertThat(outputJarFile).doesNotExist();
+
+    try (FileOutputStream outputStream = new FileOutputStream(outputJarFile)) {
+      JarOutputStream jarOutputStream = new JarOutputStream(outputStream);
+      for (CompiledSourceCode compiledSource : compiledSourceCodes) {
+
+        String formattedName = compiledSource.className.replace(".", "/");
+        if (!formattedName.endsWith(".class")) {
+          formattedName = formattedName.concat(".class");
+        }
+
+        JarEntry entry = new JarEntry(formattedName);
+        entry.setTime(System.currentTimeMillis());
+        jarOutputStream.putNextEntry(entry);
+        jarOutputStream.write(compiledSource.compiledBytecode);
+        jarOutputStream.closeEntry();
+      }
+      jarOutputStream.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/main/java/org/apache/geode/test/compiler/JavaCompiler.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/org/apache/geode/test/compiler/JavaCompiler.java b/geode-junit/src/main/java/org/apache/geode/test/compiler/JavaCompiler.java
new file mode 100644
index 0000000..8449605
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/compiler/JavaCompiler.java
@@ -0,0 +1,123 @@
+/*
+ * 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.test.compiler;
+
+import static java.util.stream.Collectors.toList;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import javax.tools.ToolProvider;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.commons.io.FileUtils;
+
+
+public class JavaCompiler {
+  private File tempDir;
+
+  public JavaCompiler() {
+    this.tempDir = Files.createTempDir();
+    tempDir.deleteOnExit();
+  }
+
+  public List<CompiledSourceCode> compile(File... sourceFiles) throws IOException {
+    String[] sourceFileContents =
+        Arrays.stream(sourceFiles).map(this::readFileToString).toArray(String[]::new);
+
+    return compile(sourceFileContents);
+  }
+
+  public List<CompiledSourceCode> compile(String... sourceFileContents) throws IOException {
+    UncompiledSourceCode[] uncompiledSourceCodes = Arrays.stream(sourceFileContents)
+        .map(UncompiledSourceCode::fromSourceCode).toArray(UncompiledSourceCode[]::new);
+
+    return compile(uncompiledSourceCodes);
+  }
+
+  public List<CompiledSourceCode> compile(UncompiledSourceCode... uncompiledSources)
+      throws IOException {
+    File temporarySourcesDirectory = createSubdirectory(tempDir, "sources");
+    File temporaryClassesDirectory = createSubdirectory(tempDir, "classes");
+
+    List<String> options = Stream.of("-d", temporaryClassesDirectory.getAbsolutePath(),
+        "-classpath", System.getProperty("java.class.path")).collect(toList());
+
+    try {
+      for (UncompiledSourceCode sourceCode : uncompiledSources) {
+        File sourceFile = new File(temporarySourcesDirectory, sourceCode.simpleClassName + ".java");
+        FileUtils.writeStringToFile(sourceFile, sourceCode.sourceCode, Charsets.UTF_8);
+        options.add(sourceFile.getAbsolutePath());
+      }
+
+      int exitCode = ToolProvider.getSystemJavaCompiler().run(System.in, System.out, System.err,
+          options.toArray(new String[] {}));
+
+      if (exitCode != 0) {
+        throw new RuntimeException(
+            "Unable to compile the given source code. See System.err for details.");
+      }
+
+      List<CompiledSourceCode> compiledSourceCodes = new ArrayList<>();
+      addCompiledClasses(compiledSourceCodes, "", temporaryClassesDirectory);
+      return compiledSourceCodes;
+    } finally {
+      FileUtils.deleteDirectory(temporaryClassesDirectory);
+    }
+  }
+
+  private static void addCompiledClasses(List<CompiledSourceCode> ret, String pkgName, File dir)
+      throws IOException {
+    for (File file : dir.listFiles()) {
+      String filename = file.getName();
+
+      if (file.isDirectory()) {
+        String qname = pkgName + filename + ".";
+        addCompiledClasses(ret, qname, file);
+      } else if (filename.endsWith(".class")) {
+        String qname = pkgName + filename.substring(0, filename.length() - 6);
+        ret.add(new CompiledSourceCode(qname, FileUtils.readFileToByteArray(file)));
+      } else {
+        System.err.println("Unexpected file : " + file.getAbsolutePath());
+      }
+    }
+  }
+
+  private File createSubdirectory(File parent, String directoryName) {
+    File subdirectory = parent.toPath().resolve(directoryName).toFile();
+    if (!subdirectory.exists()) {
+      subdirectory.mkdirs();
+    }
+
+    if (!subdirectory.exists() || !subdirectory.isDirectory()) {
+      throw new IllegalArgumentException("Invalid directory" + subdirectory.getAbsolutePath());
+    }
+
+    return subdirectory;
+  }
+
+  private String readFileToString(File file) {
+    try {
+      return FileUtils.readFileToString(file, Charsets.UTF_8);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/main/java/org/apache/geode/test/compiler/UncompiledSourceCode.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/org/apache/geode/test/compiler/UncompiledSourceCode.java b/geode-junit/src/main/java/org/apache/geode/test/compiler/UncompiledSourceCode.java
new file mode 100644
index 0000000..4161f5e
--- /dev/null
+++ b/geode-junit/src/main/java/org/apache/geode/test/compiler/UncompiledSourceCode.java
@@ -0,0 +1,71 @@
+/*
+ * 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.test.compiler;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.SystemUtils;
+
+public class UncompiledSourceCode {
+  public String simpleClassName;
+  public String sourceCode;
+
+  private UncompiledSourceCode(String simpleClassName, String sourceCode) {
+    this.simpleClassName = simpleClassName;
+    this.sourceCode = sourceCode;
+  }
+
+  public static UncompiledSourceCode fromSourceCode(String sourceCode) {
+    String simpleClassName = new ClassNameExtractor().extractFromSourceCode(sourceCode);
+    return new UncompiledSourceCode(simpleClassName, sourceCode);
+  }
+
+  public static UncompiledSourceCode fromClassName(String fullyQualifiedClassName) {
+    ClassNameWithPackage classNameWithPackage = ClassNameWithPackage.of(fullyQualifiedClassName);
+    boolean isPackageSpecified = StringUtils.isNotBlank(classNameWithPackage.packageName);
+
+    StringBuilder sourceCode = new StringBuilder();
+    if (isPackageSpecified) {
+      sourceCode.append(String.format("package %s;", classNameWithPackage.packageName));
+      sourceCode.append(SystemUtils.LINE_SEPARATOR);
+    }
+
+    sourceCode.append(String.format("public class %s {}", classNameWithPackage.simpleClassName));
+
+    return new UncompiledSourceCode(classNameWithPackage.simpleClassName, sourceCode.toString());
+  }
+
+  private static class ClassNameWithPackage {
+    String packageName;
+    String simpleClassName;
+
+    static ClassNameWithPackage of(String fqClassName) {
+      int indexOfLastDot = fqClassName.lastIndexOf(".");
+
+      if (indexOfLastDot == -1) {
+        return new ClassNameWithPackage("", fqClassName);
+      } else {
+        String specifiedPackage = fqClassName.substring(0, indexOfLastDot);
+        String simpleClassName = fqClassName.substring(indexOfLastDot + 1);
+
+        return new ClassNameWithPackage(specifiedPackage, simpleClassName);
+      }
+    }
+
+    private ClassNameWithPackage(String packageName, String simpleClassName) {
+      this.packageName = packageName;
+      this.simpleClassName = simpleClassName;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/test/java/org/apache/geode/test/compiler/ClassNameExtractorTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/org/apache/geode/test/compiler/ClassNameExtractorTest.java b/geode-junit/src/test/java/org/apache/geode/test/compiler/ClassNameExtractorTest.java
new file mode 100644
index 0000000..db060f8
--- /dev/null
+++ b/geode-junit/src/test/java/org/apache/geode/test/compiler/ClassNameExtractorTest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.test.compiler;
+
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class ClassNameExtractorTest {
+  private static final String SPACE = " ";
+  private static final String CLASS_NAME_TO_FIND = "MyClassNameToFind";
+
+  @Test
+  public void extractsClassNames() throws Exception {
+    SoftAssertions softAssertions = new SoftAssertions();
+    ClassNameExtractor classNameExtractor = new ClassNameExtractor();
+
+    Set<List<String>> permutationsToTest = Sets.cartesianProduct(
+        ImmutableSet.of("public ", "private ", "protected ", ""), ImmutableSet.of("abstract ", ""),
+        ImmutableSet.of("static ", ""), ImmutableSet.of("class ", "interface "),
+        ImmutableSet.of("extends Foo ", ""), ImmutableSet.of("implements Bar ", ""));
+
+    for (List<String> permutation : permutationsToTest) {
+      String firstLineOfSource =
+          permutation.get(0) + permutation.get(1) + permutation.get(2) + permutation.get(3)
+              + CLASS_NAME_TO_FIND + SPACE + permutation.get(4) + permutation.get(5) + " {";
+
+      String className = classNameExtractor.extractFromSourceCode(firstLineOfSource);
+      softAssertions.assertThat(className).isEqualTo(CLASS_NAME_TO_FIND);
+    }
+
+    softAssertions.assertAll();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/test/java/org/apache/geode/test/compiler/JarBuilderTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/org/apache/geode/test/compiler/JarBuilderTest.java b/geode-junit/src/test/java/org/apache/geode/test/compiler/JarBuilderTest.java
new file mode 100644
index 0000000..58adc02
--- /dev/null
+++ b/geode-junit/src/test/java/org/apache/geode/test/compiler/JarBuilderTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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.test.compiler;
+
+import static java.util.stream.Collectors.toSet;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.Set;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+
+import org.assertj.core.api.Assertions;
+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.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class JarBuilderTest {
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private JarBuilder jarBuilder;
+  private File outputJar;
+
+  @Before
+  public void setup() {
+    jarBuilder = new JarBuilder();
+    outputJar = new File(temporaryFolder.getRoot(), "output.jar");
+  }
+
+  @Test
+  public void jarWithSingleClass() throws Exception {
+    File classContents = loadTestResource("AbstractClass.java");
+    jarBuilder.buildJar(outputJar, classContents);
+
+    Set<String> jarEntryNames = jarEntryNamesFromFile(outputJar);
+    assertThat(jarEntryNames)
+        .containsExactlyInAnyOrder("org/apache/geode/test/compiler/AbstractClass.class");
+  }
+
+  @Test
+  public void jarWithTwoDependentClasses() throws Exception {
+    File sourceFileOne = loadTestResource("AbstractClass.java");
+    File sourceFileTwo = loadTestResource("ConcreteClass.java");
+
+    jarBuilder.buildJar(outputJar, sourceFileOne, sourceFileTwo);
+
+    Set<String> jarEntryNames = jarEntryNamesFromFile(outputJar);
+
+    assertThat(jarEntryNames).containsExactlyInAnyOrder(
+        "org/apache/geode/test/compiler/AbstractClass.class",
+        "org/apache/geode/test/compiler/ConcreteClass.class");
+  }
+
+  @Test
+  public void jarWithClassInDefaultPackage() throws Exception {
+    String classInFooBarPackage = "package foo.bar; public class ClassInFooBarPackage {}";
+    String classInDefaultPackage = "public class ClassInDefaultPackage {}";
+    jarBuilder.buildJar(outputJar, classInFooBarPackage, classInDefaultPackage);
+
+    Set<String> jarEntryNames = jarEntryNamesFromFile(outputJar);
+    assertThat(jarEntryNames).containsExactlyInAnyOrder("ClassInDefaultPackage.class",
+        "foo/bar/ClassInFooBarPackage.class");
+  }
+
+
+  @Test
+  public void jarFromOnlyClassNames() throws Exception {
+    String defaultPackageClassName = "DefaultClass";
+    String otherPackageClassName = "foo.bar.OtherClass";
+    jarBuilder.buildJarFromClassNames(outputJar, defaultPackageClassName, otherPackageClassName);
+
+    Set<String> jarEntryNames = jarEntryNamesFromFile(outputJar);
+    assertThat(jarEntryNames).containsExactlyInAnyOrder("DefaultClass.class",
+        "foo/bar/OtherClass.class");
+  }
+
+  @Test
+  public void canLoadClassesFromJar() throws Exception {
+    String defaultPackageClassName = "DefaultClass";
+    String otherPackageClassName = "foo.bar.OtherClass";
+    jarBuilder.buildJarFromClassNames(outputJar, defaultPackageClassName, otherPackageClassName);
+
+    URLClassLoader jarClassLoader = new URLClassLoader(new URL[] {outputJar.toURL()});
+
+    jarClassLoader.loadClass("DefaultClass");
+    jarClassLoader.loadClass("foo.bar.OtherClass");
+  }
+
+  private Set<String> jarEntryNamesFromFile(File jarFile) throws Exception {
+    assertThat(jarFile).exists();
+
+    Enumeration<JarEntry> jarEntries = new JarFile(jarFile).entries();
+    return Collections.list(jarEntries).stream().map(JarEntry::getName).collect(toSet());
+  }
+
+  private File loadTestResource(String fileName) throws URISyntaxException {
+    URL resourceFileURL = this.getClass().getResource(fileName);
+    assertThat(resourceFileURL).isNotNull();
+
+    URI resourceUri = resourceFileURL.toURI();
+    return new File(resourceUri);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/test/java/org/apache/geode/test/compiler/JavaCompilerTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/org/apache/geode/test/compiler/JavaCompilerTest.java b/geode-junit/src/test/java/org/apache/geode/test/compiler/JavaCompilerTest.java
new file mode 100644
index 0000000..425169d
--- /dev/null
+++ b/geode-junit/src/test/java/org/apache/geode/test/compiler/JavaCompilerTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.test.compiler;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.File;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.assertj.core.api.Assertions;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class JavaCompilerTest {
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Test
+  public void compileSingleClass() throws Exception {
+    File implementsFunctionSourceFile = getFileFromTestResources("AbstractClass.java");
+    String classContents = FileUtils.readFileToString(implementsFunctionSourceFile, "UTF-8");
+
+    List<CompiledSourceCode> compiledSourceCodes = new JavaCompiler().compile(classContents);
+
+    assertThat(compiledSourceCodes).hasSize(1);
+  }
+
+  @Test
+  public void compileTwoDependentClasses() throws Exception {
+    File sourceFileOne = getFileFromTestResources("AbstractClass.java");
+    File sourceFileTwo = getFileFromTestResources("ConcreteClass.java");
+
+    List<CompiledSourceCode> compiledSourceCodes =
+        new JavaCompiler().compile(sourceFileOne, sourceFileTwo);
+
+    assertThat(compiledSourceCodes).hasSize(2);
+  }
+
+  @Test
+  public void invalidSourceThrowsException() throws Exception {
+    JavaCompiler javaCompiler = new JavaCompiler();
+    String sourceCode = "public class foo {this is not valid java source code}";
+    assertThatThrownBy(() -> javaCompiler.compile(sourceCode)).isInstanceOf(Exception.class);
+  }
+
+  private File getFileFromTestResources(String fileName) throws URISyntaxException {
+    URL resourceFileURL = this.getClass().getResource(fileName);
+    assertThat(resourceFileURL).isNotNull();
+
+    URI resourceUri = resourceFileURL.toURI();
+    File file = new File(resourceUri);
+
+    assertThat(file).exists();
+    return file;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/test/java/org/apache/geode/test/compiler/UncompiledSourceCodeTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/org/apache/geode/test/compiler/UncompiledSourceCodeTest.java b/geode-junit/src/test/java/org/apache/geode/test/compiler/UncompiledSourceCodeTest.java
new file mode 100644
index 0000000..3db8a72
--- /dev/null
+++ b/geode-junit/src/test/java/org/apache/geode/test/compiler/UncompiledSourceCodeTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.test.compiler;
+
+import static org.assertj.core.api.Assertions.*;
+
+import org.apache.commons.lang.SystemUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class UncompiledSourceCodeTest {
+  @Test
+  public void fromClassNameWithNoPackage() throws Exception {
+    UncompiledSourceCode uncompiledSourceCode = UncompiledSourceCode.fromClassName("NoPackage");
+    assertThat(uncompiledSourceCode.simpleClassName).isEqualTo("NoPackage");
+    assertThat(uncompiledSourceCode.sourceCode).isEqualTo("public class NoPackage {}");
+  }
+
+  @Test
+  public void fromClassNameWithPackage() throws Exception {
+    UncompiledSourceCode uncompiledSourceCode =
+        UncompiledSourceCode.fromClassName("foo.bar.ClassName");
+    assertThat(uncompiledSourceCode.simpleClassName).isEqualTo("ClassName");
+    assertThat(uncompiledSourceCode.sourceCode)
+        .isEqualTo("package foo.bar;" + SystemUtils.LINE_SEPARATOR + "public class ClassName {}");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/test/resources/org/apache/geode/test/compiler/AbstractClass.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/resources/org/apache/geode/test/compiler/AbstractClass.java b/geode-junit/src/test/resources/org/apache/geode/test/compiler/AbstractClass.java
new file mode 100644
index 0000000..24ed0c6
--- /dev/null
+++ b/geode-junit/src/test/resources/org/apache/geode/test/compiler/AbstractClass.java
@@ -0,0 +1,18 @@
+/*
+ * 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.test.compiler;
+
+public abstract class AbstractClass {
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/06b839c2/geode-junit/src/test/resources/org/apache/geode/test/compiler/ConcreteClass.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/resources/org/apache/geode/test/compiler/ConcreteClass.java b/geode-junit/src/test/resources/org/apache/geode/test/compiler/ConcreteClass.java
new file mode 100644
index 0000000..bf98672
--- /dev/null
+++ b/geode-junit/src/test/resources/org/apache/geode/test/compiler/ConcreteClass.java
@@ -0,0 +1,19 @@
+/*
+ * 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.test.compiler;
+
+
+public class ConcreteClass extends AbstractClass {
+}


[05/10] geode git commit: GEODE-3397: Fixed issue with Tomcat locators in cache-client.xml file

Posted by ud...@apache.org.
GEODE-3397: Fixed issue with Tomcat locators in cache-client.xml file

This closes #688


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

Branch: refs/heads/feature/GEODE-3416
Commit: 1bd15f8a27b9875d65910bd8ad51bd32122bc002
Parents: 8a6e309
Author: David Anuta <da...@gmail.com>
Authored: Fri Aug 4 13:26:24 2017 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Thu Aug 10 13:11:18 2017 -0700

----------------------------------------------------------------------
 .../geode/session/tests/ContainerInstall.java   | 20 ++++++++++++++++----
 .../geode/session/tests/ServerContainer.java    |  2 +-
 2 files changed, 17 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/1bd15f8a/geode-assembly/src/test/java/org/apache/geode/session/tests/ContainerInstall.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/session/tests/ContainerInstall.java b/geode-assembly/src/test/java/org/apache/geode/session/tests/ContainerInstall.java
index 45b8668..9d03417 100644
--- a/geode-assembly/src/test/java/org/apache/geode/session/tests/ContainerInstall.java
+++ b/geode-assembly/src/test/java/org/apache/geode/session/tests/ContainerInstall.java
@@ -419,13 +419,25 @@ public abstract class ContainerInstall {
       DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
       Document doc = docBuilder.parse(XMLPath);
 
+      Node node = null;
       // Get node with specified tagId
-      Node node = findNodeWithAttribute(doc, tagName, "id", tagId);
+      if (tagId != null) {
+        node = findNodeWithAttribute(doc, tagName, "id", tagId);
+      } else if (writeOnSimilarAttributeNames) {
+        NodeList nodes = doc.getElementsByTagName(tagName);
+        for (int i = 0; i < nodes.getLength(); i++) {
+          Node n = nodes.item(i);
+          if (nodeHasExactAttributes(n, attributes, false)) {
+            node = n;
+            break;
+          }
+        }
+      }
       // If no node is found
-      if (node != null
-          || (writeOnSimilarAttributeNames && nodeHasExactAttributes(node, attributes, false))) {
+      if (node != null) {
         rewriteNodeAttributes(node, attributes);
-        ((Element) node).setAttribute("id", tagId);
+        if (tagId != null)
+          ((Element) node).setAttribute("id", tagId);
       } else {
         Element e = doc.createElement(tagName);
         // Set id attribute

http://git-wip-us.apache.org/repos/asf/geode/blob/1bd15f8a/geode-assembly/src/test/java/org/apache/geode/session/tests/ServerContainer.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/session/tests/ServerContainer.java b/geode-assembly/src/test/java/org/apache/geode/session/tests/ServerContainer.java
index 94a13e9..dbd438a 100644
--- a/geode-assembly/src/test/java/org/apache/geode/session/tests/ServerContainer.java
+++ b/geode-assembly/src/test/java/org/apache/geode/session/tests/ServerContainer.java
@@ -368,7 +368,7 @@ public abstract class ServerContainer {
       attributes.put("port", Integer.toString(locatorPort));
 
       ContainerInstall.editXMLFile(getSystemProperty("cache-xml-file"), "locator", "pool",
-          attributes);
+          attributes, true);
     } else {
       setSystemProperty("locators", locatorAddress + "[" + locatorPort + "]");
     }


[09/10] geode git commit: GEODE-3416: Reduce synchronization blockages in SocketCloser. Remove synchronization blocks around HashMap. Replace that implementation with simpler ThreadPool that is not unbounded and does not grow as the number of remoteAddre

Posted by ud...@apache.org.
GEODE-3416: Reduce synchronization blockages in SocketCloser.
Remove synchronization blocks around HashMap. Replace that implementation
with simpler ThreadPool that is not unbounded and does not grow as the
number of remoteAddress (clients/peers) are added


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

Branch: refs/heads/feature/GEODE-3416
Commit: 31e4db0f7630c4350519c29f2ce4061fc0c2b1ee
Parents: 7352fcc
Author: Dave Barnes <db...@pivotal.io>
Authored: Thu Aug 10 17:11:50 2017 -0700
Committer: Udo Kohlmeyer <uk...@pivotal.io>
Committed: Fri Aug 11 09:46:15 2017 -0700

----------------------------------------------------------------------
 .../master_middleman/bookbinder_helpers.rb      | 298 -------------------
 .../cache/tier/sockets/CacheClientProxy.java    |  51 +---
 .../apache/geode/internal/net/SocketCloser.java | 176 ++++-------
 .../apache/geode/internal/tcp/Connection.java   |   4 +-
 .../geode/internal/tcp/ConnectionTable.java     |   4 -
 .../internal/net/SocketCloserJUnitTest.java     | 155 +++-------
 6 files changed, 115 insertions(+), 573 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/31e4db0f/geode-book/master_middleman/bookbinder_helpers.rb
----------------------------------------------------------------------
diff --git a/geode-book/master_middleman/bookbinder_helpers.rb b/geode-book/master_middleman/bookbinder_helpers.rb
deleted file mode 100644
index 817875c..0000000
--- a/geode-book/master_middleman/bookbinder_helpers.rb
+++ /dev/null
@@ -1,298 +0,0 @@
-# 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.
-
-require 'bookbinder/code_example_reader'
-require 'bookbinder/ingest/cloner_factory'
-require 'bookbinder/ingest/git_accessor'
-require 'bookbinder/local_filesystem_accessor'
-require 'date'
-require_relative 'archive_drop_down_menu'
-require_relative 'quicklinks_renderer'
-
-I18n.enforce_available_locales = false
-
-module Bookbinder
-  class Helpers < ::Middleman::Extension
-    # class << self
-    #   def registered(app)
-    #     app.helpers HelperMethods
-    #   end
-
-    #   alias :included :registered
-    # end
-
-    module HelperMethods
-
-      def yield_for_code_snippet(from: nil, at: nil)
-        cloner_factory = Ingest::ClonerFactory.new({out: $stdout},
-                                                   LocalFilesystemAccessor.new,
-                                                   Ingest::GitAccessor.new)
-
-        cloner = cloner_factory.produce(config[:local_repo_dir])
-        code_example_reader = CodeExampleReader.new({out: $stdout},
-                                                    LocalFilesystemAccessor.new)
-        working_copy = cloner.call(source_repo_name: from,
-                                   source_ref: 'master',
-                                   destination_parent_dir: config[:workspace])
-
-        snippet, language = code_example_reader.get_snippet_and_language_at(at, working_copy)
-
-        delimiter = '```'
-
-        snippet.prepend("#{delimiter}#{language}\n").concat("\n#{delimiter}")
-      end
-
-      def elastic_search?
-        !!config[:elastic_search]
-      end
-
-      def yield_for_subnav
-        partial "subnavs/#{subnav_template_name}"
-      end
-
-      def yield_for_archive_drop_down_menu
-        menu = ArchiveDropDownMenu.new(
-          config[:archive_menu],
-          current_path: current_page.path
-        )
-        unless menu.empty?
-          partial 'archive_menus/default', locals: { menu_title: menu.title,
-                                                     dropdown_links: menu.dropdown_links }
-        end
-      end
-
-      def exclude_feedback
-        current_page.add_metadata({page: {feedback_disabled: true}})
-      end
-
-      def yield_for_feedback
-        partial 'layouts/feedback' if config[:feedback_enabled] && !current_page.metadata[:page][:feedback_disabled]
-      end
-
-      def exclude_repo_link
-        current_page.add_metadata({page: {repo_link_disabled: true}})
-      end
-
-      def render_repo_link
-        if config[:repo_link_enabled] && repo_url && !current_page.metadata[:page][:repo_link_disabled]
-          "<a id='repo-link' href='#{repo_url}'>View the source for this page in GitHub</a>"
-        end
-      end
-
-      def mermaid_diagram(&blk)
-        escaped_text = capture(&blk).gsub('-','\-')
-
-        @_out_buf.concat "<div class='mermaid'>#{escaped_text}</div>"
-      end
-
-      def modified_date(default_date: nil)
-        parsed_default_date = Time.parse(default_date).utc if default_date
-
-        date = page_last_modified_date || parsed_default_date
-
-        "Page last updated: <span data-behavior=\"DisplayModifiedDate\" data-modified-date=\"#{date.to_i}000\"></span>" if date
-      end
-
-      def breadcrumbs
-        page_chain = add_ancestors_of(current_page, [])
-        breadcrumbs = page_chain.map do |page|
-          make_breadcrumb(page, page == current_page)
-        end.compact
-        return if breadcrumbs.size < 2
-        return content_tag :ul, breadcrumbs.reverse.join(' '), class: 'breadcrumbs'
-      end
-
-      def vars
-        OpenStruct.new config[:template_variables]
-      end
-
-      ## Geode helpers (start)
-      def geode_product_name
-        current_page.data.title= vars.geode_product_name
-      end
-
-      def geode_product_name_long
-        current_page.data.title= vars.geode_product_name_long
-      end
-
-      def geode_product_version
-        current_page.data.title= vars.geode_product_version
-      end
-
-      def set_title(*args)
-        current_page.data.title= args.join(' ')
-      end
-      ## Geode helpers (end)
-
-      def product_info
-        config[:product_info].fetch(template_key, {})
-      end
-
-      def production_host
-        config[:production_host]
-      end
-
-      def quick_links
-        page_src = File.read(current_page.source_file)
-        quicklinks_renderer = QuicklinksRenderer.new(vars)
-        Redcarpet::Markdown.new(quicklinks_renderer).render(page_src)
-      end
-
-      def owners
-        html_resources = sitemap.resources.select { |r| r.path.end_with?('.html') }
-        html_resources.each.with_object({}) { |resource, owners|
-          owners[resource.path] = Array(resource.data['owner'])
-        }
-      end
-
-      def template_key
-        decreasingly_specific_namespaces.detect { |ns|
-          config[:subnav_templates].has_key?(ns)
-        }
-      end
-
-      def body_classes(path=current_path.dup, options={})
-        if path.is_a? Hash
-          options = path
-          path = current_path.dup
-        end
-        basename = File.basename(path)
-        dirname = File.dirname(path).gsub('.', '_')
-        page_classes(File.join(dirname, basename), options)
-      end
-
-      private
-
-      def subnav_template_name
-        config[:subnav_templates][template_key] || 'default'
-      end
-
-      def decreasingly_specific_namespaces
-        body_classes(numeric_prefix: numeric_class_prefix).
-          split(' ').reverse.drop(1).
-          map {|ns| ns.sub(/^#{numeric_class_prefix}/, '')}
-      end
-
-      def numeric_class_prefix
-        'NUMERIC_CLASS_PREFIX'
-      end
-
-      def page_last_modified_date
-        git_accessor = Ingest::GitAccessor.new
-
-        current_date = if current_page.data.dita
-          git_accessor.author_date(preprocessing_path(current_page.source_file), dita: true)
-        else
-          git_accessor.author_date(current_page.source_file)
-        end
-
-        current_date.utc if current_date
-      end
-
-      def repo_url
-        nested_dir, filename = parse_out_nested_dir_and_filename
-
-        repo_dir = match_repo_dir(nested_dir)
-        page_repo_config = config[:repo_links][repo_dir]
-
-        if page_repo_config && page_repo_config['ref']
-          org_repo = Pathname(page_repo_config['repo'])
-          ref = Pathname(page_repo_config['ref'])
-          at_path = at_path(page_repo_config)
-          nested_dir = extract_nested_directory(nested_dir, repo_dir)
-
-          "http://github.com/#{org_repo.join(Pathname('tree'), ref, Pathname(nested_dir), at_path, source_file(filename))}"
-        end
-      end
-
-      def match_repo_dir(nested_dir)
-        config[:repo_links].keys
-          .select{ |key| nested_dir.match(/^#{key}/) }
-          .sort_by{ |key| key.length }
-          .last
-      end
-
-      def source_file(filename)
-        fs = LocalFilesystemAccessor.new
-
-        if current_page.data.dita
-          source_filename = "#{filename}.xml"
-
-          if fs.source_file_exists?(Pathname(preprocessing_path(current_page.source_file)).dirname,
-                                             source_filename)
-            source_filename
-          else
-            ''
-          end
-        else
-          "#{filename}.html.md.erb"
-        end
-      end
-
-      def preprocessing_path(current_source_path)
-        root_path, nested_repo_path = current_source_path.split('source')
-
-        root_path.gsub!('/output/master_middleman', '')
-
-        "#{root_path}output/preprocessing/sections#{nested_repo_path}"
-      end
-
-      def parse_out_nested_dir_and_filename
-        current_page.path
-          .match(/\/?(.*?)\/?([^\/]*)\.html$?/)
-          .captures
-      end
-
-      def extract_nested_directory(nested_dir, repo_dir)
-        nested_dir = nested_dir.gsub("#{repo_dir}", '')
-        nested_dir = nested_dir.sub('/', '') if nested_dir[0] == '/'
-
-        nested_dir
-      end
-
-      def at_path(page_repo_config)
-        path = page_repo_config['at_path'] || ""
-
-        Pathname(path)
-      end
-
-      def add_ancestors_of(page, ancestors)
-        if page
-          add_ancestors_of(page.parent, ancestors + [page])
-        else
-          ancestors
-        end
-      end
-
-      def make_breadcrumb(page, is_current_page)
-        return nil unless (text = page.data.breadcrumb || page.data.title)
-        if is_current_page
-          css_class = 'active'
-          link = content_tag :span, text
-        else
-          link = link_to(text, '/' + page.path)
-        end
-        content_tag :li, link, :class => css_class
-      end
-    end
-    
-    helpers HelperMethods
-    
-
-  end
-end
-::Middleman::Extensions.register(:bookbinder, Bookbinder::Helpers)

http://git-wip-us.apache.org/repos/asf/geode/blob/31e4db0f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
index d7e3548..98bfed9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
@@ -181,11 +181,7 @@ public class CacheClientProxy implements ClientSession {
    * True if we are connected to a client.
    */
   private volatile boolean connected = false;
-  // /**
-  // * A string representing interest in all keys
-  // */
-  // protected static final String ALL_KEYS = "ALL_KEYS";
-  //
+
   /**
    * True if a marker message is still in the ha queue.
    */
@@ -459,47 +455,6 @@ public class CacheClientProxy implements ClientSession {
     return this.proxyID;
   }
 
-  // the following code was commented out simply because it was not used
-  // /**
-  // * Determines if the proxy represents the client host (and only the host, not
-  // * necessarily the exact VM running on the host)
-  // *
-  // * @return Whether the proxy represents the client host
-  // */
-  // protected boolean representsClientHost(String clientHost)
-  // {
-  // // [bruce] TODO BUGBUGBUG: this should compare InetAddresses, not Strings
-  // return this._remoteHostAddress.equals(clientHost);
-  // }
-
-  // protected boolean representsClientVM(DistributedMember remoteMember)
-  // {
-  // // logger.warn("Is input port " + clientPort + " contained in " +
-  // // logger.warn("Does input host " + clientHost + " equal " +
-  // // this._remoteHostAddress+ ": " + representsClientHost(clientHost));
-  // // logger.warn("representsClientVM: " +
-  // // (representsClientHost(clientHost) && containsPort(clientPort)));
-  // return (proxyID.getDistributedMember().equals(remoteMember));
-  // }
-
-  // /**
-  // * Determines if the CacheClientUpdater proxied by this instance is listening
-  // * on the input clientHost and clientPort
-  // *
-  // * @param clientHost
-  // * The host name of the client to compare
-  // * @param clientPort
-  // * The port number of the client to compare
-  // *
-  // * @return Whether the CacheClientUpdater proxied by this instance is
-  // * listening on the input clientHost and clientPort
-  // */
-  // protected boolean representsCacheClientUpdater(String clientHost,
-  // int clientPort)
-  // {
-  // return (clientPort == this._socket.getPort() && representsClientHost(clientHost));
-  // }
-
   protected boolean isMember(ClientProxyMembershipID memberId) {
     return this.proxyID.equals(memberId);
   }
@@ -994,8 +949,7 @@ public class CacheClientProxy implements ClientSession {
   private void closeSocket() {
     if (this._socketClosed.compareAndSet(false, true)) {
       // Close the socket
-      this._cacheClientNotifier.getSocketCloser().asyncClose(this._socket, this._remoteHostAddress,
-          null);
+      this._cacheClientNotifier.getSocketCloser().asyncClose(this._socket, null);
       getCacheClientNotifier().getAcceptorStats().decCurrentQueueConnections();
     }
   }
@@ -1009,7 +963,6 @@ public class CacheClientProxy implements ClientSession {
     {
       String remoteHostAddress = this._remoteHostAddress;
       if (remoteHostAddress != null) {
-        this._cacheClientNotifier.getSocketCloser().releaseResourcesForAddress(remoteHostAddress);
         this._remoteHostAddress = null;
       }
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/31e4db0f/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java
index 6d86fd8..fbbe797 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java
@@ -14,11 +14,15 @@
  */
 package org.apache.geode.internal.net;
 
+import org.apache.geode.SystemFailure;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.logging.LoggingThreadGroup;
+import org.apache.logging.log4j.Logger;
+
 import java.io.IOException;
 import java.net.Socket;
-import java.util.HashMap;
-import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
@@ -26,12 +30,6 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.SystemFailure;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.LoggingThreadGroup;
-
 /**
  * This class allows sockets to be closed without blocking. In some cases we have seen a call of
  * socket.close block for minutes. This class maintains a thread pool for every other member we have
@@ -51,28 +49,27 @@ public class SocketCloser {
    * minutes).
    */
   static final long ASYNC_CLOSE_POOL_KEEP_ALIVE_SECONDS =
-      Long.getLong("p2p.ASYNC_CLOSE_POOL_KEEP_ALIVE_SECONDS", 120).longValue();
+      Long.getLong("p2p.ASYNC_CLOSE_POOL_KEEP_ALIVE_SECONDS", 120);
   /**
    * Maximum number of threads that can be doing a socket close. Any close requests over this max
    * will queue up waiting for a thread.
    */
-  static final int ASYNC_CLOSE_POOL_MAX_THREADS =
-      Integer.getInteger("p2p.ASYNC_CLOSE_POOL_MAX_THREADS", 8).intValue();
+  private static final int ASYNC_CLOSE_POOL_MAX_THREADS =
+      Integer.getInteger("p2p.ASYNC_CLOSE_POOL_MAX_THREADS", 8);
   /**
    * How many milliseconds the synchronous requester waits for the async close to happen. Default is
    * 0. Prior releases waited 50ms.
    */
-  static final long ASYNC_CLOSE_WAIT_MILLISECONDS =
-      Long.getLong("p2p.ASYNC_CLOSE_WAIT_MILLISECONDS", 0).longValue();
+  private static final long ASYNC_CLOSE_WAIT_MILLISECONDS =
+      Long.getLong("p2p.ASYNC_CLOSE_WAIT_MILLISECONDS", 0);
 
 
-  /** map of thread pools of async close threads */
-  private final HashMap<String, ThreadPoolExecutor> asyncCloseExecutors = new HashMap<>();
   private final long asyncClosePoolKeepAliveSeconds;
   private final int asyncClosePoolMaxThreads;
   private final long asyncCloseWaitTime;
   private final TimeUnit asyncCloseWaitUnits;
   private boolean closed;
+  private final ExecutorService socketCloserThreadPool;
 
   public SocketCloser() {
     this(ASYNC_CLOSE_POOL_KEEP_ALIVE_SECONDS, ASYNC_CLOSE_POOL_MAX_THREADS,
@@ -90,53 +87,25 @@ public class SocketCloser {
     this.asyncClosePoolMaxThreads = asyncClosePoolMaxThreads;
     this.asyncCloseWaitTime = asyncCloseWaitTime;
     this.asyncCloseWaitUnits = asyncCloseWaitUnits;
+
+    final ThreadGroup threadGroup =
+        LoggingThreadGroup.createThreadGroup("Socket asyncClose", logger);
+    ThreadFactory threadFactory = command -> {
+      Thread thread = new Thread(threadGroup, command);
+      thread.setDaemon(true);
+      return thread;
+    };
+    socketCloserThreadPool = new ThreadPoolExecutor(this.asyncClosePoolMaxThreads,
+        this.asyncClosePoolMaxThreads, this.asyncClosePoolKeepAliveSeconds, TimeUnit.SECONDS,
+        new LinkedBlockingQueue<>(), threadFactory);
   }
 
   public int getMaxThreads() {
     return this.asyncClosePoolMaxThreads;
   }
 
-  private ThreadPoolExecutor getAsyncThreadExecutor(String address) {
-    synchronized (asyncCloseExecutors) {
-      ThreadPoolExecutor pool = asyncCloseExecutors.get(address);
-      if (pool == null) {
-        final ThreadGroup tg = LoggingThreadGroup.createThreadGroup("Socket asyncClose", logger);
-        ThreadFactory tf = new ThreadFactory() {
-          public Thread newThread(final Runnable command) {
-            Thread thread = new Thread(tg, command);
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-        BlockingQueue<Runnable> workQueue = new LinkedBlockingQueue<Runnable>();
-        pool = new ThreadPoolExecutor(this.asyncClosePoolMaxThreads, this.asyncClosePoolMaxThreads,
-            this.asyncClosePoolKeepAliveSeconds, TimeUnit.SECONDS, workQueue, tf);
-        pool.allowCoreThreadTimeOut(true);
-        asyncCloseExecutors.put(address, pool);
-      }
-      return pool;
-    }
-  }
-
-  /**
-   * Call this method if you know all the resources in the closer for the given address are no
-   * longer needed. Currently a thread pool is kept for each address and if you know that an address
-   * no longer needs its pool then you should call this method.
-   */
-  public void releaseResourcesForAddress(String address) {
-    synchronized (asyncCloseExecutors) {
-      ThreadPoolExecutor pool = asyncCloseExecutors.get(address);
-      if (pool != null) {
-        pool.shutdown();
-        asyncCloseExecutors.remove(address);
-      }
-    }
-  }
-
   private boolean isClosed() {
-    synchronized (asyncCloseExecutors) {
-      return this.closed;
-    }
+    return this.closed;
   }
 
   /**
@@ -144,34 +113,9 @@ public class SocketCloser {
    * called then the asyncClose will be done synchronously.
    */
   public void close() {
-    synchronized (asyncCloseExecutors) {
-      if (!this.closed) {
-        this.closed = true;
-        for (ThreadPoolExecutor pool : asyncCloseExecutors.values()) {
-          pool.shutdown();
-        }
-        asyncCloseExecutors.clear();
-      }
-    }
-  }
-
-  private void asyncExecute(String address, Runnable r) {
-    // Waiting 50ms for the async close request to complete is what the old (close per thread)
-    // code did. But now that we will not create a thread for every close request
-    // it seems better to let the thread that requested the close to move on quickly.
-    // So the default has changed to not wait. The system property p2p.ASYNC_CLOSE_WAIT_MILLISECONDS
-    // can be set to how many milliseconds to wait.
-    if (this.asyncCloseWaitTime == 0) {
-      getAsyncThreadExecutor(address).execute(r);
-    } else {
-      Future<?> future = getAsyncThreadExecutor(address).submit(r);
-      try {
-        future.get(this.asyncCloseWaitTime, this.asyncCloseWaitUnits);
-      } catch (InterruptedException | ExecutionException | TimeoutException e) {
-        // We want this code to wait at most 50ms for the close to happen.
-        // It is ok to ignore these exception and let the close continue
-        // in the background.
-      }
+    if (!this.closed) {
+      this.closed = true;
+      socketCloserThreadPool.shutdown();
     }
   }
 
@@ -181,34 +125,40 @@ public class SocketCloser {
    * this method may block for a certain amount of time. If it is called after the SocketCloser is
    * closed then a normal synchronous close is done.
    * 
-   * @param sock the socket to close
-   * @param address identifies who the socket is connected to
-   * @param extra an optional Runnable with stuff to execute in the async thread
+   * @param socket the socket to close
+   * @param runnableCode an optional Runnable with stuff to execute in the async thread
    */
-  public void asyncClose(final Socket sock, final String address, final Runnable extra) {
-    if (sock == null || sock.isClosed()) {
+  public void asyncClose(final Socket socket, final Runnable runnableCode) {
+    if (socket == null || socket.isClosed()) {
       return;
     }
+
     boolean doItInline = false;
     try {
-      synchronized (asyncCloseExecutors) {
-        if (isClosed()) {
-          // this SocketCloser has been closed so do a synchronous, inline, close
-          doItInline = true;
-        } else {
-          asyncExecute(address, new Runnable() {
-            public void run() {
-              Thread.currentThread().setName("AsyncSocketCloser for " + address);
-              try {
-                if (extra != null) {
-                  extra.run();
-                }
-                inlineClose(sock);
-              } finally {
-                Thread.currentThread().setName("unused AsyncSocketCloser");
+      if (isClosed()) {
+        // this SocketCloser has been closed so do a synchronous, inline, close
+        doItInline = true;
+      } else {
+        socketCloserThreadPool.execute(() -> {
+          if (runnableCode != null) {
+            runnableCode.run();
+          }
+          inlineClose(socket);
+        });
+        if (this.asyncCloseWaitTime != 0) {
+          try {
+            Future future = socketCloserThreadPool.submit(() -> {
+              if (runnableCode != null) {
+                runnableCode.run();
               }
-            }
-          });
+              inlineClose(socket);
+            });
+            future.get(this.asyncCloseWaitTime, this.asyncCloseWaitUnits);
+          } catch (InterruptedException | ExecutionException | TimeoutException e) {
+            // We want this code to wait at most 50ms for the close to happen.
+            // It is ok to ignore these exception and let the close continue
+            // in the background.
+          }
         }
       }
     } catch (OutOfMemoryError ignore) {
@@ -217,10 +167,10 @@ public class SocketCloser {
       doItInline = true;
     }
     if (doItInline) {
-      if (extra != null) {
-        extra.run();
+      if (runnableCode != null) {
+        runnableCode.run();
       }
-      inlineClose(sock);
+      inlineClose(socket);
     }
   }
 
@@ -228,19 +178,19 @@ public class SocketCloser {
   /**
    * Closes the specified socket
    * 
-   * @param sock the socket to close
+   * @param socket the socket to close
    */
-  private static void inlineClose(final Socket sock) {
+  private void inlineClose(final Socket socket) {
     // the next two statements are a mad attempt to fix bug
     // 36041 - segv in jrockit in pthread signaling code. This
     // seems to alleviate the problem.
     try {
-      sock.shutdownInput();
-      sock.shutdownOutput();
+      socket.shutdownInput();
+      socket.shutdownOutput();
     } catch (Exception e) {
     }
     try {
-      sock.close();
+      socket.close();
     } catch (IOException ignore) {
     } catch (VirtualMachineError err) {
       SystemFailure.initiateFailure(err);

http://git-wip-us.apache.org/repos/asf/geode/blob/31e4db0f/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java b/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
index 0ecb3bf..954a33c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
@@ -562,7 +562,7 @@ public class Connection implements Runnable {
       } catch (IOException io) {
         logger.fatal(LocalizedMessage
             .create(LocalizedStrings.Connection_UNABLE_TO_GET_P2P_CONNECTION_STREAMS), io);
-        t.getSocketCloser().asyncClose(socket, this.remoteAddr.toString(), null);
+        t.getSocketCloser().asyncClose(socket, null);
         throw io;
       }
     }
@@ -847,7 +847,7 @@ public class Connection implements Runnable {
         Socket s = this.socket;
         if (s != null && !s.isClosed()) {
           prepareForAsyncClose();
-          this.owner.getSocketCloser().asyncClose(s, String.valueOf(this.remoteAddr), null);
+          this.owner.getSocketCloser().asyncClose(s, null);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/31e4db0f/geode-core/src/main/java/org/apache/geode/internal/tcp/ConnectionTable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/tcp/ConnectionTable.java b/geode-core/src/main/java/org/apache/geode/internal/tcp/ConnectionTable.java
index 044ab42..11c3bb3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/tcp/ConnectionTable.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/tcp/ConnectionTable.java
@@ -929,10 +929,6 @@ public class ConnectionTable {
               owner.getDM().getMembershipManager().getShutdownCause());
         }
       }
-
-      if (remoteAddress != null) {
-        this.socketCloser.releaseResourcesForAddress(remoteAddress.toString());
-      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/31e4db0f/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java
index 942cad4..b6dbfe2 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java
@@ -14,22 +14,21 @@
  */
 package org.apache.geode.internal.net;
 
-import static org.junit.Assert.*;
-
-import java.net.Socket;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.awaitility.Awaitility;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.internal.net.SocketCloser;
-import org.apache.geode.test.dunit.Wait;
-import org.apache.geode.test.dunit.WaitCriterion;
-import org.apache.geode.test.junit.categories.UnitTest;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Tests the default SocketCloser.
@@ -62,86 +61,49 @@ public class SocketCloserJUnitTest {
    */
   @Test
   public void testAsync() {
-    final CountDownLatch cdl = new CountDownLatch(1);
+    final CountDownLatch countDownLatch = new CountDownLatch(1);
     final AtomicInteger waitingToClose = new AtomicInteger(0);
-    Runnable r = new Runnable() {
-      @Override
-      public void run() {
-        try {
-          waitingToClose.incrementAndGet();
-          cdl.await();
-        } catch (InterruptedException e) {
-        }
-      }
-    };
 
     final int SOCKET_COUNT = 100;
-    final Socket[] aSockets = new Socket[SOCKET_COUNT];
-    for (int i = 0; i < SOCKET_COUNT; i++) {
-      aSockets[i] = createClosableSocket();
-    }
-    // Schedule a 100 sockets for async close.
-    // They should all be stuck on cdl.
-    for (int i = 0; i < SOCKET_COUNT; i++) {
-      this.socketCloser.asyncClose(aSockets[i], "A", r);
-    }
-    // Make sure the sockets have not been closed
-    for (int i = 0; i < SOCKET_COUNT; i++) {
-      assertEquals(false, aSockets[i].isClosed());
-    }
-    final Socket[] bSockets = new Socket[SOCKET_COUNT];
-    for (int i = 0; i < SOCKET_COUNT; i++) {
-      bSockets[i] = createClosableSocket();
-    }
+    final int REMOTE_CLIENT_COUNT = 200;
+
+    List<Socket> trackedSockets = new ArrayList<>();
     // Schedule a 100 sockets for async close.
-    // They should all be stuck on cdl.
-    for (int i = 0; i < SOCKET_COUNT; i++) {
-      this.socketCloser.asyncClose(bSockets[i], "B", r);
-    }
-    // Make sure the sockets have not been closed
-    for (int i = 0; i < SOCKET_COUNT; i++) {
-      assertEquals(false, bSockets[i].isClosed());
+    // They should all be stuck on countDownLatch.
+    for (int i = 0; i < REMOTE_CLIENT_COUNT; i++) {
+      Socket[] aSockets = new Socket[SOCKET_COUNT];
+
+      for (int j = 0; j < SOCKET_COUNT; j++) {
+        aSockets[j] = createClosableSocket();
+        trackedSockets.add(aSockets[j]);
+        this.socketCloser.asyncClose(aSockets[j], () -> {
+          try {
+            waitingToClose.incrementAndGet();
+            countDownLatch.await();
+          } catch (InterruptedException e) {
+          }
+        });
+      }
     }
+
     // close the socketCloser first to verify that the sockets
     // that have already been scheduled will be still be closed.
-    this.socketCloser.releaseResourcesForAddress("A");
     this.socketCloser.close();
-    // Each thread pool (one for A and one for B) has a max of 8 threads.
-    // So verify that this many are currently waiting on cdl.
-    {
-      final int maxThreads = this.socketCloser.getMaxThreads();
-      WaitCriterion wc = new WaitCriterion() {
-        public boolean done() {
-          return waitingToClose.get() == 2 * maxThreads;
-        }
-
-        public String description() {
-          return "expected " + 2 * maxThreads + " waiters but found only " + waitingToClose.get();
-        }
-      };
-      Wait.waitForCriterion(wc, 5000, 10, true);
-    }
-    // now count down the latch that allows the sockets to close
-    cdl.countDown();
+    countDownLatch.countDown();
     // now all the sockets should get closed; use a wait criteria
     // since a thread pool is doing to closes
-    {
-      WaitCriterion wc = new WaitCriterion() {
-        public boolean done() {
-          for (int i = 0; i < SOCKET_COUNT; i++) {
-            if (!aSockets[i].isClosed() || !bSockets[i].isClosed()) {
-              return false;
-            }
-          }
-          return true;
+    Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> {
+      boolean areAllClosed = true;
+      for (Iterator<Socket> iterator = trackedSockets.iterator(); iterator.hasNext();) {
+        Socket socket = iterator.next();
+        if (socket.isClosed()) {
+          iterator.remove();
+          continue;
         }
-
-        public String description() {
-          return "one or more sockets did not close";
-        }
-      };
-      Wait.waitForCriterion(wc, 5000, 10, true);
-    }
+        areAllClosed = false;
+      }
+      return areAllClosed;
+    });
   }
 
   /**
@@ -150,18 +112,11 @@ public class SocketCloserJUnitTest {
   @Test
   public void testClosedSocket() throws Exception {
     final AtomicBoolean runnableCalled = new AtomicBoolean();
-    Runnable r = new Runnable() {
-      @Override
-      public void run() {
-        runnableCalled.set(true);
-      }
-    };
 
     Socket s = createClosableSocket();
     s.close();
-    this.socketCloser.asyncClose(s, "A", r);
-    Wait.pause(10);
-    assertEquals(false, runnableCalled.get());
+    this.socketCloser.asyncClose(s, () -> runnableCalled.set(true));
+    Awaitility.await().atMost(1, TimeUnit.SECONDS).until(() -> !runnableCalled.get());
   }
 
   /**
@@ -170,25 +125,11 @@ public class SocketCloserJUnitTest {
   @Test
   public void testClosedSocketCloser() {
     final AtomicBoolean runnableCalled = new AtomicBoolean();
-    Runnable r = new Runnable() {
-      @Override
-      public void run() {
-        runnableCalled.set(true);
-      }
-    };
 
-    final Socket s = createClosableSocket();
+    final Socket closableSocket = createClosableSocket();
     this.socketCloser.close();
-    this.socketCloser.asyncClose(s, "A", r);
-    WaitCriterion wc = new WaitCriterion() {
-      public boolean done() {
-        return runnableCalled.get() && s.isClosed();
-      }
-
-      public String description() {
-        return "runnable was not called or socket was not closed";
-      }
-    };
-    Wait.waitForCriterion(wc, 5000, 10, true);
+    this.socketCloser.asyncClose(closableSocket, () -> runnableCalled.set(true));
+    Awaitility.await().atMost(5, TimeUnit.SECONDS)
+        .until(() -> runnableCalled.get() && closableSocket.isClosed());
   }
 }


[04/10] geode git commit: GEODE-3328 Properties to set Key/Trust Store Type for SSL configuration - add to docs This closes #703

Posted by ud...@apache.org.
GEODE-3328 Properties to set Key/Trust Store Type for SSL configuration - add to docs
This closes #703


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

Branch: refs/heads/feature/GEODE-3416
Commit: 8a6e309fcf083ee0420c2cbd0d080754ab3a0070
Parents: 08154dd
Author: Dave Barnes <db...@pivotal.io>
Authored: Wed Aug 9 15:22:22 2017 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Thu Aug 10 12:07:48 2017 -0700

----------------------------------------------------------------------
 geode-docs/managing/security/implementing_ssl.html.md.erb  | 3 +++
 geode-docs/reference/topics/gemfire_properties.html.md.erb | 7 +++++++
 2 files changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/8a6e309f/geode-docs/managing/security/implementing_ssl.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/managing/security/implementing_ssl.html.md.erb b/geode-docs/managing/security/implementing_ssl.html.md.erb
index 4894de7..5cf2517 100644
--- a/geode-docs/managing/security/implementing_ssl.html.md.erb
+++ b/geode-docs/managing/security/implementing_ssl.html.md.erb
@@ -101,6 +101,9 @@ any protocol that is enabled by default in the configured JSSE provider.</dd>
 <dt>**ssl-truststore, ssl-truststore-password**</dt>
 <dd>The path to the trust store and the trust store password, specified as strings</dd>
 
+<dt>**ssl-keystore-type, ssl-truststore-type**</dt>
+<dd>The types of the key store and trust store, specified as strings. The default for both is "JKS", indicating a Java key store or trust store.</dd>
+
 ### Example: secure communications throughout
 
 To implement secure SSL communications throughout an entire distributed system, each process should

http://git-wip-us.apache.org/repos/asf/geode/blob/8a6e309f/geode-docs/reference/topics/gemfire_properties.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/reference/topics/gemfire_properties.html.md.erb b/geode-docs/reference/topics/gemfire_properties.html.md.erb
index c5030b9..238803e 100644
--- a/geode-docs/reference/topics/gemfire_properties.html.md.erb
+++ b/geode-docs/reference/topics/gemfire_properties.html.md.erb
@@ -653,6 +653,13 @@ Any security-related (properties that begin with <code class="ph codeph">securit
 <td></td>
 </tr>
 
+<tr>
+<td>ssl-keystore-type, ssl-truststore-type</td>
+<td>Strings. Type of key store or trust store. "JKS" indicates Java. One common alternative is "pkcs12".</td>
+<td>S, L</td>
+<td>JKS</td>
+</tr>
+
 <tr class="even">
 <td>start-dev-rest-api</td>
 <td>If set to true, then the developer REST API service will be started when cache is created. REST service can be configured using <code class="ph codeph">http-service-port</code> and <code class="ph codeph">http-service-bind-address</code> properties.</td>


[10/10] geode git commit: GEODE-3416: Cleanup SocketCloser code to reduce the synchronization Old code would force single threaded behavior, new code will synchronize on the checking or changing or the closed flag

Posted by ud...@apache.org.
GEODE-3416: Cleanup SocketCloser code to reduce the synchronization
Old code would force single threaded behavior, new code will synchronize
on the checking or changing or the closed flag


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

Branch: refs/heads/feature/GEODE-3416
Commit: 5d567b1f3712127a44b3f1b556080d054423bd94
Parents: 31e4db0
Author: Udo Kohlmeyer <uk...@pivotal.io>
Authored: Fri Aug 11 09:45:45 2017 -0700
Committer: Udo Kohlmeyer <uk...@pivotal.io>
Committed: Fri Aug 11 09:46:47 2017 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/CacheClientProxy.java    |   4 +-
 .../apache/geode/internal/net/SocketCloser.java | 192 +++++++++++++------
 .../apache/geode/internal/tcp/Connection.java   |   4 +-
 .../internal/net/SocketCloserJUnitTest.java     |  10 +-
 4 files changed, 140 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/5d567b1f/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
index 98bfed9..34f232d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxy.java
@@ -949,7 +949,8 @@ public class CacheClientProxy implements ClientSession {
   private void closeSocket() {
     if (this._socketClosed.compareAndSet(false, true)) {
       // Close the socket
-      this._cacheClientNotifier.getSocketCloser().asyncClose(this._socket, null);
+      this._cacheClientNotifier.getSocketCloser().asyncClose(this._socket, this._remoteHostAddress,
+          null);
       getCacheClientNotifier().getAcceptorStats().decCurrentQueueConnections();
     }
   }
@@ -963,6 +964,7 @@ public class CacheClientProxy implements ClientSession {
     {
       String remoteHostAddress = this._remoteHostAddress;
       if (remoteHostAddress != null) {
+        this._cacheClientNotifier.getSocketCloser().releaseResourcesForAddress(remoteHostAddress);
         this._remoteHostAddress = null;
       }
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/5d567b1f/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java
index fbbe797..46d69a8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCloser.java
@@ -21,8 +21,10 @@ import org.apache.logging.log4j.Logger;
 
 import java.io.IOException;
 import java.net.Socket;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
@@ -49,27 +51,32 @@ public class SocketCloser {
    * minutes).
    */
   static final long ASYNC_CLOSE_POOL_KEEP_ALIVE_SECONDS =
-      Long.getLong("p2p.ASYNC_CLOSE_POOL_KEEP_ALIVE_SECONDS", 120);
+      Long.getLong("p2p.ASYNC_CLOSE_POOL_KEEP_ALIVE_SECONDS", 120).longValue();
   /**
    * Maximum number of threads that can be doing a socket close. Any close requests over this max
    * will queue up waiting for a thread.
    */
-  private static final int ASYNC_CLOSE_POOL_MAX_THREADS =
-      Integer.getInteger("p2p.ASYNC_CLOSE_POOL_MAX_THREADS", 8);
+  static final int ASYNC_CLOSE_POOL_MAX_THREADS =
+      Integer.getInteger("p2p.ASYNC_CLOSE_POOL_MAX_THREADS", 4).intValue();
   /**
    * How many milliseconds the synchronous requester waits for the async close to happen. Default is
    * 0. Prior releases waited 50ms.
    */
-  private static final long ASYNC_CLOSE_WAIT_MILLISECONDS =
-      Long.getLong("p2p.ASYNC_CLOSE_WAIT_MILLISECONDS", 0);
+  static final long ASYNC_CLOSE_WAIT_MILLISECONDS =
+      Long.getLong("p2p.ASYNC_CLOSE_WAIT_MILLISECONDS", 0).longValue();
 
 
+  /**
+   * map of thread pools of async close threads
+   */
+  private final ConcurrentHashMap<String, ExecutorService>
+      asyncCloseExecutors =
+      new ConcurrentHashMap<>();
   private final long asyncClosePoolKeepAliveSeconds;
   private final int asyncClosePoolMaxThreads;
   private final long asyncCloseWaitTime;
   private final TimeUnit asyncCloseWaitUnits;
-  private boolean closed;
-  private final ExecutorService socketCloserThreadPool;
+  private Boolean closed = Boolean.FALSE;
 
   public SocketCloser() {
     this(ASYNC_CLOSE_POOL_KEEP_ALIVE_SECONDS, ASYNC_CLOSE_POOL_MAX_THREADS,
@@ -82,30 +89,71 @@ public class SocketCloser {
   }
 
   public SocketCloser(long asyncClosePoolKeepAliveSeconds, int asyncClosePoolMaxThreads,
-      long asyncCloseWaitTime, TimeUnit asyncCloseWaitUnits) {
+                      long asyncCloseWaitTime, TimeUnit asyncCloseWaitUnits) {
     this.asyncClosePoolKeepAliveSeconds = asyncClosePoolKeepAliveSeconds;
     this.asyncClosePoolMaxThreads = asyncClosePoolMaxThreads;
     this.asyncCloseWaitTime = asyncCloseWaitTime;
     this.asyncCloseWaitUnits = asyncCloseWaitUnits;
-
-    final ThreadGroup threadGroup =
-        LoggingThreadGroup.createThreadGroup("Socket asyncClose", logger);
-    ThreadFactory threadFactory = command -> {
-      Thread thread = new Thread(threadGroup, command);
-      thread.setDaemon(true);
-      return thread;
-    };
-    socketCloserThreadPool = new ThreadPoolExecutor(this.asyncClosePoolMaxThreads,
-        this.asyncClosePoolMaxThreads, this.asyncClosePoolKeepAliveSeconds, TimeUnit.SECONDS,
-        new LinkedBlockingQueue<>(), threadFactory);
   }
 
   public int getMaxThreads() {
     return this.asyncClosePoolMaxThreads;
   }
 
-  private boolean isClosed() {
-    return this.closed;
+  private ExecutorService getAsyncThreadExecutor(String address) {
+    ExecutorService executorService = asyncCloseExecutors.get(address);
+    if (executorService == null) {
+      //To be used for pre-1.8 jdk releases.
+//      createThreadPool();
+
+      executorService = Executors.newWorkStealingPool(asyncClosePoolMaxThreads);
+
+      ExecutorService
+          previousThreadPoolExecutor =
+          asyncCloseExecutors.putIfAbsent(address, executorService);
+
+      if (previousThreadPoolExecutor != null) {
+        executorService.shutdownNow();
+        return previousThreadPoolExecutor;
+      }
+    }
+    return executorService;
+  }
+
+  /**
+   * @deprecated this method is to be used for pre 1.8 jdk.
+   */
+  @Deprecated
+  private void createThreadPool() {
+    ExecutorService executorService;
+    final ThreadGroup
+        threadGroup =
+        LoggingThreadGroup.createThreadGroup("Socket asyncClose", logger);
+    ThreadFactory threadFactory = new ThreadFactory() {
+      public Thread newThread(final Runnable command) {
+        Thread thread = new Thread(threadGroup, command);
+        thread.setDaemon(true);
+        return thread;
+      }
+    };
+
+    executorService =
+        new ThreadPoolExecutor(asyncClosePoolMaxThreads, asyncClosePoolMaxThreads,
+            asyncCloseWaitTime,
+            asyncCloseWaitUnits, new LinkedBlockingQueue<>(), threadFactory);
+  }
+
+  /**
+   * Call this method if you know all the resources in the closer for the given address are no
+   * longer needed. Currently a thread pool is kept for each address and if you know that an address
+   * no longer needs its pool then you should call this method.
+   */
+
+  public void releaseResourcesForAddress(String address) {
+    ExecutorService executorService = asyncCloseExecutors.remove(address);
+    if (executorService != null) {
+      executorService.shutdown();
+    }
   }
 
   /**
@@ -113,84 +161,104 @@ public class SocketCloser {
    * called then the asyncClose will be done synchronously.
    */
   public void close() {
-    if (!this.closed) {
-      this.closed = true;
-      socketCloserThreadPool.shutdown();
+    synchronized (closed) {
+      if (!this.closed) {
+        this.closed = true;
+      } else {
+        return;
+      }
+    }
+    for (ExecutorService executorService : asyncCloseExecutors.values()) {
+      executorService.shutdown();
+      asyncCloseExecutors.clear();
     }
   }
 
+  private Future asyncExecute(String address, Runnable runnableToExecute) {
+    ExecutorService asyncThreadExecutor = getAsyncThreadExecutor(address);
+    return asyncThreadExecutor.submit(runnableToExecute);
+  }
+
   /**
    * Closes the specified socket in a background thread. In some cases we see close hang (see bug
    * 33665). Depending on how the SocketCloser is configured (see ASYNC_CLOSE_WAIT_MILLISECONDS)
    * this method may block for a certain amount of time. If it is called after the SocketCloser is
    * closed then a normal synchronous close is done.
-   * 
-   * @param socket the socket to close
-   * @param runnableCode an optional Runnable with stuff to execute in the async thread
+   * @param sock the socket to close
+   * @param address identifies who the socket is connected to
+   * @param extra an optional Runnable with stuff to execute in the async thread
    */
-  public void asyncClose(final Socket socket, final Runnable runnableCode) {
-    if (socket == null || socket.isClosed()) {
+  public void asyncClose(final Socket sock, final String address, final Runnable extra) {
+    if (sock == null || sock.isClosed()) {
       return;
     }
-
     boolean doItInline = false;
     try {
-      if (isClosed()) {
-        // this SocketCloser has been closed so do a synchronous, inline, close
-        doItInline = true;
-      } else {
-        socketCloserThreadPool.execute(() -> {
-          if (runnableCode != null) {
-            runnableCode.run();
-          }
-          inlineClose(socket);
-        });
-        if (this.asyncCloseWaitTime != 0) {
-          try {
-            Future future = socketCloserThreadPool.submit(() -> {
-              if (runnableCode != null) {
-                runnableCode.run();
+      Future submittedTask = null;
+      synchronized (closed) {
+        if (closed) {
+          // this SocketCloser has been closed so do a synchronous, inline, close
+          doItInline = true;
+        } else {
+          submittedTask = asyncExecute(address, new Runnable() {
+            public void run() {
+              Thread.currentThread().setName("AsyncSocketCloser for " + address);
+              try {
+                if (extra != null) {
+                  extra.run();
+                }
+                inlineClose(sock);
+              } finally {
+                Thread.currentThread().setName("unused AsyncSocketCloser");
               }
-              inlineClose(socket);
-            });
-            future.get(this.asyncCloseWaitTime, this.asyncCloseWaitUnits);
-          } catch (InterruptedException | ExecutionException | TimeoutException e) {
-            // We want this code to wait at most 50ms for the close to happen.
-            // It is ok to ignore these exception and let the close continue
-            // in the background.
-          }
+            }
+          });
         }
       }
+      if (submittedTask != null) {
+        waitForFutureTaskWithTimeout(submittedTask);
+      }
     } catch (OutOfMemoryError ignore) {
       // If we can't start a thread to close the socket just do it inline.
       // See bug 50573.
       doItInline = true;
     }
     if (doItInline) {
-      if (runnableCode != null) {
-        runnableCode.run();
+      if (extra != null) {
+        extra.run();
       }
-      inlineClose(socket);
+      inlineClose(sock);
     }
   }
 
+  private void waitForFutureTaskWithTimeout(Future submittedTask) {
+    if (this.asyncCloseWaitTime != 0) {
+      try {
+        submittedTask.get(this.asyncCloseWaitTime, this.asyncCloseWaitUnits);
+      } catch (InterruptedException | ExecutionException | TimeoutException e) {
+        // We want this code to wait at most 50ms for the close to happen.
+        // It is ok to ignore these exception and let the close continue
+        // in the background.
+      }
+    }
+  }
 
   /**
    * Closes the specified socket
-   * 
-   * @param socket the socket to close
+   * @param sock the socket to close
    */
-  private void inlineClose(final Socket socket) {
+
+  private static void inlineClose(final Socket sock) {
     // the next two statements are a mad attempt to fix bug
     // 36041 - segv in jrockit in pthread signaling code. This
     // seems to alleviate the problem.
     try {
-      socket.shutdownInput();
-      socket.shutdownOutput();
+      sock.shutdownInput();
+      sock.shutdownOutput();
     } catch (Exception e) {
     }
     try {
-      socket.close();
+      sock.close();
     } catch (IOException ignore) {
     } catch (VirtualMachineError err) {
       SystemFailure.initiateFailure(err);

http://git-wip-us.apache.org/repos/asf/geode/blob/5d567b1f/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java b/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
index 954a33c..0ecb3bf 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/tcp/Connection.java
@@ -562,7 +562,7 @@ public class Connection implements Runnable {
       } catch (IOException io) {
         logger.fatal(LocalizedMessage
             .create(LocalizedStrings.Connection_UNABLE_TO_GET_P2P_CONNECTION_STREAMS), io);
-        t.getSocketCloser().asyncClose(socket, null);
+        t.getSocketCloser().asyncClose(socket, this.remoteAddr.toString(), null);
         throw io;
       }
     }
@@ -847,7 +847,7 @@ public class Connection implements Runnable {
         Socket s = this.socket;
         if (s != null && !s.isClosed()) {
           prepareForAsyncClose();
-          this.owner.getSocketCloser().asyncClose(s, null);
+          this.owner.getSocketCloser().asyncClose(s, String.valueOf(this.remoteAddr), null);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/5d567b1f/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java
index b6dbfe2..90315ce 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/net/SocketCloserJUnitTest.java
@@ -72,11 +72,11 @@ public class SocketCloserJUnitTest {
     // They should all be stuck on countDownLatch.
     for (int i = 0; i < REMOTE_CLIENT_COUNT; i++) {
       Socket[] aSockets = new Socket[SOCKET_COUNT];
-
+      String address = i + "";
       for (int j = 0; j < SOCKET_COUNT; j++) {
         aSockets[j] = createClosableSocket();
         trackedSockets.add(aSockets[j]);
-        this.socketCloser.asyncClose(aSockets[j], () -> {
+        this.socketCloser.asyncClose(aSockets[j], address, () -> {
           try {
             waitingToClose.incrementAndGet();
             countDownLatch.await();
@@ -94,7 +94,7 @@ public class SocketCloserJUnitTest {
     // since a thread pool is doing to closes
     Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> {
       boolean areAllClosed = true;
-      for (Iterator<Socket> iterator = trackedSockets.iterator(); iterator.hasNext();) {
+      for (Iterator<Socket> iterator = trackedSockets.iterator(); iterator.hasNext(); ) {
         Socket socket = iterator.next();
         if (socket.isClosed()) {
           iterator.remove();
@@ -115,7 +115,7 @@ public class SocketCloserJUnitTest {
 
     Socket s = createClosableSocket();
     s.close();
-    this.socketCloser.asyncClose(s, () -> runnableCalled.set(true));
+    this.socketCloser.asyncClose(s, "A", () -> runnableCalled.set(true));
     Awaitility.await().atMost(1, TimeUnit.SECONDS).until(() -> !runnableCalled.get());
   }
 
@@ -128,7 +128,7 @@ public class SocketCloserJUnitTest {
 
     final Socket closableSocket = createClosableSocket();
     this.socketCloser.close();
-    this.socketCloser.asyncClose(closableSocket, () -> runnableCalled.set(true));
+    this.socketCloser.asyncClose(closableSocket, "A", () -> runnableCalled.set(true));
     Awaitility.await().atMost(5, TimeUnit.SECONDS)
         .until(() -> runnableCalled.get() && closableSocket.isClosed());
   }


[07/10] geode git commit: GEODE-3328: refactor ConnectCommand

Posted by ud...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartMemberUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartMemberUtils.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartMemberUtils.java
index fd95387..3e1053a 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartMemberUtils.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartMemberUtils.java
@@ -83,16 +83,17 @@ public class StartMemberUtils {
   }
 
   static void addGemFirePropertyFile(final List<String> commandLine,
-      final String gemfirePropertiesPathname) {
-    if (org.apache.geode.internal.lang.StringUtils.isNotBlank(gemfirePropertiesPathname)) {
-      commandLine.add("-DgemfirePropertyFile=" + gemfirePropertiesPathname);
+      final File gemfirePropertiesFile) {
+    if (gemfirePropertiesFile != null) {
+      commandLine.add("-DgemfirePropertyFile=" + gemfirePropertiesFile.getAbsolutePath());
     }
   }
 
   static void addGemFireSecurityPropertyFile(final List<String> commandLine,
-      final String gemfireSecurityPropertiesPathname) {
-    if (org.apache.geode.internal.lang.StringUtils.isNotBlank(gemfireSecurityPropertiesPathname)) {
-      commandLine.add("-DgemfireSecurityPropertyFile=" + gemfireSecurityPropertiesPathname);
+      final File gemfireSecurityPropertiesFile) {
+    if (gemfireSecurityPropertiesFile != null) {
+      commandLine
+          .add("-DgemfireSecurityPropertyFile=" + gemfireSecurityPropertiesFile.getAbsolutePath());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
index 432a065..9b743c8 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
@@ -142,14 +142,14 @@ public class StartServerCommand implements GfshCommand {
           help = CliStrings.START_SERVER__MESSAGE__TIME__TO__LIVE__HELP) final Integer messageTimeToLive,
       @CliOption(key = CliStrings.START_SERVER__OFF_HEAP_MEMORY_SIZE,
           help = CliStrings.START_SERVER__OFF_HEAP_MEMORY_SIZE__HELP) final String offHeapMemorySize,
-      @CliOption(key = CliStrings.START_SERVER__PROPERTIES, optionContext = ConverterHint.FILE_PATH,
-          help = CliStrings.START_SERVER__PROPERTIES__HELP) String gemfirePropertiesPathname,
+      @CliOption(key = CliStrings.START_SERVER__PROPERTIES, optionContext = ConverterHint.FILE,
+          help = CliStrings.START_SERVER__PROPERTIES__HELP) File gemfirePropertiesFile,
       @CliOption(key = CliStrings.START_SERVER__REBALANCE, unspecifiedDefaultValue = "false",
           specifiedDefaultValue = "true",
           help = CliStrings.START_SERVER__REBALANCE__HELP) final Boolean rebalance,
       @CliOption(key = CliStrings.START_SERVER__SECURITY_PROPERTIES,
-          optionContext = ConverterHint.FILE_PATH,
-          help = CliStrings.START_SERVER__SECURITY_PROPERTIES__HELP) String gemfireSecurityPropertiesPathname,
+          optionContext = ConverterHint.FILE,
+          help = CliStrings.START_SERVER__SECURITY_PROPERTIES__HELP) File gemfireSecurityPropertiesFile,
       @CliOption(key = CliStrings.START_SERVER__SERVER_BIND_ADDRESS,
           unspecifiedDefaultValue = CacheServer.DEFAULT_BIND_ADDRESS,
           help = CliStrings.START_SERVER__SERVER_BIND_ADDRESS__HELP) final String serverBindAddress,
@@ -206,23 +206,16 @@ public class StartServerCommand implements GfshCommand {
             CliStrings.format(CliStrings.CACHE_XML_NOT_FOUND_MESSAGE, cacheXmlPathname));
       }
 
-      gemfirePropertiesPathname = CliUtil.resolvePathname(gemfirePropertiesPathname);
-
-      if (StringUtils.isNotBlank(gemfirePropertiesPathname)
-          && !IOUtils.isExistingPathname(gemfirePropertiesPathname)) {
+      if (gemfirePropertiesFile != null && !gemfirePropertiesFile.exists()) {
         return ResultBuilder.createUserErrorResult(
             CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, StringUtils.EMPTY,
-                gemfirePropertiesPathname));
+                gemfirePropertiesFile.getAbsolutePath()));
       }
 
-      gemfireSecurityPropertiesPathname =
-          CliUtil.resolvePathname(gemfireSecurityPropertiesPathname);
-
-      if (StringUtils.isNotBlank(gemfireSecurityPropertiesPathname)
-          && !IOUtils.isExistingPathname(gemfireSecurityPropertiesPathname)) {
+      if (gemfireSecurityPropertiesFile != null && !gemfireSecurityPropertiesFile.exists()) {
         return ResultBuilder.createUserErrorResult(
             CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ",
-                gemfireSecurityPropertiesPathname));
+                gemfireSecurityPropertiesFile.getAbsolutePath()));
       }
 
       File serverPidFile = new File(workingDirectory, ProcessType.SERVER.getPidFileName());
@@ -309,9 +302,8 @@ public class StartServerCommand implements GfshCommand {
       ServerLauncher serverLauncher = serverLauncherBuilder.build();
 
       String[] serverCommandLine = createStartServerCommandLine(serverLauncher,
-          gemfirePropertiesPathname, gemfireSecurityPropertiesPathname, gemfireProperties,
-          classpath, includeSystemClasspath, jvmArgsOpts, disableExitWhenOutOfMemory, initialHeap,
-          maxHeap);
+          gemfirePropertiesFile, gemfireSecurityPropertiesFile, gemfireProperties, classpath,
+          includeSystemClasspath, jvmArgsOpts, disableExitWhenOutOfMemory, initialHeap, maxHeap);
 
       if (getGfsh().getDebug()) {
         getGfsh().logInfo(StringUtils.join(serverCommandLine, StringUtils.SPACE), null);
@@ -428,7 +420,7 @@ public class StartServerCommand implements GfshCommand {
   }
 
   String[] createStartServerCommandLine(final ServerLauncher launcher,
-      final String gemfirePropertiesPathname, final String gemfireSecurityPropertiesPathname,
+      final File gemfirePropertiesFile, final File gemfireSecurityPropertiesFile,
       final Properties gemfireProperties, final String userClasspath,
       final Boolean includeSystemClasspath, final String[] jvmArgsOpts,
       final Boolean disableExitWhenOutOfMemory, final String initialHeap, final String maxHeap)
@@ -441,8 +433,8 @@ public class StartServerCommand implements GfshCommand {
     commandLine.add(getServerClasspath(Boolean.TRUE.equals(includeSystemClasspath), userClasspath));
 
     StartMemberUtils.addCurrentLocators(this, commandLine, gemfireProperties);
-    StartMemberUtils.addGemFirePropertyFile(commandLine, gemfirePropertiesPathname);
-    StartMemberUtils.addGemFireSecurityPropertyFile(commandLine, gemfireSecurityPropertiesPathname);
+    StartMemberUtils.addGemFirePropertyFile(commandLine, gemfirePropertiesFile);
+    StartMemberUtils.addGemFireSecurityPropertyFile(commandLine, gemfireSecurityPropertiesFile);
     StartMemberUtils.addGemFireSystemProperties(commandLine, gemfireProperties);
     StartMemberUtils.addJvmArgumentsAndOptions(commandLine, jvmArgsOpts);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UserInputProperty.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UserInputProperty.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UserInputProperty.java
new file mode 100644
index 0000000..5169dab
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UserInputProperty.java
@@ -0,0 +1,120 @@
+/*
+ * 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.cli.commands;
+
+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 org.apache.commons.lang.StringUtils;
+
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.internal.security.SecurableCommunicationChannel;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+import org.apache.geode.management.internal.security.ResourceConstants;
+
+public class UserInputProperty {
+  public static UserInputProperty USERNAME =
+      new UserInputProperty(ResourceConstants.USER_NAME, "user", false);
+  public static UserInputProperty PASSWORD =
+      new UserInputProperty(ResourceConstants.PASSWORD, "password", "", true);
+  public static UserInputProperty KEYSTORE =
+      new UserInputProperty(SSL_KEYSTORE, "key-store", "", false);
+  public static UserInputProperty KEYSTORE_PASSWORD =
+      new UserInputProperty(SSL_KEYSTORE_PASSWORD, "key-store-password", "", true);
+  public static UserInputProperty KEYSTORE_TYPE =
+      new UserInputProperty(SSL_KEYSTORE_TYPE, "key-store-type", "JKS", false);
+  public static UserInputProperty TRUSTSTORE =
+      new UserInputProperty(SSL_TRUSTSTORE, "trust-store", "", false);
+  public static UserInputProperty TRUSTSTORE_PASSWORD =
+      new UserInputProperty(SSL_TRUSTSTORE_PASSWORD, "trust-store-password", "", true);
+  public static UserInputProperty TRUSTSTORE_TYPE =
+      new UserInputProperty(SSL_TRUSTSTORE_TYPE, "trust-store-type", "JKS", false);
+  public static UserInputProperty CIPHERS = new UserInputProperty(SSL_CIPHERS, "ssl-ciphers",
+      DistributionConfig.DEFAULT_SSL_CIPHERS, false);
+  public static UserInputProperty PROTOCOL = new UserInputProperty(SSL_PROTOCOLS, "ssl-protocols",
+      DistributionConfig.DEFAULT_SSL_PROTOCOLS, false);
+  public static UserInputProperty COMPONENT = new UserInputProperty(SSL_ENABLED_COMPONENTS,
+      "ssl-enabled-components", SecurableCommunicationChannel.ALL.getConstant(), false);
+
+  private final String key;
+  private final String prompt;
+  private final boolean isMasked;
+  private final String defaultValue;
+
+  // use this if this property does not allow an empty string and has no default value
+  UserInputProperty(String key, String prompt, boolean isMasked) {
+    this(key, prompt, null, isMasked);
+  }
+
+  // if you allow an empty string for this property, supply a default value of ""
+  UserInputProperty(String key, String prompt, String defaultValue, boolean isMasked) {
+    this.key = key;
+    this.prompt = prompt;
+    this.defaultValue = defaultValue;
+    this.isMasked = isMasked;
+  }
+
+  public String promptForAcceptableValue(Gfsh gfsh) {
+    if (gfsh.isQuietMode() || gfsh.isHeadlessMode()) {
+      return defaultValue == null ? "" : defaultValue;
+    }
+
+    String value = promptForUserInput(gfsh);
+
+    if (value.length() > 0) {
+      return value;
+    }
+
+    // when user input an empty string and a default value is supplied, return the default value
+    if (value.length() == 0 && defaultValue != null) {
+      return defaultValue;
+    }
+
+    // otherwise prompt till we get a non-empty value, only when this property has no default value
+    while (value.length() == 0) {
+      value = promptForUserInput(gfsh);
+    }
+    return value;
+  }
+
+  private String promptForUserInput(Gfsh gfsh) {
+    String promptText = (StringUtils.isBlank(defaultValue)) ? prompt + ": "
+        : prompt + "(default: " + defaultValue + ")" + ": ";
+    String value;
+
+    if (isMasked) {
+      value = gfsh.readPassword(promptText);
+    } else {
+      value = gfsh.readText(promptText);
+    }
+    // when gfsh is mocked or quiet mode, the above would return null
+    if (value == null) {
+      value = "";
+    }
+    return value;
+  }
+
+  public String getKey() {
+    return key;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index 58c8ef7..7562005 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -544,9 +544,9 @@ public class CliStrings {
       "Network address of the Locator in the form: host[port].";
   public static final String CONNECT__URL = "url";
   public static final String CONNECT__DEFAULT_BASE_URL =
-      "http://localhost:" + DistributionConfig.DEFAULT_HTTP_SERVICE_PORT + "/gemfire/v1";
+      "http://localhost:" + DistributionConfig.DEFAULT_HTTP_SERVICE_PORT + "/geode-mgmt/v1";
   public static final String CONNECT__DEFAULT_SSL_BASE_URL =
-      "https://localhost:" + DistributionConfig.DEFAULT_HTTP_SERVICE_PORT + "/gemfire/v1";
+      "https://localhost:" + DistributionConfig.DEFAULT_HTTP_SERVICE_PORT + "/geode-mgmt/v1";
   public static final String CONNECT__URL__HELP =
       "Indicates the base URL to the Manager's HTTP service.  For example: 'http://<host>:<port>/gemfire/v1' Default is '"
           + CONNECT__DEFAULT_BASE_URL + "'";

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
index a69ce36..503c252 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
@@ -475,14 +475,14 @@ public class Gfsh extends JLineShell {
     return signalHandler;
   }
 
-  public String readPassword(String textToPrompt) throws IOException {
+  public String readPassword(String textToPrompt) {
     if (isHeadlessMode && isQuietMode())
       return null;
 
     return readWithMask(textToPrompt, '*');
   }
 
-  public String readText(String textToPrompt) throws IOException {
+  public String readText(String textToPrompt) {
     if (isHeadlessMode && isQuietMode())
       return null;
 
@@ -618,12 +618,20 @@ public class Gfsh extends JLineShell {
     return success;
   }
 
-  public String interact(String textToPrompt) throws IOException {
-    return reader.readLine(textToPrompt);
+  public String interact(String textToPrompt) {
+    try {
+      return reader.readLine(textToPrompt);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
   }
 
-  public String readWithMask(String textToPrompt, Character mask) throws IOException {
-    return reader.readLine(textToPrompt, mask);
+  public String readWithMask(String textToPrompt, Character mask) {
+    try {
+      return reader.readLine(textToPrompt, mask);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/JmxOperationInvoker.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/JmxOperationInvoker.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/JmxOperationInvoker.java
index 7ae7c3b..5066496 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/JmxOperationInvoker.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/JmxOperationInvoker.java
@@ -17,35 +17,17 @@ package org.apache.geode.management.internal.cli.shell;
 import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_PREFIX;
 import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_PREFIX;
 
-import org.apache.commons.lang.StringUtils;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.internal.util.ArrayUtils;
-import org.apache.geode.internal.util.IOUtils;
-import org.apache.geode.management.DistributedSystemMXBean;
-import org.apache.geode.management.MemberMXBean;
-import org.apache.geode.management.internal.MBeanJMXAdapter;
-import org.apache.geode.management.internal.ManagementConstants;
-import org.apache.geode.management.internal.cli.CliUtil;
-import org.apache.geode.management.internal.cli.CommandRequest;
-import org.apache.geode.management.internal.cli.LogWrapper;
-import org.apache.geode.management.internal.cli.commands.ShellCommands;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-
-import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 import java.net.MalformedURLException;
-import java.net.URL;
 import java.text.MessageFormat;
 import java.util.Arrays;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.atomic.AtomicBoolean;
+
 import javax.management.AttributeNotFoundException;
 import javax.management.InstanceNotFoundException;
 import javax.management.JMX;
@@ -63,6 +45,18 @@ import javax.management.remote.JMXConnectorFactory;
 import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.internal.admin.SSLConfig;
+import org.apache.geode.internal.net.SSLConfigurationFactory;
+import org.apache.geode.internal.security.SecurableCommunicationChannel;
+import org.apache.geode.internal.util.ArrayUtils;
+import org.apache.geode.management.DistributedSystemMXBean;
+import org.apache.geode.management.MemberMXBean;
+import org.apache.geode.management.internal.MBeanJMXAdapter;
+import org.apache.geode.management.internal.ManagementConstants;
+import org.apache.geode.management.internal.cli.CommandRequest;
+import org.apache.geode.management.internal.cli.LogWrapper;
+
 /**
  * OperationInvoker JMX Implementation
  * 
@@ -99,9 +93,8 @@ public class JmxOperationInvoker implements OperationInvoker {
 
   private int clusterId = CLUSTER_ID_WHEN_NOT_CONNECTED;
 
-  public JmxOperationInvoker(final String host, final int port, final String userName,
-      final String password, final Map<String, String> sslConfigProps,
-      String gfSecurityPropertiesPath) throws Exception {
+  public JmxOperationInvoker(final String host, final int port, Properties gfProperties)
+      throws Exception {
     final Set<String> propsToClear = new TreeSet<String>();
     try {
       this.managerHost = host;
@@ -112,36 +105,41 @@ public class JmxOperationInvoker implements OperationInvoker {
       final Map<String, Object> env = new HashMap<String, Object>();
 
       env.put(JMXConnectionListener.CHECK_PERIOD_PROP, JMXConnectionListener.CHECK_PERIOD);
+      env.put(JMXConnector.CREDENTIALS, gfProperties);
 
-      if (userName != null && userName.length() > 0) {
-        env.put(JMXConnector.CREDENTIALS, new String[] {userName, password});
-      }
-      Set<Entry<String, String>> entrySet = sslConfigProps.entrySet();
-      for (Iterator<Entry<String, String>> it = entrySet.iterator(); it.hasNext();) {
-        Entry<String, String> entry = it.next();
-        String key = entry.getKey();
-        String value = entry.getValue();
-        key = checkforSystemPropertyPrefix(key);
-        if ((key.equals(Gfsh.SSL_ENABLED_CIPHERS) || key.equals(Gfsh.SSL_ENABLED_PROTOCOLS))
-            && "any".equals(value)) {
-          continue;
-        }
-        System.setProperty(key, value);
-        propsToClear.add(key);
-      }
+      SSLConfig sslConfig = SSLConfigurationFactory.getSSLConfigForComponent(gfProperties,
+          SecurableCommunicationChannel.JMX);
 
-      if (!sslConfigProps.isEmpty()) {
-        if (System.getProperty(Gfsh.SSL_KEYSTORE) != null
-            || System.getProperty(Gfsh.SSL_TRUSTSTORE) != null) {
-          // use ssl to connect
-          env.put("com.sun.jndi.rmi.factory.socket", new SslRMIClientSocketFactory());
+      if (sslConfig.isEnabled()) {
+        if (sslConfig.getKeystore() != null) {
+          System.setProperty(SSLConfigurationFactory.JAVAX_KEYSTORE, sslConfig.getKeystore());
+          propsToClear.add(SSLConfigurationFactory.JAVAX_KEYSTORE);
         }
-      }
-
-      // Check for JMX Credentials if empty put properties instance directly so that
-      // jmx management interceptor can read it for custom security properties
-      if (!env.containsKey(JMXConnector.CREDENTIALS)) {
-        env.put(JMXConnector.CREDENTIALS, readProperties(gfSecurityPropertiesPath));
+        if (sslConfig.getKeystorePassword() != null) {
+          System.setProperty(SSLConfigurationFactory.JAVAX_KEYSTORE_PASSWORD,
+              sslConfig.getKeystorePassword());
+          propsToClear.add(SSLConfigurationFactory.JAVAX_KEYSTORE_PASSWORD);
+        }
+        if (sslConfig.getKeystoreType() != null) {
+          System.setProperty(SSLConfigurationFactory.JAVAX_KEYSTORE_TYPE,
+              sslConfig.getKeystoreType());
+          propsToClear.add(SSLConfigurationFactory.JAVAX_KEYSTORE_TYPE);
+        }
+        if (sslConfig.getTruststore() != null) {
+          System.setProperty(SSLConfigurationFactory.JAVAX_TRUSTSTORE, sslConfig.getTruststore());
+          propsToClear.add(SSLConfigurationFactory.JAVAX_TRUSTSTORE);
+        }
+        if (sslConfig.getTruststorePassword() != null) {
+          System.setProperty(SSLConfigurationFactory.JAVAX_TRUSTSTORE_PASSWORD,
+              sslConfig.getTruststorePassword());
+          propsToClear.add(SSLConfigurationFactory.JAVAX_TRUSTSTORE_PASSWORD);
+        }
+        if (sslConfig.getTruststoreType() != null) {
+          System.setProperty(SSLConfigurationFactory.JAVAX_TRUSTSTORE_TYPE,
+              sslConfig.getTruststoreType());
+          propsToClear.add(SSLConfigurationFactory.JAVAX_TRUSTSTORE_TYPE);
+        }
+        env.put("com.sun.jndi.rmi.factory.socket", new SslRMIClientSocketFactory());
       }
 
       this.url = new JMXServiceURL(MessageFormat.format(JMX_URL_FORMAT,
@@ -184,57 +182,6 @@ public class JmxOperationInvoker implements OperationInvoker {
     }
   }
 
-  // Copied from ShellCommands.java
-  private Properties readProperties(String gfSecurityPropertiesPath) throws MalformedURLException {
-    Gfsh gfshInstance = Gfsh.getCurrentInstance();
-    // reference to hold resolved gfSecurityPropertiesPath
-    String gfSecurityPropertiesPathToUse = CliUtil.resolvePathname(gfSecurityPropertiesPath);
-    URL gfSecurityPropertiesUrl = null;
-
-    // Case 1: User has specified gfSecurity properties file
-    if (StringUtils.isNotBlank(gfSecurityPropertiesPathToUse)) {
-      // User specified gfSecurity properties doesn't exist
-      if (!IOUtils.isExistingPathname(gfSecurityPropertiesPathToUse)) {
-        gfshInstance
-            .printAsSevere(CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
-                "Security ", gfSecurityPropertiesPathToUse));
-      } else {
-        gfSecurityPropertiesUrl = new File(gfSecurityPropertiesPathToUse).toURI().toURL();
-      }
-    } else if (gfSecurityPropertiesPath == null) {
-      // Use default "gfsecurity.properties"
-      // in current dir, user's home or classpath
-      gfSecurityPropertiesUrl = ShellCommands.getFileUrl("gfsecurity.properties");
-    }
-    // if 'gfSecurityPropertiesPath' OR gfsecurity.properties has resolvable path
-    if (gfSecurityPropertiesUrl != null) {
-      gfshInstance.logToFile("Using security properties file : "
-          + CliUtil.decodeWithDefaultCharSet(gfSecurityPropertiesUrl.getPath()), null);
-      return loadPropertiesFromURL(gfSecurityPropertiesUrl);
-    }
-    return null;
-  }
-
-  static Properties loadPropertiesFromURL(URL gfSecurityPropertiesUrl) {
-    Properties props = new Properties();
-    if (gfSecurityPropertiesUrl != null) {
-      InputStream inputStream = null;
-      try {
-
-        inputStream = gfSecurityPropertiesUrl.openStream();
-        props.load(inputStream);
-      } catch (IOException io) {
-        throw new RuntimeException(
-            CliStrings.format(CliStrings.CONNECT__MSG__COULD_NOT_READ_CONFIG_FROM_0,
-                CliUtil.decodeWithDefaultCharSet(gfSecurityPropertiesUrl.getPath())),
-            io);
-      } finally {
-        IOUtils.close(inputStream);
-      }
-    }
-    return props;
-  }
-
   private String checkforSystemPropertyPrefix(String key) {
     String returnKey = key;
     if (key.startsWith("javax.")) {

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/GfshConsoleReader.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/GfshConsoleReader.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/GfshConsoleReader.java
index 9251d7e..a635c50 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/GfshConsoleReader.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/GfshConsoleReader.java
@@ -15,8 +15,6 @@
 
 package org.apache.geode.management.internal.cli.util;
 
-import java.io.IOException;
-
 import org.apache.geode.internal.GfeConsoleReaderFactory.GfeConsoleReader;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 
@@ -44,11 +42,7 @@ public class GfshConsoleReader extends GfeConsoleReader {
   public String readLine(String textToPrompt) {
     String lineRead = null;
     if (isSupported()) {
-      try {
-        lineRead = gfsh.interact(textToPrompt);
-      } catch (IOException e) {
-        lineRead = null;
-      }
+      lineRead = gfsh.interact(textToPrompt);
     }
     return lineRead;
   }
@@ -56,12 +50,8 @@ public class GfshConsoleReader extends GfeConsoleReader {
   public char[] readPassword(String textToPrompt) {
     char[] password = null;
     if (isSupported()) {
-      try {
-        String passwordString = gfsh.readWithMask(textToPrompt, '*');
-        password = passwordString.toCharArray();
-      } catch (IOException e) {
-        password = null;
-      }
+      String passwordString = gfsh.readWithMask(textToPrompt, '*');
+      password = passwordString.toCharArray();
     }
     return password;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/main/java/org/apache/geode/management/internal/configuration/utils/ClusterConfigurationStatusRetriever.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/utils/ClusterConfigurationStatusRetriever.java b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/utils/ClusterConfigurationStatusRetriever.java
index 3080809..86681b1 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/configuration/utils/ClusterConfigurationStatusRetriever.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/configuration/utils/ClusterConfigurationStatusRetriever.java
@@ -14,6 +14,11 @@
  */
 package org.apache.geode.management.internal.configuration.utils;
 
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Properties;
+import java.util.Set;
+
 import org.apache.geode.distributed.LocatorLauncher;
 import org.apache.geode.distributed.internal.tcpserver.TcpClient;
 import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
@@ -21,22 +26,17 @@ import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.configuration.messages.SharedConfigurationStatusRequest;
 import org.apache.geode.management.internal.configuration.messages.SharedConfigurationStatusResponse;
 
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.Set;
-
 public class ClusterConfigurationStatusRetriever {
   private static final int NUM_ATTEMPTS_FOR_SHARED_CONFIGURATION_STATUS = 3;
 
-  public static String fromLocator(String locatorHostName, int locatorPort)
+  public static String fromLocator(String locatorHostName, int locatorPort, Properties configProps)
       throws ClassNotFoundException, IOException {
     final StringBuilder buffer = new StringBuilder();
 
     try {
       final InetAddress networkAddress = InetAddress.getByName(locatorHostName);
 
-      TcpClient client = new TcpClient();
+      TcpClient client = new TcpClient(configProps);
       SharedConfigurationStatusResponse statusResponse =
           (SharedConfigurationStatusResponse) client.requestToServer(networkAddress, locatorPort,
               new SharedConfigurationStatusRequest(), 10000, true);
@@ -114,6 +114,7 @@ public class ClusterConfigurationStatusRetriever {
 
   public static String fromLocator(LocatorLauncher.LocatorState locatorState)
       throws ClassNotFoundException, IOException {
-    return fromLocator(locatorState.getHost(), Integer.parseInt(locatorState.getPort()));
+    return fromLocator(locatorState.getHost(), Integer.parseInt(locatorState.getPort()),
+        new Properties());
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/test/java/org/apache/geode/internal/net/SSLConfigurationFactoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/net/SSLConfigurationFactoryJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/net/SSLConfigurationFactoryJUnitTest.java
index 31c2469..47f0d2b 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/net/SSLConfigurationFactoryJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/net/SSLConfigurationFactoryJUnitTest.java
@@ -32,22 +32,24 @@ import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTOR
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTORE_PASSWORD;
 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.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertEquals;
 
+import java.util.Properties;
+
 import org.apache.commons.lang.StringUtils;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.DistributionConfigImpl;
-import org.apache.geode.internal.admin.SSLConfig;
-import org.apache.geode.internal.security.SecurableCommunicationChannel;
-import org.apache.geode.test.junit.categories.MembershipTest;
-import org.apache.geode.test.junit.categories.UnitTest;
 import org.junit.After;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 
-import java.util.Properties;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionConfigImpl;
+import org.apache.geode.internal.admin.SSLConfig;
+import org.apache.geode.internal.security.SecurableCommunicationChannel;
+import org.apache.geode.test.junit.categories.MembershipTest;
+import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category({UnitTest.class, MembershipTest.class})
 public class SSLConfigurationFactoryJUnitTest {
@@ -280,6 +282,27 @@ public class SSLConfigurationFactoryJUnitTest {
     assertEquals(true, sslConfig.isEnabled());
   }
 
+  @Test
+  public void setDistributionConfig() throws Exception {
+    Properties properties = new Properties();
+    properties.setProperty(SSL_ENABLED_COMPONENTS, "all");
+    properties.setProperty(SSL_KEYSTORE, "someKeyStore");
+    DistributionConfigImpl distributionConfig = new DistributionConfigImpl(properties);
+    SSLConfigurationFactory.setDistributionConfig(distributionConfig);
+
+    SSLConfig sslConfig =
+        SSLConfigurationFactory.getSSLConfigForComponent(SecurableCommunicationChannel.LOCATOR);
+    assertThat(sslConfig.isEnabled()).isTrue();
+    assertThat(sslConfig.getKeystore()).isEqualTo("someKeyStore");
+
+    properties.setProperty(SSL_ENABLED_COMPONENTS, SecurableCommunicationChannel.JMX.getConstant());
+    properties.setProperty(SSL_KEYSTORE, "someOtherKeyStore");
+    sslConfig = SSLConfigurationFactory.getSSLConfigForComponent(properties,
+        SecurableCommunicationChannel.LOCATOR);
+    assertThat(sslConfig.isEnabled()).isFalse();
+    assertThat(sslConfig.getKeystore()).isEqualTo("someOtherKeyStore");
+  }
+
   private void assertSSLConfig(final Properties properties, final SSLConfig sslConfig,
       final SecurableCommunicationChannel expectedSecurableComponent,
       final DistributionConfigImpl distributionConfig) {

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorSSLDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorSSLDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorSSLDUnitTest.java
deleted file mode 100644
index 866f4ef..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorSSLDUnitTest.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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;
-
-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_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.util.test.TestUtil.getResourcePath;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.OutputStream;
-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.RuleChain;
-import org.junit.rules.TemporaryFolder;
-
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.security.SecurableCommunicationChannels;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.rules.CleanupDUnitVMsRule;
-import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
-import org.apache.geode.test.dunit.rules.MemberVM;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
-
-@Category(DistributedTest.class)
-public class ConnectToLocatorSSLDUnitTest {
-  private TemporaryFolder folder = new SerializableTemporaryFolder();
-  private LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
-  private CleanupDUnitVMsRule cleanupDUnitVMsRule = new CleanupDUnitVMsRule();
-
-  @Rule
-  public RuleChain ruleChain =
-      RuleChain.outerRule(folder).around(cleanupDUnitVMsRule).around(lsRule);
-
-  private File jks = null;
-  protected File securityPropsFile = null;
-  private Properties securityProps;
-  protected MemberVM locator;
-
-  @Before
-  public void before() throws Exception {
-    jks = new File(getResourcePath(getClass(), "/ssl/trusted.keystore"));
-    securityPropsFile = folder.newFile("security.properties");
-    securityProps = new Properties();
-  }
-
-  protected void connect() throws Exception {
-    final int locatorPort = locator.getPort();
-    final String securityPropsFilePath = securityPropsFile.getCanonicalPath();
-
-    // when gfsh uses SSL, it leaves SSL state behind to contaminate other tests. So we pushed
-    // gfsh into a VM and uses a CleanupDUnitVM rule to clean it up after each test.
-    Host.getHost(0).getVM(1).invoke(() -> {
-      GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
-      gfshConnector.connectAndVerify(locatorPort, GfshShellConnectionRule.PortType.locator,
-          CliStrings.CONNECT__SECURITY_PROPERTIES, securityPropsFilePath);
-      gfshConnector.executeAndVerifyCommand("list members");
-      gfshConnector.close();
-    });
-
-  }
-
-  @Test
-  public void testConnectToLocator_withSSL() throws Exception {
-    securityProps.setProperty(SSL_ENABLED_COMPONENTS, SecurableCommunicationChannels.ALL);
-    securityProps.setProperty(SSL_KEYSTORE, jks.getCanonicalPath());
-    securityProps.setProperty(SSL_KEYSTORE_PASSWORD, "password");
-    securityProps.setProperty(SSL_TRUSTSTORE, jks.getCanonicalPath());
-    securityProps.setProperty(SSL_TRUSTSTORE_PASSWORD, "password");
-    securityProps.setProperty(SSL_PROTOCOLS, "TLSv1.2");
-    securityProps.setProperty(SSL_CIPHERS, "any");
-
-    // start up the locator
-    locator = lsRule.startLocatorVM(0, securityProps);
-    // saving the securityProps to a file
-    OutputStream out = new FileOutputStream(securityPropsFile);
-    securityProps.store(out, null);
-
-    connect();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorWithLegacySSLDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorWithLegacySSLDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorWithLegacySSLDUnitTest.java
deleted file mode 100644
index d7db489..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/ConnectToLocatorWithLegacySSLDUnitTest.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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;
-
-import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_ENABLED;
-import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_KEYSTORE;
-import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_KEYSTORE_PASSWORD;
-import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_KEYSTORE_TYPE;
-import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_TRUSTSTORE;
-import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_TRUSTSTORE_PASSWORD;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_ENABLED;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_KEYSTORE;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_KEYSTORE_PASSWORD;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_KEYSTORE_TYPE;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_TRUSTSTORE;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_TRUSTSTORE_PASSWORD;
-import static org.apache.geode.util.test.TestUtil.getResourcePath;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.OutputStream;
-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.RuleChain;
-import org.junit.rules.TemporaryFolder;
-
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.rules.CleanupDUnitVMsRule;
-import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
-import org.apache.geode.test.dunit.rules.MemberVM;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
-
-@Category(DistributedTest.class)
-public class ConnectToLocatorWithLegacySSLDUnitTest {
-  private TemporaryFolder folder = new SerializableTemporaryFolder();
-  private LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
-  private CleanupDUnitVMsRule cleanupDUnitVMsRule = new CleanupDUnitVMsRule();
-
-  @Rule
-  public RuleChain ruleChain =
-      RuleChain.outerRule(folder).around(cleanupDUnitVMsRule).around(lsRule);
-
-  private File jks = null;
-  protected File securityPropsFile = null;
-  private Properties securityProps;
-  protected MemberVM locator;
-
-  @Before
-  public void before() throws Exception {
-    jks = new File(getResourcePath(getClass(), "/ssl/trusted.keystore"));
-    securityPropsFile = folder.newFile("security.properties");
-    securityProps = new Properties();
-  }
-
-  protected void startUpLocatorAndConnect(Properties properties) throws Exception {
-    locator = lsRule.startLocatorVM(0, securityProps);
-    // saving the securityProps to a file
-    OutputStream out = new FileOutputStream(securityPropsFile);
-    securityProps.store(out, null);
-
-    final int locatorPort = locator.getPort();
-    final String securityPropsFilePath = securityPropsFile.getCanonicalPath();
-
-    // when gfsh uses SSL, it leaves SSL state behind to contaminate other tests. So we pushed
-    // gfsh into a VM and uses a CleanupDUnitVM rule to clean it up after each test.
-    Host.getHost(0).getVM(1).invoke(() -> {
-      GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
-      gfshConnector.connectAndVerify(locatorPort, GfshShellConnectionRule.PortType.locator,
-          CliStrings.CONNECT__SECURITY_PROPERTIES, securityPropsFilePath);
-      gfshConnector.executeAndVerifyCommand("list members");
-      gfshConnector.close();
-    });
-  }
-
-  @Test
-  public void testConnectToLocator_withLegacyClusterSSL() throws Exception {
-    securityProps.setProperty(CLUSTER_SSL_ENABLED, "true");
-    securityProps.setProperty(CLUSTER_SSL_KEYSTORE, jks.getCanonicalPath());
-    securityProps.setProperty(CLUSTER_SSL_KEYSTORE_PASSWORD, "password");
-    securityProps.setProperty(CLUSTER_SSL_KEYSTORE_TYPE, "JKS");
-    securityProps.setProperty(CLUSTER_SSL_TRUSTSTORE, jks.getCanonicalPath());
-    securityProps.setProperty(CLUSTER_SSL_TRUSTSTORE_PASSWORD, "password");
-
-    startUpLocatorAndConnect(securityProps);
-  }
-
-  @Test
-  public void testConnectToLocator_withLegacyJMXManagerSSL() throws Exception {
-    securityProps.setProperty(JMX_MANAGER_SSL_ENABLED, "true");
-    securityProps.setProperty(JMX_MANAGER_SSL_KEYSTORE, jks.getCanonicalPath());
-    securityProps.setProperty(JMX_MANAGER_SSL_KEYSTORE_PASSWORD, "password");
-    securityProps.setProperty(JMX_MANAGER_SSL_KEYSTORE_TYPE, "JKS");
-    securityProps.setProperty(JMX_MANAGER_SSL_TRUSTSTORE, jks.getCanonicalPath());
-    securityProps.setProperty(JMX_MANAGER_SSL_TRUSTSTORE_PASSWORD, "password");
-
-    startUpLocatorAndConnect(securityProps);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserConverterTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserConverterTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserConverterTest.java
index 3c42e0b..1ad22d4 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserConverterTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserConverterTest.java
@@ -17,7 +17,16 @@ package org.apache.geode.management.internal.cli;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.spy;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.springframework.shell.event.ParseResult;
 
 import org.apache.geode.management.internal.cli.converters.DiskStoreNameConverter;
 import org.apache.geode.management.internal.cli.converters.FilePathConverter;
@@ -25,15 +34,6 @@ import org.apache.geode.management.internal.cli.converters.FilePathStringConvert
 import org.apache.geode.management.internal.cli.converters.RegionPathConverter;
 import org.apache.geode.test.dunit.rules.GfshParserRule;
 import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.springframework.shell.event.ParseResult;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
 
 @Category(IntegrationTest.class)
 public class GfshParserConverterTest {
@@ -152,12 +152,12 @@ public class GfshParserConverterTest {
     doReturn(roots).when(spy).getRoots();
     doReturn(siblings).when(spy).getSiblings(any());
 
-    String command = "start server --properties-file=";
+    String command = "start server --cache-xml-file=";
     commandCandidate = parser.complete(command);
     assertThat(commandCandidate.size()).isEqualTo(2);
     assertThat(commandCandidate.getFirstCandidate()).isEqualTo(command + "/logs");
 
-    command = "start server --properties-file=sibling";
+    command = "start server --cache-xml-file=sibling";
     commandCandidate = parser.complete(command);
     assertThat(commandCandidate.size()).isEqualTo(2);
     assertThat(commandCandidate.getFirstCandidate()).isEqualTo(command + "1");
@@ -170,6 +170,7 @@ public class GfshParserConverterTest {
     assertThat(commandCandidate.getFirstCandidate()).isEqualTo(command + "1");
   }
 
+
   @Test
   public void testRegionPathConverter() throws Exception {
     RegionPathConverter spy = parser.spyConverter(RegionPathConverter.class);

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandTest.java
new file mode 100644
index 0000000..3a5c584
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandTest.java
@@ -0,0 +1,300 @@
+/*
+ * 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.cli.commands;
+
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_KEYSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_KEYSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_KEYSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE;
+import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE_PASSWORD;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+import org.apache.geode.management.internal.cli.shell.OperationInvoker;
+import org.apache.geode.test.dunit.rules.GfshParserRule;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class ConnectCommandTest {
+
+  @ClassRule
+  public static GfshParserRule gfshParserRule = new GfshParserRule();
+
+  private ConnectCommand connectCommand;
+
+  private Gfsh gfsh = mock(Gfsh.class);
+
+  private CommandResult result;
+  private Properties properties;
+  private ArgumentCaptor<File> fileCaptor;
+
+  @Before
+  public void before() throws Exception {
+    properties = new Properties();
+    when(gfsh.getOperationInvoker()).thenReturn(mock(OperationInvoker.class));
+    // using spy instead of mock because we want to call the real method when we do connect
+    connectCommand = spy(ConnectCommand.class);
+    when(connectCommand.getGfsh()).thenReturn(gfsh);
+    doReturn(properties).when(connectCommand).loadProperties(any());
+    CommandResult result = mock(CommandResult.class);
+    when(connectCommand.httpConnect(any(), any())).thenReturn(result);
+    when(connectCommand.jmxConnect(any(), anyBoolean(), any(), any(), anyBoolean()))
+        .thenReturn(result);
+    fileCaptor = ArgumentCaptor.forClass(File.class);
+  }
+
+  @Test
+  public void whenGfshIsAlreadyConnected() throws Exception {
+    when(gfsh.isConnectedAndReady()).thenReturn(true);
+    result = gfshParserRule.executeCommandWithInstance(connectCommand, "connect");
+    assertThat(result.nextLine()).contains("Already connected to");
+  }
+
+  @Test
+  public void promptForPasswordIfUsernameIsGiven() throws Exception {
+    doReturn(properties).when(connectCommand).resolveSslProperties(any(), anyBoolean(), any(),
+        any());
+    result = gfshParserRule.executeCommandWithInstance(connectCommand, "connect --user=user");
+    verify(gfsh).readPassword(CliStrings.CONNECT__PASSWORD + ": ");
+
+    assertThat(properties.getProperty("security-username")).isEqualTo("user");
+    assertThat(properties.getProperty("security-password")).isEqualTo("");
+  }
+
+  @Test
+  public void notPromptForPasswordIfUsernameIsGiven() throws Exception {
+    doReturn(properties).when(connectCommand).resolveSslProperties(any(), anyBoolean(), any(),
+        any());
+    result = gfshParserRule.executeCommandWithInstance(connectCommand,
+        "connect --user=user --password=pass");
+    verify(gfsh, times(0)).readPassword(CliStrings.CONNECT__PASSWORD + ": ");
+
+    assertThat(properties.getProperty("security-username")).isEqualTo("user");
+    assertThat(properties.getProperty("security-password")).isEqualTo("pass");
+  }
+
+  @Test
+  public void notPromptForPasswordIfuserNameisGivenInFile() throws Exception {
+    // username specified in property file won't promot for password
+    properties.setProperty("security-username", "user");
+    doReturn(properties).when(connectCommand).loadProperties(any(File.class));
+
+    result = gfshParserRule.executeCommandWithInstance(connectCommand, "connect");
+    verify(gfsh, times(0)).readPassword(CliStrings.CONNECT__PASSWORD + ": ");
+
+    assertThat(properties).doesNotContainKey("security-password");
+  }
+
+  @Test
+  public void plainConnectNotLoadFileNotPrompt() throws Exception {
+    result = gfshParserRule.executeCommandWithInstance(connectCommand, "connect");
+    // will not try to load from any file
+    verify(connectCommand).loadProperties(null, null);
+
+    // will not try to prompt
+    verify(gfsh, times(0)).readText(any());
+    verify(gfsh, times(0)).readPassword(any());
+  }
+
+  @Test
+  public void connectUseSsl() throws Exception {
+    result = gfshParserRule.executeCommandWithInstance(connectCommand, "connect --use-ssl");
+
+    // will not try to load from any file
+    verify(connectCommand).loadProperties(null, null);
+
+    // gfsh will prompt for the all the ssl properties
+    verify(gfsh).readText("key-store: ");
+    verify(gfsh).readPassword("key-store-password: ");
+    verify(gfsh).readText("key-store-type(default: JKS): ");
+    verify(gfsh).readText("trust-store: ");
+    verify(gfsh).readPassword("trust-store-password: ");
+    verify(gfsh).readText("trust-store-type(default: JKS): ");
+    verify(gfsh).readText("ssl-ciphers(default: any): ");
+    verify(gfsh).readText("ssl-protocols(default: any): ");
+
+    // verify the resulting properties has correct values
+    assertThat(properties).hasSize(9);
+    assertThat(properties.getProperty("ssl-keystore")).isEqualTo("");
+    assertThat(properties.getProperty("ssl-keystore-password")).isEqualTo("");
+    assertThat(properties.getProperty("ssl-keystore-type")).isEqualTo("JKS");
+    assertThat(properties.getProperty("ssl-truststore")).isEqualTo("");
+    assertThat(properties.getProperty("ssl-truststore-password")).isEqualTo("");
+    assertThat(properties.getProperty("ssl-truststore-type")).isEqualTo("JKS");
+    assertThat(properties.getProperty("ssl-ciphers")).isEqualTo("any");
+    assertThat(properties.getProperty("ssl-protocols")).isEqualTo("any");
+    assertThat(properties.getProperty("ssl-enabled-components")).isEqualTo("all");
+  }
+
+  @Test
+  public void securityFileContainsSSLPropsAndNoUseSSL() throws Exception {
+    properties.setProperty(SSL_KEYSTORE, "keystore");
+    result = gfshParserRule.executeCommandWithInstance(connectCommand,
+        "connect --security-properties-file=test");
+
+    // will try to load from this file
+    verify(connectCommand).loadProperties(any(), fileCaptor.capture());
+    assertThat(fileCaptor.getValue()).hasName("test");
+
+    // it will prompt for missing properties
+    verify(gfsh, times(6)).readText(any());
+    verify(gfsh, times(2)).readPassword(any());
+  }
+
+  @Test
+  public void securityFileContainsNoSSLPropsAndNoUseSSL() throws Exception {
+    result = gfshParserRule.executeCommandWithInstance(connectCommand,
+        "connect --security-properties-file=test");
+
+    // will try to load from this file
+    verify(connectCommand).loadProperties(any(), fileCaptor.capture());
+    assertThat(fileCaptor.getValue()).hasName("test");
+
+    // it will prompt for missing properties
+    verify(gfsh, times(0)).readText(any());
+    verify(gfsh, times(0)).readPassword(any());
+  }
+
+  @Test
+  public void connectUseLegacySecurityPropertiesFile() throws Exception {
+    properties.setProperty(JMX_MANAGER_SSL_KEYSTORE, "jmx-keystore");
+    result = gfshParserRule.executeCommandWithInstance(connectCommand,
+        "connect --security-properties-file=test --key-store=keystore --key-store-password=password");
+
+    // wil try to load from this file
+    verify(connectCommand).loadProperties(fileCaptor.capture());
+    assertThat(fileCaptor.getValue()).hasName("test");
+
+    // it will not prompt for missing properties
+    verify(gfsh, times(0)).readText(any());
+    verify(gfsh, times(0)).readPassword(any());
+
+    // the command option will be ignored
+    assertThat(properties).hasSize(1);
+    assertThat(properties.get(JMX_MANAGER_SSL_KEYSTORE)).isEqualTo("jmx-keystore");
+  }
+
+  @Test
+  public void connectUseSecurityPropertiesFile_promptForMissing() throws Exception {
+    properties.setProperty(SSL_KEYSTORE, "keystore");
+    properties.setProperty(SSL_KEYSTORE_PASSWORD, "password");
+    result = gfshParserRule.executeCommandWithInstance(connectCommand,
+        "connect --security-properties-file=test");
+
+    // since nothing is loaded, will prompt for all missing values
+    verify(gfsh, times(6)).readText(any());
+    verify(gfsh, times(1)).readPassword(any());
+  }
+
+  @Test
+  public void connectUseSecurityPropertiesFileAndOption_promptForMissing() throws Exception {
+    properties.setProperty(SSL_KEYSTORE, "keystore");
+    properties.setProperty(SSL_KEYSTORE_PASSWORD, "password");
+    result = gfshParserRule.executeCommandWithInstance(connectCommand,
+        "connect --security-properties-file=test --key-store=keystore2 --trust-store=truststore2");
+
+    // since nothing is loaded, will prompt for all missing values
+    verify(gfsh, times(5)).readText(any());
+    verify(gfsh, times(1)).readPassword(any());
+
+    assertThat(properties).hasSize(9);
+    assertThat(properties.getProperty("ssl-keystore")).isEqualTo("keystore2");
+    assertThat(properties.getProperty("ssl-keystore-password")).isEqualTo("password");
+    assertThat(properties.getProperty("ssl-keystore-type")).isEqualTo("JKS");
+    assertThat(properties.getProperty("ssl-truststore")).isEqualTo("truststore2");
+    assertThat(properties.getProperty("ssl-truststore-password")).isEqualTo("");
+    assertThat(properties.getProperty("ssl-truststore-type")).isEqualTo("JKS");
+    assertThat(properties.getProperty("ssl-ciphers")).isEqualTo("any");
+    assertThat(properties.getProperty("ssl-protocols")).isEqualTo("any");
+    assertThat(properties.getProperty("ssl-enabled-components")).isEqualTo("all");
+  }
+
+  @Test
+  public void containsLegacySSLConfigTest_ssl() throws Exception {
+    properties.setProperty(SSL_KEYSTORE, "keystore");
+    assertThat(connectCommand.containsLegacySSLConfig(properties)).isFalse();
+  }
+
+  @Test
+  public void containsLegacySSLConfigTest_cluster() throws Exception {
+    properties.setProperty(CLUSTER_SSL_KEYSTORE, "cluster-keystore");
+    assertThat(connectCommand.containsLegacySSLConfig(properties)).isTrue();
+  }
+
+  @Test
+  public void containsLegacySSLConfigTest_jmx() throws Exception {
+    properties.setProperty(JMX_MANAGER_SSL_KEYSTORE, "jmx-keystore");
+    assertThat(connectCommand.containsLegacySSLConfig(properties)).isTrue();
+  }
+
+  @Test
+  public void containsLegacySSLConfigTest_http() throws Exception {
+    properties.setProperty(HTTP_SERVICE_SSL_KEYSTORE, "http-keystore");
+    assertThat(connectCommand.containsLegacySSLConfig(properties)).isTrue();
+  }
+
+  @Test
+  public void loadPropertiesWithNull() throws Exception {
+    doCallRealMethod().when(connectCommand).loadProperties(any());
+    assertThat(connectCommand.loadProperties(null, null)).isEmpty();
+  }
+
+  @Test
+  public void isSslImpliedByOptions() throws Exception {
+    assertThat(connectCommand.isSslImpliedBySslOptions(null)).isFalse();
+
+    assertThat(connectCommand.isSslImpliedBySslOptions(null, null, null)).isFalse();
+
+    assertThat(connectCommand.isSslImpliedBySslOptions(null, "test")).isTrue();
+  }
+
+  @Test
+  public void resolveSslProperties() throws Exception {
+    // assume properties loaded from either file has an ssl property
+    properties.setProperty(SSL_KEYSTORE, "keystore");
+    properties = connectCommand.resolveSslProperties(gfsh, false, null, null);
+    assertThat(properties).hasSize(9);
+
+    properties.clear();
+
+    properties.setProperty(SSL_KEYSTORE, "keystore");
+    properties =
+        connectCommand.resolveSslProperties(gfsh, false, null, null, "keystore2", "password");
+    assertThat(properties).hasSize(9);
+    assertThat(properties.getProperty(SSL_KEYSTORE)).isEqualTo("keystore2");
+    assertThat(properties.getProperty(SSL_KEYSTORE_PASSWORD)).isEqualTo("password");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
index 67c3f5a..da60c7a 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
@@ -29,6 +29,24 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
+import java.io.File;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import org.jmock.Expectations;
+import org.jmock.Mockery;
+import org.jmock.lib.concurrent.Synchroniser;
+import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.distributed.DistributedMember;
@@ -41,23 +59,6 @@ import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.util.MemberNotFoundException;
 import org.apache.geode.test.junit.categories.UnitTest;
-import org.jmock.Expectations;
-import org.jmock.Mockery;
-import org.jmock.lib.concurrent.Synchroniser;
-import org.jmock.lib.legacy.ClassImposteriser;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.io.File;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
 
 /**
  * The GfshCommandJUnitTest class is a test suite of test cases testing the contract and
@@ -412,11 +413,7 @@ public class GfshCommandJUnitTest {
     assertTrue(commandLine.isEmpty());
     StartMemberUtils.addGemFirePropertyFile(commandLine, null);
     assertTrue(commandLine.isEmpty());
-    StartMemberUtils.addGemFirePropertyFile(commandLine, org.apache.commons.lang.StringUtils.EMPTY);
-    assertTrue(commandLine.isEmpty());
-    StartMemberUtils.addGemFirePropertyFile(commandLine, " ");
-    assertTrue(commandLine.isEmpty());
-    StartMemberUtils.addGemFirePropertyFile(commandLine, "/path/to/gemfire.properties");
+    StartMemberUtils.addGemFirePropertyFile(commandLine, new File("/path/to/gemfire.properties"));
     assertFalse(commandLine.isEmpty());
     assertTrue(commandLine.contains("-DgemfirePropertyFile=/path/to/gemfire.properties"));
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/UserInputPropertyTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/UserInputPropertyTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/UserInputPropertyTest.java
new file mode 100644
index 0000000..3527afd
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/UserInputPropertyTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cli.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class UserInputPropertyTest {
+
+  private UserInputProperty userInputProperty;
+  private Gfsh gfsh;
+
+  @Before
+  public void setUp() throws Exception {
+    gfsh = mock(Gfsh.class);
+  }
+
+  @Test
+  public void propertyWithNoDefaultVaueWillPromptTillAValueIsSupplied_interactive()
+      throws Exception {
+    userInputProperty = new UserInputProperty("key", "prompt", false);
+
+    when(gfsh.readText(any())).thenReturn("").thenReturn("").thenReturn("value");
+    String input = userInputProperty.promptForAcceptableValue(gfsh);
+
+    assertThat(input).isEqualTo("value");
+    verify(gfsh, times(0)).readPassword(any());
+    verify(gfsh, times(3)).readText(any());
+  }
+
+  @Test
+  public void propertyWithNoDefaultValue_quietMode() throws Exception {
+    when(gfsh.isQuietMode()).thenReturn(true);
+    userInputProperty = new UserInputProperty("key", "prompt", false);
+    String input = userInputProperty.promptForAcceptableValue(gfsh);
+    assertThat(input).isEqualTo("");
+    verify(gfsh, times(0)).readPassword(any());
+    verify(gfsh, times(0)).readText(any());
+  }
+
+  @Test
+  public void propertyWithDefaultValue_Interactive() throws Exception {
+    userInputProperty = new UserInputProperty("key", "prompt", "value", false);
+    String input = userInputProperty.promptForAcceptableValue(gfsh);
+    assertThat(input).isEqualTo("value");
+    verify(gfsh, times(0)).readPassword(any());
+    verify(gfsh).readText(any());
+  }
+
+  @Test
+  public void propertyWithEmptyDefaultValue_Interactive() throws Exception {
+    userInputProperty = new UserInputProperty("key", "prompt", "", false);
+    String input = userInputProperty.promptForAcceptableValue(gfsh);
+    assertThat(input).isEqualTo("");
+    verify(gfsh, times(0)).readPassword(any());
+    verify(gfsh).readText(any());
+  }
+
+  @Test
+  public void propertyWithDefaultValue_Interactive_masked() throws Exception {
+    userInputProperty = new UserInputProperty("key", "prompt", "value", true);
+    String input = userInputProperty.promptForAcceptableValue(gfsh);
+    assertThat(input).isEqualTo("value");
+    verify(gfsh).readPassword(any());
+    verify(gfsh, times(0)).readText(any());
+  }
+
+  @Test
+  public void propertyWithDefaultValue_Quiet() throws Exception {
+    when(gfsh.isQuietMode()).thenReturn(true);
+    userInputProperty = new UserInputProperty("key", "prompt", "value", false);
+    String input = userInputProperty.promptForAcceptableValue(gfsh);
+    assertThat(input).isEqualTo("value");
+    verify(gfsh, times(0)).readPassword(any());
+    verify(gfsh, times(0)).readText(any());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-cq/src/test/java/org/apache/geode/management/CacheServerManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/management/CacheServerManagementDUnitTest.java b/geode-cq/src/test/java/org/apache/geode/management/CacheServerManagementDUnitTest.java
index 2cd69dd..a8d3997 100644
--- a/geode-cq/src/test/java/org/apache/geode/management/CacheServerManagementDUnitTest.java
+++ b/geode-cq/src/test/java/org/apache/geode/management/CacheServerManagementDUnitTest.java
@@ -14,17 +14,41 @@
  */
 package org.apache.geode.management;
 
-import org.junit.Ignore;
-import org.junit.experimental.categories.Category;
-import org.junit.Test;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_HTTP_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
-import static org.junit.Assert.*;
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Properties;
 
-import org.apache.geode.test.junit.categories.DistributedTest;
+import javax.management.InstanceNotFoundException;
+import javax.management.MBeanServer;
+import javax.management.Notification;
+import javax.management.NotificationListener;
+import javax.management.ObjectName;
+
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.client.internal.LocatorTestBase;
-import org.apache.geode.cache.query.*;
+import org.apache.geode.cache.query.IndexExistsException;
+import org.apache.geode.cache.query.IndexInvalidException;
+import org.apache.geode.cache.query.IndexNameConflictException;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.RegionNotFoundException;
 import org.apache.geode.cache.query.cq.dunit.CqQueryDUnitTest;
 import org.apache.geode.cache.query.internal.cq.CqService;
 import org.apache.geode.cache.server.CacheServer;
@@ -36,17 +60,15 @@ import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.management.internal.JmxManagerLocatorRequest;
 import org.apache.geode.management.internal.MBeanJMXAdapter;
 import org.apache.geode.management.internal.SystemManagementService;
-import org.apache.geode.test.dunit.*;
-
-import javax.management.*;
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Collections;
-import java.util.Properties;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import org.apache.geode.test.dunit.Assert;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.LogWriterUtils;
+import org.apache.geode.test.dunit.NetworkUtils;
+import org.apache.geode.test.dunit.SerializableRunnable;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.Wait;
+import org.apache.geode.test.dunit.WaitCriterion;
+import org.apache.geode.test.junit.categories.DistributedTest;
 
 /**
  * Cache Server related management test cases
@@ -219,7 +241,7 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
 
     // Step 4:
     JmxManagerLocatorRequest.send(locator.getHost().getHostName(), locatorPort,
-        CONNECT_LOCATOR_TIMEOUT_MS, Collections.<String, String>emptyMap());
+        CONNECT_LOCATOR_TIMEOUT_MS, new Properties());
 
     // Step 5:
     locator.invoke("Check locator", () -> {

http://git-wip-us.apache.org/repos/asf/geode/blob/7352fcc7/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandIntegrationTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandIntegrationTest.java
new file mode 100644
index 0000000..bead33d
--- /dev/null
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandIntegrationTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.cli.commands;
+
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class ConnectCommandIntegrationTest {
+
+  @ClassRule
+  public static LocatorStarterRule locator = new LocatorStarterRule().withAutoStart();
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @Test
+  public void connectToLocator() throws Exception {
+    gfsh.connectAndVerify(locator);
+  }
+
+  @Test
+  public void connectOverJmx() throws Exception {
+    gfsh.connectAndVerify(locator.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger);
+  }
+
+  @Test
+  public void connectOverHttp() throws Exception {
+    gfsh.connectAndVerify(locator.getHttpPort(), GfshShellConnectionRule.PortType.http);
+  }
+}


[03/10] geode git commit: GEODE-3111 GatewayReceiver - DEFAULT_MANUAL_START value is ambiguous This closes #705

Posted by ud...@apache.org.
GEODE-3111 GatewayReceiver - DEFAULT_MANUAL_START value is ambiguous
This closes #705


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

Branch: refs/heads/feature/GEODE-3416
Commit: 08154dd7731910803aa5eb2caf0e284b3a239755
Parents: 06b839c
Author: Dave Barnes <db...@pivotal.io>
Authored: Wed Aug 9 16:17:14 2017 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Thu Aug 10 09:42:39 2017 -0700

----------------------------------------------------------------------
 .../apache/geode/cache/wan/GatewayReceiver.java |  2 +-
 .../gfsh/command-pages/create.html.md.erb       |  4 +--
 .../setting_up_a_multisite_system.html.md.erb   | 32 ++++++--------------
 .../multisite_overview.html.md.erb              |  3 +-
 4 files changed, 15 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/08154dd7/geode-core/src/main/java/org/apache/geode/cache/wan/GatewayReceiver.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/wan/GatewayReceiver.java b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewayReceiver.java
index 8c02af1..fd6f039 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/wan/GatewayReceiver.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/wan/GatewayReceiver.java
@@ -64,7 +64,7 @@ public interface GatewayReceiver {
   public static final String DEFAULT_HOSTNAME_FOR_SENDERS = "";
 
   /**
-   * The default value (true) for manually starting a <code>GatewayReceiver</code>.
+   * The default value for manually starting a <code>GatewayReceiver</code>.
    * 
    * @since GemFire 8.1
    */

http://git-wip-us.apache.org/repos/asf/geode/blob/08154dd7/geode-docs/tools_modules/gfsh/command-pages/create.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/tools_modules/gfsh/command-pages/create.html.md.erb b/geode-docs/tools_modules/gfsh/command-pages/create.html.md.erb
index b23ca56..01efcd8 100644
--- a/geode-docs/tools_modules/gfsh/command-pages/create.html.md.erb
+++ b/geode-docs/tools_modules/gfsh/command-pages/create.html.md.erb
@@ -392,7 +392,7 @@ create gateway-receiver [--group=value(,value)*] [--member=value(,value)*]
 </tr>
 <tr class="odd">
 <td><span class="keyword parmname">\-\-manual-start</span></td>
-<td>Boolean value that specifies whether you need to manually start the gateway receiver. If you supply a null value, the default is &quot;true&quot; and you will need to start the gateway receiver manually.</td>
+<td>Boolean value that specifies whether you need to manually start the gateway receiver. If you supply a null value, the default is &quot;false&quot; the gateway receiver starts automatically.</td>
 <td>true</td>
 </tr>
 <tr class="even">
@@ -517,7 +517,7 @@ create gateway-sender --id=value --remote-distributed-system-id=value
 </tr>
 <tr class="odd">
 <td><span class="keyword parmname">\-\-manual-start</span></td>
-<td>Boolean value that specifies whether you need to manually start the gateway sender. If you supply a null value, the default is &quot;false&quot; and the gateway sender attempts to start automatically.</td>
+<td>Boolean value that specifies whether you need to manually start the gateway sender. If you supply a null value, the default is &quot;false&quot; and the gateway sender starts automatically.</td>
 <td>false</td>
 </tr>
 <tr class="even">

http://git-wip-us.apache.org/repos/asf/geode/blob/08154dd7/geode-docs/topologies_and_comm/multi_site_configuration/setting_up_a_multisite_system.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/topologies_and_comm/multi_site_configuration/setting_up_a_multisite_system.html.md.erb b/geode-docs/topologies_and_comm/multi_site_configuration/setting_up_a_multisite_system.html.md.erb
index 0b5091d..dd2bc3a 100644
--- a/geode-docs/topologies_and_comm/multi_site_configuration/setting_up_a_multisite_system.html.md.erb
+++ b/geode-docs/topologies_and_comm/multi_site_configuration/setting_up_a_multisite_system.html.md.erb
@@ -167,7 +167,7 @@ See [WAN Configuration](../../reference/topics/elements_ref.html#topic_7B1CABCAD
          dispatcher-threads="2" order-policy="partition" manual-start="true" />
         ```
 
-        By default, this is set to false on gateway senders and the senders are started automatically.
+        By default, `manual-start` is set to false on gateway senders and the senders are started automatically.
 
 4.  If you have configured the gateway sender with `manual-start` equal to true, then start the gateway sender when appropriate. For persistent gateway senders, the appropriate time to start them is when all other members hosting persistent regions are started. In gfsh:
 
@@ -238,7 +238,8 @@ A gateway receiver configuration specifies a range of possible port numbers on w
 
 You can optionally configure gateway receivers to provide a specific IP address or host name for gateway sender connections. If you configure hostname-for-senders, locators will use the provided host name or IP address when instructing gateway senders on how to connect to gateway receivers. If you provide "" or null as the value, by default the gateway receiver's bind-address will be sent to clients.
 
-In addition, you can configure gateway receivers to start automatically or to require a manual start. By default, gateway receivers must be started manually.
+In addition, you can configure gateway receivers to start automatically or, by setting `manual-start` to true, to require a manual start. 
+By default, gateway receivers start automatically.
 
 **Note:**
 To configure a gateway receiver, you can use gfsh, cache.xml or Java API configurations as described below. For more information on configuring gateway receivers in gfsh, see [create gateway-receiver](../../tools_modules/gfsh/command-pages/create.html#topic_a4x_pb1_dk).
@@ -255,7 +256,7 @@ To configure a gateway receiver, you can use gfsh, cache.xml or Java API configu
 
     ``` pre
     <cache>
-      <gateway-receiver start-port="1530" end-port="1551" hostname-for-senders="gateway1.mycompany.com" manual-start="false" /> 
+      <gateway-receiver start-port="1530" end-port="1551" hostname-for-senders="gateway1.mycompany.com" /> 
        ... 
     </cache>
     ```
@@ -272,33 +273,20 @@ To configure a gateway receiver, you can use gfsh, cache.xml or Java API configu
     gateway.setEndPort(1551);
     gateway.setHostnameForSenders("gateway1.mycompany.com");
     GatewayReceiver receiver = gateway.create();
-    receiver.start();
     ```
 
     **Note:**
-    When using the Java API, you must create any region that might receive events from a remote site before you start the gateway receiver. Otherwise, batches of events could arrive from remote sites before the regions for those events have been created. If this occurs, the local site will throw exceptions because the receiving region does not exist yet. If you define regions in `cache.xml`, the correct startup order is handled automatically.
+    When using the Java API, you must create any region that might receive events from a remote site before you create the gateway receiver. Otherwise, batches of events could arrive from remote sites before the regions for those events have been created. If this occurs, the local site will throw exceptions because the receiving region does not yet exist. If you define regions in `cache.xml`, the correct startup order is handled automatically.
 
--   By default, `manual-start` is configured to **true** on gateway receivers. If you have not changed this configuration (either in gfsh or cache.xml), you must start your gateway receiver manually. For example, in gfsh:
+After starting new gateway receivers, you can execute the [load-balance gateway-sender](../../tools_modules/gfsh/command-pages/load-balance.html) command in `gfsh` so that a specific gateway sender will be able to rebalance its connections and connect new remote gateway receivers. Invoking this command redistributes gateway sender connections more evenly among all the gateway receivers.
 
-    ``` pre
-    gfsh>create gateway-receiver --manual-start=false
-    ```
-
--   In gfsh, if you have not disabled manual-start, execute the following command to start the gateway receiver:
-
-    ``` pre
-    gfsh>start gateway-receiver
-    ```
-
--   After starting new gateway receivers, you can execute the [load-balance gateway-sender](../../tools_modules/gfsh/command-pages/load-balance.html) command in `gfsh` so that a specific gateway sender will be able to rebalance its connections and connect new remote gateway receivers. Invoking this command redistributes gateway sender connections more evenly among all the gateway receivers.
-
-    Another option is to use the `GatewaySender.rebalance` Java API.
+Another option is to use the `GatewaySender.rebalance` Java API.
 
 As an example, assume the following scenario:
 
-1.  start 1 receiver in site ny
-2.  start 4 senders in site ln
-3.  start 3 additional receiver in NY
+1.  Create 1 receiver in site NY.
+2.  Create 4 senders in site LN.
+3.  Create 3 additional receivers in NY.
 
 You can then execute the following in gfsh to see the effects of rebalancing:
 

http://git-wip-us.apache.org/repos/asf/geode/blob/08154dd7/geode-docs/topologies_and_comm/topology_concepts/multisite_overview.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/topologies_and_comm/topology_concepts/multisite_overview.html.md.erb b/geode-docs/topologies_and_comm/topology_concepts/multisite_overview.html.md.erb
index f250f80..e7bb61d 100644
--- a/geode-docs/topologies_and_comm/topology_concepts/multisite_overview.html.md.erb
+++ b/geode-docs/topologies_and_comm/topology_concepts/multisite_overview.html.md.erb
@@ -108,7 +108,8 @@ A gateway receiver applies each region event to the same region or partition tha
 
 Gateway senders use any available gateway receiver in the target cluster to send region events. You can deploy gateway receiver configurations to multiple Geode members as needed for high availability and load balancing, however you can only host one gateway receiver per member.
 
-After you create a gateway receiver, you can configure the gateway receiver to start automatically or to require a manual start. The current default is to require a manual start for the gateway receiver (`manual-start` is set to true).
+After you create a gateway receiver, you can configure the gateway receiver to start automatically or to require a manual start. 
+By default, the gateway receiver starts automatically (`manual-start` is set to false).
 
 After you create and start a new gateway receiver at one WAN site, you can execute the [load-balance gateway-sender](../../tools_modules/gfsh/command-pages/load-balance.html#concept_fn2_qls_5q) command in `gfsh` for existing remote gateway senders so that the new receiver can pick up connections to gateway senders at different sites. You invoke this command on the gateway senders to redistribute connections more evenly among all the gateway receivers. Another option is to use the `GatewaySender.rebalance` Java API.