You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2016/10/24 19:15:54 UTC

incubator-geode git commit: GEODE-2025: do not use 8080 as the default http-server-port since we defined default in DistributionConfig

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 892d6d33f -> 259ceb77d


GEODE-2025: do not use 8080 as the default http-server-port since we defined default in DistributionConfig


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

Branch: refs/heads/develop
Commit: 259ceb77d0221772801f2b0e8f535530e346c371
Parents: 892d6d3
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Oct 21 09:38:43 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Oct 24 12:14:18 2016 -0700

----------------------------------------------------------------------
 .../rest/internal/web/GeodeRestClient.java      |  28 ++---
 .../web/RestSecurityIntegrationTest.java        |   4 +-
 .../rest/internal/web/RestServersJUnitTest.java |  68 ++++++++++++
 .../internal/web/SwaggerVerificationTest.java   |   4 +-
 .../apache/geode/cache/server/CacheServer.java  |   7 --
 .../util/FindRestEnabledServersFunction.java    |   9 +-
 .../geode/management/internal/RestAgent.java    |  40 +++----
 .../cli/commands/LauncherLifecycleCommands.java | 111 +++++++++++--------
 8 files changed, 179 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/259ceb77/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/GeodeRestClient.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/GeodeRestClient.java b/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/GeodeRestClient.java
index 2889c67..071b95c 100644
--- a/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/GeodeRestClient.java
+++ b/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/GeodeRestClient.java
@@ -36,6 +36,9 @@ import org.apache.http.impl.client.BasicAuthCache;
 import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClients;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
 import org.json.JSONTokener;
 import org.junit.Assert;
 
@@ -49,12 +52,13 @@ import java.nio.charset.StandardCharsets;
 public class GeodeRestClient {
 
   public final static String PROTOCOL = "http";
-  public final static String HOSTNAME = "localhost";
   public final static String CONTEXT = "/geode/v1";
 
   private int restPort = 0;
+  private String bindAddress = null;
 
-  public GeodeRestClient(int restPort) {
+  public GeodeRestClient(String bindAddress, int restPort) {
+    this.bindAddress = bindAddress;
     this.restPort = restPort;
   }
 
@@ -113,21 +117,19 @@ public class GeodeRestClient {
     return response.getStatusLine().getStatusCode();
   }
 
-  public static JSONTokener getResponseBody(HttpResponse response) throws IOException {
-    HttpEntity entity = response.getEntity();
-    InputStream content = entity.getContent();
-    BufferedReader reader = new BufferedReader(new InputStreamReader(content));
-    String line;
-    StringBuilder str = new StringBuilder();
-    while ((line = reader.readLine()) != null) {
-      str.append(line);
-    }
-    return new JSONTokener(str.toString());
+  public static JSONObject getJsonObject(HttpResponse response) throws IOException, JSONException {
+    JSONTokener tokener = new JSONTokener(new InputStreamReader(response.getEntity().getContent()));
+    return new JSONObject(tokener);
+  }
+
+  public static JSONArray getJsonArray(HttpResponse response) throws IOException, JSONException {
+    JSONTokener tokener = new JSONTokener(new InputStreamReader(response.getEntity().getContent()));
+    return new JSONArray(tokener);
   }
 
   public HttpResponse doRequest(HttpRequestBase request, String username, String password)
       throws MalformedURLException {
-    HttpHost targetHost = new HttpHost(HOSTNAME, restPort, PROTOCOL);
+    HttpHost targetHost = new HttpHost(bindAddress, restPort, PROTOCOL);
     CloseableHttpClient httpclient = HttpClients.custom().build();
     HttpClientContext clientContext = HttpClientContext.create();
     // if username is null, do not put in authentication

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/259ceb77/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/RestSecurityIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/RestSecurityIntegrationTest.java b/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/RestSecurityIntegrationTest.java
index 5b3f7ce..2dffcb7 100644
--- a/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/RestSecurityIntegrationTest.java
+++ b/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/RestSecurityIntegrationTest.java
@@ -59,7 +59,7 @@ public class RestSecurityIntegrationTest {
 
   @ClassRule
   public static ServerStarter serverStarter = new ServerStarter(properties);
-  private final GeodeRestClient restClient = new GeodeRestClient(restPort);
+  private final GeodeRestClient restClient = new GeodeRestClient("localhost", restPort);
 
   @BeforeClass
   public static void before() throws Exception {
@@ -199,7 +199,7 @@ public class RestSecurityIntegrationTest {
     assertEquals("A '200 - OK' was expected", 200, restClient.getCode(response));
     assertEquals(MediaType.APPLICATION_JSON_UTF8_VALUE, restClient.getContentType(response));
 
-    JSONObject jsonObject = new JSONObject(restClient.getResponseBody(response));
+    JSONObject jsonObject = restClient.getJsonObject(response);
     JSONArray regions = jsonObject.getJSONArray("regions");
     assertNotNull(regions);
     assertTrue(regions.length() > 0);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/259ceb77/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/RestServersJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/RestServersJUnitTest.java b/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/RestServersJUnitTest.java
new file mode 100644
index 0000000..552a184
--- /dev/null
+++ b/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/RestServersJUnitTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.rest.internal.web;
+
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.START_DEV_REST_API;
+
+import org.apache.geode.test.dunit.Assert;
+import org.apache.geode.test.dunit.rules.ServerStarter;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.apache.http.HttpResponse;
+import org.json.JSONArray;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Properties;
+
+@Category(IntegrationTest.class)
+public class RestServersJUnitTest {
+
+  private static int defaultPort = 7070;
+  static Properties properties = new Properties() {
+    {
+      setProperty(START_DEV_REST_API, "true");
+      setProperty(HTTP_SERVICE_BIND_ADDRESS, "localhost");
+    }
+  };
+
+  @ClassRule
+  public static ServerStarter serverStarter = new ServerStarter(properties);
+  private static GeodeRestClient restClient;
+
+  @BeforeClass
+  public static void before() throws Exception {
+    serverStarter.startServer();
+    restClient = new GeodeRestClient("localhost", defaultPort);
+  }
+
+  @Test
+  public void testDefaultPort() throws Exception {
+    // make sure the server is started on the default port and we can connect using the default port
+    HttpResponse response = restClient.doGet("/", null, null);
+    Assert.assertEquals(200, GeodeRestClient.getCode(response));
+  }
+
+  @Test
+  public void testServers() throws Exception {
+    HttpResponse response = restClient.doGet("/servers", null, null);
+    JSONArray body = GeodeRestClient.getJsonArray(response);
+    Assert.assertEquals(1, body.length());
+    Assert.assertEquals("http://localhost:7070", body.getString(0));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/259ceb77/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/SwaggerVerificationTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/SwaggerVerificationTest.java b/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/SwaggerVerificationTest.java
index 3412331..55cc26e 100644
--- a/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/SwaggerVerificationTest.java
+++ b/geode-assembly/src/test/java/org/apache/geode/rest/internal/web/SwaggerVerificationTest.java
@@ -49,7 +49,7 @@ public class SwaggerVerificationTest {
 
   @ClassRule
   public static ServerStarter serverStarter = new ServerStarter(properties);
-  private final GeodeRestClient restClient = new GeodeRestClient(restPort);
+  private final GeodeRestClient restClient = new GeodeRestClient("localhost", restPort);
 
   @BeforeClass
   public static void before() throws Exception {
@@ -65,7 +65,7 @@ public class SwaggerVerificationTest {
     // Check the JSON
     response = restClient.doGetRequest("/geode/v2/api-docs");
     assertThat(GeodeRestClient.getCode(response), is(200));
-    JSONObject json = new JSONObject(GeodeRestClient.getResponseBody(response));
+    JSONObject json = GeodeRestClient.getJsonObject(response);
     assertThat(json.get("swagger"), is("2.0"));
 
     JSONObject info = json.getJSONObject("info");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/259ceb77/geode-core/src/main/java/org/apache/geode/cache/server/CacheServer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/server/CacheServer.java b/geode-core/src/main/java/org/apache/geode/cache/server/CacheServer.java
index 8a5eb96..bd00170 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/server/CacheServer.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/server/CacheServer.java
@@ -20,8 +20,6 @@ import java.util.Set;
 import org.apache.geode.cache.ClientSession;
 import org.apache.geode.cache.InterestRegistrationListener;
 import org.apache.geode.cache.client.Pool;
-import org.apache.geode.cache.server.ClientSubscriptionConfig;
-import org.apache.geode.cache.server.ServerLoadProbe;
 import org.apache.geode.cache.server.internal.ConnectionCountProbe;
 import org.apache.geode.distributed.DistributedMember;
 
@@ -137,11 +135,6 @@ public interface CacheServer {
    */
   public static final boolean DEFAULT_TCP_NO_DELAY = true;
 
-
-  public static final int HTTP_DEFAULT_PORT = 8080;
-  public static final String HTTP_SERVICE_DEFAULT_BIND_ADDRESS = "";
-
-
   /**
    * Returns the port on which this cache server listens for clients.
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/259ceb77/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java
index 6863782..5da63ad 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java
@@ -22,12 +22,13 @@ import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.InternalEntity;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.management.internal.RestAgent;
 
 /**
  * The FindRestEnabledServersFunction class is a gemfire function that gives details about REST
  * enabled gemfire servers.
  * <p/>
- * 
+ *
  * @since GemFire 8.1
  */
 
@@ -42,11 +43,13 @@ public class FindRestEnabledServersFunction extends FunctionAdapter implements I
       GemFireCacheImpl c = (GemFireCacheImpl) CacheFactory.getAnyInstance();
       DistributionConfig config = InternalDistributedSystem.getAnyInstance().getConfig();
 
+      String bindAddress = RestAgent.getBindAddressForHttpService(config);
+
       final String protocolType = config.getHttpServiceSSLEnabled() ? "https" : "http";
 
       if (c.isRESTServiceRunning()) {
-        context.getResultSender().lastResult(protocolType + "://"
-            + config.getHttpServiceBindAddress() + ":" + config.getHttpServicePort());
+        context.getResultSender()
+            .lastResult(protocolType + "://" + bindAddress + ":" + config.getHttpServicePort());
 
       } else {
         context.getResultSender().lastResult("");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/259ceb77/geode-core/src/main/java/org/apache/geode/management/internal/RestAgent.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/RestAgent.java b/geode-core/src/main/java/org/apache/geode/management/internal/RestAgent.java
index 3f630ce..07e6e25 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/RestAgent.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/RestAgent.java
@@ -114,7 +114,7 @@ public class RestAgent {
   public void startHttpService() {
     // TODO: add a check that will make sure that we start HTTP service on
     // non-manager data node
-    String httpServiceBindAddress = getBindAddressForHttpService();
+    String httpServiceBindAddress = getBindAddressForHttpService(this.config);
     logger.info("Attempting to start HTTP service on port ({}) at bind-address ({})...",
         this.config.getHttpServicePort(), httpServiceBindAddress);
 
@@ -157,25 +157,25 @@ public class RestAgent {
     }
   }
 
-  private String getBindAddressForHttpService() {
-    java.lang.String bindAddress = this.config.getHttpServiceBindAddress();
-    if (StringUtils.isBlank(bindAddress)) {
-      if (StringUtils.isBlank(this.config.getServerBindAddress())) {
-        if (StringUtils.isBlank(this.config.getBindAddress())) {
-          try {
-            bindAddress = SocketCreator.getLocalHost().getHostAddress();
-            logger.info("RestAgent.getBindAddressForHttpService.localhost: "
-                + SocketCreator.getLocalHost().getHostAddress());
-          } catch (UnknownHostException e) {
-            logger.error("LocalHost could not be found.", e);
-            return bindAddress;
-          }
-        } else {
-          bindAddress = this.config.getBindAddress();
-        }
-      } else {
-        bindAddress = this.config.getServerBindAddress();
-      }
+  public static String getBindAddressForHttpService(DistributionConfig config) {
+    String bindAddress = config.getHttpServiceBindAddress();
+    if (!StringUtils.isBlank(bindAddress))
+      return bindAddress;
+
+    bindAddress = config.getServerBindAddress();
+    if (!StringUtils.isBlank(bindAddress))
+      return bindAddress;
+
+    bindAddress = config.getBindAddress();
+    if (!StringUtils.isBlank(bindAddress))
+      return bindAddress;
+
+    try {
+      bindAddress = SocketCreator.getLocalHost().getHostAddress();
+      logger.info("RestAgent.getBindAddressForHttpService.localhost: "
+          + SocketCreator.getLocalHost().getHostAddress());
+    } catch (UnknownHostException e) {
+      logger.error("LocalHost could not be found.", e);
     }
     return bindAddress;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/259ceb77/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java
index 9e23f98..912d8cc 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java
@@ -14,48 +14,33 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.distributed.ConfigurationProperties.BIND_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.CACHE_XML_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_CONFIGURATION_DIR;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_TIME_STATISTICS;
+import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.LOAD_CLUSTER_CONFIGURATION_FROM_DIR;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATOR_WAIT_TIME;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCK_MEMORY;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.MEMCACHED_BIND_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.MEMCACHED_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.MEMCACHED_PROTOCOL;
+import static org.apache.geode.distributed.ConfigurationProperties.OFF_HEAP_MEMORY_SIZE;
+import static org.apache.geode.distributed.ConfigurationProperties.REDIS_BIND_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.REDIS_PASSWORD;
+import static org.apache.geode.distributed.ConfigurationProperties.REDIS_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.START_DEV_REST_API;
+import static org.apache.geode.distributed.ConfigurationProperties.STATISTIC_ARCHIVE_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.USE_CLUSTER_CONFIGURATION;
 import static org.apache.geode.management.internal.cli.i18n.CliStrings.START_SERVER__PASSWORD;
 
-import java.awt.Desktop;
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.FileNotFoundException;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.net.InetAddress;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.EmptyStackException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.Stack;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.Query;
-import javax.management.QueryExp;
-import javax.net.ssl.SSLException;
-import javax.net.ssl.SSLHandshakeException;
-
-import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
 import org.apache.geode.GemFireException;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.server.CacheServer;
@@ -71,13 +56,13 @@ import org.apache.geode.distributed.internal.tcpserver.TcpClient;
 import org.apache.geode.internal.DistributionLocator;
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.OSProcess;
-import org.apache.geode.internal.net.*;
 import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.lang.ClassUtils;
 import org.apache.geode.internal.lang.ObjectUtils;
 import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.internal.lang.SystemUtils;
+import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.process.ClusterConfigurationNotAvailableException;
 import org.apache.geode.internal.process.ProcessLauncherContext;
 import org.apache.geode.internal.process.ProcessStreamReader;
@@ -116,13 +101,50 @@ import org.apache.geode.management.internal.configuration.messages.SharedConfigu
 import org.apache.geode.management.internal.configuration.messages.SharedConfigurationStatusResponse;
 import org.apache.geode.management.internal.security.ResourceConstants;
 import org.apache.geode.security.AuthenticationFailedException;
+import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import java.awt.Desktop;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.net.InetAddress;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EmptyStackException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.Stack;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.Query;
+import javax.management.QueryExp;
+import javax.net.ssl.SSLException;
+import javax.net.ssl.SSLHandshakeException;
 
 /**
  * The LauncherLifecycleCommands class encapsulates all GemFire launcher commands for GemFire tools
  * (like starting GemFire Monitor (GFMon) and Visual Statistics Display (VSD)) as well external
  * tools (like jconsole).
  * <p>
- * 
+ *
  * @see org.apache.geode.distributed.LocatorLauncher
  * @see org.apache.geode.distributed.ServerLauncher
  * @see org.apache.geode.management.internal.cli.commands.AbstractCommandsSupport
@@ -1460,11 +1482,10 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
       @CliOption(key = CliStrings.START_SERVER__REST_API, unspecifiedDefaultValue = "false",
           specifiedDefaultValue = "true",
           help = CliStrings.START_SERVER__REST_API__HELP) final Boolean startRestApi,
-      @CliOption(key = CliStrings.START_SERVER__HTTP_SERVICE_PORT,
-          unspecifiedDefaultValue = ("" + CacheServer.HTTP_DEFAULT_PORT),
+      @CliOption(key = CliStrings.START_SERVER__HTTP_SERVICE_PORT, unspecifiedDefaultValue = "",
           help = CliStrings.START_SERVER__HTTP_SERVICE_PORT__HELP) final String httpServicePort,
       @CliOption(key = CliStrings.START_SERVER__HTTP_SERVICE_BIND_ADDRESS,
-          unspecifiedDefaultValue = CacheServer.HTTP_SERVICE_DEFAULT_BIND_ADDRESS,
+          unspecifiedDefaultValue = "",
           help = CliStrings.START_SERVER__HTTP_SERVICE_BIND_ADDRESS__HELP) final String httpServiceBindAddress,
       @CliOption(key = CliStrings.START_SERVER__USERNAME, unspecifiedDefaultValue = "",
           help = CliStrings.START_SERVER__USERNAME__HELP) final String userName,