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 2019/01/25 16:44:13 UTC

[geode] branch develop updated: GEODE-6283: expose HttpService on InternalCache so that different com… (#3110)

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

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


The following commit(s) were added to refs/heads/develop by this push:
     new f253b59  GEODE-6283: expose HttpService on InternalCache so that different com… (#3110)
f253b59 is described below

commit f253b590e844063c23a2e399d1a903df6fb33b29
Author: jinmeiliao <ji...@pivotal.io>
AuthorDate: Fri Jan 25 08:44:03 2019 -0800

    GEODE-6283: expose HttpService on InternalCache so that different com… (#3110)
    
    * GEODE-6283: expose HttpService on InternalCache so that different component can add their own webapps independently.
    
    Co-authored-by: Jens Deppe <jd...@pivotal.io>
---
 extensions/geode-modules-assembly/build.gradle     |   1 +
 .../release/session/bin/modify_war                 |   4 +
 geode-assembly/build.gradle                        |   2 +
 .../apache/geode/session/tests/TomcatInstall.java  |   3 +-
 .../rest/RegionManagementAcceptanceTest.java       |  39 ++++
 .../controllers/RestAPIsAndInterOpsDUnitTest.java  |  14 +-
 .../internal/HttpServiceIntegrationTest.java       |  55 +++++
 .../internal/RegionManagementIntegrationTest.java  |   5 +
 .../distributed/internal/InternalLocator.java      |  58 +++--
 .../geode/internal/cache/GemFireCacheImpl.java     |  32 ++-
 .../cache/HttpService.java}                        | 124 +++++-----
 .../apache/geode/internal/cache/InternalCache.java |   2 +
 .../cache/InternalCacheForClientAccess.java        |   5 +
 .../internal/cache/xmlcache/CacheCreation.java     |   6 +
 .../geode/management/internal/ManagementAgent.java | 250 +++++----------------
 .../geode/management/internal/RestAgent.java       |  88 +-------
 .../internal/SystemManagementService.java          |   4 +-
 .../support/LoginHandlerInterceptor.java           |   4 +-
 ...perJUnitTest.java => HttpServiceJunitTest.java} |  21 +-
 .../cache/client/internal/LocatorTestBase.java     |   5 +-
 .../java/org/apache/geode/test/fake/Fakes.java     |   3 +
 .../geode/tools/pulse/tests/rules/ServerRule.java  |  12 +-
 .../web/controllers/RestAccessControllerTest.java  |   4 +-
 .../web/security/GeodeAuthenticationProvider.java  |   4 +-
 .../internal/web/security/RestSecurityService.java |   4 +-
 .../controllers/AbstractManagementController.java  |   6 +-
 .../controllers/RegionManagementController.java    |   5 +
 .../rest/security/GeodeAuthenticationProvider.java |   4 +-
 .../rest/security/RestSecurityService.java         |   4 +-
 29 files changed, 355 insertions(+), 413 deletions(-)

diff --git a/extensions/geode-modules-assembly/build.gradle b/extensions/geode-modules-assembly/build.gradle
index 9e2e5cc..aac5f7a 100644
--- a/extensions/geode-modules-assembly/build.gradle
+++ b/extensions/geode-modules-assembly/build.gradle
@@ -211,6 +211,7 @@ task distAppServer(type: Zip, dependsOn: [':extensions:geode-modules-session:jar
       filter(ReplaceTokens, tokens:['ANTLR_VERSION': project.'antlr.version'])
       filter(ReplaceTokens, tokens:['TX_VERSION': project.'javax.transaction-api.version'])
       filter(ReplaceTokens, tokens:['JGROUPS_VERSION': project.'jgroups.version'])
+      filter(ReplaceTokens, tokens:['JETTY_VERSION': project.'jetty.version'])
       filter(ReplaceTokens, tokens:['SHIRO_VERSION': project.'shiro.version'])
       filter(ReplaceTokens, tokens:['COMMONS_IO_VERSION': project.'commons-io.version'])
       filter(ReplaceTokens, tokens:['COMMONS_LANG_VERSION': project.'commons-lang3.version'])
diff --git a/extensions/geode-modules-assembly/release/session/bin/modify_war b/extensions/geode-modules-assembly/release/session/bin/modify_war
index aa7f71a..bdafa4c 100755
--- a/extensions/geode-modules-assembly/release/session/bin/modify_war
+++ b/extensions/geode-modules-assembly/release/session/bin/modify_war
@@ -271,6 +271,10 @@ OTHER_JARS=(${GEODE}/lib/geode-core-${VERSION}.jar \
     ${GEODE}/lib/log4j-jul-@LOG4J_VERSION@.jar \
     ${GEODE}/lib/fastutil-@FASTUTIL_VERSION@.jar \
     ${GEODE}/lib/javax.transaction-api-@TX_VERSION@.jar \
+    ${GEODE}/lib/jetty-http-@JETTY_VERSION@.jar \
+    ${GEODE}/lib/jetty-io-@JETTY_VERSION@.jar \
+    ${GEODE}/lib/jetty-server-@JETTY_VERSION@.jar \
+    ${GEODE}/lib/jetty-util-@JETTY_VERSION@.jar \
     ${GEODE}/lib/jgroups-@JGROUPS_VERSION@.jar \
     ${GEODE}/lib/commons-io-@COMMONS_IO_VERSION@.jar \
     ${GEODE}/lib/commons-lang3-@COMMONS_LANG_VERSION@.jar \
diff --git a/geode-assembly/build.gradle b/geode-assembly/build.gradle
index 37740d3..db01c2d 100755
--- a/geode-assembly/build.gradle
+++ b/geode-assembly/build.gradle
@@ -200,6 +200,8 @@ dependencies {
   acceptanceTestCompile(project(':geode-dunit')) {
     exclude module: 'geode-core'
   }
+  acceptanceTestCompile(project(':geode-assembly:geode-assembly-test'))
+  acceptanceTestCompile('org.apache.httpcomponents:httpclient')
 
 
   uiTestCompile(project(':geode-core'))
diff --git a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatInstall.java b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatInstall.java
index 8d5da4c..31958bc 100644
--- a/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatInstall.java
+++ b/geode-assembly/geode-assembly-test/src/main/java/org/apache/geode/session/tests/TomcatInstall.java
@@ -92,7 +92,8 @@ public class TomcatInstall extends ContainerInstall {
   private static final String[] tomcatRequiredJars =
       {"antlr", "commons-io", "commons-lang", "commons-validator", "fastutil", "geode-common",
           "geode-core", "geode-management", "javax.transaction-api", "jgroups", "log4j-api",
-          "log4j-core", "log4j-jul", "shiro-core"};
+          "log4j-core", "log4j-jul", "shiro-core", "jetty-server", "jetty-util", "jetty-http",
+          "jetty-io"};
 
   private final TomcatVersion version;
 
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/RegionManagementAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/RegionManagementAcceptanceTest.java
new file mode 100644
index 0000000..877bbd4
--- /dev/null
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/management/internal/rest/RegionManagementAcceptanceTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.rest;
+
+import org.junit.Rule;
+import org.junit.Test;
+
+import org.apache.geode.test.junit.rules.GeodeDevRestClient;
+import org.apache.geode.test.junit.rules.gfsh.GfshRule;
+import org.apache.geode.test.junit.rules.gfsh.GfshScript;
+
+public class RegionManagementAcceptanceTest {
+
+  @Rule
+  public GfshRule gfsh = new GfshRule();
+
+  @Test
+  public void sanityCheck() throws Exception {
+    GfshScript.of("start locator --port=0").execute(gfsh);
+
+    // verify the management rest api is started correctly
+    GeodeDevRestClient client =
+        new GeodeDevRestClient("/geode-management/v2", "localhost", 7070, false);
+
+    client.doGetAndAssert("/ping").hasStatusCode(200).hasResponseBody().isEqualTo("pong");
+  }
+}
diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/rest/internal/web/controllers/RestAPIsAndInterOpsDUnitTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/rest/internal/web/controllers/RestAPIsAndInterOpsDUnitTest.java
index d5d6d3b..8b3ee05 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/rest/internal/web/controllers/RestAPIsAndInterOpsDUnitTest.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/rest/internal/web/controllers/RestAPIsAndInterOpsDUnitTest.java
@@ -155,21 +155,13 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
     final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
 
     // create Cache of given VM and start HTTP service with REST APIs service
-    startBridgeServer(hostName, serverPort, groups, locators, regions, probe);
-
-    return "http://" + hostName + ":" + serverPort + this.urlContext + "/v1";
-  }
-
-  private int startBridgeServer(String hostName, int restServicerPort, final String[] groups,
-      final String locators, final String[] regions, final ServerLoadProbe probe)
-      throws IOException {
 
     Properties props = new Properties();
     props.setProperty(MCAST_PORT, String.valueOf(0));
     props.setProperty(LOCATORS, locators);
     props.setProperty(START_DEV_REST_API, "true");
     props.setProperty(HTTP_SERVICE_BIND_ADDRESS, hostName);
-    props.setProperty(HTTP_SERVICE_PORT, String.valueOf(restServicerPort));
+    props.setProperty(HTTP_SERVICE_PORT, String.valueOf(serverPort));
 
     DistributedSystem ds = getSystem(props);
     InternalCache cache = (InternalCache) CacheFactory.create(ds);
@@ -190,7 +182,9 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
     server.start();
 
     remoteObjects.put(CACHE_KEY, cache);
-    return server.getPort();
+    server.getPort();
+
+    return "http://" + hostName + ":" + serverPort + this.urlContext + "/v1";
   }
 
   private void doPutsInClientCache() {
diff --git a/geode-assembly/src/integrationTest/java/org/apache/geode/management/internal/HttpServiceIntegrationTest.java b/geode-assembly/src/integrationTest/java/org/apache/geode/management/internal/HttpServiceIntegrationTest.java
new file mode 100644
index 0000000..0021a0f
--- /dev/null
+++ b/geode-assembly/src/integrationTest/java/org/apache/geode/management/internal/HttpServiceIntegrationTest.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;
+
+
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.test.junit.rules.GeodeDevRestClient;
+import org.apache.geode.test.junit.rules.RequiresGeodeHome;
+import org.apache.geode.test.junit.rules.ServerStarterRule;
+
+
+public class HttpServiceIntegrationTest {
+
+  @ClassRule
+  public static RequiresGeodeHome requiresGeodeHome = new RequiresGeodeHome();
+
+  @ClassRule
+  public static ServerStarterRule server =
+      new ServerStarterRule().withRestService().withJMXManager().withAutoStart();
+
+  @Test
+  public void devRestIsAvailable() throws Exception {
+    GeodeDevRestClient client =
+        new GeodeDevRestClient("/geode/v1", "localhost", server.getHttpPort(), false);
+    client.doGetAndAssert("/servers").hasStatusCode(200);
+  }
+
+  @Test
+  public void adminRestIsAvailable() throws Exception {
+    GeodeDevRestClient client =
+        new GeodeDevRestClient("/geode-mgmt/v1", "localhost", server.getHttpPort(), false);
+    client.doGetAndAssert("/version").hasStatusCode(200);
+  }
+
+  @Test
+  public void pulseIsAvailable() throws Exception {
+    GeodeDevRestClient client =
+        new GeodeDevRestClient("/pulse", "localhost", server.getHttpPort(), false);
+    client.doGetAndAssert("/index.html").hasStatusCode(200);
+  }
+}
diff --git a/geode-assembly/src/integrationTest/java/org/apache/geode/management/internal/RegionManagementIntegrationTest.java b/geode-assembly/src/integrationTest/java/org/apache/geode/management/internal/RegionManagementIntegrationTest.java
index 7e0a8a2..8adf993 100644
--- a/geode-assembly/src/integrationTest/java/org/apache/geode/management/internal/RegionManagementIntegrationTest.java
+++ b/geode-assembly/src/integrationTest/java/org/apache/geode/management/internal/RegionManagementIntegrationTest.java
@@ -64,4 +64,9 @@ public class RegionManagementIntegrationTest {
     assertThat(result.getPersistenceStatus().getMessage())
         .isEqualTo("no members found to create cache element");
   }
+
+  @Test
+  public void ping() throws Exception {
+    restClient.doGetAndAssert("/ping").hasStatusCode(200).hasResponseBody().isEqualTo("pong");
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index f26342e..89d5599 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -67,6 +67,7 @@ import org.apache.geode.distributed.internal.tcpserver.TcpServer;
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.admin.remote.DistributionLocatorId;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.tier.sockets.TcpServerFactory;
 import org.apache.geode.internal.cache.wan.WANServiceProvider;
@@ -83,11 +84,8 @@ import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.net.SocketCreatorFactory;
 import org.apache.geode.internal.statistics.StatisticsConfig;
 import org.apache.geode.management.internal.AgentUtil;
-import org.apache.geode.management.internal.JettyHelper;
 import org.apache.geode.management.internal.JmxManagerLocator;
 import org.apache.geode.management.internal.JmxManagerLocatorRequest;
-import org.apache.geode.management.internal.ManagementAgent;
-import org.apache.geode.management.internal.SystemManagementService;
 import org.apache.geode.management.internal.api.ClusterManagementService;
 import org.apache.geode.management.internal.api.LocatorClusterManagementService;
 import org.apache.geode.management.internal.configuration.domain.SharedConfigurationStatus;
@@ -261,7 +259,8 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
    * startPeerLocation on the locator object.
    *
    * @param port the tcp/ip port to listen on
-   * @param loggingSession the LoggingSession to use, may be a NullLoggingSession which does nothing
+   * @param loggingSession the LoggingSession to use, may be a NullLoggingSession which does
+   *        nothing
    * @param logFile the file that log messages should be written to
    * @param logger a log writer that should be used (logFile parameter is ignored)
    * @param securityLogger the logger to be used for security related log messages
@@ -271,8 +270,10 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
    */
   public static InternalLocator createLocator(int port, LoggingSession loggingSession, File logFile,
       InternalLogWriter logger,
-      InternalLogWriter securityLogger, InetAddress bindAddress, String hostnameForClients,
-      Properties distributedSystemProperties, boolean startDistributedSystem) {
+      InternalLogWriter securityLogger,
+      InetAddress bindAddress, String hostnameForClients,
+      Properties distributedSystemProperties,
+      boolean startDistributedSystem) {
     synchronized (locatorLock) {
       if (hasLocator()) {
         throw new IllegalStateException(
@@ -312,8 +313,11 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
    * @param hostnameForClients the name to give to clients for connecting to this locator
    */
   public static InternalLocator startLocator(int port, File logFile, InternalLogWriter logger,
-      InternalLogWriter securityLogger, InetAddress bindAddress, boolean startDistributedSystem,
-      Properties dsProperties, String hostnameForClients) throws IOException {
+      InternalLogWriter securityLogger,
+      InetAddress bindAddress,
+      boolean startDistributedSystem,
+      Properties dsProperties, String hostnameForClients)
+      throws IOException {
     System.setProperty(FORCE_LOCATOR_DM_TYPE, "true");
     InternalLocator newLocator = null;
 
@@ -415,7 +419,8 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
    */
   private InternalLocator(int port, LoggingSession loggingSession, File logFile,
       InternalLogWriter logWriter, InternalLogWriter securityLogWriter,
-      InetAddress bindAddress, String hostnameForClients, Properties distributedSystemProperties,
+      InetAddress bindAddress, String hostnameForClients,
+      Properties distributedSystemProperties,
       DistributionConfigImpl cfg, boolean startDistributedSystem) {
 
     // TODO: the following three assignments are already done in superclass
@@ -572,8 +577,10 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
    */
   @Deprecated
   public static InternalLocator startLocator(int locatorPort, File logFile,
-      InternalLogWriter logger, InternalLogWriter logger1, InetAddress addr,
-      Properties dsProperties, boolean peerLocator, boolean serverLocator, String s, boolean b1)
+      InternalLogWriter logger, InternalLogWriter logger1,
+      InetAddress addr,
+      Properties dsProperties, boolean peerLocator,
+      boolean serverLocator, String s, boolean b1)
       throws IOException {
     return startLocator(locatorPort, logFile, logger, logger1, addr, true, dsProperties, s);
   }
@@ -602,7 +609,6 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
       sb.append('[').append(getPort()).append(']');
       String thisLocator = sb.toString();
 
-
       if (this.peerLocator) {
         // append this locator to the locators list from the config properties
         // this.logger.config("ensuring that this locator is in the locators list");
@@ -695,26 +701,20 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
       return;
     }
 
-    ManagementAgent managementAgent =
-        ((SystemManagementService) SystemManagementService.getExistingManagementService(myCache))
-            .getManagementAgent();
-
-    if (managementAgent == null) {
-      logger.info(
-          "management service needs to be started for ClusterManagementService to be running.");
-      return;
-    }
-
     Pair<String, Object> securityServiceAttr =
-        new ImmutablePair<>(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM,
+        new ImmutablePair<>(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM,
             myCache.getSecurityService());
     Pair<String, Object> cmServiceAttr =
-        new ImmutablePair<>(JettyHelper.CLUSTER_MANAGEMENT_SERVICE_CONTEXT_PARAM,
+        new ImmutablePair<>(HttpService.CLUSTER_MANAGEMENT_SERVICE_CONTEXT_PARAM,
             clusterManagementService);
-    managementWebapp =
-        managementAgent
-            .addWebApplication("/geode-management", gemfireManagementWar, securityServiceAttr,
-                cmServiceAttr);
+
+    try {
+      myCache.getHttpService()
+          .addWebApplication("/geode-management", gemfireManagementWar, securityServiceAttr,
+              cmServiceAttr);
+    } catch (Exception e) {
+      logger.error(e.getMessage(), e);
+    }
   }
 
   /**
@@ -722,7 +722,6 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
    * and distributed system are started.
    *
    * @param distributedSystem The distributed system to use for the statistics.
-   *
    * @since GemFire 5.7
    */
   void endStartLocator(InternalDistributedSystem distributedSystem) {
@@ -749,7 +748,6 @@ public class InternalLocator extends Locator implements ConnectListener, LogConf
    *
    * @param distributedSystem The distributed system which the server location services should use.
    *        If null, the method will try to find an already connected distributed system.
-   *
    * @since GemFire 5.7
    */
   void startServerLocation(InternalDistributedSystem distributedSystem) throws IOException {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index ba2501d..1268098 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -214,8 +214,10 @@ import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.LoggingExecutors;
 import org.apache.geode.internal.logging.LoggingThread;
 import org.apache.geode.internal.monitoring.ThreadsMonitoring;
+import org.apache.geode.internal.net.SSLConfigurationFactory;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.offheap.MemoryAllocator;
+import org.apache.geode.internal.security.SecurableCommunicationChannel;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.internal.security.SecurityServiceFactory;
 import org.apache.geode.internal.sequencelog.SequenceLoggerImpl;
@@ -602,6 +604,8 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
   private final ClusterConfigurationLoader ccLoader = new ClusterConfigurationLoader();
 
+  private HttpService httpService;
+
   static {
     // this works around jdk bug 6427854, reported in ticket #44434
     String propertyName = "sun.nio.ch.bugLevel";
@@ -871,11 +875,12 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
         this.securityService = SecurityServiceFactory.create();
       }
 
+      DistributionConfig systemConfig = system.getConfig();
       if (!this.isClient && PoolManager.getAll().isEmpty()) {
         // We only support management on members of a distributed system
         // Should do this: if (!getSystem().isLoner()) {
         // but it causes quickstart.CqClientTest to hang
-        boolean disableJmx = system.getConfig().getDisableJmx();
+        boolean disableJmx = systemConfig.getDisableJmx();
         if (disableJmx) {
           logger.info("Running with JMX disabled.");
         } else {
@@ -983,13 +988,22 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
       SystemFailure.signalCacheCreate();
 
-      this.diskMonitor = new DiskStoreMonitor(system.getConfig().getLogFile());
+      this.diskMonitor = new DiskStoreMonitor(systemConfig.getLogFile());
 
       addRegionEntrySynchronizationListener(new GatewaySenderQueueEntrySynchronizationListener());
       backupService = new BackupService(this);
+      if (!this.isClient) {
+        httpService = new HttpService(systemConfig.getHttpServiceBindAddress(),
+            systemConfig.getHttpServicePort(), SSLConfigurationFactory
+                .getSSLConfigForComponent(systemConfig, SecurableCommunicationChannel.WEB));
+      }
     } // synchronized
   }
 
+  public HttpService getHttpService() {
+    return httpService;
+  }
+
   @Override
   public void reLoadClusterConfiguration() throws IOException, ClassNotFoundException {
     this.configurationResponse = requestSharedConfiguration();
@@ -1290,7 +1304,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
   }
 
   private void startRestAgentServer(GemFireCacheImpl cache) {
-    if (this.system.getConfig().getStartDevRestApi() && isNotJmxManager() && isServerNode()) {
+    if (this.system.getConfig().getStartDevRestApi() && isServerNode()) {
       this.restAgent = new RestAgent(this.system.getConfig(), this.securityService);
       this.restAgent.start(cache);
     } else {
@@ -2214,7 +2228,9 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
           stopRedisServer();
 
-          stopRestAgentServer();
+          if (httpService != null) {
+            httpService.stop();
+          }
 
           // no need to track PR instances since we won't create any more
           // cacheServers or gatewayHubs
@@ -2480,14 +2496,6 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
       this.redisServer.shutdown();
   }
 
-  private void stopRestAgentServer() {
-    if (this.restAgent != null) {
-      logger.info("Rest Server on port {} is shutting down",
-          new Object[] {this.system.getConfig().getHttpServicePort()});
-      this.restAgent.stop();
-    }
-  }
-
   private void prepareDiskStoresForClose() {
     String pdxDSName = TypeRegistry.getPdxDiskStoreName(this);
     DiskStoreImpl pdxDiskStore = null;
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/JettyHelper.java b/geode-core/src/main/java/org/apache/geode/internal/cache/HttpService.java
similarity index 72%
rename from geode-core/src/main/java/org/apache/geode/management/internal/JettyHelper.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/HttpService.java
index afbb051..d248675 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/JettyHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/HttpService.java
@@ -12,12 +12,13 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.management.internal;
+package org.apache.geode.internal.cache;
 
 import java.io.File;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.CountDownLatch;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
@@ -38,25 +39,21 @@ import org.eclipse.jetty.webapp.WebAppContext;
 import org.apache.geode.GemFireConfigException;
 import org.apache.geode.internal.admin.SSLConfig;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.management.internal.SSLUtil;
 
-/**
- * @since GemFire 8.1
- */
-@SuppressWarnings("unused")
-public class JettyHelper {
+public class HttpService {
 
   private static final Logger logger = LogService.getLogger();
-
+  private Server httpServer;
+  private String bindAddress = "0.0.0.0";
+  private int port;
+  private SSLConfig sslConfig;
   private static final String FILE_PATH_SEPARATOR = System.getProperty("file.separator");
   private static final String USER_DIR = System.getProperty("user.dir");
   private static final String USER_NAME = System.getProperty("user.name");
 
   private static final String HTTPS = "https";
 
-  private static String bindAddress = "0.0.0.0";
-
-  private static int port = 0;
-
   public static final String SECURITY_SERVICE_SERVLET_CONTEXT_PARAM =
       "org.apache.geode.securityService";
 
@@ -64,13 +61,19 @@ public class JettyHelper {
   public static final String CLUSTER_MANAGEMENT_SERVICE_CONTEXT_PARAM =
       "org.apache.geode.sslConfig";
 
-  public static Server initJetty(final String bindAddress, final int port, SSLConfig sslConfig) {
+  private List<WebAppContext> webApps = new ArrayList<>();
 
-    final Server jettyServer = new Server();
+  public HttpService(String bindAddress, int port, SSLConfig sslConfig) {
+    if (port == 0) {
+      return;
+    }
+    this.sslConfig = sslConfig;
+
+    this.httpServer = new Server();
 
     // Add a handler collection here, so that each new context adds itself
     // to this collection.
-    jettyServer.setHandler(new HandlerCollection(true));
+    httpServer.setHandler(new HandlerCollection(true));
     ServerConnector connector = null;
 
     HttpConfiguration httpConfig = new HttpConfiguration();
@@ -99,7 +102,6 @@ public class JettyHelper {
         logger.warn("SSL Protocol could not be determined. SSL settings might not work correctly");
       }
 
-
       if (StringUtils.isBlank(sslConfig.getKeystore())) {
         throw new GemFireConfigException(
             "Key store can't be empty if SSL is enabled for HttpService");
@@ -134,43 +136,43 @@ public class JettyHelper {
 
       // Somehow With HTTP_2.0 Jetty throwing NPE. Need to investigate further whether all GemFire
       // web application(Pulse, REST) can do with HTTP_1.1
-      connector = new ServerConnector(jettyServer,
+      connector = new ServerConnector(httpServer,
           new SslConnectionFactory(sslContextFactory, HttpVersion.HTTP_1_1.asString()),
           new HttpConnectionFactory(httpConfig));
 
-
       connector.setPort(port);
     } else {
-      connector = new ServerConnector(jettyServer, new HttpConnectionFactory(httpConfig));
+      connector = new ServerConnector(httpServer, new HttpConnectionFactory(httpConfig));
 
       connector.setPort(port);
     }
 
-    jettyServer.setConnectors(new Connector[] {connector});
+    httpServer.setConnectors(new Connector[] {connector});
 
     if (StringUtils.isNotBlank(bindAddress)) {
       connector.setHost(bindAddress);
     }
 
-
     if (bindAddress != null && !bindAddress.isEmpty()) {
-      JettyHelper.bindAddress = bindAddress;
+      this.bindAddress = bindAddress;
     }
-
-    JettyHelper.port = port;
-
-    return jettyServer;
+    this.port = port;
   }
 
-
-  public static Server startJetty(final Server jetty) throws Exception {
-    jetty.start();
-    return jetty;
+  public Server getHttpServer() {
+    return httpServer;
   }
 
-  public static WebAppContext addWebApplication(final Server jetty, final String webAppContext,
-      final String warFilePath,
-      Pair<String, Object>... attributeNameValuePairs) {
+  public synchronized void addWebApplication(String webAppContext, String warFilePath,
+      Pair<String, Object>... attributeNameValuePairs)
+      throws Exception {
+    if (httpServer == null) {
+      logger.info(
+          String.format("unable to add %s webapp. Http service is not started on this member.",
+              webAppContext));
+      return;
+    }
+
     WebAppContext webapp = new WebAppContext();
     webapp.setContextPath(webAppContext);
     webapp.setWar(warFilePath);
@@ -186,20 +188,21 @@ public class JettyHelper {
     File tmpPath = new File(getWebAppBaseDirectory(webAppContext));
     tmpPath.mkdirs();
     webapp.setTempDirectory(tmpPath);
-    ((HandlerCollection) jetty.getHandler()).addHandler(webapp);
-    // if we are adding this webapp after the jetty server has already started, we will need to
-    // manually start the webapp.
-    if (jetty.isStarted()) {
-      try {
-        webapp.start();
-      } catch (Exception e) {
-        logger.error(e.getMessage(), e);
-      }
+    logger.info("Adding webapp " + webAppContext);
+    ((HandlerCollection) httpServer.getHandler()).addHandler(webapp);
+
+    // if the server is not started yet start the server, otherwise, start the webapp alone
+    if (!httpServer.isStarted()) {
+      logger.info("Attempting to start HTTP service on port ({}) at bind-address ({})...",
+          this.port, this.bindAddress);
+      httpServer.start();
+    } else {
+      webapp.start();
     }
-    return webapp;
+    webApps.add(webapp);
   }
 
-  private static String getWebAppBaseDirectory(final String context) {
+  private String getWebAppBaseDirectory(final String context) {
     String underscoredContext = context.replace("/", "_");
     String uuid = UUID.randomUUID().toString().substring(0, 8);
     final String workingDirectory = USER_DIR.concat(FILE_PATH_SEPARATOR)
@@ -211,15 +214,30 @@ public class JettyHelper {
     return workingDirectory;
   }
 
-  private static final CountDownLatch latch = new CountDownLatch(1);
-
-  private static String normalizeWebAppArchivePath(final String webAppArchivePath) {
-    return (webAppArchivePath.startsWith(File.separator) ? new File(webAppArchivePath)
-        : new File(".", webAppArchivePath)).getAbsolutePath();
-  }
+  public void stop() {
+    if (this.httpServer == null) {
+      return;
+    }
 
-  private static String normalizeWebAppContext(final String webAppContext) {
-    return (webAppContext.startsWith("/") ? webAppContext : "/" + webAppContext);
+    logger.debug("Stopping the HTTP service...");
+    try {
+      for (WebAppContext webapp : webApps) {
+        webapp.stop();
+      }
+      this.httpServer.stop();
+    } catch (Exception e) {
+      logger.warn("Failed to stop the HTTP service because: {}", e.getMessage(), e);
+    } finally {
+      try {
+        this.httpServer.destroy();
+      } catch (Exception ignore) {
+        logger.info("Failed to properly release resources held by the HTTP service: {}",
+            ignore.getMessage(), ignore);
+      } finally {
+        this.httpServer = null;
+        System.clearProperty("catalina.base");
+        System.clearProperty("catalina.home");
+      }
+    }
   }
-
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
index d8ca5af..3d129f9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
@@ -370,4 +370,6 @@ public interface InternalCache extends Cache, Extensible<Cache>, CacheTime {
    * by Geode will not be accessible from the returned cache.
    */
   InternalCacheForClientAccess getCacheForProcessingClientRequests();
+
+  HttpService getHttpService();
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java
index 759aa1f..9a305f3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java
@@ -1218,4 +1218,9 @@ public class InternalCacheForClientAccess implements InternalCache {
   public InternalCacheForClientAccess getCacheForProcessingClientRequests() {
     return this;
   }
+
+  @Override
+  public HttpService getHttpService() {
+    return delegate.getHttpService();
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
index b58f8a4..9ee0604 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
@@ -119,6 +119,7 @@ import org.apache.geode.internal.cache.DistributedRegion;
 import org.apache.geode.internal.cache.ExpirationScheduler;
 import org.apache.geode.internal.cache.FilterProfile;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.cache.InitialImageOperation;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalCacheForClientAccess;
@@ -2422,4 +2423,9 @@ public class CacheCreation implements InternalCache {
   public InternalCacheForClientAccess getCacheForProcessingClientRequests() {
     throw new UnsupportedOperationException("Should not be invoked");
   }
+
+  @Override
+  public HttpService getHttpService() {
+    throw new UnsupportedOperationException("Should not be invoked");
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java b/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java
index 62cb055..ed99159 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/ManagementAgent.java
@@ -47,15 +47,12 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.logging.log4j.Logger;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.webapp.WebAppContext;
 
 import org.apache.geode.GemFireConfigException;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.admin.SSLConfig;
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.net.SSLConfigurationFactory;
@@ -100,7 +97,6 @@ public class ManagementAgent {
   private final DistributionConfig config;
   private final SecurityService securityService;
   private final InternalCache cache;
-  private boolean isHttpServiceRunning = false;
   private RMIClientSocketFactory rmiClientSocketFactory;
   private RMIServerSocketFactory rmiServerSocketFactory;
   private int port;
@@ -126,38 +122,9 @@ public class ManagementAgent {
     return this.running;
   }
 
-  synchronized boolean isHttpServiceRunning() {
-    return isHttpServiceRunning;
-  }
-
-  private synchronized void setHttpServiceRunning(boolean isHttpServiceRunning) {
-    this.isHttpServiceRunning = isHttpServiceRunning;
-  }
-
-  private boolean isAPIRestServiceRunning(InternalCache cache) {
-    return (cache != null && cache.getRestAgent() != null && cache.getRestAgent().isRunning());
-  }
 
-  private boolean isServerNode(InternalCache cache) {
-    return (cache.getInternalDistributedSystem().getDistributedMember()
-        .getVmKind() != ClusterDistributionManager.LOCATOR_DM_TYPE
-        && cache.getInternalDistributedSystem().getDistributedMember()
-            .getVmKind() != ClusterDistributionManager.ADMIN_ONLY_DM_TYPE
-        && !cache.isClient());
-  }
-
-  public synchronized void startAgent(InternalCache cache) {
-    // Do not start Management REST service if developer REST service is already
-    // started.
-
-    if (!isAPIRestServiceRunning(cache)) {
-      startHttpService(isServerNode(cache));
-    } else {
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "Developer REST APIs webapp is already running, Not Starting M&M REST and pulse!");
-      }
-    }
+  public synchronized void startAgent() {
+    loadWebApplications();
 
     if (!this.running && this.config.getJmxManagerPort() != 0) {
       try {
@@ -170,8 +137,6 @@ public class ManagementAgent {
   }
 
   public synchronized void stopAgent() {
-    stopHttpService();
-
     if (!this.running) {
       return;
     }
@@ -189,123 +154,70 @@ public class ManagementAgent {
     this.running = false;
   }
 
-  public WebAppContext addWebApplication(String webAppContext, String warFilePath,
-      Pair<String, Object>... attributeNameValuePairs) {
-    if (httpServer == null) {
-      logger.info(
-          String.format("unable to add %s webapp. Http service is not started on this member.",
-              webAppContext));
-      return null;
-    }
-
-    return JettyHelper
-        .addWebApplication(httpServer, webAppContext, warFilePath, attributeNameValuePairs);
-  }
-
-  private Server httpServer;
   private final String GEMFIRE_VERSION = GemFireVersion.getGemFireVersion();
   private final AgentUtil agentUtil = new AgentUtil(GEMFIRE_VERSION);
 
-  private void startHttpService(boolean isServer) {
+  private void loadWebApplications() {
     final SystemManagementService managementService = (SystemManagementService) ManagementService
         .getManagementService(cache);
 
     final ManagerMXBean managerBean = managementService.getManagerMXBean();
 
-    if (this.config.getHttpServicePort() != 0) {
-      if (logger.isDebugEnabled()) {
-        logger.debug("Attempting to start HTTP service on port ({}) at bind-address ({})...",
-            this.config.getHttpServicePort(), this.config.getHttpServiceBindAddress());
-      }
+    if (this.config.getHttpServicePort() == 0) {
+      setStatusMessage(managerBean,
+          "Embedded HTTP server configured not to start (http-service-port=0) or (jmx-manager-http-port=0)");
+      return;
+    }
 
-      // Find the Management WAR file
-      final String gemfireWar = agentUtil.findWarLocation("geode-web");
-      if (gemfireWar == null) {
-        if (logger.isDebugEnabled()) {
-          logger.debug(
-              "Unable to find Geode V1 Management REST API WAR file; the Management REST Interface for Geode will not be accessible.");
-        }
+    // Find the Management rest WAR file
+    final String adminRestWar = agentUtil.findWarLocation("geode-web");
+    if (adminRestWar == null) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "Unable to find Geode V1 Management REST API WAR file; the Management REST Interface for Geode will not be accessible.");
       }
+    }
 
-      // Find the Pulse WAR file
-      final String pulseWar = agentUtil.findWarLocation("geode-pulse");
+    // Find the Pulse WAR file
+    final String pulseWar = agentUtil.findWarLocation("geode-pulse");
 
-      if (pulseWar == null) {
-        final String message =
-            "Unable to find Pulse web application WAR file; Pulse for Geode will not be accessible";
-        setStatusMessage(managerBean, message);
-        if (logger.isDebugEnabled()) {
-          logger.debug(message);
-        }
-      } else {
-        String pwFile = this.config.getJmxManagerPasswordFile();
-        if (securityService.isIntegratedSecurity() || StringUtils.isNotBlank(pwFile)) {
-          System.setProperty("spring.profiles.active", "pulse.authentication.gemfire");
-        }
+    if (pulseWar == null) {
+      final String message =
+          "Unable to find Pulse web application WAR file; Pulse for Geode will not be accessible";
+      setStatusMessage(managerBean, message);
+      if (logger.isDebugEnabled()) {
+        logger.debug(message);
       }
-
-      // Find developer REST WAR file
-      final String gemfireAPIWar = agentUtil.findWarLocation("geode-web-api");
-      if (gemfireAPIWar == null) {
-        final String message =
-            "Unable to find Geode Developer REST API WAR file; the Developer REST Interface for Geode will not be accessible.";
-        setStatusMessage(managerBean, message);
-        if (logger.isDebugEnabled()) {
-          logger.debug(message);
-        }
+    } else {
+      String pwFile = this.config.getJmxManagerPasswordFile();
+      if (securityService.isIntegratedSecurity() || StringUtils.isNotBlank(pwFile)) {
+        System.setProperty("spring.profiles.active", "pulse.authentication.gemfire");
       }
+    }
 
-      try {
-        if (agentUtil.isAnyWarFileAvailable(gemfireWar, pulseWar, gemfireAPIWar)) {
-
-          final String bindAddress = this.config.getHttpServiceBindAddress();
-          final int port = this.config.getHttpServicePort();
-
-          boolean isRestWebAppAdded = false;
-
-          this.httpServer = JettyHelper.initJetty(bindAddress, port, SSLConfigurationFactory
-              .getSSLConfigForComponent(config, SecurableCommunicationChannel.WEB));
-
-          Pair<String, Object> securityServiceAttr =
-              new ImmutablePair<>(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM,
-                  securityService);
-          Pair<String, Object> sslConfigAttr =
-              new ImmutablePair<>(JettyHelper.GEODE_SSLCONFIG_SERVLET_CONTEXT_PARAM,
-                  createSslProps());
-
-          // if jmx manager is running, admin rest should be available, either on locator or server
-          if (agentUtil.isAnyWarFileAvailable(gemfireWar)) {
-            addWebApplication("/gemfire", gemfireWar, securityServiceAttr);
-            addWebApplication("/geode-mgmt", gemfireWar, securityServiceAttr);
-          }
-
-          // if jmx manager is running, pulse should be available, either on locator or server
-          // we need to pass in the sllConfig to pulse because it needs it to make jmx connection
-          if (agentUtil.isAnyWarFileAvailable(pulseWar)) {
-            addWebApplication("/pulse", pulseWar, securityServiceAttr, sslConfigAttr);
-          }
-
-          // the dev rest api is only available on servers
-          if (isServer && this.config.getStartDevRestApi()) {
-            if (agentUtil.isAnyWarFileAvailable(gemfireAPIWar)) {
-              addWebApplication("/geode", gemfireAPIWar, securityServiceAttr);
-              addWebApplication("/gemfire-api", gemfireAPIWar, securityServiceAttr);
-              isRestWebAppAdded = true;
-            }
-          } else {
-            final String message =
-                "Developer REST API web application will not start when start-dev-rest-api is not set and node is not server";
-            setStatusMessage(managerBean, message);
-            if (logger.isDebugEnabled()) {
-              logger.debug(message);
-            }
-          }
-
-          if (logger.isDebugEnabled()) {
-            logger.debug("Starting HTTP embedded server on port ({}) at bind-address ({})...",
-                ((ServerConnector) this.httpServer.getConnectors()[0]).getPort(), bindAddress);
-          }
+    try {
+      if (agentUtil.isAnyWarFileAvailable(adminRestWar, pulseWar)) {
+
+        final String bindAddress = this.config.getHttpServiceBindAddress();
+        final int port = this.config.getHttpServicePort();
+
+        Pair<String, Object> securityServiceAttr =
+            new ImmutablePair<>(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM,
+                securityService);
+        Pair<String, Object> sslConfigAttr =
+            new ImmutablePair<>(HttpService.GEODE_SSLCONFIG_SERVLET_CONTEXT_PARAM,
+                createSslProps());
+
+        HttpService httpService = cache.getHttpService();
+        // if jmx manager is running, admin rest should be available, either on locator or server
+        if (agentUtil.isAnyWarFileAvailable(adminRestWar)) {
+          httpService.addWebApplication("/gemfire", adminRestWar, securityServiceAttr);
+          httpService.addWebApplication("/geode-mgmt", adminRestWar, securityServiceAttr);
+        }
 
+        // if jmx manager is running, pulse should be available, either on locator or server
+        // we need to pass in the sllConfig to pulse because it needs it to make jmx connection
+        if (agentUtil.isAnyWarFileAvailable(pulseWar)) {
           System.setProperty(PULSE_EMBEDDED_PROP, "true");
           System.setProperty(PULSE_HOST_PROP, "" + config.getJmxManagerBindAddress());
           System.setProperty(PULSE_PORT_PROP, "" + config.getJmxManagerPort());
@@ -316,38 +228,16 @@ public class ManagementAgent {
               .getSocketCreatorForComponent(SecurableCommunicationChannel.LOCATOR);
           System.setProperty(PULSE_USESSL_MANAGER, jmxSocketCreator.useSSL() + "");
           System.setProperty(PULSE_USESSL_LOCATOR, locatorSocketCreator.useSSL() + "");
+          httpService.addWebApplication("/pulse", pulseWar, securityServiceAttr, sslConfigAttr);
 
-          this.httpServer = JettyHelper.startJetty(this.httpServer);
-
-          // now, that Tomcat has been started, we can set the URL used by web
-          // clients to connect to Pulse
-          if (agentUtil.isAnyWarFileAvailable(pulseWar)) {
-            managerBean.setPulseURL("http://".concat(getHost(bindAddress)).concat(":")
-                .concat(String.valueOf(port)).concat("/pulse/"));
-          }
-
-          // set cache property for developer REST service running
-          if (isRestWebAppAdded) {
-            cache.setRESTServiceRunning(true);
-            // create region to hold query information (queryId, queryString).
-            // Added for the developer REST APIs
-            RestAgent.createParameterizedQueryRegion();
-          }
-
-          // set true for HTTP service running
-          setHttpServiceRunning(true);
+          managerBean.setPulseURL("http://".concat(getHost(bindAddress)).concat(":")
+              .concat(String.valueOf(port)).concat("/pulse/"));
         }
-      } catch (Exception e) {
-        stopHttpService();// Jetty needs to be stopped even if it has failed to
-        // start. Some of the threads are left behind even if
-        // server.start() fails due to an exception
-        setStatusMessage(managerBean, "HTTP service failed to start with "
-            + e.getClass().getSimpleName() + " '" + e.getMessage() + "'");
-        throw new ManagementException("HTTP service failed to start", e);
       }
-    } else {
-      setStatusMessage(managerBean,
-          "Embedded HTTP server configured not to start (http-service-port=0) or (jmx-manager-http-port=0)");
+    } catch (Exception e) {
+      setStatusMessage(managerBean, "HTTP service failed to start with "
+          + e.getClass().getSimpleName() + " '" + e.getMessage() + "'");
+      throw new ManagementException("HTTP service failed to start", e);
     }
   }
 
@@ -403,30 +293,6 @@ public class ManagementAgent {
     mBean.setStatusMessage(message);
   }
 
-  private void stopHttpService() {
-    if (this.httpServer != null) {
-      if (logger.isDebugEnabled()) {
-        logger.debug("Stopping the HTTP service...");
-      }
-      try {
-        this.httpServer.stop();
-      } catch (Exception e) {
-        logger.warn("Failed to stop the HTTP service because: {}", e.getMessage(), e);
-      } finally {
-        try {
-          this.httpServer.destroy();
-        } catch (Exception ignore) {
-          logger.info("Failed to properly release resources held by the HTTP service: {}",
-              ignore.getMessage(), ignore);
-        } finally {
-          this.httpServer = null;
-          System.clearProperty("catalina.base");
-          System.clearProperty("catalina.home");
-        }
-      }
-    }
-  }
-
   /**
    * http://docs.oracle.com/javase/6/docs/technotes/guides/management/agent.html #gdfvq
    * https://blogs.oracle.com/jmxetc/entry/java_5_premain_rmi_connectors
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 26af1b6..7dc06f6 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
@@ -20,8 +20,6 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.logging.log4j.Logger;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.ServerConnector;
 
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.CacheFactory;
@@ -30,14 +28,12 @@ import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.GemFireVersion;
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.InternalRegionArguments;
 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.security.SecurableCommunicationChannel;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.ManagementService;
 
 /**
  * Agent implementation that controls the HTTP server end points used for REST clients to connect
@@ -64,18 +60,11 @@ public class RestAgent {
     return this.running;
   }
 
-  private boolean isManagementRestServiceRunning(InternalCache cache) {
-    final SystemManagementService managementService =
-        (SystemManagementService) ManagementService.getManagementService(cache);
-    return (managementService.getManagementAgent() != null
-        && managementService.getManagementAgent().isHttpServiceRunning());
-  }
 
   public synchronized void start(InternalCache cache) {
-    if (!this.running && this.config.getHttpServicePort() != 0
-        && !isManagementRestServiceRunning(cache)) {
+    if (!this.running && this.config.getHttpServicePort() != 0) {
       try {
-        startHttpService();
+        startHttpService(cache);
         this.running = true;
         cache.setRESTServiceRunning(true);
 
@@ -89,21 +78,6 @@ public class RestAgent {
     }
   }
 
-  public synchronized void stop() {
-    if (this.running) {
-      stopHttpService();
-      if (logger.isDebugEnabled()) {
-        logger.debug("Gemfire Rest Http service stopped");
-      }
-      this.running = false;
-    } else {
-      if (logger.isDebugEnabled()) {
-        logger.debug("Attempt to stop Gemfire Rest Http service which is not running");
-      }
-    }
-  }
-
-  private Server httpServer;
   private final String GEMFIRE_VERSION = GemFireVersion.getGemFireVersion();
   private AgentUtil agentUtil = new AgentUtil(GEMFIRE_VERSION);
 
@@ -113,13 +87,7 @@ public class RestAgent {
   }
 
   // Start HTTP service in embedded mode
-  public void startHttpService() {
-    // TODO: add a check that will make sure that we start HTTP service on
-    // non-manager data node
-    String httpServiceBindAddress = getBindAddressForHttpService(this.config);
-    logger.info("Attempting to start HTTP service on port ({}) at bind-address ({})...",
-        this.config.getHttpServicePort(), httpServiceBindAddress);
-
+  public void startHttpService(InternalCache cache) {
     // Find the developer REST WAR file
     final String gemfireAPIWar = agentUtil.findWarLocation("geode-web-api");
     if (gemfireAPIWar == null) {
@@ -134,32 +102,16 @@ public class RestAgent {
             "Detected presence of catalina system properties. HTTP service will not be started. To enable the GemFire Developer REST API, please deploy the /geode-web-api WAR file in your application server.");
       } else if (agentUtil.isAnyWarFileAvailable(gemfireAPIWar)) {
 
-        final int port = this.config.getHttpServicePort();
-
-        this.httpServer = JettyHelper.initJetty(httpServiceBindAddress, port,
-            SSLConfigurationFactory.getSSLConfigForComponent(SecurableCommunicationChannel.WEB));
-
         Pair<String, Object> securityServiceAttr =
-            new ImmutablePair<>(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM,
+            new ImmutablePair<>(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM,
                 securityService);
 
-        JettyHelper
-            .addWebApplication(httpServer, "/gemfire-api", gemfireAPIWar, securityServiceAttr);
-        JettyHelper.addWebApplication(httpServer, "/geode", gemfireAPIWar, securityServiceAttr);
-
-        if (logger.isDebugEnabled()) {
-          logger.debug("Starting HTTP embedded server on port ({}) at bind-address ({})...",
-              ((ServerConnector) this.httpServer.getConnectors()[0]).getPort(),
-              httpServiceBindAddress);
-        }
-
-        JettyHelper.startJetty(this.httpServer);
-        logger.info("HTTP service started successfully...!!");
+        HttpService httpService = cache.getHttpService();
+        httpService
+            .addWebApplication("/gemfire-api", gemfireAPIWar, securityServiceAttr);
+        httpService.addWebApplication("/geode", gemfireAPIWar, securityServiceAttr);
       }
     } catch (Exception e) {
-      stopHttpService();// Jetty needs to be stopped even if it has failed to
-                        // start. Some of the threads are left behind even if
-                        // server.start() fails due to an exception
       throw new RuntimeException("HTTP service failed to start due to " + e.getMessage());
     }
   }
@@ -187,28 +139,6 @@ public class RestAgent {
     return bindAddress;
   }
 
-  private void stopHttpService() {
-    if (this.httpServer != null) {
-      logger.info("Stopping the HTTP service...");
-      try {
-        this.httpServer.stop();
-      } catch (Exception e) {
-        logger.warn("Failed to stop the HTTP service because: {}", e.getMessage(), e);
-      } finally {
-        try {
-          this.httpServer.destroy();
-        } catch (Exception ignore) {
-          logger.error("Failed to properly release resources held by the HTTP service: {}",
-              ignore.getMessage(), ignore);
-        } finally {
-          this.httpServer = null;
-          System.clearProperty("catalina.base");
-          System.clearProperty("catalina.home");
-        }
-      }
-    }
-  }
-
   /**
    * This method will create a REPLICATED region named _ParameterizedQueries__. In developer REST
    * APIs, this region will be used to store the queryId and queryString as a key and value
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java b/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
index 281d889..023f182 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/SystemManagementService.java
@@ -429,7 +429,7 @@ public class SystemManagementService extends BaseManagementService {
           system.handleResourceEvent(ResourceEvent.MANAGER_START, null);
           federatingManager.startManager();
           if (this.agent != null) {
-            this.agent.startAgent(getInternalCache());
+            this.agent.startAgent();
           }
           getInternalCache().getJmxManagerAdvisor().broadcastChange();
           started = true;
@@ -479,7 +479,7 @@ public class SystemManagementService extends BaseManagementService {
         federatingManager.stopManager();
         system.handleResourceEvent(ResourceEvent.MANAGER_STOP, null);
         getInternalCache().getJmxManagerAdvisor().broadcastChange();
-        if (this.agent != null && (this.agent.isRunning() || this.agent.isHttpServiceRunning())) {
+        if (this.agent != null && this.agent.isRunning()) {
           this.agent.stopAgent();
         }
       }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/support/LoginHandlerInterceptor.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/support/LoginHandlerInterceptor.java
index a28cdad..53a1c50 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/support/LoginHandlerInterceptor.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/support/LoginHandlerInterceptor.java
@@ -30,9 +30,9 @@ import org.springframework.web.servlet.handler.HandlerInterceptorAdapter;
 
 import org.apache.geode.annotations.VisibleForTesting;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.internal.JettyHelper;
 import org.apache.geode.management.internal.security.ResourceConstants;
 
 /**
@@ -128,6 +128,6 @@ public class LoginHandlerInterceptor extends HandlerInterceptorAdapter
   @Override
   public void setServletContext(ServletContext servletContext) {
     securityService = (SecurityService) servletContext
-        .getAttribute(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
+        .getAttribute(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/JettyHelperJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/HttpServiceJunitTest.java
similarity index 75%
rename from geode-core/src/test/java/org/apache/geode/management/internal/JettyHelperJUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/management/internal/HttpServiceJunitTest.java
index 7ecc227..7907caa 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/JettyHelperJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/HttpServiceJunitTest.java
@@ -18,7 +18,6 @@ import static org.assertj.core.api.Assertions.assertThat;
 
 import java.util.Properties;
 
-import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
 import org.junit.After;
 import org.junit.Before;
@@ -26,17 +25,17 @@ import org.junit.Test;
 
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.net.SSLConfigurationFactory;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
 
 /**
- * The JettyHelperJUnitTest class is a test suite of test cases testing the contract and
+ * The HttpServiceJunitTest class is a test suite of test cases testing the contract and
  * functionality of the JettyHelper class. Does not start Jetty.
  *
- * @see org.apache.geode.management.internal.JettyHelper
  * @see org.junit.Test
  */
-public class JettyHelperJUnitTest {
+public class HttpServiceJunitTest {
   private DistributionConfig distributionConfig;
 
   @Before
@@ -52,19 +51,21 @@ public class JettyHelperJUnitTest {
 
   @Test
   public void testSetPortNoBindAddress() {
-    final Server jetty = JettyHelper.initJetty(null, 8090, SSLConfigurationFactory
+    final HttpService jetty = new HttpService(null, 8090, SSLConfigurationFactory
         .getSSLConfigForComponent(distributionConfig, SecurableCommunicationChannel.WEB));
     assertThat(jetty).isNotNull();
-    assertThat(jetty.getConnectors()[0]).isNotNull();
-    assertThat(((ServerConnector) jetty.getConnectors()[0]).getPort()).isEqualTo(8090);
+    assertThat(jetty.getHttpServer().getConnectors()[0]).isNotNull();
+    assertThat(((ServerConnector) jetty.getHttpServer().getConnectors()[0]).getPort())
+        .isEqualTo(8090);
   }
 
   @Test
   public void testSetPortWithBindAddress() {
-    final Server jetty = JettyHelper.initJetty("10.123.50.1", 10480, SSLConfigurationFactory
+    final HttpService jetty = new HttpService("10.123.50.1", 10480, SSLConfigurationFactory
         .getSSLConfigForComponent(distributionConfig, SecurableCommunicationChannel.WEB));
     assertThat(jetty).isNotNull();
-    assertThat(jetty.getConnectors()[0]).isNotNull();
-    assertThat(((ServerConnector) jetty.getConnectors()[0]).getPort()).isEqualTo(10480);
+    assertThat(jetty.getHttpServer().getConnectors()[0]).isNotNull();
+    assertThat(((ServerConnector) jetty.getHttpServer().getConnectors()[0]).getPort())
+        .isEqualTo(10480);
   }
 }
diff --git a/geode-dunit/src/main/java/org/apache/geode/cache/client/internal/LocatorTestBase.java b/geode-dunit/src/main/java/org/apache/geode/cache/client/internal/LocatorTestBase.java
index 86996fe..62ae405 100644
--- a/geode-dunit/src/main/java/org/apache/geode/cache/client/internal/LocatorTestBase.java
+++ b/geode-dunit/src/main/java/org/apache/geode/cache/client/internal/LocatorTestBase.java
@@ -16,6 +16,7 @@ package org.apache.geode.cache.client.internal;
 
 import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
 import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_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;
@@ -46,6 +47,7 @@ import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.cache.server.ServerLoadProbe;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.Locator;
+import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.cache.PoolFactoryImpl;
 import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.Invoke;
@@ -115,13 +117,14 @@ public abstract class LocatorTestBase extends JUnit4DistributedTestCase {
   protected void postTearDownLocatorTestBase() throws Exception {}
 
   protected int startLocator(final String hostName, final String otherLocators) throws Exception {
-    final String testName = getUniqueName();
     disconnectFromDS();
+    final int httpPort = AvailablePortHelper.getRandomAvailableTCPPort();
     Properties props = new Properties();
     props.put(MCAST_PORT, String.valueOf(0));
     props.put(LOCATORS, otherLocators);
     props.put(LOG_LEVEL, LogWriterUtils.getDUnitLogLevel());
     props.put(ENABLE_CLUSTER_CONFIGURATION, "false");
+    props.put(HTTP_SERVICE_PORT, String.valueOf(httpPort));
     File logFile = new File("");
     InetAddress bindAddr = null;
     try {
diff --git a/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java b/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java
index 95dea38..74095c2 100644
--- a/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java
+++ b/geode-junit/src/main/java/org/apache/geode/test/fake/Fakes.java
@@ -38,6 +38,7 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.TXManagerImpl;
+import org.apache.geode.internal.security.SecurableCommunicationChannel;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.pdx.PdxInstanceFactory;
 import org.apache.geode.pdx.internal.TypeRegistry;
@@ -67,6 +68,8 @@ public class Fakes {
     GemFireCacheImpl cache = mock(GemFireCacheImpl.class);
     InternalDistributedSystem system = mock(InternalDistributedSystem.class);
     DistributionConfig config = mock(DistributionConfig.class);
+    when(config.getSecurableCommunicationChannels())
+        .thenReturn(new SecurableCommunicationChannel[] {SecurableCommunicationChannel.ALL});
     ClusterDistributionManager distributionManager = mock(ClusterDistributionManager.class);
     PdxInstanceFactory pdxInstanceFactory = mock(PdxInstanceFactory.class);
     TypeRegistry pdxRegistryMock = mock(TypeRegistry.class);
diff --git a/geode-pulse/geode-pulse-test/src/main/java/org/apache/geode/tools/pulse/tests/rules/ServerRule.java b/geode-pulse/geode-pulse-test/src/main/java/org/apache/geode/tools/pulse/tests/rules/ServerRule.java
index 509ad6f..e67ce31 100644
--- a/geode-pulse/geode-pulse-test/src/main/java/org/apache/geode/tools/pulse/tests/rules/ServerRule.java
+++ b/geode-pulse/geode-pulse-test/src/main/java/org/apache/geode/tools/pulse/tests/rules/ServerRule.java
@@ -15,7 +15,6 @@
 package org.apache.geode.tools.pulse.tests.rules;
 
 
-import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -25,7 +24,7 @@ import org.junit.rules.ExternalResource;
 
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.admin.SSLConfig;
-import org.apache.geode.management.internal.JettyHelper;
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.tools.pulse.internal.data.PulseConstants;
 import org.apache.geode.tools.pulse.tests.Server;
 
@@ -33,7 +32,7 @@ public class ServerRule extends ExternalResource {
   private static final String LOCALHOST = "localhost";
   private static final String PULSE_CONTEXT = "/pulse/";
 
-  private org.eclipse.jetty.server.Server jetty;
+  private HttpService jetty;
   private Server server;
   private String pulseURL;
   private String jsonAuthFile;
@@ -50,7 +49,6 @@ public class ServerRule extends ExternalResource {
   protected void before() throws Throwable {
     startServer();
     startJetty();
-    await().until(() -> jetty.isStarted());
   }
 
   @Override
@@ -79,11 +77,9 @@ public class ServerRule extends ExternalResource {
         String.valueOf(Boolean.TRUE));
 
     int httpPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    jetty = JettyHelper.initJetty(LOCALHOST, httpPort, new SSLConfig());
-    JettyHelper.addWebApplication(jetty, PULSE_CONTEXT, getPulseWarPath());
+    jetty = new HttpService(LOCALHOST, httpPort, new SSLConfig());
+    jetty.addWebApplication(PULSE_CONTEXT, getPulseWarPath());
     pulseURL = "http://" + LOCALHOST + ":" + httpPort + PULSE_CONTEXT;
-    System.out.println("Pulse started at " + pulseURL);
-    jetty.start();
   }
 
   private void stopServer() throws Exception {
diff --git a/geode-web-api/src/integrationTest/java/org/apache/geode/rest/internal/web/controllers/RestAccessControllerTest.java b/geode-web-api/src/integrationTest/java/org/apache/geode/rest/internal/web/controllers/RestAccessControllerTest.java
index ec177bf..023a41f 100644
--- a/geode-web-api/src/integrationTest/java/org/apache/geode/rest/internal/web/controllers/RestAccessControllerTest.java
+++ b/geode-web-api/src/integrationTest/java/org/apache/geode/rest/internal/web/controllers/RestAccessControllerTest.java
@@ -70,7 +70,7 @@ import org.apache.geode.cache.RegionEvent;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.TimeoutException;
 import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.management.internal.JettyHelper;
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.management.internal.RestAgent;
 import org.apache.geode.test.junit.rules.ServerStarterRule;
 
@@ -935,7 +935,7 @@ class TestContextLoader extends GenericXmlWebContextLoader {
   protected void loadBeanDefinitions(GenericWebApplicationContext context,
       WebMergedContextConfiguration webMergedConfig) {
     super.loadBeanDefinitions(context, webMergedConfig);
-    context.getServletContext().setAttribute(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM,
+    context.getServletContext().setAttribute(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM,
         RestAccessControllerTest.rule.getCache().getSecurityService());
   }
 
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/security/GeodeAuthenticationProvider.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/security/GeodeAuthenticationProvider.java
index db5bf5c..1789ee6 100644
--- a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/security/GeodeAuthenticationProvider.java
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/security/GeodeAuthenticationProvider.java
@@ -29,8 +29,8 @@ import org.springframework.security.core.authority.AuthorityUtils;
 import org.springframework.stereotype.Component;
 import org.springframework.web.context.ServletContextAware;
 
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.internal.JettyHelper;
 import org.apache.geode.management.internal.security.ResourceConstants;
 import org.apache.geode.security.GemFireSecurityException;
 
@@ -71,6 +71,6 @@ public class GeodeAuthenticationProvider implements AuthenticationProvider, Serv
   @Override
   public void setServletContext(ServletContext servletContext) {
     securityService = (SecurityService) servletContext
-        .getAttribute(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
+        .getAttribute(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
   }
 }
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/security/RestSecurityService.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/security/RestSecurityService.java
index 4020ddc..4e9cad5 100644
--- a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/security/RestSecurityService.java
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/security/RestSecurityService.java
@@ -19,8 +19,8 @@ import javax.servlet.ServletContext;
 import org.springframework.stereotype.Component;
 import org.springframework.web.context.ServletContextAware;
 
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.internal.JettyHelper;
 import org.apache.geode.security.GemFireSecurityException;
 import org.apache.geode.security.ResourcePermission;
 import org.apache.geode.security.ResourcePermission.Operation;
@@ -78,6 +78,6 @@ public class RestSecurityService implements ServletContextAware {
   @Override
   public void setServletContext(ServletContext servletContext) {
     securityService = (SecurityService) servletContext
-        .getAttribute(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
+        .getAttribute(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
   }
 }
diff --git a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/AbstractManagementController.java b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/AbstractManagementController.java
index e03e713..7012ee3 100644
--- a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/AbstractManagementController.java
+++ b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/AbstractManagementController.java
@@ -29,9 +29,9 @@ import org.springframework.web.bind.annotation.ExceptionHandler;
 import org.springframework.web.bind.annotation.InitBinder;
 import org.springframework.web.context.ServletContextAware;
 
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.internal.JettyHelper;
 import org.apache.geode.management.internal.api.ClusterManagementResult;
 import org.apache.geode.management.internal.api.LocatorClusterManagementService;
 import org.apache.geode.security.AuthenticationFailedException;
@@ -46,9 +46,9 @@ public class AbstractManagementController implements ServletContextAware {
   @Override
   public void setServletContext(ServletContext servletContext) {
     securityService = (SecurityService) servletContext
-        .getAttribute(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
+        .getAttribute(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
     clusterManagementService = (LocatorClusterManagementService) servletContext
-        .getAttribute(JettyHelper.CLUSTER_MANAGEMENT_SERVICE_CONTEXT_PARAM);
+        .getAttribute(HttpService.CLUSTER_MANAGEMENT_SERVICE_CONTEXT_PARAM);
   }
 
   private static final Logger logger = LogService.getLogger();
diff --git a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java
index d6bc916..f2ec03e 100644
--- a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java
+++ b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/controllers/RegionManagementController.java
@@ -41,4 +41,9 @@ public class RegionManagementController extends AbstractManagementController {
     return new ResponseEntity<>(result,
         result.isSuccessful() ? HttpStatus.CREATED : HttpStatus.INTERNAL_SERVER_ERROR);
   }
+
+  @RequestMapping(method = RequestMethod.GET, value = "/ping")
+  public ResponseEntity<String> ping() {
+    return new ResponseEntity<>("pong", HttpStatus.OK);
+  }
 }
diff --git a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/security/GeodeAuthenticationProvider.java b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/security/GeodeAuthenticationProvider.java
index 4ec684e..7b17bd2 100644
--- a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/security/GeodeAuthenticationProvider.java
+++ b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/security/GeodeAuthenticationProvider.java
@@ -28,8 +28,8 @@ import org.springframework.security.core.authority.AuthorityUtils;
 import org.springframework.stereotype.Component;
 import org.springframework.web.context.ServletContextAware;
 
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.internal.JettyHelper;
 import org.apache.geode.management.internal.security.ResourceConstants;
 import org.apache.geode.security.GemFireSecurityException;
 
@@ -71,6 +71,6 @@ public class GeodeAuthenticationProvider implements AuthenticationProvider, Serv
   @Override
   public void setServletContext(ServletContext servletContext) {
     securityService = (SecurityService) servletContext
-        .getAttribute(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
+        .getAttribute(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
   }
 }
diff --git a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/security/RestSecurityService.java b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/security/RestSecurityService.java
index f0fd792..4860ab8 100644
--- a/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/security/RestSecurityService.java
+++ b/geode-web-management/src/main/java/org/apache/geode/management/internal/rest/security/RestSecurityService.java
@@ -19,8 +19,8 @@ import javax.servlet.ServletContext;
 import org.springframework.stereotype.Component;
 import org.springframework.web.context.ServletContextAware;
 
+import org.apache.geode.internal.cache.HttpService;
 import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.management.internal.JettyHelper;
 import org.apache.geode.security.GemFireSecurityException;
 import org.apache.geode.security.ResourcePermission;
 import org.apache.geode.security.ResourcePermission.Operation;
@@ -78,6 +78,6 @@ public class RestSecurityService implements ServletContextAware {
   @Override
   public void setServletContext(ServletContext servletContext) {
     securityService = (SecurityService) servletContext
-        .getAttribute(JettyHelper.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
+        .getAttribute(HttpService.SECURITY_SERVICE_SERVLET_CONTEXT_PARAM);
   }
 }