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 2017/09/11 20:18:10 UTC

[geode] branch develop updated: GEODE-3096: pulling in refactoring work on HttpOperationInvoker

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 9b308b6  GEODE-3096: pulling in refactoring work on HttpOperationInvoker
9b308b6 is described below

commit 9b308b6cc2fa9b7324ea62622906bdf0913db768
Author: Jinmei Liao <ji...@pivotal.io>
AuthorDate: Thu Aug 10 13:33:13 2017 -0700

    GEODE-3096: pulling in refactoring work on HttpOperationInvoker
    
    * Use HttpOperationInvoker to replace RestHttpOperationInvoker and SimpleHttpOperationInvoker
    * Use one single ShellCommandController to replace all command controllers
    * do not allow execution of commands that require client side file data gathering to be executed only on the locator/server
    * deprecate CommandService and CommandStatement
    * simplify CommandRequest, delete geode's ClientHttpRequest
    * fix tests
---
 .../org/apache/geode/annotations/TestingOnly.java  |  21 +-
 geode-core/build.gradle                            |   2 +
 .../apache/geode/internal/lang/Initializer.java    |  45 --
 .../geode/internal/util/ArgumentRedactor.java      |   6 +-
 .../org/apache/geode/management/MemberMXBean.java  |   3 +-
 .../apache/geode/management/cli/CliMetaData.java   |  10 +-
 .../geode/management/cli/CommandService.java       |   4 +
 .../geode/management/cli/CommandStatement.java     |   4 +-
 .../management/internal/beans/MemberMBean.java     |   8 +-
 .../internal/beans/MemberMBeanBridge.java          |  63 +-
 .../management/internal/cli/CommandManager.java    |  33 +-
 .../management/internal/cli/CommandRequest.java    |  82 +-
 .../management/internal/cli/CommandResponse.java   |  20 +-
 .../internal/cli/CommandResponseBuilder.java       |   2 +-
 .../management/internal/cli/GfshParseResult.java   |  19 +-
 .../internal/cli/commands/ConnectCommand.java      |  24 +-
 .../internal/cli/commands/DeployCommand.java       |  81 +-
 .../internal/cli/commands/ExportConfigCommand.java |  76 +-
 .../ExportImportClusterConfigurationCommands.java  |  44 +-
 .../internal/cli/commands/ExportLogsCommand.java   |   3 +-
 .../internal/cli/commands/GfshCommand.java         |  53 +-
 .../internal/cli/commands/QueryCommand.java        |  30 -
 .../internal/cli/remote/CommandExecutor.java       |  58 ++
 .../internal/cli/remote/CommandProcessor.java      | 188 -----
 .../internal/cli/remote/CommandStatementImpl.java  |  55 +-
 .../internal/cli/remote/MemberCommandService.java  |  23 +-
 .../cli/remote/OnlineCommandProcessor.java         | 133 +++
 .../cli/remote/RemoteExecutionStrategy.java        | 142 ----
 .../internal/cli/result/AbstractResultData.java    |   7 +-
 .../internal/cli/result/CommandResult.java         |  19 +
 .../internal/cli/result/DownloadFileResult.java    |  73 ++
 .../internal/cli/result/ErrorResultData.java       |   6 +-
 .../internal/cli/result/InfoResultData.java        |  18 +-
 .../internal/cli/result/ResultBuilder.java         |  20 +-
 .../geode/management/internal/cli/shell/Gfsh.java  |  76 +-
 .../internal/cli/shell/GfshExecutionStrategy.java  | 107 +--
 .../internal/cli/shell/JmxOperationInvoker.java    |  45 +-
 .../internal/cli/shell/ScriptExecutionDetails.java |  79 ++
 .../controllers/AbstractCommandsController.java    | 178 +---
 .../web/controllers/ClientCommandsController.java  |  59 --
 .../web/controllers/ClusterCommandsController.java |  46 --
 .../web/controllers/ConfigCommandsController.java  | 222 -----
 .../web/controllers/DataCommandsController.java    | 238 ------
 .../web/controllers/DeployCommandsController.java  | 106 ---
 .../controllers/DiskStoreCommandsController.java   | 198 -----
 .../DurableClientCommandsController.java           | 163 ----
 .../web/controllers/ExportLogController.java       | 138 ----
 .../controllers/FunctionCommandsController.java    | 141 ----
 .../web/controllers/IndexCommandsController.java   | 174 ----
 .../LauncherLifecycleCommandsController.java       |  53 --
 .../web/controllers/MemberCommandsController.java  |  67 --
 .../MiscellaneousCommandsController.java           | 249 ------
 .../web/controllers/PdxCommandsController.java     | 128 ---
 .../web/controllers/QueueCommandsController.java   | 157 ----
 .../web/controllers/RegionCommandsController.java  | 401 ---------
 .../web/controllers/ShellCommandsController.java   | 234 ++----
 .../web/controllers/WanCommandsController.java     | 465 -----------
 .../geode/management/internal/web/domain/Link.java | 165 ----
 .../management/internal/web/domain/LinkIndex.java  | 138 ----
 .../internal/web/http/ClientHttpRequest.java       | 497 ------------
 .../management/internal/web/http/HttpHeader.java   |  99 ---
 .../web/http/support/SimpleHttpRequester.java      |  23 +-
 .../web/shell/AbstractHttpOperationInvoker.java    | 895 ---------------------
 .../internal/web/shell/HttpOperationInvoker.java   | 670 ++++++++++++++-
 .../web/shell/RestHttpOperationInvoker.java        | 464 -----------
 .../web/shell/SimpleHttpOperationInvoker.java      | 163 ----
 .../management/internal/web/util/ConvertUtils.java |  88 +-
 .../dunit/QueryParamsAuthorizationDUnitTest.java   |  87 +-
 .../geode/internal/lang/InitializerJUnitTest.java  |  78 --
 .../internal/util/ArgumentRedactorJUnitTest.java   |  20 +-
 .../internal/cli/CommandRequestTest.java           |  11 +-
 .../internal/cli/GfshParserParsingTest.java        |   4 +-
 .../management/internal/cli/HeadlessGfsh.java      |  22 +-
 .../cli/commands/ConfigCommandsDUnitTest.java      |  11 +-
 .../cli/commands/GfshCommandJUnitTest.java         |  54 +-
 .../cli/commands/ShellCommandsDUnitTest.java       |  40 +-
 .../cli/commands/ShowMetricsDUnitTest.java         |  34 +-
 .../internal/cli/remote/CommandExecutorTest.java   |  97 +++
 .../cli/remote/OnlineCommandProcessorTest.java     |  93 +++
 .../internal/cli/result/CommandResultTest.java     |  75 ++
 .../result/DownloadFileResultTest.java}            |  41 +-
 ...nitTest.java => GfshExecutionStrategyTest.java} |  11 +-
 .../security/DeployCommandsSecurityTest.java       |  14 +-
 .../internal/security/MultiUserDUnitTest.java      |  22 +-
 .../web/controllers/ExportLogControllerTest.java   |  56 --
 .../ShellCommandsControllerProcessCommandTest.java | 118 +++
 .../WanCommandsControllerJUnitTest.java            | 144 ----
 .../web/shell/HttpOperationInvokerTest.java        |  73 ++
 .../test/dunit/rules/GfshShellConnectionRule.java  |  26 +-
 geode-junit/build.gradle                           |   1 +
 .../org/apache/geode/test/junit/ResultCaptor.java  |  29 +-
 .../internal/cli/commands/CommandOverHttpTest.java |  91 +++
 .../ConnectCommandWithHttpAndSSLDUnitTest.java     | 349 --------
 .../commands/ConnectCommandWithSecurityTest.java   |  11 +-
 .../cli/commands/QueryNamesOverHttpDUnitTest.java  |  60 --
 .../internal/security/LogNoPasswordTest.java       |  25 +-
 .../internal/web/AbstractWebTestCase.java          |  96 ---
 .../ShellCommandsControllerJUnitTest.java          | 253 ------
 .../internal/web/domain/LinkIndexJUnitTest.java    | 275 -------
 .../internal/web/domain/LinkJUnitTest.java         | 129 ---
 .../web/http/ClientHttpRequestJUnitTest.java       | 556 -------------
 .../HttpOperationInvokerMBeanOperationTest.java    |  93 +++
 .../shell/HttpOperationInvokerSecurityTest.java    | 102 +++
 .../shell/RestHttpOperationInvokerJUnitTest.java   | 470 -----------
 .../shell/SimpleHttpOperationInvokerJUnitTest.java | 205 -----
 .../internal/web/util/UriUtilsJUnitTest.java       | 116 ---
 106 files changed, 2469 insertions(+), 9824 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/http/HttpMethod.java b/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
similarity index 71%
rename from geode-core/src/main/java/org/apache/geode/management/internal/web/http/HttpMethod.java
rename to geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
index 2ba364b..f2a21a1 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/http/HttpMethod.java
+++ b/geode-common/src/main/java/org/apache/geode/annotations/TestingOnly.java
@@ -12,16 +12,17 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.management.internal.web.http;
 
-/**
- * The HttpMethod enum is an enumeration of all HTTP methods (POST, GET, PUT, DELETE, HEADERS, etc).
- * <p/>
- * 
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-public enum HttpMethod {
-  CONNECT, DELETE, GET, HEAD, OPTIONS, POST, PUT, TRACE
+package org.apache.geode.annotations;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Target;
+
+@Documented
+@Target({ElementType.CONSTRUCTOR, ElementType.METHOD})
+public @interface TestingOnly {
 
+  /** Optional description */
+  String value() default "";
 }
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index 8145a63..9f5a242 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -140,6 +140,8 @@ dependencies {
   testRuntime 'commons-validator:commons-validator:' + project.'commons-validator.version'
   testCompile 'net.spy:spymemcached:' + project.'spymemcached.version'
   testCompile 'redis.clients:jedis:' + project.'jedis.version'
+  testCompile 'org.springframework:spring-test:' + project.'springframework.version'
+
 
   testCompile 'com.pholser:junit-quickcheck-core:' + project.'junit-quickcheck.version'
   testCompile 'com.pholser:junit-quickcheck-generators:' + project.'junit-quickcheck.version'
diff --git a/geode-core/src/main/java/org/apache/geode/internal/lang/Initializer.java b/geode-core/src/main/java/org/apache/geode/internal/lang/Initializer.java
deleted file mode 100644
index 2037371..0000000
--- a/geode-core/src/main/java/org/apache/geode/internal/lang/Initializer.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.lang;
-
-/**
- * The Initializer class is a utility class to identify Initable objects and initialize them by
- * calling their init method.
- * <p/>
- * 
- * @see org.apache.geode.internal.lang.Initable
- * @since GemFire 8.0
- */
-public class Initializer {
-
-  /**
-   * Initializes the specified Object by calling it's init method if and only if the Object
-   * implements the Initable interface.
-   * <p/>
-   * 
-   * @param initableObj the Object targeted to be initialized.
-   * @return true if the target Object was initialized using an init method; false otherwise.
-   * @see org.apache.geode.internal.lang.Initable#init()
-   */
-  public static boolean init(final Object initableObj) {
-    if (initableObj instanceof Initable) {
-      ((Initable) initableObj).init();
-      return true;
-    }
-
-    return false;
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/util/ArgumentRedactor.java b/geode-core/src/main/java/org/apache/geode/internal/util/ArgumentRedactor.java
index df82dff..020e81d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/util/ArgumentRedactor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/util/ArgumentRedactor.java
@@ -69,7 +69,7 @@ public class ArgumentRedactor {
    * Single value: "password=secret" or "--password=secret" Multiple values: "-Dflag -Dkey=value
    * --classpath=."
    * 
-   * @param line The input to be parsed
+   * @param line The argument input to be parsed
    * @return A redacted string that has sensitive information obscured.
    */
   public static String redact(String line) {
@@ -106,6 +106,10 @@ public class ArgumentRedactor {
     return redacted.toString().trim();
   }
 
+  public static String redactScriptLine(String line) {
+    return line.replaceAll("password=[\\S]+", "password=********");
+  }
+
   /**
    * Return a redacted value if the key indicates redaction is necessary. Otherwise, return the
    * value unchanged.
diff --git a/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java b/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
index ca7c2a2..d0b05d1 100644
--- a/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/MemberMXBean.java
@@ -209,7 +209,8 @@ public interface MemberMXBean {
   String processCommand(String commandString, Map<String, String> env);
 
   /**
-   * Executes a command on the member.
+   * Executes a command on the member. this is the method that's used by the HttpOperationInvoker
+   * and JmxOperationInvoker
    * 
    * @param commandString Command to be execute.
    * @param env Environmental properties to use during command execution.
diff --git a/geode-core/src/main/java/org/apache/geode/management/cli/CliMetaData.java b/geode-core/src/main/java/org/apache/geode/management/cli/CliMetaData.java
index 226086f..0e838e6 100644
--- a/geode-core/src/main/java/org/apache/geode/management/cli/CliMetaData.java
+++ b/geode-core/src/main/java/org/apache/geode/management/cli/CliMetaData.java
@@ -14,13 +14,13 @@
  */
 package org.apache.geode.management.cli;
 
-import org.apache.geode.management.internal.cli.CliAroundInterceptor;
-
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
+import org.apache.geode.management.internal.cli.CliAroundInterceptor;
+
 /**
  * An annotation to define additional meta-data for commands.
  *
@@ -52,6 +52,12 @@ public @interface CliMetaData {
   boolean isFileDownloadOverHttp() default false;
 
   /**
+   * Indicates whether this command would require fileData to be sent from the client. If this is
+   * true, the preExecution of the interceptor needs to return a FileResult
+   */
+  boolean isFileUploaded() default false;
+
+  /**
    * Indicates that the effect of the command is persisted or the commands affects the persistent
    * configuration
    */
diff --git a/geode-core/src/main/java/org/apache/geode/management/cli/CommandService.java b/geode-core/src/main/java/org/apache/geode/management/cli/CommandService.java
index 20f1c75..698461a 100644
--- a/geode-core/src/main/java/org/apache/geode/management/cli/CommandService.java
+++ b/geode-core/src/main/java/org/apache/geode/management/cli/CommandService.java
@@ -33,6 +33,8 @@ import org.apache.geode.management.internal.cli.remote.MemberCommandService;
  *
  *
  * @since GemFire 7.0
+ *
+ * @deprecated since 1.3 use OnlineCommandProcessor directly
  */
 public abstract class CommandService {
   protected static final Map<String, String> EMPTY_ENV = Collections.emptyMap();
@@ -78,6 +80,7 @@ public abstract class CommandService {
    *         command.
    *
    * @see CommandStatement#process()
+   * @deprecated since Geode 1.3, simply call processCommand to execute the command
    */
   public abstract CommandStatement createCommandStatement(String commandString);
 
@@ -91,6 +94,7 @@ public abstract class CommandService {
    *         command.
    *
    * @see CommandStatement#process()
+   * @deprecated since Geode 1.3, simply call processCommand to execute the command
    */
   protected abstract CommandStatement createCommandStatement(String commandString,
       Map<String, String> env);
diff --git a/geode-core/src/main/java/org/apache/geode/management/cli/CommandStatement.java b/geode-core/src/main/java/org/apache/geode/management/cli/CommandStatement.java
index a01f08c..5d8c830 100755
--- a/geode-core/src/main/java/org/apache/geode/management/cli/CommandStatement.java
+++ b/geode-core/src/main/java/org/apache/geode/management/cli/CommandStatement.java
@@ -22,6 +22,8 @@ import java.util.Map;
  * 
  * 
  * @since GemFire 7.0
+ *
+ * @deprecated since Geode 1.3. simply use commandProcessor to process the command
  */
 public interface CommandStatement {
 
@@ -45,7 +47,7 @@ public interface CommandStatement {
 
   /**
    * Returns whether the command statement is well formed.
-   * 
+   *
    * @return True if the command statement is well formed, false otherwise.
    */
   public boolean validate();
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBean.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBean.java
index 271dce1..f3febc1 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBean.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBean.java
@@ -24,7 +24,6 @@ import org.apache.geode.management.GemFireProperties;
 import org.apache.geode.management.JVMMetrics;
 import org.apache.geode.management.MemberMXBean;
 import org.apache.geode.management.OSMetrics;
-import org.apache.geode.management.internal.cli.remote.CommandExecutionContext;
 
 /**
  * This MBean is a gateway to cache and a member
@@ -404,12 +403,7 @@ public class MemberMBean extends NotificationBroadcasterSupport implements Membe
 
   @Override
   public String processCommand(String commandString, Map<String, String> env, Byte[][] binaryData) {
-    try {
-      CommandExecutionContext.setBytesFromShell(ArrayUtils.toBytes(binaryData));
-      return bridge.processCommand(commandString, env);
-    } finally {
-      CommandExecutionContext.clear();
-    }
+    return bridge.processCommand(commandString, env, ArrayUtils.toBytes(binaryData));
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
index 5105c3d..5b911d5 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
@@ -44,7 +44,6 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsType;
-import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.DiskStore;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.execute.FunctionService;
@@ -64,6 +63,7 @@ import org.apache.geode.distributed.internal.locks.DLockStats;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.PureJavaMode;
+import org.apache.geode.internal.cache.BackupManager;
 import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.DirectoryHolder;
 import org.apache.geode.internal.cache.DiskDirectoryStats;
@@ -77,7 +77,6 @@ import org.apache.geode.internal.cache.PartitionedRegionStats;
 import org.apache.geode.internal.cache.control.ResourceManagerStats;
 import org.apache.geode.internal.cache.execute.FunctionServiceStats;
 import org.apache.geode.internal.cache.lru.LRUStatistics;
-import org.apache.geode.internal.cache.BackupManager;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
@@ -99,14 +98,11 @@ import org.apache.geode.internal.statistics.platform.SolarisSystemStats;
 import org.apache.geode.internal.statistics.platform.WindowsSystemStats;
 import org.apache.geode.internal.stats50.VMStats50;
 import org.apache.geode.internal.tcp.ConnectionTable;
-import org.apache.geode.management.DependenciesNotFoundException;
 import org.apache.geode.management.DiskBackupResult;
 import org.apache.geode.management.GemFireProperties;
 import org.apache.geode.management.JVMMetrics;
 import org.apache.geode.management.ManagementException;
 import org.apache.geode.management.OSMetrics;
-import org.apache.geode.management.cli.CommandService;
-import org.apache.geode.management.cli.CommandServiceException;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.ManagementConstants;
 import org.apache.geode.management.internal.ManagementStrings;
@@ -122,11 +118,8 @@ import org.apache.geode.management.internal.beans.stats.StatsLatency;
 import org.apache.geode.management.internal.beans.stats.StatsRate;
 import org.apache.geode.management.internal.beans.stats.VMStatsMonitor;
 import org.apache.geode.management.internal.cli.CommandResponseBuilder;
-import org.apache.geode.management.internal.cli.remote.CommandExecutionContext;
-import org.apache.geode.management.internal.cli.remote.MemberCommandService;
+import org.apache.geode.management.internal.cli.remote.OnlineCommandProcessor;
 import org.apache.geode.management.internal.cli.result.CommandResult;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
 
 /**
  * This class acts as an Bridge between MemberMBean and GemFire Cache and Distributed System
@@ -169,7 +162,7 @@ public class MemberMBeanBridge {
   /**
    * Command Service
    */
-  private CommandService commandService;
+  private OnlineCommandProcessor commandProcessor;
 
   private String commandServiceInitError;
 
@@ -345,20 +338,12 @@ public class MemberMBeanBridge {
 
     this.config = system.getConfig();
     try {
-      this.commandService = CommandService.createLocalCommandService(cache);
-    } catch (CacheClosedException e) {
-      commandServiceInitError = e.getMessage();
-      // LOG:CONFIG:
-      logger.info(LogMarker.CONFIG, "Command Service could not be initialized. {}", e.getMessage());
-    } catch (CommandServiceException e) {
-      commandServiceInitError = e.getMessage();
-      // LOG:CONFIG:
-      logger.info(LogMarker.CONFIG, "Command Service could not be initialized. {}", e.getMessage());
-    } catch (DependenciesNotFoundException e) {
+      this.commandProcessor =
+          new OnlineCommandProcessor(system.getProperties(), cache.getSecurityService());
+    } catch (Exception e) {
       commandServiceInitError = e.getMessage();
-      // log as error for dedicated cache server - launched through script
-      // LOG:CONFIG:
-      logger.info(LogMarker.CONFIG, "Command Service could not be initialized. {}", e.getMessage());
+      logger.info(LogMarker.CONFIG, "Command processor could not be initialized. {}",
+          e.getMessage());
     }
 
     intitGemfireProperties();
@@ -1582,39 +1567,15 @@ public class MemberMBeanBridge {
    * @param env environment information to be used for processing the command
    * @return result of the processing the given command string.
    */
-  public String processCommand(String commandString, Map<String, String> env) {
-    if (commandService == null) {
+  public String processCommand(String commandString, Map<String, String> env, byte[][] binaryData) {
+    if (commandProcessor == null) {
       throw new JMRuntimeException(
           "Command can not be processed as Command Service did not get initialized. Reason: "
               + commandServiceInitError);
     }
 
-    boolean isGfshRequest = isGfshRequest(env);
-    if (isGfshRequest) {
-      CommandExecutionContext.setShellRequest();
-    }
-
-    Result result = ((MemberCommandService) commandService).processCommand(commandString, env);
-    if (!(result instanceof CommandResult)) {// TODO - Abhishek - Shouldn't be needed
-      while (result.hasNextLine()) {
-        result = ResultBuilder.createInfoResult(result.nextLine());
-      }
-    }
-
-    if (isGfshRequest) {
-      return CommandResponseBuilder.createCommandResponseJson(getMember(), (CommandResult) result);
-    } else {
-      return ResultBuilder.resultAsString(result);
-    }
-  }
-
-  private boolean isGfshRequest(Map<String, String> env) {
-    String appName = null;
-    if (env != null) {
-      appName = env.get(Gfsh.ENV_APP_NAME);
-    }
-
-    return Gfsh.GFSH_APP_NAME.equals(appName);
+    Result result = commandProcessor.executeCommand(commandString, env, binaryData);
+    return CommandResponseBuilder.createCommandResponseJson(getMember(), (CommandResult) result);
   }
 
   public long getTotalDiskUsage() {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandManager.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandManager.java
index 3c8f6cf..b686396 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandManager.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandManager.java
@@ -16,22 +16,6 @@ package org.apache.geode.management.internal.cli;
 
 import static org.apache.geode.distributed.ConfigurationProperties.USER_COMMAND_PACKAGES;
 
-import io.github.lukehutch.fastclasspathscanner.FastClasspathScanner;
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.geode.distributed.ConfigurationProperties;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.management.internal.cli.commands.GfshCommand;
-import org.apache.geode.management.internal.cli.help.Helper;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.cli.util.ClasspathScanLoadHelper;
-import org.springframework.shell.converters.SimpleFileConverter;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.Converter;
-import org.springframework.shell.core.MethodTarget;
-import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
-import org.springframework.shell.core.annotation.CliCommand;
-
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.HashSet;
@@ -43,6 +27,21 @@ import java.util.ServiceLoader;
 import java.util.Set;
 import java.util.StringTokenizer;
 
+import org.springframework.shell.converters.SimpleFileConverter;
+import org.springframework.shell.core.CommandMarker;
+import org.springframework.shell.core.Converter;
+import org.springframework.shell.core.MethodTarget;
+import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
+import org.springframework.shell.core.annotation.CliCommand;
+
+import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.internal.ClassPathLoader;
+import org.apache.geode.management.internal.cli.commands.GfshCommand;
+import org.apache.geode.management.internal.cli.help.Helper;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+import org.apache.geode.management.internal.cli.util.ClasspathScanLoadHelper;
+
 /**
  *
  * this only takes care of loading all available command markers and converters from the application
@@ -73,7 +72,7 @@ public class CommandManager {
 
   /**
    * this is used when getting the instance in a cache server. We are getting the
-   * user-command-package from distribution properties. used by CommandProcessor.
+   * user-command-package from distribution properties. used by OnlineCommandProcessor.
    */
   public CommandManager(final Properties cacheProperties) {
     if (cacheProperties != null) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandRequest.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandRequest.java
index 00a0587..aade959 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandRequest.java
@@ -14,13 +14,15 @@
  */
 package org.apache.geode.management.internal.cli;
 
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.cli.CliMetaData;
-
+import java.net.URI;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Map;
 
+import org.springframework.web.util.UriComponentsBuilder;
+
+import org.apache.geode.annotations.TestingOnly;
+import org.apache.geode.management.cli.CliMetaData;
+
 /**
  * The CommandRequest class encapsulates information pertaining to the command the user entered in
  * Gfsh.
@@ -31,19 +33,17 @@ import java.util.Map;
  */
 @SuppressWarnings("unused")
 public class CommandRequest {
-
+  protected static final String CMD_QUERY_PARAMETER = "cmd";
+  protected static final String REST_API_MANAGEMENT_COMMANDS_URI = "/management/commands";
   protected static final String OPTION_SPECIFIER = "--";
 
   private final byte[][] fileData;
-
   private final GfshParseResult parseResult;
-
-  private final Map<String, String> customParameters = new HashMap<String, String>();
   private final Map<String, String> env;
   private final boolean downloadFile;
 
-  private String customInput;
 
+  @TestingOnly
   public CommandRequest(final Map<String, String> env) {
     this.env = env;
     this.fileData = null;
@@ -58,10 +58,6 @@ public class CommandRequest {
     downloadFile = false;
   }
 
-  public CommandRequest(final GfshParseResult parseResult, final Map<String, String> env) {
-    this(parseResult, env, null);
-  }
-
   public CommandRequest(final GfshParseResult parseResult, final Map<String, String> env,
       final byte[][] fileData) {
     assert parseResult != null : "The Gfsh ParseResult cannot be null!";
@@ -74,38 +70,11 @@ public class CommandRequest {
     this.downloadFile = (metaData != null && metaData.isFileDownloadOverHttp());
   }
 
-  public String getName() {
-    if (getUserInput() != null) {
-      final String[] userInputTokenized = getUserInput().split("\\s");
-      final StringBuilder buffer = new StringBuilder();
-
-      for (final String token : userInputTokenized) {
-        if (!token.startsWith(OPTION_SPECIFIER)) {
-          buffer.append(token).append(StringUtils.SPACE);
-        }
-      }
-
-      return buffer.toString().trim();
-    } else {
-      return "unknown";
-    }
-  }
-
-  public String getCustomInput() {
-    return customInput;
-  }
 
   public boolean isDownloadFile() {
     return downloadFile;
   }
 
-  public void setCustomInput(final String input) {
-    this.customInput = input;
-  }
-
-  public Map<String, String> getCustomParameters() {
-    return customParameters;
-  }
 
   public Map<String, String> getEnvironment() {
     return Collections.unmodifiableMap(env);
@@ -119,27 +88,6 @@ public class CommandRequest {
     return (getFileData() != null);
   }
 
-  public String getInput() {
-    return StringUtils.defaultIfBlank(getCustomInput(), getUserInput());
-  }
-
-  public Map<String, String> getParameters() {
-    final Map<String, String> parameters = new HashMap<>();
-    for (Map.Entry<String, String> mapEntry : getUserParameters().entrySet()) {
-      String key = mapEntry.getKey();
-      String value = mapEntry.getValue();
-
-      if (hasQuotesAroundNegativeNumber(value)) {
-        String trimmed = value.substring(1, value.length() - 1);
-        parameters.put(key, trimmed);
-      } else {
-        parameters.put(key, value);
-      }
-    }
-    parameters.putAll(getCustomParameters());
-    return Collections.unmodifiableMap(parameters);
-  }
-
   protected GfshParseResult getParseResult() {
     return parseResult;
   }
@@ -148,16 +96,14 @@ public class CommandRequest {
     return getParseResult().getUserInput();
   }
 
-  public Map<String, String> getUserParameters() {
+  @TestingOnly
+  public Map<String, String> getParameters() {
     return getParseResult().getParamValueStrings();
   }
 
-  private boolean hasQuotesAroundNegativeNumber(String value) {
-    if (value == null) {
-      return false;
-    } else {
-      return value.startsWith("\"") && value.endsWith("\"") && value.matches("\"-[0-9]+\"");
-    }
+  public URI getHttpRequestUrl(String baseUrl) {
+    return UriComponentsBuilder.fromHttpUrl(baseUrl).path(REST_API_MANAGEMENT_COMMANDS_URI)
+        .queryParam(CMD_QUERY_PARAMETER, getUserInput()).build().encode().toUri();
   }
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandResponse.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandResponse.java
index 81c4958..10bdbe8 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandResponse.java
@@ -14,13 +14,13 @@
  */
 package org.apache.geode.management.internal.cli;
 
+import java.nio.file.Path;
 import java.text.DateFormat;
 
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.management.internal.cli.json.GfJsonObject;
 
 /**
- * 
  * @since GemFire 7.0
  */
 public class CommandResponse {
@@ -35,10 +35,18 @@ public class CommandResponse {
   private final String debugInfo;
   private final Data data;
   private final boolean failedToPersist;
+  private final String fileToDownload;
 
   CommandResponse(String sender, String contentType, int status, String page, String tokenAccessor,
       String debugInfo, String header, GfJsonObject content, String footer,
       boolean failedToPersist) {
+    this(sender, contentType, status, page, tokenAccessor, debugInfo, header, content, footer,
+        failedToPersist, null);
+  }
+
+  CommandResponse(String sender, String contentType, int status, String page, String tokenAccessor,
+      String debugInfo, String header, GfJsonObject content, String footer, boolean failedToPersist,
+      Path fileToDownload) {
     this.sender = sender;
     this.contentType = contentType;
     this.status = status;
@@ -49,6 +57,11 @@ public class CommandResponse {
     this.when = DateFormat.getInstance().format(new java.util.Date());
     this.version = GemFireVersion.getGemFireVersion();
     this.failedToPersist = failedToPersist;
+    if (fileToDownload != null) {
+      this.fileToDownload = fileToDownload.toString();
+    } else {
+      this.fileToDownload = null;
+    }
   }
 
   // For de-serializing
@@ -63,6 +76,7 @@ public class CommandResponse {
     this.when = jsonObject.getString("when");
     this.version = jsonObject.getString("version");
     this.failedToPersist = jsonObject.getBoolean("failedToPersist");
+    this.fileToDownload = jsonObject.getString("fileToDownload");
   }
 
   /**
@@ -100,6 +114,10 @@ public class CommandResponse {
     return page;
   }
 
+  public String getFileToDownload() {
+    return fileToDownload;
+  }
+
   /**
    * @return the when
    */
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandResponseBuilder.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandResponseBuilder.java
index 790e54b..c41d2d6 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandResponseBuilder.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandResponseBuilder.java
@@ -31,7 +31,7 @@ public class CommandResponseBuilder {
     content = result.getContent();
     return new CommandResponse(memberName, getType(result), result.getStatus().getCode(), "1/1",
         CliMetaData.ANNOTATION_NULL_VALUE, getDebugInfo(result), result.getHeader(), content,
-        result.getFooter(), result.failedToPersist());
+        result.getFooter(), result.failedToPersist(), result.getFileToDownload());
   }
 
   // De-serializing to CommandResponse
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParseResult.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParseResult.java
index b228b28..1a20c7b 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParseResult.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParseResult.java
@@ -25,6 +25,7 @@ import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 import org.springframework.shell.event.ParseResult;
 
+import org.apache.geode.annotations.TestingOnly;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.internal.cli.shell.GfshExecutionStrategy;
 import org.apache.geode.management.internal.cli.shell.OperationInvoker;
@@ -82,18 +83,9 @@ public class GfshParseResult extends ParseResult {
       } else {
         argumentAsString = argument.toString();
       }
-      // need to quote the argument with single quote if it contains white space.
-      // these will be used for the http request parameters, when turned into the
-      // commands again, the options will be quoted.
-      if (argumentAsString.contains(" ")) {
-        if (argumentAsString.contains("'")) {
-          argumentAsString = "\"" + argumentAsString + "\"";
-        } else {
-          argumentAsString = "'" + argumentAsString + "'";
-        }
-      }
-      // this always uses the first variation of the option as the key, so all the controllers
-      // should use this as the parameter key
+
+      // this maps are used for easy access of option values in String form.
+      // It's used in tests and validation of option values in pre-execution
       paramValueStringMap.put(cliOption.key()[0], argumentAsString);
     }
   }
@@ -105,11 +97,14 @@ public class GfshParseResult extends ParseResult {
     return userInput;
   }
 
+  @TestingOnly
   public String getParamValue(String param) {
     return paramValueStringMap.get(param);
   }
 
   /**
+   * Used only in tests and command pre-execution for validating arguments
+   * 
    * @return the unmodifiable paramValueStringMap
    */
   public Map<String, String> getParamValueStrings() {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
index a75eeb0..d3ff638 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.security.KeyStore;
 import java.security.SecureRandom;
 import java.util.Arrays;
-import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
 
@@ -41,7 +40,6 @@ import org.springframework.shell.core.annotation.CliOption;
 
 import org.apache.geode.internal.DSFIDFactory;
 import org.apache.geode.internal.admin.SSLConfig;
-import org.apache.geode.internal.lang.Initializer;
 import org.apache.geode.internal.net.SSLConfigurationFactory;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
 import org.apache.geode.management.cli.CliMetaData;
@@ -60,10 +58,8 @@ import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.cli.shell.JmxOperationInvoker;
 import org.apache.geode.management.internal.cli.util.ConnectionEndpoint;
 import org.apache.geode.management.internal.security.ResourceConstants;
-import org.apache.geode.management.internal.web.domain.LinkIndex;
 import org.apache.geode.management.internal.web.http.support.SimpleHttpRequester;
 import org.apache.geode.management.internal.web.shell.HttpOperationInvoker;
-import org.apache.geode.management.internal.web.shell.RestHttpOperationInvoker;
 import org.apache.geode.security.AuthenticationFailedException;
 
 public class ConnectCommand implements GfshCommand {
@@ -250,23 +246,11 @@ public class ConnectCommand implements GfshCommand {
         }
       }
 
-      // This is so that SSL termination results in https URLs being returned
-      String query = (url.startsWith("https")) ? "?scheme=https" : "";
+      // this triggers the authentication check
+      new SimpleHttpRequester(gfsh, CONNECT_LOCATOR_TIMEOUT_MS, gfProperties)
+          .exchange(url.concat("/ping"), String.class);
 
-      LogWrapper.getInstance().warning(String.format(
-          "Sending HTTP request for Link Index at (%1$s)...", url.concat("/index").concat(query)));
-
-      LinkIndex linkIndex =
-          new SimpleHttpRequester(gfsh, CONNECT_LOCATOR_TIMEOUT_MS, (Map) gfProperties)
-              .exchange(url.concat("/index").concat(query), LinkIndex.class);
-
-      LogWrapper.getInstance()
-          .warning(String.format("Received Link Index (%1$s)", linkIndex.toString()));
-
-      HttpOperationInvoker operationInvoker =
-          new RestHttpOperationInvoker(linkIndex, gfsh, url, (Map) gfProperties);
-
-      Initializer.init(operationInvoker);
+      HttpOperationInvoker operationInvoker = new HttpOperationInvoker(gfsh, url, gfProperties);
       gfsh.setOperationInvoker(operationInvoker);
 
       LogWrapper.getInstance()
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommand.java
index 4f46539..ebbc3c3 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommand.java
@@ -29,10 +29,8 @@ import org.apache.commons.lang.StringUtils;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
-import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.ConverterHint;
 import org.apache.geode.management.cli.Result;
@@ -47,7 +45,6 @@ import org.apache.geode.management.internal.cli.result.FileResult;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.result.TabularResultData;
 import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.NotAuthorizedException;
 import org.apache.geode.security.ResourcePermission;
 
 public class DeployCommand implements GfshCommand {
@@ -64,7 +61,7 @@ public class DeployCommand implements GfshCommand {
   @CliCommand(value = {CliStrings.DEPLOY}, help = CliStrings.DEPLOY__HELP)
   @CliMetaData(
       interceptor = "org.apache.geode.management.internal.cli.commands.DeployCommand$Interceptor",
-      relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
+      isFileUploaded = true, relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
   @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
       operation = ResourcePermission.Operation.MANAGE, target = ResourcePermission.Target.JAR)
   public Result deploy(
@@ -73,64 +70,48 @@ public class DeployCommand implements GfshCommand {
       @CliOption(key = {CliStrings.JAR, CliStrings.JARS},
           help = CliStrings.DEPLOY__JAR__HELP) String[] jars,
       @CliOption(key = {CliStrings.DEPLOY__DIR}, help = CliStrings.DEPLOY__DIR__HELP) String dir) {
-    try {
 
-      // since deploy function can potentially do a lot of damage to security, this action should
-      // require these following privileges
-      SecurityService securityService = getSecurityService();
+    TabularResultData tabularData = ResultBuilder.createTabularResultData();
 
-      TabularResultData tabularData = ResultBuilder.createTabularResultData();
+    byte[][] shellBytesData = CommandExecutionContext.getBytesFromShell();
+    String[] jarNames = CliUtil.bytesToNames(shellBytesData);
+    byte[][] jarBytes = CliUtil.bytesToData(shellBytesData);
 
-      byte[][] shellBytesData = CommandExecutionContext.getBytesFromShell();
-      String[] jarNames = CliUtil.bytesToNames(shellBytesData);
-      byte[][] jarBytes = CliUtil.bytesToData(shellBytesData);
+    Set<DistributedMember> targetMembers;
 
-      Set<DistributedMember> targetMembers;
+    targetMembers = CliUtil.findMembers(groups, null);
 
-      targetMembers = CliUtil.findMembers(groups, null);
+    if (targetMembers.size() > 0) {
+      // this deploys the jars to all the matching servers
+      ResultCollector<?, ?> resultCollector = CliUtil.executeFunction(this.deployFunction,
+          new Object[] {jarNames, jarBytes}, targetMembers);
 
-      if (targetMembers.size() > 0) {
-        // this deploys the jars to all the matching servers
-        ResultCollector<?, ?> resultCollector = CliUtil.executeFunction(this.deployFunction,
-            new Object[] {jarNames, jarBytes}, targetMembers);
+      List<CliFunctionResult> results =
+          CliFunctionResult.cleanResults((List<?>) resultCollector.getResult());
 
-        List<CliFunctionResult> results =
-            CliFunctionResult.cleanResults((List<?>) resultCollector.getResult());
-
-        for (CliFunctionResult result : results) {
-          if (result.getThrowable() != null) {
+      for (CliFunctionResult result : results) {
+        if (result.getThrowable() != null) {
+          tabularData.accumulate("Member", result.getMemberIdOrName());
+          tabularData.accumulate("Deployed JAR", "");
+          tabularData.accumulate("Deployed JAR Location",
+              "ERROR: " + result.getThrowable().getClass().getName() + ": "
+                  + result.getThrowable().getMessage());
+          tabularData.setStatus(Result.Status.ERROR);
+        } else {
+          String[] strings = (String[]) result.getSerializables();
+          for (int i = 0; i < strings.length; i += 2) {
             tabularData.accumulate("Member", result.getMemberIdOrName());
-            tabularData.accumulate("Deployed JAR", "");
-            tabularData.accumulate("Deployed JAR Location",
-                "ERROR: " + result.getThrowable().getClass().getName() + ": "
-                    + result.getThrowable().getMessage());
-            tabularData.setStatus(Result.Status.ERROR);
-          } else {
-            String[] strings = (String[]) result.getSerializables();
-            for (int i = 0; i < strings.length; i += 2) {
-              tabularData.accumulate("Member", result.getMemberIdOrName());
-              tabularData.accumulate("Deployed JAR", strings[i]);
-              tabularData.accumulate("Deployed JAR Location", strings[i + 1]);
-            }
+            tabularData.accumulate("Deployed JAR", strings[i]);
+            tabularData.accumulate("Deployed JAR Location", strings[i + 1]);
           }
         }
       }
-
-      Result result = ResultBuilder.buildResult(tabularData);
-      persistClusterConfiguration(result,
-          () -> getSharedConfiguration().addJarsToThisLocator(jarNames, jarBytes, groups));
-      return result;
-    } catch (NotAuthorizedException e) {
-      // for NotAuthorizedException, will catch this later in the code
-      throw e;
-    } catch (VirtualMachineError e) {
-      SystemFailure.initiateFailure(e);
-      throw e;
-    } catch (Throwable t) {
-      SystemFailure.checkFailure();
-      return ResultBuilder.createGemFireErrorResult(String
-          .format("Exception while attempting to deploy: (%1$s)", toString(t, isDebugging())));
     }
+
+    Result result = ResultBuilder.buildResult(tabularData);
+    persistClusterConfiguration(result,
+        () -> getSharedConfiguration().addJarsToThisLocator(jarNames, jarBytes, groups));
+    return result;
   }
 
   /**
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportConfigCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportConfigCommand.java
index 672ec88..5482afe 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportConfigCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportConfigCommand.java
@@ -21,10 +21,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.commons.lang.StringUtils;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
-import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.management.cli.CliMetaData;
@@ -74,36 +74,25 @@ public class ExportConfigCommand implements GfshCommand {
       return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
     }
 
-    try {
-      ResultCollector<?, ?> rc =
-          CliUtil.executeFunction(this.exportConfigFunction, null, targetMembers);
-      List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
-
-      for (CliFunctionResult result : results) {
-        if (result.getThrowable() != null) {
-          infoData.addLine(CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__EXCEPTION,
-              result.getMemberIdOrName(), result.getThrowable()));
-        } else if (result.isSuccessful()) {
-          String cacheFileName = result.getMemberIdOrName() + "-cache.xml";
-          String propsFileName = result.getMemberIdOrName() + "-gf.properties";
-          String[] fileContent = (String[]) result.getSerializables();
-          infoData.addAsFile(cacheFileName, fileContent[0], "Downloading Cache XML file: {0}",
-              false);
-          infoData.addAsFile(propsFileName, fileContent[1], "Downloading properties file: {0}",
-              false);
-        }
+    ResultCollector<?, ?> rc =
+        CliUtil.executeFunction(this.exportConfigFunction, null, targetMembers);
+    List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
+
+    for (CliFunctionResult result : results) {
+      if (result.getThrowable() != null) {
+        infoData.addLine(CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__EXCEPTION,
+            result.getMemberIdOrName(), result.getThrowable()));
+      } else if (result.isSuccessful()) {
+        String cacheFileName = result.getMemberIdOrName() + "-cache.xml";
+        String propsFileName = result.getMemberIdOrName() + "-gf.properties";
+        String[] fileContent = (String[]) result.getSerializables();
+        infoData.addAsFile(cacheFileName, fileContent[0], "Downloading Cache XML file: {0}", false);
+        infoData.addAsFile(propsFileName, fileContent[1], "Downloading properties file: {0}",
+            false);
       }
-      return ResultBuilder.buildResult(infoData);
-    } catch (VirtualMachineError e) {
-      SystemFailure.initiateFailure(e);
-      throw e;
-    } catch (Throwable th) {
-      SystemFailure.checkFailure();
-      th.printStackTrace(System.err);
-      return ResultBuilder
-          .createGemFireErrorResult(CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__EXCEPTION,
-              th.getClass().getName() + ": " + th.getMessage()));
     }
+    return ResultBuilder.buildResult(infoData);
+
   }
 
   /**
@@ -116,20 +105,23 @@ public class ExportConfigCommand implements GfshCommand {
     public Result preExecution(GfshParseResult parseResult) {
       Map<String, String> paramValueMap = parseResult.getParamValueStrings();
       String dir = paramValueMap.get("dir");
-      dir = (dir == null) ? null : dir.trim();
-
-      File saveDirFile = new File(".");
-      if (dir != null && !dir.isEmpty()) {
-        saveDirFile = new File(dir);
-        if (saveDirFile.exists()) {
-          if (!saveDirFile.isDirectory())
-            return ResultBuilder.createGemFireErrorResult(
-                CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__NOT_A_DIRECTORY, dir));
-        } else if (!saveDirFile.mkdirs()) {
-          return ResultBuilder.createGemFireErrorResult(
-              CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__CANNOT_CREATE_DIR, dir));
-        }
+      if (StringUtils.isBlank(dir)) {
+        saveDirString = new File(".").getAbsolutePath();
+        return ResultBuilder.createInfoResult("OK");
+      }
+
+      File saveDirFile = new File(dir.trim());
+
+      if (!saveDirFile.exists() && !saveDirFile.mkdirs()) {
+        return ResultBuilder.createGemFireErrorResult(
+            CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__CANNOT_CREATE_DIR, dir));
+      }
+
+      if (!saveDirFile.isDirectory()) {
+        return ResultBuilder.createGemFireErrorResult(
+            CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__NOT_A_DIRECTORY, dir));
       }
+
       try {
         if (!saveDirFile.canWrite()) {
           return ResultBuilder.createGemFireErrorResult(CliStrings.format(
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
index 83eddee..133f596 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
@@ -17,8 +17,22 @@ package org.apache.geode.management.internal.cli.commands;
 import static java.util.stream.Collectors.joining;
 import static java.util.stream.Collectors.toSet;
 
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.Logger;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.ClusterConfigurationService;
@@ -37,6 +51,7 @@ import org.apache.geode.management.internal.cli.result.ErrorResultData;
 import org.apache.geode.management.internal.cli.result.FileResult;
 import org.apache.geode.management.internal.cli.result.InfoResultData;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.management.internal.configuration.functions.GetRegionNamesFunction;
 import org.apache.geode.management.internal.configuration.functions.RecreateCacheFunction;
@@ -44,19 +59,6 @@ import org.apache.geode.management.internal.configuration.utils.ZipUtils;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.logging.log4j.Logger;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 /**
  * Commands for the cluster configuration
@@ -74,7 +76,7 @@ public class ExportImportClusterConfigurationCommands implements GfshCommand {
       mandatory = true, help = CliStrings.EXPORT_SHARED_CONFIG__FILE__HELP) String zipFileName) {
 
     InternalLocator locator = InternalLocator.getLocator();
-    if (!locator.isSharedConfigurationRunning()) {
+    if (locator == null || !locator.isSharedConfigurationRunning()) {
       return ResultBuilder.createGemFireErrorResult(CliStrings.SHARED_CONFIGURATION_NOT_STARTED);
     }
 
@@ -82,7 +84,9 @@ public class ExportImportClusterConfigurationCommands implements GfshCommand {
     try {
       tempDir = Files.createTempDirectory("clusterConfig");
     } catch (IOException e) {
-      logSevere(e);
+      if (Gfsh.getCurrentInstance() != null) {
+        Gfsh.getCurrentInstance().logSevere(e.getMessage(), e);
+      }
       ErrorResultData errorData =
           ResultBuilder.createErrorResultData().addLine("Unable to create temp directory");
       return ResultBuilder.buildResult(errorData);
@@ -106,7 +110,9 @@ public class ExportImportClusterConfigurationCommands implements GfshCommand {
     } catch (Exception e) {
       ErrorResultData errorData = ResultBuilder.createErrorResultData();
       errorData.addLine("Export failed");
-      logSevere(e);
+      if (Gfsh.getCurrentInstance() != null) {
+        Gfsh.getCurrentInstance().logSevere(e.getMessage(), e);
+      }
       result = ResultBuilder.buildResult(errorData);
     } finally {
       zipFile.delete();
@@ -119,7 +125,7 @@ public class ExportImportClusterConfigurationCommands implements GfshCommand {
       help = CliStrings.IMPORT_SHARED_CONFIG__HELP)
   @CliMetaData(
       interceptor = "org.apache.geode.management.internal.cli.commands.ExportImportClusterConfigurationCommands$ImportInterceptor",
-      relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
+      isFileUploaded = true, relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE)
   @SuppressWarnings("unchecked")
   public Result importSharedConfig(@CliOption(key = {CliStrings.IMPORT_SHARED_CONFIG__ZIP},
@@ -172,7 +178,9 @@ public class ExportImportClusterConfigurationCommands implements GfshCommand {
     } catch (Exception e) {
       ErrorResultData errorData = ResultBuilder.createErrorResultData();
       errorData.addLine("Import failed");
-      logSevere(e);
+      if (Gfsh.getCurrentInstance() != null) {
+        Gfsh.getCurrentInstance().logSevere(e.getMessage(), e);
+      }
       result = ResultBuilder.buildResult(errorData);
       // if import is unsuccessful, don't need to bounce the server.
       return result;
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogsCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogsCommand.java
index 70e1e60..45b183b 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogsCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogsCommand.java
@@ -43,6 +43,7 @@ import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.functions.ExportLogsFunction;
 import org.apache.geode.management.internal.cli.functions.SizeExportLogsFunction;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.util.ExportLogsCacheWriter;
 import org.apache.geode.management.internal.configuration.utils.ZipUtils;
@@ -199,7 +200,7 @@ public class ExportLogsCommand implements GfshCommand {
       ZipUtils.zipDirectory(exportedLogsDir, exportedLogsZipFile);
       FileUtils.deleteDirectory(tempDir.toFile());
 
-      result = ResultBuilder.createInfoResult(exportedLogsZipFile.toString());
+      result = new CommandResult(exportedLogsZipFile);
     } catch (Exception ex) {
       logger.error(ex.getMessage(), ex);
       result = ResultBuilder.createGemFireErrorResult(ex.getMessage());
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java
index c7f53b1..5455c97 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshCommand.java
@@ -24,7 +24,6 @@ import org.springframework.shell.core.CommandMarker;
 
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.execute.Execution;
-import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.ClusterConfigurationService;
@@ -101,6 +100,7 @@ public interface GfshCommand extends CommandMarker {
     return getGfsh() != null;
   }
 
+
   default InternalCache getCache() {
     return (InternalCache) CacheFactory.getAnyInstance();
   }
@@ -145,55 +145,4 @@ public interface GfshCommand extends CommandMarker {
     return FunctionService.onMembers(members);
   }
 
-  default void logInfo(final String message) {
-    logInfo(message, null);
-  }
-
-  default void logInfo(final Throwable cause) {
-    logInfo(cause.getMessage(), cause);
-  }
-
-  default void logInfo(final String message, final Throwable cause) {
-    if (isLogging()) {
-      getGfsh().logInfo(message, cause);
-    }
-  }
-
-  default void logWarning(final String message) {
-    logWarning(message, null);
-  }
-
-  default void logWarning(final Throwable cause) {
-    logWarning(cause.getMessage(), cause);
-  }
-
-  default void logWarning(final String message, final Throwable cause) {
-    if (isLogging()) {
-      getGfsh().logWarning(message, cause);
-    }
-  }
-
-  default void logSevere(final String message) {
-    logSevere(message, null);
-  }
-
-  default void logSevere(final Throwable cause) {
-    logSevere(cause.getMessage(), cause);
-  }
-
-  default void logSevere(final String message, final Throwable cause) {
-    if (isLogging()) {
-      getGfsh().logSevere(message, cause);
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  default <T extends Function> T register(T function) {
-    if (FunctionService.isRegistered(function.getId())) {
-      function = (T) FunctionService.getFunction(function.getId());
-    } else {
-      FunctionService.registerFunction(function);
-    }
-    return function;
-  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueryCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueryCommand.java
index f14f3f1..e216964 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueryCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueryCommand.java
@@ -19,7 +19,6 @@ import java.io.File;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
@@ -77,8 +76,6 @@ public class QueryCommand implements GfshCommand {
       return dataResult;
     }
 
-    Object array[] = replaceGfshEnvVar(query, CommandExecutionContext.getShellEnv());
-    query = (String) array[1];
     boolean limitAdded = false;
 
     if (!StringUtils.containsIgnoreCase(query, " limit")
@@ -179,31 +176,4 @@ public class QueryCommand implements GfshCommand {
       return result;
     }
   }
-
-  private static Object[] replaceGfshEnvVar(String query, Map<String, String> gfshEnvVarMap) {
-    boolean done = false;
-    int startIndex = 0;
-    int replacedVars = 0;
-    while (!done) {
-      int index1 = query.indexOf("${", startIndex);
-      if (index1 == -1) {
-        break;
-      }
-      int index2 = query.indexOf("}", index1);
-      if (index2 == -1) {
-        break;
-      }
-      String var = query.substring(index1 + 2, index2);
-      String value = gfshEnvVarMap.get(var);
-      if (value != null) {
-        query = query.replaceAll("\\$\\{" + var + "\\}", value);
-        replacedVars++;
-      }
-      startIndex = index2 + 1;
-      if (startIndex >= query.length()) {
-        done = true;
-      }
-    }
-    return new Object[] {replacedVars, query};
-  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java
new file mode 100755
index 0000000..22956ee
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandExecutor.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal.cli.remote;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.logging.log4j.Logger;
+import org.springframework.shell.event.ParseResult;
+import org.springframework.util.ReflectionUtils;
+
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.security.NotAuthorizedException;
+
+/**
+ * this executes the command using method reflection. It logs all possible exceptions, and generates
+ * GemfireErrorResult based on the exceptions.
+ *
+ * For AuthorizationExceptions, it logs it and then rethrow it.
+ */
+public class CommandExecutor {
+  private Logger logger = LogService.getLogger();
+
+  public Object execute(ParseResult parseResult) {
+    try {
+      Object result = invokeCommand(parseResult);
+
+      if (result == null) {
+        return ResultBuilder.createGemFireErrorResult("Command returned null: " + parseResult);
+      }
+      return result;
+    } catch (NotAuthorizedException e) {
+      logger.error("Not authorized to execute \"" + parseResult + "\".", e);
+      throw e;
+    } catch (Exception e) {
+      logger.error("Could not execute \"" + parseResult + "\".", e);
+      return ResultBuilder.createGemFireErrorResult("Error while processing command <" + parseResult
+          + "> Reason : " + ExceptionUtils.getStackTrace(e));
+    }
+  }
+
+  protected Object invokeCommand(ParseResult parseResult) {
+    return ReflectionUtils.invokeMethod(parseResult.getMethod(), parseResult.getInstance(),
+        parseResult.getArguments());
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandProcessor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandProcessor.java
deleted file mode 100755
index 2464b00..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandProcessor.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.cli.remote;
-
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.util.Map;
-import java.util.Properties;
-
-import org.springframework.shell.core.Parser;
-import org.springframework.shell.event.ParseResult;
-
-import org.apache.geode.internal.security.SecurityService;
-import org.apache.geode.internal.security.SecurityServiceFactory;
-import org.apache.geode.management.cli.CommandProcessingException;
-import org.apache.geode.management.cli.CommandStatement;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.CommandManager;
-import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.LogWrapper;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.util.CommentSkipHelper;
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.ResourcePermission;
-
-/**
- * @since GemFire 7.0
- */
-public class CommandProcessor {
-  protected RemoteExecutionStrategy executionStrategy;
-  private GfshParser gfshParser;
-  private int lastExecutionStatus;
-  private LogWrapper logWrapper;
-
-  // Lock to synchronize getters & stop
-  private final Object LOCK = new Object();
-
-  private volatile boolean isStopped = false;
-
-  private final SecurityService securityService;
-
-  public CommandProcessor() throws ClassNotFoundException, IOException {
-    this(null, SecurityServiceFactory.create());
-  }
-
-  public CommandProcessor(Properties cacheProperties, SecurityService securityService)
-      throws ClassNotFoundException, IOException {
-    this.gfshParser = new GfshParser(new CommandManager(cacheProperties));
-    this.executionStrategy = new RemoteExecutionStrategy();
-    this.logWrapper = LogWrapper.getInstance();
-    this.securityService = securityService;
-  }
-
-  protected RemoteExecutionStrategy getExecutionStrategy() {
-    synchronized (LOCK) {
-      return executionStrategy;
-    }
-  }
-
-  protected Parser getParser() {
-    synchronized (LOCK) {
-      return gfshParser;
-    }
-  }
-
-  //// stripped down AbstractShell.executeCommand
-  public ParseResult parseCommand(String commentLessLine)
-      throws CommandProcessingException, IllegalStateException {
-    if (commentLessLine != null) {
-      return getParser().parse(commentLessLine);
-    }
-    throw new IllegalStateException("Command String should not be null.");
-  }
-
-  public Result executeCommand(CommandStatement cmdStmt) {
-    Object result;
-    Result commandResult = null;
-
-    CommentSkipHelper commentSkipper = new CommentSkipHelper();
-    String commentLessLine = commentSkipper.skipComments(cmdStmt.getCommandString());
-
-    if (commentLessLine != null && !commentLessLine.isEmpty()) {
-      CommandExecutionContext.setShellEnv(cmdStmt.getEnv());
-
-      final RemoteExecutionStrategy executionStrategy = getExecutionStrategy();
-      try {
-        ParseResult parseResult = ((CommandStatementImpl) cmdStmt).getParseResult();
-
-        if (parseResult == null) {
-          parseResult = parseCommand(commentLessLine);
-          if (parseResult == null) {// TODO-Abhishek: Handle this in GfshParser Implementation
-            setLastExecutionStatus(1);
-            return ResultBuilder.createParsingErrorResult(cmdStmt.getCommandString());
-          }
-          ((CommandStatementImpl) cmdStmt).setParseResult(parseResult);
-        }
-
-        // do general authorization check here
-        Method method = parseResult.getMethod();
-        ResourceOperation resourceOperation = method.getAnnotation(ResourceOperation.class);
-        if (resourceOperation != null) {
-          this.securityService.authorize(resourceOperation.resource(),
-              resourceOperation.operation(), resourceOperation.target(), ResourcePermission.ALL);
-        }
-
-        result = executionStrategy.execute(parseResult);
-        if (result instanceof Result) {
-          commandResult = (Result) result;
-        } else {
-          if (logWrapper.fineEnabled()) {
-            logWrapper.fine("Unknown result type, using toString : " + String.valueOf(result));
-          }
-          commandResult = ResultBuilder.createInfoResult(String.valueOf(result));
-        }
-      } catch (CommandProcessingException e) { // expected from Parser
-        setLastExecutionStatus(1);
-        if (logWrapper.infoEnabled()) {
-          logWrapper.info("Could not parse \"" + cmdStmt.getCommandString() + "\".", e);
-        }
-        return ResultBuilder.createParsingErrorResult(e.getMessage());
-      } catch (NotAuthorizedException e) {
-        setLastExecutionStatus(1);
-        if (logWrapper.infoEnabled()) {
-          logWrapper.info("Could not execute \"" + cmdStmt.getCommandString() + "\".", e);
-        }
-        // for NotAuthorizedException, will catch this later in the code
-        throw e;
-      } catch (RuntimeException e) {
-        setLastExecutionStatus(1);
-        if (logWrapper.infoEnabled()) {
-          logWrapper.info("Could not execute \"" + cmdStmt.getCommandString() + "\".", e);
-        }
-        return ResultBuilder.createGemFireErrorResult("Error while processing command <"
-            + cmdStmt.getCommandString() + "> Reason : " + e.getMessage());
-      } catch (Exception e) {
-        setLastExecutionStatus(1);
-        if (logWrapper.warningEnabled()) {
-          logWrapper.warning("Could not execute \"" + cmdStmt.getCommandString() + "\".", e);
-        }
-        return ResultBuilder.createGemFireErrorResult("Unexpected error while processing command <"
-            + cmdStmt.getCommandString() + "> Reason : " + e.getMessage());
-      }
-      if (logWrapper.fineEnabled()) {
-        logWrapper.fine("Executed " + commentLessLine);
-      }
-      setLastExecutionStatus(0);
-    }
-
-    return commandResult;
-  }
-
-  public CommandStatement createCommandStatement(String commandString, Map<String, String> env) {
-    return new CommandStatementImpl(commandString, env, this);
-  }
-
-  public int getLastExecutionStatus() {
-    return lastExecutionStatus;
-  }
-
-  public void setLastExecutionStatus(int lastExecutionStatus) {
-    this.lastExecutionStatus = lastExecutionStatus;
-  }
-
-  public boolean isStopped() {
-    return isStopped;
-  }
-
-  public void stop() {
-    synchronized (LOCK) {
-      this.gfshParser = null;
-      this.executionStrategy = null;
-      this.isStopped = true;
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandStatementImpl.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandStatementImpl.java
index ac510d1..c9a717e 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandStatementImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandStatementImpl.java
@@ -14,11 +14,8 @@
  */
 package org.apache.geode.management.internal.cli.remote;
 
-import java.util.Collections;
 import java.util.Map;
 
-import org.springframework.shell.event.ParseResult;
-
 import org.apache.geode.management.cli.CommandStatement;
 import org.apache.geode.management.cli.Result;
 
@@ -26,81 +23,41 @@ import org.apache.geode.management.cli.Result;
  * 
  * 
  * @since GemFire 7.0
+ *
+ * @deprecated since Geode 1.3. simply use commandProcessor to process the command
  */
 public class CommandStatementImpl implements CommandStatement {
 
-  private CommandProcessor cmdProcessor;
+  private OnlineCommandProcessor cmdProcessor;
   private String commandString;
   private Map<String, String> env;
-  private ParseResult parseResult;
 
   CommandStatementImpl(String commandString, Map<String, String> env,
-      CommandProcessor cmdProcessor) {
+      OnlineCommandProcessor cmdProcessor) {
     this.commandString = commandString;
     this.env = env;
     this.cmdProcessor = cmdProcessor;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.management.internal.cli.remote.CommandStatement#getCommandString()
-   */
   @Override
   public String getCommandString() {
     return commandString;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.management.internal.cli.remote.CommandStatement#getEnv()
-   */
   @Override
   public Map<String, String> getEnv() {
     return env;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.management.internal.cli.remote.CommandStatement#process()
-   */
   @Override
   public Result process() {
-    return cmdProcessor.executeCommand(this);
-  }
-
-  /**
-   * @return the parseResult
-   */
-  ParseResult getParseResult() {
-    return parseResult;
+    return cmdProcessor.executeCommand(commandString, env, null);
   }
 
-  /**
-   * @param parseResult the parseResult to set
-   */
-  void setParseResult(ParseResult parseResult) {
-    this.parseResult = parseResult;
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.management.internal.cli.remote.CommandStatement#validate()
-   */
-  @Override
   public boolean validate() {
-    // TODO-Abhishek: is not null check enough?
-    return parseResult != null;
+    return true;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see java.lang.Object#toString()
-   */
   @Override
   public String toString() {
     return CommandStatement.class.getSimpleName() + "[commandString=" + commandString + ", env="
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/MemberCommandService.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/MemberCommandService.java
index 25ff549..6130117 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/MemberCommandService.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/MemberCommandService.java
@@ -14,7 +14,6 @@
  */
 package org.apache.geode.management.internal.cli.remote;
 
-import java.io.IOException;
 import java.util.Map;
 
 import org.apache.geode.internal.cache.InternalCache;
@@ -23,23 +22,23 @@ import org.apache.geode.management.cli.CommandServiceException;
 import org.apache.geode.management.cli.CommandStatement;
 import org.apache.geode.management.cli.Result;
 
+/**
+ * @deprecated since 1.3 use OnlineCommandProcessor directly
+ */
+
 public class MemberCommandService extends CommandService {
   private final Object modLock = new Object();
 
   private InternalCache cache;
-  private CommandProcessor commandProcessor;
+  private OnlineCommandProcessor onlineCommandProcessor;
 
   public MemberCommandService(InternalCache cache) throws CommandServiceException {
     this.cache = cache;
     try {
-      this.commandProcessor = new CommandProcessor(cache.getDistributedSystem().getProperties(),
-          cache.getSecurityService());
-    } catch (ClassNotFoundException e) {
-      throw new CommandServiceException("Could not load commands.", e);
-    } catch (IOException e) {
+      this.onlineCommandProcessor = new OnlineCommandProcessor(
+          cache.getDistributedSystem().getProperties(), cache.getSecurityService());
+    } catch (Exception e) {
       throw new CommandServiceException("Could not load commands.", e);
-    } catch (IllegalStateException e) {
-      throw new CommandServiceException(e.getMessage(), e);
     }
   }
 
@@ -48,19 +47,21 @@ public class MemberCommandService extends CommandService {
   }
 
   public Result processCommand(String commandString, Map<String, String> env) {
-    return createCommandStatement(commandString, env).process();
+    return onlineCommandProcessor.executeCommand(commandString, env, null);
   }
 
+  @Deprecated
   public CommandStatement createCommandStatement(String commandString) {
     return this.createCommandStatement(commandString, EMPTY_ENV);
   }
 
+  @Deprecated
   public CommandStatement createCommandStatement(String commandString, Map<String, String> env) {
     if (!isUsable()) {
       throw new IllegalStateException("Cache instance is not available.");
     }
     synchronized (modLock) {
-      return commandProcessor.createCommandStatement(commandString, env);
+      return new CommandStatementImpl(commandString, env, onlineCommandProcessor);
     }
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessor.java
new file mode 100644
index 0000000..7fa4acb
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessor.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal.cli.remote;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Properties;
+
+import org.springframework.shell.core.Parser;
+import org.springframework.shell.event.ParseResult;
+import org.springframework.util.StringUtils;
+
+import org.apache.geode.annotations.TestingOnly;
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.internal.security.SecurityServiceFactory;
+import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.cli.CommandProcessingException;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.CommandManager;
+import org.apache.geode.management.internal.cli.GfshParser;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.util.CommentSkipHelper;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+/**
+ * @since GemFire 7.0
+ */
+public class OnlineCommandProcessor {
+  protected final CommandExecutor executor;
+  private final GfshParser gfshParser;
+
+  // Lock to synchronize getters & stop
+  private final Object LOCK = new Object();
+
+  private final SecurityService securityService;
+
+  @TestingOnly
+  public OnlineCommandProcessor() throws ClassNotFoundException, IOException {
+    this(new Properties(), SecurityServiceFactory.create());
+  }
+
+  public OnlineCommandProcessor(Properties cacheProperties, SecurityService securityService)
+      throws ClassNotFoundException, IOException {
+    this(cacheProperties, securityService, new CommandExecutor());
+  }
+
+  @TestingOnly
+  public OnlineCommandProcessor(Properties cacheProperties, SecurityService securityService,
+      CommandExecutor commandExecutor) {
+    this.gfshParser = new GfshParser(new CommandManager(cacheProperties));
+    this.executor = commandExecutor;
+    this.securityService = securityService;
+  }
+
+  protected CommandExecutor getCommandExecutor() {
+    synchronized (LOCK) {
+      return executor;
+    }
+  }
+
+  protected Parser getParser() {
+    synchronized (LOCK) {
+      return gfshParser;
+    }
+  }
+
+  public ParseResult parseCommand(String commentLessLine)
+      throws CommandProcessingException, IllegalStateException {
+    if (commentLessLine != null) {
+      return getParser().parse(commentLessLine);
+    }
+    throw new IllegalStateException("Command String should not be null.");
+  }
+
+  public Result executeCommand(String command) {
+    return executeCommand(command, Collections.emptyMap(), null);
+  }
+
+  public Result executeCommand(String command, Map<String, String> env) {
+    return executeCommand(command, env, null);
+  }
+
+  public Result executeCommand(String command, Map<String, String> env, byte[][] binaryData) {
+    CommentSkipHelper commentSkipper = new CommentSkipHelper();
+    String commentLessLine = commentSkipper.skipComments(command);
+    if (StringUtils.isEmpty(commentLessLine)) {
+      return null;
+    }
+
+    CommandExecutionContext.setShellEnv(env);
+    CommandExecutionContext.setBytesFromShell(binaryData);
+
+    final CommandExecutor commandExecutor = getCommandExecutor();
+    ParseResult parseResult = parseCommand(commentLessLine);
+
+    if (parseResult == null) {
+      return ResultBuilder.createParsingErrorResult(command);
+    }
+
+    Method method = parseResult.getMethod();
+
+    // do general authorization check here
+    ResourceOperation resourceOperation = method.getAnnotation(ResourceOperation.class);
+    if (resourceOperation != null) {
+      this.securityService.authorize(resourceOperation.resource(), resourceOperation.operation(),
+          resourceOperation.target(), ResourcePermission.ALL);
+    }
+
+    // this command processor does not exeucte command that needs fileData passed from client
+    CliMetaData metaData = method.getAnnotation(CliMetaData.class);
+    if (metaData != null && metaData.isFileUploaded() && binaryData == null) {
+      return ResultBuilder
+          .createUserErrorResult(command + " can not be executed only from server side");
+    }
+
+    return (Result) commandExecutor.execute(parseResult);
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/RemoteExecutionStrategy.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/RemoteExecutionStrategy.java
deleted file mode 100644
index 75dce47..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/RemoteExecutionStrategy.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.cli.remote;
-
-import java.lang.reflect.Method;
-
-import org.springframework.shell.event.ParseResult;
-import org.springframework.util.Assert;
-import org.springframework.util.ReflectionUtils;
-
-import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.management.ManagementService;
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.cli.Result.Status;
-import org.apache.geode.management.internal.cli.CliAroundInterceptor;
-import org.apache.geode.management.internal.cli.GfshParseResult;
-import org.apache.geode.management.internal.cli.LogWrapper;
-import org.apache.geode.management.internal.cli.result.FileResult;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-
-/**
- * 
- * 
- * @since GemFire 7.0
- */
-// Doesn't have to be org.springframework.roo.shell.ExecutionStrategy
-public class RemoteExecutionStrategy {
-  private LogWrapper logWrapper = LogWrapper.getInstance();
-
-  public Object execute(ParseResult parseResult) throws RuntimeException {
-    Result result = null;
-    try {
-
-      Assert.notNull(parseResult, "Parse result required");
-      if (!GfshParseResult.class.isInstance(parseResult)) {
-        // Remote command means implemented for Gfsh and ParseResult should be GfshParseResult.
-        // TODO: should this message be more specific?
-        throw new IllegalArgumentException("Command Configuration/Definition error.");
-      }
-
-      GfshParseResult gfshParseResult = (GfshParseResult) parseResult;
-
-      Method method = gfshParseResult.getMethod();
-
-      if (!isShellOnly(method, gfshParseResult)) {
-        Boolean fromShell = CommandExecutionContext.isShellRequest();
-        boolean sentFromShell = fromShell != null && fromShell.booleanValue();
-        String interceptorClass = getInterceptor(gfshParseResult.getMethod());
-        CliAroundInterceptor interceptor = null;
-
-        // 1. Pre Execution
-        if (!sentFromShell && !CliMetaData.ANNOTATION_NULL_VALUE.equals(interceptorClass)) {
-          try {
-            interceptor = (CliAroundInterceptor) ClassPathLoader.getLatest()
-                .forName(interceptorClass).newInstance();
-          } catch (InstantiationException e) {
-            logWrapper.info(e.getMessage());
-          } catch (IllegalAccessException e) {
-            logWrapper.info(e.getMessage());
-          } catch (ClassNotFoundException e) {
-            logWrapper.info(e.getMessage());
-          }
-          if (interceptor != null) {
-            Result preExecResult = interceptor.preExecution(gfshParseResult);
-            if (Status.ERROR.equals(preExecResult.getStatus())) {
-              return preExecResult;
-            } else if (preExecResult instanceof FileResult) {
-              FileResult fileResult = (FileResult) preExecResult;
-              byte[][] fileData = fileResult.toBytes();
-              CommandExecutionContext.setBytesFromShell(fileData);
-            }
-          } else {
-            return ResultBuilder
-                .createBadConfigurationErrorResult("Interceptor Configuration Error");
-          }
-        }
-        logWrapper.info("Executing " + gfshParseResult.getUserInput());
-
-        result = (Result) ReflectionUtils.invokeMethod(gfshParseResult.getMethod(),
-            gfshParseResult.getInstance(), gfshParseResult.getArguments());
-
-        if (result != null && Status.ERROR.equals(result.getStatus())) {
-          logWrapper
-              .info("Error occurred while executing \"" + gfshParseResult.getUserInput() + "\".");
-        }
-
-        if (interceptor != null) {
-          Result postExecResult = interceptor.postExecution(gfshParseResult, result, null);
-          if (postExecResult != null) {
-            if (Status.ERROR.equals(postExecResult.getStatus())) {
-              logWrapper.warning(postExecResult.toString(), null);
-            } else if (logWrapper.fineEnabled()) {
-              logWrapper.fine(String.valueOf(postExecResult));
-            }
-            result = postExecResult;
-          }
-          CommandExecutionContext.setBytesFromShell(null); // for remote commands with bytes
-        }
-      } else {
-        throw new IllegalArgumentException(
-            "Only Remote command can be executed through " + ManagementService.class.getSimpleName()
-                + ".processCommand() or ManagementMBean's processCommand "
-                + "operation. Please refer documentation for the list of " + "commands.");
-      }
-    } catch (RuntimeException e) {
-      throw e;
-    }
-    return result;
-  }
-
-  private boolean isShellOnly(Method method, GfshParseResult result) {
-    CliMetaData cliMetadata = method.getAnnotation(CliMetaData.class);
-    if (cliMetadata == null) {
-      return false;
-    }
-    if (cliMetadata.shellOnly()) {
-      return true;
-    }
-
-    return false;
-  }
-
-  private String getInterceptor(Method method) {
-    CliMetaData cliMetadata = method.getAnnotation(CliMetaData.class);
-    return cliMetadata != null ? cliMetadata.interceptor() : CliMetaData.ANNOTATION_NULL_VALUE;
-  }
-
-  public void terminate() {}
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/AbstractResultData.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/AbstractResultData.java
index 0bb96cf..847273a 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/AbstractResultData.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/AbstractResultData.java
@@ -72,9 +72,6 @@ public abstract class AbstractResultData implements ResultData {
     this.contentObject = gfJsonObject.getJSONObject(RESULT_CONTENT);
   }
 
-  /**
-   * @return the gfJsonObject
-   */
   public GfJsonObject getGfJsonObject() {
     return gfJsonObject;
   }
@@ -108,9 +105,7 @@ public abstract class AbstractResultData implements ResultData {
   }
 
   /**
-   * 
-   * @param footerText
-   * @return this ResultData
+   *
    * @throws ResultDataException If the value is non-finite number or if the key is null.
    */
   public AbstractResultData setFooter(String footerText) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/CommandResult.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/CommandResult.java
index bbb59d0..32685d5 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/CommandResult.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/CommandResult.java
@@ -15,6 +15,7 @@
 package org.apache.geode.management.internal.cli.result;
 
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -52,6 +53,12 @@ public class CommandResult implements Result {
 
   private transient int numTimesSaved;
 
+  public Path getFileToDownload() {
+    return fileToDownload;
+  }
+
+  private Path fileToDownload;
+
 
   public CommandResult(ResultData resultData) {
     this.resultData = resultData;
@@ -60,6 +67,15 @@ public class CommandResult implements Result {
     this.resultLines = new Vector<>();
   }
 
+  public CommandResult(Path fileToDownload) {
+    this(new InfoResultData(fileToDownload.toString()));
+    this.fileToDownload = fileToDownload.toAbsolutePath();
+  }
+
+  public boolean hasFileToDownload() {
+    return fileToDownload != null;
+  }
+
   @Override
   public Status getStatus() {
     return this.status;
@@ -629,4 +645,7 @@ public class CommandResult implements Result {
     this.failedToPersist = !commandPersisted;
   }
 
+  public void setFileToDownload(Path fileToDownload) {
+    this.fileToDownload = fileToDownload;
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/DownloadFileResult.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/DownloadFileResult.java
new file mode 100644
index 0000000..74b2619
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/DownloadFileResult.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.result;
+
+import java.io.IOException;
+
+import org.apache.commons.lang.Validate;
+
+import org.apache.geode.management.cli.Result;
+
+public class DownloadFileResult implements Result {
+  private String filePath = null;
+  private boolean hasLine = true;
+
+  public DownloadFileResult(String filePath) {
+    Validate.notNull(filePath);
+    this.filePath = filePath;
+  }
+
+  @Override
+  public Status getStatus() {
+    return Status.OK;
+  }
+
+  @Override
+  public void resetToFirstLine() {}
+
+  @Override
+  public boolean hasNextLine() {
+    return hasLine;
+  }
+
+  @Override
+  public String nextLine() {
+    if (hasLine) {
+      hasLine = false;
+      return filePath;
+    }
+
+    throw new IndexOutOfBoundsException();
+  }
+
+  @Override
+  public boolean hasIncomingFiles() {
+    return true;
+  }
+
+  @Override
+  public void saveIncomingFiles(String directory) throws IOException {}
+
+  @Override
+  public boolean failedToPersist() {
+    return false;
+  }
+
+  @Override
+  public void setCommandPersisted(boolean commandPersisted) {
+
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/ErrorResultData.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/ErrorResultData.java
index 7ae0d80..9edcdbf 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/ErrorResultData.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/ErrorResultData.java
@@ -26,10 +26,14 @@ import org.apache.geode.management.internal.cli.json.GfJsonObject;
 public class ErrorResultData extends InfoResultData {
   private static final String ERROR_CODE = "errorCode";
 
-  /* package */ ErrorResultData() {
+  public ErrorResultData() {
     super();
   }
 
+  public ErrorResultData(String message) {
+    super(message);
+  }
+
   // Useful on client/gfsh side to reconstruct the object
   /* package */ ErrorResultData(GfJsonObject gfJsonObject) {
     super(gfJsonObject);
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/InfoResultData.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/InfoResultData.java
index f399a54..bd12f02 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/InfoResultData.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/InfoResultData.java
@@ -18,33 +18,32 @@ import org.apache.geode.management.internal.cli.json.GfJsonException;
 import org.apache.geode.management.internal.cli.json.GfJsonObject;
 
 /**
- * 
- * 
  * @since GemFire 7.0
  */
 public class InfoResultData extends AbstractResultData {
   public static final String RESULT_CONTENT_MESSAGE = "message";
 
-  /* package */ InfoResultData() {
+  InfoResultData() {
     super();
   }
 
-  /* package */ InfoResultData(GfJsonObject gfJsonObject) {
+  InfoResultData(GfJsonObject gfJsonObject) {
     super(gfJsonObject);
   }
 
+  public InfoResultData(String message) {
+    this();
+    addLine(message);
+  }
+
   /**
-   * 
-   * @param headerText
    * @return this InfoResultData
-   * @throws ResultDataException
    */
   public InfoResultData setHeader(String headerText) {
     return (InfoResultData) super.setHeader(headerText);
   }
 
   /**
-   * 
    * @param line message to add
    * @return this InfoResultData
    */
@@ -59,10 +58,7 @@ public class InfoResultData extends AbstractResultData {
   }
 
   /**
-   * 
-   * @param footerText
    * @return this InfoResultData
-   * @throws ResultDataException
    */
   public InfoResultData setFooter(String footerText) {
     return (InfoResultData) super.setFooter(footerText);
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/ResultBuilder.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/ResultBuilder.java
index 2518793..12e799f 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/ResultBuilder.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/result/ResultBuilder.java
@@ -14,9 +14,12 @@
  */
 package org.apache.geode.management.internal.cli.result;
 
+import java.nio.file.Paths;
 import java.util.Collection;
 import java.util.List;
 
+import org.apache.commons.lang.StringUtils;
+
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.json.GfJsonException;
 import org.apache.geode.management.internal.cli.json.GfJsonObject;
@@ -115,7 +118,7 @@ public class ResultBuilder {
    * @param message Message to be shown to the user
    * @return Result for unreadable command response.
    */
-  public static Result createBadResponseErrorResult(String message) {
+  public static CommandResult createBadResponseErrorResult(String message) {
     return createErrorResult(ERRORCODE_BADRESPONSE_ERROR,
         "Could not read command response. " + message);
   }
@@ -131,7 +134,7 @@ public class ResultBuilder {
    * @return Result object with the given error code & message. If there's an exception while
    *         building result object, returns {@link #ERROR_RESULT_DEFAULT}
    */
-  private static Result createErrorResult(int errorCode, String message) {
+  private static CommandResult createErrorResult(int errorCode, String message) {
     ErrorResultData errorResultData = new ErrorResultData();
     errorResultData.setErrorCode(errorCode);
     errorResultData.addLine(message);
@@ -193,7 +196,7 @@ public class ResultBuilder {
    * @param resultData data to use to build Result
    * @return Result object built from the given ResultData
    */
-  public static Result buildResult(ResultData resultData) {
+  public static CommandResult buildResult(ResultData resultData) {
     return new CommandResult(resultData);
   }
 
@@ -205,7 +208,7 @@ public class ResultBuilder {
    * @param gfJsonObject GemFire JSON Object to use to prepare Result
    * @return Result from the given GemFire JSON Object
    */
-  public static Result fromJson(GfJsonObject gfJsonObject) {
+  public static CommandResult fromJson(GfJsonObject gfJsonObject) {
     return fromJson(gfJsonObject.toString());
   }
 
@@ -218,8 +221,8 @@ public class ResultBuilder {
    * @return Result object prepare from the JSON string. If it fails, creates an error Result for
    *         Bad Response.
    */
-  public static Result fromJson(String json) {
-    Result result;
+  public static CommandResult fromJson(String json) {
+    CommandResult result;
     try {
       GfJsonObject jsonObject = new GfJsonObject(json);
       String contentType = jsonObject.getString("contentType");
@@ -244,6 +247,11 @@ public class ResultBuilder {
 
       result = buildResult(resultData);
 
+      String fileToDownloadPath = jsonObject.getString("fileToDownload");
+      if (StringUtils.isNotBlank(fileToDownloadPath) && !fileToDownloadPath.equals("null")) {
+        result.setFileToDownload(Paths.get(fileToDownloadPath));
+      }
+
     } catch (GfJsonException e) {
       result = createBadResponseErrorResult(json);
     }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
index 2d55f42..1cf1fae 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
@@ -59,8 +59,6 @@ import org.apache.geode.management.internal.cli.LogWrapper;
 import org.apache.geode.management.internal.cli.converters.RegionPathConverter;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.CommandResult;
-import org.apache.geode.management.internal.cli.result.CompositeResultData;
-import org.apache.geode.management.internal.cli.result.CompositeResultData.SectionResultData;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.shell.jline.ANSIHandler;
 import org.apache.geode.management.internal.cli.shell.jline.ANSIHandler.ANSIStyle;
@@ -72,7 +70,8 @@ import org.apache.geode.management.internal.cli.util.CommentSkipHelper;
 /**
  * Extends an interactive shell provided by
  * <a href="https://github.com/SpringSource/spring-shell">Spring Shell</a> library.
- * <p />
+ *
+ * <p>
  * This class is used to plug-in implementations of the following Spring (Roo) Shell components
  * customized to suite GemFire Command Line Interface (CLI) requirements:
  * <ul>
@@ -83,6 +82,8 @@ import org.apache.geode.management.internal.cli.util.CommentSkipHelper;
  * Additionally, this class is used to maintain GemFire SHell (gfsh) specific information like:
  * environment
  *
+ * <p>
+ * Additionally, this class is used to maintain GemFire SHell (gfsh) specific information
  *
  * @since GemFire 7.0
  */
@@ -122,13 +123,7 @@ public class Gfsh extends JLineShell {
   public static final String ENV_SYS_OS_LINE_SEPARATOR = "SYS_OS_LINE_SEPARATOR";
   public static final String ENV_SYS_GEODE_HOME_DIR = "SYS_GEODE_HOME_DIR";
 
-  // SSL Configuration properties. keystore/truststore type is not include
-  public static final String SSL_KEYSTORE = "javax.net.ssl.keyStore";
-  public static final String SSL_KEYSTORE_PASSWORD = "javax.net.ssl.keyStorePassword";
-  public static final String SSL_TRUSTSTORE = "javax.net.ssl.trustStore";
-  public static final String SSL_TRUSTSTORE_PASSWORD = "javax.net.ssl.trustStorePassword";
-  public static final String SSL_ENABLED_CIPHERS = "javax.rmi.ssl.client.enabledCipherSuites";
-  public static final String SSL_ENABLED_PROTOCOLS = "javax.rmi.ssl.client.enabledProtocols";
+
   private static final String DEFAULT_SECONDARY_PROMPT = ">";
   private static final int DEFAULT_HEIGHT = 100;
   private static final Object INSTANCE_LOCK = new Object();
@@ -161,7 +156,7 @@ public class Gfsh extends JLineShell {
   private boolean isScriptRunning;
   private AbstractSignalNotificationHandler signalHandler;
 
-  protected Gfsh() {
+  public Gfsh() {
     this(null);
   }
 
@@ -590,7 +585,7 @@ public class Gfsh extends JLineShell {
         expandedPropCommandsMap.put(withPropsExpanded, line);
       }
       if (gfshFileLogger.fineEnabled()) {
-        gfshFileLogger.fine(logMessage + withPropsExpanded);
+        gfshFileLogger.fine(logMessage + ArgumentRedactor.redactScriptLine(withPropsExpanded));
       }
       success = super.executeScriptLine(withPropsExpanded);
     } catch (Exception e) {
@@ -1132,60 +1127,3 @@ public class Gfsh extends JLineShell {
     return output;
   }
 }
-
-
-class ScriptExecutionDetails {
-  private final String filePath;
-  private final List<CommandAndStatus> commandAndStatusList;
-
-  ScriptExecutionDetails(String filePath) {
-    this.filePath = filePath;
-    this.commandAndStatusList = new ArrayList<>();
-  }
-
-  void addCommandAndStatus(String command, String status) {
-    this.commandAndStatusList.add(new CommandAndStatus(command, status));
-  }
-
-  Result getResult() {
-    CompositeResultData compositeResultData = ResultBuilder.createCompositeResultData();
-    compositeResultData.setHeader(
-        "************************* Execution Summary ***********************\nScript file: "
-            + filePath);
-
-    for (int i = 0; i < this.commandAndStatusList.size(); i++) {
-      int commandSrNo = i + 1;
-      SectionResultData section = compositeResultData.addSection("" + (i + 1));
-      CommandAndStatus commandAndStatus = commandAndStatusList.get(i);
-      section.addData("Command-" + String.valueOf(commandSrNo), commandAndStatus.command);
-      section.addData("Status", commandAndStatus.status);
-      if (commandAndStatus.status.equals("FAILED")) {
-        compositeResultData.setStatus(Result.Status.ERROR);
-      }
-      if (i != this.commandAndStatusList.size()) {
-        section.setFooter(Gfsh.LINE_SEPARATOR);
-      }
-    }
-
-    return ResultBuilder.buildResult(compositeResultData);
-  }
-
-  void logScriptExecutionInfo(LogWrapper logWrapper, Result result) {
-    logWrapper.info(ResultBuilder.resultAsString(result));
-  }
-
-  static class CommandAndStatus {
-    private final String command;
-    private final String status;
-
-    public CommandAndStatus(String command, String status) {
-      this.command = command;
-      this.status = status;
-    }
-
-    @Override
-    public String toString() {
-      return command + "     " + status;
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
index 735143f..42faac0 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
@@ -23,11 +23,9 @@ import org.springframework.shell.core.ExecutionStrategy;
 import org.springframework.shell.core.Shell;
 import org.springframework.shell.event.ParseResult;
 import org.springframework.util.Assert;
-import org.springframework.util.ReflectionUtils;
 
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.CommandProcessingException;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.cli.Result.Status;
 import org.apache.geode.management.internal.cli.CliAroundInterceptor;
@@ -37,13 +35,13 @@ import org.apache.geode.management.internal.cli.CommandResponseBuilder;
 import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.LogWrapper;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.remote.CommandExecutor;
 import org.apache.geode.management.internal.cli.result.FileResult;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.security.NotAuthorizedException;
 
 /**
- * Defines the {@link ExecutionStrategy} for commands that are executed in GemFire SHell (gfsh).
- * 
+ * Defines the {@link ExecutionStrategy} for commands that are executed in GemFire Shell (gfsh).
  * 
  * @since GemFire 7.0
  */
@@ -57,8 +55,6 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
     this.logWrapper = LogWrapper.getInstance();
   }
 
-  //////////////// ExecutionStrategy interface Methods Start ///////////////////
-  ///////////////////////// Implemented Methods ////////////////////////////////
   /**
    * Executes the method indicated by the {@link ParseResult} which would always be
    * {@link GfshParseResult} for GemFire defined commands. If the command Method is decorated with
@@ -71,46 +67,25 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
    */
   @Override
   public Object execute(ParseResult parseResult) {
-    Result result = null;
+    Result result;
     Method method = parseResult.getMethod();
-    try {
 
-      // check if it's a shell only command
-      if (isShellOnly(method)) {
-        Assert.notNull(parseResult, "Parse result required");
-        synchronized (mutex) {
-          Assert.isTrue(isReadyForCommands(),
-              "ProcessManagerHostedExecutionStrategy not yet ready for commands");
-          return ReflectionUtils.invokeMethod(parseResult.getMethod(), parseResult.getInstance(),
-              parseResult.getArguments());
-        }
-      }
+    // check if it's a shell only command
+    if (isShellOnly(method)) {
+      Assert.notNull(parseResult, "Parse result required");
+      synchronized (mutex) {
+        Assert.isTrue(isReadyForCommands(), "Not yet ready for commands");
 
-      // check if it's a GfshParseResult
-      if (!GfshParseResult.class.isInstance(parseResult)) {
-        throw new IllegalStateException("Configuration error!");
+        return new CommandExecutor().execute(parseResult);
       }
+    }
 
-      result = executeOnRemote((GfshParseResult) parseResult);
-    } catch (NotAuthorizedException e) {
-      result = ResultBuilder
-          .createGemFireUnAuthorizedErrorResult("Unauthorized. Reason: " + e.getMessage());
-    } catch (JMXInvocationException | IllegalStateException e) {
-      Gfsh.getCurrentInstance().logWarning(e.getMessage(), e);
-    } catch (CommandProcessingException e) {
-      Gfsh.getCurrentInstance().logWarning(e.getMessage(), null);
-      Object errorData = e.getErrorData();
-      if (errorData != null && errorData instanceof Throwable) {
-        logWrapper.warning(e.getMessage(), (Throwable) errorData);
-      } else {
-        logWrapper.warning(e.getMessage());
-      }
-    } catch (Exception e) {
-      Gfsh.getCurrentInstance().logWarning("Unexpected exception occurred. " + e.getMessage(), e);
-      // Log other exceptions in gfsh log
-      logWrapper.warning("Unexpected error occurred while executing command : "
-          + ((GfshParseResult) parseResult).getUserInput(), e);
+    // check if it's a GfshParseResult
+    if (!GfshParseResult.class.isInstance(parseResult)) {
+      throw new IllegalStateException("Configuration error!");
     }
+
+    result = executeOnRemote((GfshParseResult) parseResult);
     return result;
   }
 
@@ -151,21 +126,18 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
   public void terminate() {
     shell = null;
   }
-  //////////////// ExecutionStrategy interface Methods End /////////////////////
 
   /**
    * Sends the user input (command string) via {@link OperationInvoker} to a remote GemFire node for
    * processing & execution.
    *
-   * @param parseResult
-   * 
    * @return result of execution/processing of the command
-   * 
    * @throws IllegalStateException if gfsh doesn't have an active connection.
    */
   private Result executeOnRemote(GfshParseResult parseResult) {
     Result commandResult = null;
     Object response = null;
+    Path tempFile = null;
 
     if (!shell.isConnectedAndReady()) {
       shell.logWarning(
@@ -189,17 +161,21 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
       } catch (InstantiationException | ClassNotFoundException | IllegalAccessException e) {
         shell.logWarning("Configuration error", e);
       }
-      if (interceptor != null) {
-        Result preExecResult = interceptor.preExecution(parseResult);
-        if (Status.ERROR.equals(preExecResult.getStatus())) {
-          return preExecResult;
-        } else if (preExecResult instanceof FileResult) {
-          FileResult fileResult = (FileResult) preExecResult;
-          fileData = fileResult.toBytes();
-        }
-      } else {
+
+      if (interceptor == null) {
         return ResultBuilder.createBadConfigurationErrorResult("Interceptor Configuration Error");
       }
+
+      Result preExecResult = interceptor.preExecution(parseResult);
+      if (Status.ERROR.equals(preExecResult.getStatus())) {
+        return preExecResult;
+      }
+
+      // when the preExecution yields a FileResult, we will get the fileData out of it
+      if (preExecResult instanceof FileResult) {
+        FileResult fileResult = (FileResult) preExecResult;
+        fileData = fileResult.toBytes();
+      }
     }
 
     // 2. Remote Execution
@@ -208,23 +184,22 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
       response = shell.getOperationInvoker()
           .processCommand(new CommandRequest(parseResult, env, fileData));
 
+      if (response == null) {
+        return ResultBuilder
+            .createBadResponseErrorResult("Response was null for: " + parseResult.getUserInput());
+      }
     } catch (NotAuthorizedException e) {
       return ResultBuilder
           .createGemFireUnAuthorizedErrorResult("Unauthorized. Reason : " + e.getMessage());
     } catch (Exception e) {
       shell.logSevere(e.getMessage(), e);
+      e.printStackTrace();
+      return ResultBuilder.createBadResponseErrorResult(
+          "Error occurred while executing \"" + parseResult.getUserInput() + "\" on manager.");
     } finally {
       env.clear();
     }
 
-    if (response == null) {
-      shell.logWarning("Response was null for: \"" + parseResult.getUserInput()
-          + "\". (gfsh.isConnected=" + shell.isConnectedAndReady() + ")", null);
-      return ResultBuilder.createBadResponseErrorResult(
-          " Error occurred while " + "executing \"" + parseResult.getUserInput() + "\" on manager. "
-              + "Please check manager logs for error.");
-    }
-
     // the response could be a string which is a json representation of the CommandResult object
     // it can also be a Path to a temp file downloaded from the rest http request
     if (response instanceof String) {
@@ -246,10 +221,7 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
         LogWrapper.getInstance().info(debugInfo);
       }
       commandResult = ResultBuilder.fromJson((String) response);
-    }
-
-    Path tempFile = null;
-    if (response instanceof Path) {
+    } else if (response instanceof Path) {
       tempFile = (Path) response;
     }
 
@@ -268,6 +240,11 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
       }
     }
 
+    if (commandResult == null) {
+      commandResult = ResultBuilder
+          .createGemFireErrorResult("Unable to build commandResult using the remote response.");
+    }
+
     return commandResult;
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/JmxOperationInvoker.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/JmxOperationInvoker.java
index d2407b3..1fc96a7 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/JmxOperationInvoker.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/JmxOperationInvoker.java
@@ -14,9 +14,6 @@
  */
 package org.apache.geode.management.internal.cli.shell;
 
-import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_PREFIX;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_SSL_PREFIX;
-
 import java.io.IOException;
 import java.text.MessageFormat;
 import java.util.Arrays;
@@ -44,7 +41,6 @@ import javax.management.remote.JMXConnectorFactory;
 import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
-import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.admin.SSLConfig;
 import org.apache.geode.internal.net.SSLConfigurationFactory;
 import org.apache.geode.internal.security.SecurableCommunicationChannel;
@@ -175,30 +171,6 @@ public class JmxOperationInvoker implements OperationInvoker {
     }
   }
 
-  private String checkforSystemPropertyPrefix(String key) {
-    String returnKey = key;
-    if (key.startsWith("javax.")) {
-      returnKey = key;
-    }
-    if (key.startsWith(CLUSTER_SSL_PREFIX) || key.startsWith(JMX_MANAGER_SSL_PREFIX)
-        || key.startsWith(DistributionConfig.SSL_PREFIX)) {
-      if (key.endsWith("keystore")) {
-        returnKey = Gfsh.SSL_KEYSTORE;
-      } else if (key.endsWith("keystore-password")) {
-        returnKey = Gfsh.SSL_KEYSTORE_PASSWORD;
-      } else if (key.endsWith("ciphers")) {
-        returnKey = Gfsh.SSL_ENABLED_CIPHERS;
-      } else if (key.endsWith("truststore-password")) {
-        returnKey = Gfsh.SSL_TRUSTSTORE_PASSWORD;
-      } else if (key.endsWith("truststore")) {
-        returnKey = Gfsh.SSL_TRUSTSTORE;
-      } else if (key.endsWith("protocols")) {
-        returnKey = Gfsh.SSL_ENABLED_PROTOCOLS;
-      }
-    }
-    return returnKey;
-  }
-
   @Override
   public Object getAttribute(String resourceName, String attributeName)
       throws JMXInvocationException {
@@ -274,13 +246,12 @@ public class JmxOperationInvoker implements OperationInvoker {
 
   @Override
   public Object processCommand(final CommandRequest commandRequest) throws JMXInvocationException {
+    Byte[][] binaryData = null;
     if (commandRequest.hasFileData()) {
-      return memberMXBeanProxy.processCommand(commandRequest.getInput(),
-          commandRequest.getEnvironment(), ArrayUtils.toByteArray(commandRequest.getFileData()));
-    } else {
-      return memberMXBeanProxy.processCommand(commandRequest.getInput(),
-          commandRequest.getEnvironment());
+      binaryData = ArrayUtils.toByteArray(commandRequest.getFileData());
     }
+    return memberMXBeanProxy.processCommand(commandRequest.getUserInput(),
+        commandRequest.getEnvironment(), binaryData);
   }
 
   @Override
@@ -311,14 +282,6 @@ public class JmxOperationInvoker implements OperationInvoker {
     return this.url;
   }
 
-  public String getManagerHost() {
-    return managerHost;
-  }
-
-  public int getManagerPort() {
-    return managerPort;
-  }
-
   public <T> T getMBeanProxy(final ObjectName objectName, final Class<T> mbeanInterface) {
     if (DistributedSystemMXBean.class.equals(mbeanInterface)
         && ManagementConstants.OBJECTNAME__DISTRIBUTEDSYSTEM_MXBEAN.equals(objectName.toString())) {
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/ScriptExecutionDetails.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/ScriptExecutionDetails.java
new file mode 100644
index 0000000..c267d49
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/ScriptExecutionDetails.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal.cli.shell;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.LogWrapper;
+import org.apache.geode.management.internal.cli.result.CompositeResultData;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+
+class ScriptExecutionDetails {
+  private final String filePath;
+  private final List<CommandAndStatus> commandAndStatusList;
+
+  ScriptExecutionDetails(String filePath) {
+    this.filePath = filePath;
+    this.commandAndStatusList = new ArrayList<CommandAndStatus>();
+  }
+
+  void addCommandAndStatus(String command, String status) {
+    this.commandAndStatusList.add(new CommandAndStatus(command, status));
+  }
+
+  Result getResult() {
+    CompositeResultData compositeResultData = ResultBuilder.createCompositeResultData();
+    compositeResultData.setHeader(
+        "************************* Execution Summary ***********************\nScript file: "
+            + filePath);
+
+    for (int i = 0; i < this.commandAndStatusList.size(); i++) {
+      int commandSrNo = i + 1;
+      CompositeResultData.SectionResultData section = compositeResultData.addSection("" + (i + 1));
+      CommandAndStatus commandAndStatus = commandAndStatusList.get(i);
+      section.addData("Command-" + String.valueOf(commandSrNo), commandAndStatus.command);
+      section.addData("Status", commandAndStatus.status);
+      if (commandAndStatus.status.equals("FAILED")) {
+        compositeResultData.setStatus(Result.Status.ERROR);
+      }
+      if (i != this.commandAndStatusList.size()) {
+        section.setFooter(Gfsh.LINE_SEPARATOR);
+      }
+    }
+
+    return ResultBuilder.buildResult(compositeResultData);
+  }
+
+  void logScriptExecutionInfo(LogWrapper logWrapper, Result result) {
+    logWrapper.info(ResultBuilder.resultAsString(result));
+  }
+
+  static class CommandAndStatus {
+    private final String command;
+    private final String status;
+
+    public CommandAndStatus(String command, String status) {
+      this.command = command;
+      this.status = status;
+    }
+
+    @Override
+    public String toString() {
+      return command + "     " + status;
+    }
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/AbstractCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/AbstractCommandsController.java
index 0a18ec5..aa530a5 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/AbstractCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/AbstractCommandsController.java
@@ -14,6 +14,35 @@
  */
 package org.apache.geode.management.internal.web.controllers;
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.management.ManagementFactory;
+import java.net.URI;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import javax.management.JMX;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.Query;
+import javax.management.QueryExp;
+
+import org.apache.logging.log4j.Logger;
+import org.springframework.beans.propertyeditors.StringArrayPropertyEditor;
+import org.springframework.http.HttpStatus;
+import org.springframework.http.ResponseEntity;
+import org.springframework.web.bind.WebDataBinder;
+import org.springframework.web.bind.annotation.ExceptionHandler;
+import org.springframework.web.bind.annotation.InitBinder;
+import org.springframework.web.bind.annotation.ResponseBody;
+import org.springframework.web.bind.annotation.ResponseStatus;
+import org.springframework.web.context.request.WebRequest;
+import org.springframework.web.servlet.support.ServletUriComponentsBuilder;
+
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.lang.StringUtils;
@@ -27,38 +56,9 @@ import org.apache.geode.management.internal.MBeanJMXAdapter;
 import org.apache.geode.management.internal.ManagementConstants;
 import org.apache.geode.management.internal.SystemManagementService;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.management.internal.web.controllers.support.LoginHandlerInterceptor;
 import org.apache.geode.management.internal.web.util.UriUtils;
 import org.apache.geode.security.NotAuthorizedException;
-import org.apache.logging.log4j.Logger;
-import org.springframework.beans.propertyeditors.StringArrayPropertyEditor;
-import org.springframework.http.HttpStatus;
-import org.springframework.http.ResponseEntity;
-import org.springframework.web.bind.WebDataBinder;
-import org.springframework.web.bind.annotation.ExceptionHandler;
-import org.springframework.web.bind.annotation.InitBinder;
-import org.springframework.web.bind.annotation.ResponseBody;
-import org.springframework.web.bind.annotation.ResponseStatus;
-import org.springframework.web.context.request.WebRequest;
-import org.springframework.web.servlet.support.ServletUriComponentsBuilder;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.management.ManagementFactory;
-import java.net.URI;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import javax.management.JMX;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.Query;
-import javax.management.QueryExp;
 
 /**
  * The AbstractCommandsController class is the abstract base class encapsulating common
@@ -292,7 +292,7 @@ public abstract class AbstractCommandsController {
    * @see java.net.URI
    * @see org.springframework.web.servlet.support.ServletUriComponentsBuilder
    */
-  protected /* static */ URI toUri(final String path, final String scheme) {
+  public static URI toUri(final String path, final String scheme) {
     return ServletUriComponentsBuilder.fromCurrentContextPath().path(REST_API_VERSION).path(path)
         .scheme(scheme).build().toUri();
   }
@@ -503,120 +503,6 @@ public abstract class AbstractCommandsController {
   }
 
   /**
-   * Adds the named option to the command String to be processed if the named option has value or
-   * the named option is present in the HTTP request.
-   * 
-   * @param request the WebRequest object encapsulating the details (headers, request parameters and
-   *        message body) of the user HTTP request.
-   * @param command the Gfsh command String to append options and process.
-   * @param optionName the name of the command option.
-   * @param optionValue the value for the named command option.
-   * @see #hasValue(Object)
-   * @see #hasValue(String[])
-   * @see org.apache.geode.management.internal.cli.util.CommandStringBuilder
-   * @see org.springframework.web.context.request.WebRequest
-   */
-  protected void addCommandOption(final WebRequest request, final CommandStringBuilder command,
-      final String optionName, final Object optionValue) {
-    assertNotNull(command, "The command to append options to cannot be null!");
-    assertNotNull(optionName, "The name of the option to add to the command cannot be null!");
-
-    if (hasValue(optionValue)) {
-      final String optionValueString = (optionValue instanceof String[]
-          ? StringUtils.join((String[]) optionValue, StringUtils.COMMA_DELIMITER)
-          : String.valueOf(optionValue));
-      command.addOption(optionName, optionValueString);
-    } else if (request != null && request.getParameterMap().containsKey(optionName)) {
-      command.addOption(optionName);
-    } else {
-      // do nothing!
-    }
-  }
-
-  /**
-   * Executes the specified command as entered by the user using the GemFire Shell (Gfsh). Note,
-   * Gfsh performs validation of the command during parsing before sending the command to the
-   * Manager for processing.
-   *
-   * @param command a String value containing a valid command String as would be entered by the user
-   *        in Gfsh.
-   * @return a result of the command execution as a String, typically marshalled in JSON to be
-   *         serialized back to Gfsh.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   * @see LoginHandlerInterceptor#getEnvironment()
-   * @see #getEnvironment()
-   * @see #processCommand(String, java.util.Map, byte[][])
-   */
-  protected String processCommand(final String command) {
-    return processCommand(command, getEnvironment(), null);
-  }
-
-  protected Callable<ResponseEntity<String>> getProcessCommandCallable(final String command) {
-    return getProcessCommandCallable(command, getEnvironment(), null);
-  }
-
-  protected Callable<ResponseEntity<String>> getProcessCommandCallable(final String command,
-      final Map<String, String> environment, final byte[][] fileData) {
-    Callable callable = new Callable<ResponseEntity<String>>() {
-      @Override
-      public ResponseEntity<String> call() throws Exception {
-        String result = null;
-        try {
-          result = processCommand(command, environment, fileData);
-        } catch (NotAuthorizedException ex) {
-          return new ResponseEntity<String>(ex.getMessage(), HttpStatus.FORBIDDEN);
-        } catch (Exception ex) {
-          return new ResponseEntity<String>(ex.getMessage(), HttpStatus.INTERNAL_SERVER_ERROR);
-        }
-        return new ResponseEntity<String>(result, HttpStatus.OK);
-      }
-    };
-    return getCache().getSecurityService().associateWith(callable);
-  }
-
-  /**
-   * Executes the specified command as entered by the user using the GemFire Shell (Gfsh). Note,
-   * Gfsh performs validation of the command during parsing before sending the command to the
-   * Manager for processing.
-   * 
-   * @param command a String value containing a valid command String as would be entered by the user
-   *        in Gfsh.
-   * @param fileData is a two-dimensional byte array containing the pathnames and contents of file
-   *        data streamed to the Manager, usually for the 'deploy' Gfsh command.
-   * @return a result of the command execution as a String, typically marshalled in JSON to be
-   *         serialized back to Gfsh.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   * @see LoginHandlerInterceptor#getEnvironment()
-   * @see #getEnvironment()
-   * @see #processCommand(String, java.util.Map, byte[][])
-   */
-  protected String processCommand(final String command, final byte[][] fileData) {
-    return processCommand(command, getEnvironment(), fileData);
-  }
-
-  /**
-   * Executes the specified command as entered by the user using the GemFire Shell (Gfsh). Note,
-   * Gfsh performs validation of the command during parsing before sending the command to the
-   * Manager for processing.
-   * 
-   * @param command a String value containing a valid command String as would be entered by the user
-   *        in Gfsh.
-   * @param environment a Map containing any environment configuration settings to be used by the
-   *        Manager during command execution. For example, when executing commands originating from
-   *        Gfsh, the key/value pair (APP_NAME=gfsh) is a specified mapping in the "environment.
-   *        Note, it is common for the REST API to act as a bridge, or an adapter between Gfsh and
-   *        the Manager, and thus need to specify this key/value pair mapping.
-   * @return a result of the command execution as a String, typically marshalled in JSON to be
-   *         serialized back to Gfsh.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   * @see LoginHandlerInterceptor#getEnvironment()
-   * @see #processCommand(String, java.util.Map, byte[][])
-   */
-  protected String processCommand(final String command, final Map<String, String> environment) {
-    return processCommand(command, environment, null);
-  }
-
-  /**
    * Executes the specified command as entered by the user using the GemFire Shell (Gfsh). Note,
    * Gfsh performs validation of the command during parsing before sending the command to the
    * Manager for processing.
@@ -636,9 +522,9 @@ public abstract class AbstractCommandsController {
    */
   protected String processCommand(final String command, final Map<String, String> environment,
       final byte[][] fileData) {
-    logger.info(LogMarker.CONFIG,
+    logger.debug(LogMarker.CONFIG,
         "Processing Command ({}) with Environment ({}) having File Data ({})...", command,
-        environment, (fileData != null));
+        environment, (fileData != null && fileData.length > 0));
     return getManagingMemberMXBean().processCommand(command, environment,
         ArrayUtils.toByteArray(fileData));
   }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ClientCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ClientCommandsController.java
deleted file mode 100644
index e8df505..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ClientCommandsController.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The ShellCommandsController class implements GemFire REST API calls for Gfsh Shell Commands.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.cli.commands.ListClientCommand
- * @see org.apache.geode.management.internal.cli.commands.DescribeClientCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("clientController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class ClientCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/clients")
-  @ResponseBody
-  public String listClients() {
-    return processCommand(CliStrings.LIST_CLIENTS);
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/clients/{clientID}")
-  @ResponseBody
-  public String describeClient(@PathVariable("clientID") final String clientId) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESCRIBE_CLIENT);
-    command.addOption(CliStrings.DESCRIBE_CLIENT__ID, decode(clientId));
-    return processCommand(command.toString());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ClusterCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ClusterCommandsController.java
deleted file mode 100644
index fae10de..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ClusterCommandsController.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-/**
- * The ClusterCommandsController class implements GemFire REST API web service calls for the Gfsh
- * Cluster (System)-based commands.
- *
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @since GemFire 8.0
- */
-@Controller("clusterController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class ClusterCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/services/cluster-config")
-  @ResponseBody
-  public String statusClusterConfig() {
-    return processCommand(CliStrings.STATUS_SHARED_CONFIG);
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ConfigCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ConfigCommandsController.java
deleted file mode 100644
index d223a9f..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ConfigCommandsController.java
+++ /dev/null
@@ -1,222 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import java.io.IOException;
-import java.util.concurrent.Callable;
-
-import org.springframework.http.ResponseEntity;
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-import org.springframework.web.multipart.MultipartFile;
-
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-import org.apache.geode.management.internal.web.util.ConvertUtils;
-
-/**
- * The ConfigCommandsController class implements GemFire Management REST API web service endpoints
- * for the Gfsh Config Commands.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.cli.commands.AlterRuntimeConfigCommand
- * @see org.apache.geode.management.internal.cli.commands.DescribeConfigCommand
- * @see org.apache.geode.management.internal.cli.commands.ExportConfigCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractMultiPartCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("configController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class ConfigCommandsController extends AbstractMultiPartCommandsController {
-
-  @RequestMapping(method = RequestMethod.POST, value = "/config")
-  @ResponseBody
-  public String alterRuntime(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String group,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT,
-          required = false) final Integer archiveDiskSpaceLimit,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT,
-          required = false) final Integer archiveFileSizeLimit,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT,
-          required = false) final Integer logDiskSpaceLimit,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT,
-          required = false) final Integer logFileSizeLimit,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL,
-          required = false) final String logLevel,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE,
-          required = false) final String statisticsArchiveFile,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE,
-          required = false) final Integer statisticsSampleRate,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLING__ENABLED,
-          required = false) final Boolean enableStatistics,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ,
-          required = false) final Boolean copyOnRead,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE,
-          required = false) final Integer lockLease,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT,
-          required = false) final Integer lockTimeout,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL,
-          required = false) final Integer messageSyncInterval,
-      @RequestParam(value = CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT,
-          required = false) final Integer searchTimeout) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.ALTER_RUNTIME_CONFIG);
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(group)) {
-      command.addOption(CliStrings.GROUP, group);
-    }
-
-    if (hasValue(archiveDiskSpaceLimit)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT,
-          Integer.toString(archiveDiskSpaceLimit));
-    }
-
-    if (hasValue(archiveFileSizeLimit)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT,
-          Integer.toString(archiveFileSizeLimit));
-    }
-
-    if (hasValue(logDiskSpaceLimit)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT,
-          Integer.toString(logDiskSpaceLimit));
-    }
-
-    if (hasValue(logFileSizeLimit)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT,
-          Integer.toString(logFileSizeLimit));
-    }
-
-    if (hasValue(logLevel)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL, logLevel);
-    }
-
-    if (hasValue(statisticsArchiveFile)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE,
-          statisticsArchiveFile);
-    }
-
-    if (hasValue(statisticsSampleRate)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE,
-          Integer.toString(statisticsSampleRate));
-    }
-
-    if (hasValue(enableStatistics)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLING__ENABLED,
-          Boolean.toString(enableStatistics));
-    }
-
-    if (hasValue(copyOnRead)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ,
-          Boolean.toString(copyOnRead));
-    }
-
-    if (hasValue(lockLease)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE, Integer.toString(lockLease));
-    }
-
-    if (hasValue(lockTimeout)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT,
-          Integer.toString(lockTimeout));
-    }
-
-    if (hasValue(messageSyncInterval)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL,
-          Integer.toString(messageSyncInterval));
-    }
-
-    if (hasValue(searchTimeout)) {
-      command.addOption(CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT,
-          Integer.toString(searchTimeout));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/members/{member}/config")
-  @ResponseBody
-  public String describeConfig(@PathVariable("member") final String memberNameId,
-      @RequestParam(value = CliStrings.DESCRIBE_CONFIG__HIDE__DEFAULTS,
-          defaultValue = "true") final Boolean hideDefaults) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESCRIBE_CONFIG);
-
-    command.addOption(CliStrings.MEMBER, decode(memberNameId));
-    command.addOption(CliStrings.DESCRIBE_CONFIG__HIDE__DEFAULTS, String.valueOf(hideDefaults));
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/config")
-  public Callable<ResponseEntity<String>> exportConfig(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members,
-      @RequestParam(value = CliStrings.EXPORT_CONFIG__DIR,
-          required = false) final String directory) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.EXPORT_CONFIG);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(directory)) {
-      command.addOption(CliStrings.EXPORT_CONFIG__DIR, decode(directory));
-    }
-
-    return getProcessCommandCallable(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/config/cluster")
-  public Callable<ResponseEntity<String>> exportClusterConfig(
-      @RequestParam(CliStrings.EXPORT_SHARED_CONFIG__FILE) final String zipFile) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.EXPORT_SHARED_CONFIG);
-
-    command.addOption(CliStrings.EXPORT_SHARED_CONFIG__FILE, zipFile);
-
-    return getProcessCommandCallable(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/config/cluster")
-  public Callable<ResponseEntity<String>> importClusterConfig(
-      @RequestParam(RESOURCES_REQUEST_PARAMETER) final MultipartFile[] zipFileResources,
-      @RequestParam(value = CliStrings.IMPORT_SHARED_CONFIG__ZIP) final String zipFileName)
-      throws IOException {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.IMPORT_SHARED_CONFIG);
-
-    command.addOption(CliStrings.IMPORT_SHARED_CONFIG__ZIP, zipFileName);
-
-    return getProcessCommandCallable(command.toString(), getEnvironment(),
-        ConvertUtils.convert(zipFileResources));
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DataCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DataCommandsController.java
deleted file mode 100644
index 3c58f50..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DataCommandsController.java
+++ /dev/null
@@ -1,238 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import java.util.concurrent.Callable;
-
-import org.apache.commons.lang.StringUtils;
-import org.springframework.http.ResponseEntity;
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-import org.springframework.web.context.request.WebRequest;
-
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The DataCommandsController class implements GemFire Management REST API web service endpoints for
- * the Gfsh Data Commands.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.cli.commands.ExportDataCommand
- * @see org.apache.geode.management.internal.cli.commands.GetCommand
- * @see org.apache.geode.management.internal.cli.commands.ImportDataCommand
- * @see org.apache.geode.management.internal.cli.commands.LocateEntryCommand
- * @see org.apache.geode.management.internal.cli.commands.PutCommand
- * @see org.apache.geode.management.internal.cli.commands.RebalanceCommand
- * @see org.apache.geode.management.internal.cli.commands.RemoveCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("dataController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class DataCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/regions/{region}/data")
-  @ResponseBody
-  public String get(@PathVariable("region") final String regionNamePath,
-      @RequestParam(CliStrings.GET__KEY) final String key,
-      @RequestParam(value = CliStrings.GET__KEYCLASS, required = false) final String keyClassName,
-      @RequestParam(value = CliStrings.GET__VALUEKLASS,
-          required = false) final String valueClassName,
-      @RequestParam(value = CliStrings.GET__LOAD,
-          defaultValue = "true") final Boolean loadOnCacheMiss) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.GET);
-
-    command.addOption(CliStrings.GET__REGIONNAME, decode(regionNamePath));
-    command.addOption(CliStrings.GET__KEY, key);
-    command.addOption(CliStrings.GET__LOAD, String.valueOf(Boolean.TRUE.equals(loadOnCacheMiss)));
-
-    if (hasValue(keyClassName)) {
-      command.addOption(CliStrings.GET__KEYCLASS, keyClassName);
-    }
-
-    if (hasValue(valueClassName)) {
-      command.addOption(CliStrings.GET__VALUEKLASS, valueClassName);
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.PUT, value = "/regions/{region}/data")
-  @ResponseBody
-  public String put(@PathVariable("region") final String regionNamePath,
-      @RequestParam(CliStrings.PUT__KEY) final String key,
-      @RequestParam(value = CliStrings.PUT__KEYCLASS, required = false) final String keyClassName,
-      @RequestParam(CliStrings.PUT__VALUE) final String value,
-      @RequestParam(value = CliStrings.PUT__VALUEKLASS,
-          required = false) final String valueClassName,
-      @RequestParam(value = CliStrings.PUT__PUTIFABSENT,
-          defaultValue = "false") final Boolean putIfAbsent) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.PUT);
-
-    command.addOption(CliStrings.PUT__REGIONNAME, decode(regionNamePath));
-    command.addOption(CliStrings.PUT__KEY, key);
-    command.addOption(CliStrings.PUT__VALUE, decode(value));
-
-    if (hasValue(keyClassName)) {
-      command.addOption(CliStrings.PUT__KEYCLASS, keyClassName);
-    }
-
-    if (hasValue(valueClassName)) {
-      command.addOption(CliStrings.PUT__VALUEKLASS, valueClassName);
-    }
-
-    command.addOption(CliStrings.PUT__PUTIFABSENT, String.valueOf(putIfAbsent));
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.DELETE, value = "/regions/{region}/data")
-  @ResponseBody
-  public String remove(@PathVariable("region") final String regionNamePath,
-      @RequestParam(value = CliStrings.REMOVE__ALL, defaultValue = "false") final Boolean allKeys,
-      @RequestParam(value = CliStrings.REMOVE__KEY, required = false) final String key,
-      @RequestParam(value = CliStrings.REMOVE__KEYCLASS,
-          required = false) final String keyClassName) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.REMOVE);
-
-    command.addOption(CliStrings.REMOVE__REGION, decode(regionNamePath));
-    command.addOption(CliStrings.REMOVE__ALL, String.valueOf(allKeys));
-
-    if (key != null) {
-      command.addOption(CliStrings.REMOVE__KEY, key);
-    }
-
-    if (hasValue(keyClassName)) {
-      command.addOption(CliStrings.REMOVE__KEYCLASS, keyClassName);
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/members/{member}/regions/{region}/data")
-  public Callable<ResponseEntity<String>> exportData(
-      @PathVariable("member") final String memberNameId,
-      @PathVariable("region") final String regionNamePath,
-      @RequestParam(CliStrings.EXPORT_DATA__FILE) final String file) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.EXPORT_DATA);
-
-    command.addOption(CliStrings.MEMBER, decode(memberNameId));
-    command.addOption(CliStrings.EXPORT_DATA__REGION, decode(regionNamePath));
-    command.addOption(CliStrings.EXPORT_DATA__FILE, decode(file));
-
-    return getProcessCommandCallable(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/members/{member}/regions/{region}/data")
-  public Callable<ResponseEntity<String>> importData(
-      @PathVariable("member") final String memberNameId,
-      @PathVariable("region") final String regionNamePath,
-      @RequestParam(value = CliStrings.IMPORT_DATA__INVOKE_CALLBACKS,
-          required = false) final boolean invokeCallbacks,
-      @RequestParam(CliStrings.IMPORT_DATA__FILE) final String file) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.IMPORT_DATA);
-
-    command.addOption(CliStrings.MEMBER, decode(memberNameId));
-    command.addOption(CliStrings.IMPORT_DATA__REGION, decode(regionNamePath));
-    command.addOption(CliStrings.IMPORT_DATA__FILE, decode(file));
-    command.addOption(CliStrings.IMPORT_DATA__INVOKE_CALLBACKS, invokeCallbacks + "");
-
-    return getProcessCommandCallable(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/regions/{region}/data/location")
-  @ResponseBody
-  public String locateEntry(@PathVariable("region") final String regionNamePath,
-      @RequestParam(CliStrings.LOCATE_ENTRY__KEY) final String key,
-      @RequestParam(value = CliStrings.LOCATE_ENTRY__KEYCLASS,
-          required = false) final String keyClassName,
-      @RequestParam(value = CliStrings.LOCATE_ENTRY__VALUEKLASS,
-          required = false) final String valueClassName,
-      @RequestParam(value = CliStrings.LOCATE_ENTRY__RECURSIVE,
-          defaultValue = "false") final Boolean recursive) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.LOCATE_ENTRY);
-
-    command.addOption(CliStrings.LOCATE_ENTRY__REGIONNAME, decode(regionNamePath));
-    command.addOption(CliStrings.LOCATE_ENTRY__KEY, key);
-
-    if (hasValue(keyClassName)) {
-      command.addOption(CliStrings.LOCATE_ENTRY__KEYCLASS, keyClassName);
-    }
-
-    if (hasValue(valueClassName)) {
-      command.addOption(CliStrings.LOCATE_ENTRY__VALUEKLASS, valueClassName);
-    }
-
-    command.addOption(CliStrings.LOCATE_ENTRY__RECURSIVE, String.valueOf(recursive));
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/regions/data/query")
-  public Callable<ResponseEntity<String>> query(final WebRequest request,
-      @RequestParam(CliStrings.QUERY__QUERY) final String oql,
-      @RequestParam(value = CliStrings.QUERY__INTERACTIVE,
-          defaultValue = "true") final Boolean interactive) {
-
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.QUERY);
-
-    command.addOption(CliStrings.QUERY__QUERY, decode(oql));
-    command.addOption(CliStrings.QUERY__INTERACTIVE,
-        String.valueOf(Boolean.TRUE.equals(interactive)));
-
-    return getProcessCommandCallable(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/regions/data", params = "op=rebalance")
-  public Callable<ResponseEntity<String>> rebalance(
-      @RequestParam(value = CliStrings.REBALANCE__INCLUDEREGION,
-          required = false) final String[] includedRegions,
-      @RequestParam(value = CliStrings.REBALANCE__EXCLUDEREGION,
-          required = false) final String[] excludedRegions,
-      @RequestParam(value = CliStrings.REBALANCE__SIMULATE,
-          defaultValue = "false") final Boolean simulate,
-      @RequestParam(value = CliStrings.REBALANCE__TIMEOUT,
-          defaultValue = "-1") final Long timeout) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.REBALANCE);
-
-    if (hasValue(includedRegions)) {
-      command.addOption(CliStrings.REBALANCE__INCLUDEREGION,
-          StringUtils.join(includedRegions, ","));
-    }
-
-    if (hasValue(excludedRegions)) {
-      command.addOption(CliStrings.REBALANCE__EXCLUDEREGION,
-          StringUtils.join(excludedRegions, ","));
-    }
-
-    command.addOption(CliStrings.REBALANCE__SIMULATE, String.valueOf(simulate));
-    command.addOption(CliStrings.REBALANCE__TIMEOUT, String.valueOf(timeout));
-
-    return getProcessCommandCallable(command.toString());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DeployCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DeployCommandsController.java
deleted file mode 100644
index 9d3e086..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DeployCommandsController.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import java.io.IOException;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-import org.springframework.web.multipart.MultipartFile;
-
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-import org.apache.geode.management.internal.web.util.ConvertUtils;
-
-/**
- * The DeployCommandsController class implements the GemFire Management REST API web service
- * endpoints for the Gfsh Deploy Commands.
- * <p/>
- *
- * @see org.apache.geode.management.internal.cli.commands.DeployCommand
- * @see org.apache.geode.management.internal.cli.commands.UndeployCommand
- * @see org.apache.geode.management.internal.cli.commands.ListDeployedCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractMultiPartCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("deployController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class DeployCommandsController extends AbstractMultiPartCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/deployed")
-  @ResponseBody
-  public String listDeployed(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.LIST_DEPLOYED);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/deployed")
-  @ResponseBody
-  public String deploy(
-      @RequestParam(RESOURCES_REQUEST_PARAMETER) final MultipartFile[] jarFileResources,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.JAR, required = false) final String jarFileName,
-      @RequestParam(value = CliStrings.DEPLOY__DIR, required = false) final String directory)
-      throws IOException {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.DEPLOY);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(jarFileName)) {
-      command.addOption(CliStrings.JAR, jarFileName);
-    }
-
-    if (hasValue(directory)) {
-      command.addOption(CliStrings.DEPLOY__DIR, directory);
-    }
-    return processCommand(command.toString(), ConvertUtils.convert(jarFileResources));
-  }
-
-  @RequestMapping(method = RequestMethod.DELETE, value = "/deployed")
-  @ResponseBody
-  public String undeploy(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.JAR, required = false) final String[] jarFileNames) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.UNDEPLOY);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(jarFileNames)) {
-      command.addOption(CliStrings.JAR,
-          StringUtils.join(jarFileNames, StringUtils.COMMA_DELIMITER));
-    }
-    return processCommand(command.toString());
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DiskStoreCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DiskStoreCommandsController.java
deleted file mode 100644
index 1f646d6..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DiskStoreCommandsController.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import java.util.concurrent.Callable;
-
-import org.springframework.http.ResponseEntity;
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.commands.ListDiskStoresCommand;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The DiskStoreCommandsController class implements GemFire Management REST API web service
- * endpoints for the Gfsh Disk Store Commands.
- * <p/>
- *
- * @see org.apache.geode.management.internal.cli.commands.AlterOfflineDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.BackupDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.CompactDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.CompactOfflineDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.CreateDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.DescribeDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.DescribeOfflineDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.DestroyDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.ExportOfflineDiskStoreCommand
- * @see ListDiskStoresCommand
- * @see org.apache.geode.management.internal.cli.commands.RevokeMissingDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.ShowMissingDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.UpgradeOfflineDiskStoreCommand
- * @see org.apache.geode.management.internal.cli.commands.ValidateDiskStoreCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("diskStoreController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class DiskStoreCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/diskstores")
-  @ResponseBody
-  public String listDiskStores() {
-    return processCommand(CliStrings.LIST_DISK_STORE);
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/diskstores", params = "op=backup")
-  public Callable<ResponseEntity<String>> backupDiskStore(
-      @RequestParam(value = CliStrings.BACKUP_DISK_STORE__DISKDIRS) final String dir,
-      @RequestParam(value = CliStrings.BACKUP_DISK_STORE__BASELINEDIR,
-          required = false) final String baselineDir) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.BACKUP_DISK_STORE);
-
-    command.addOption(CliStrings.BACKUP_DISK_STORE__DISKDIRS, decode(dir));
-
-    if (hasValue(baselineDir)) {
-      command.addOption(CliStrings.BACKUP_DISK_STORE__BASELINEDIR, decode(baselineDir));
-    }
-
-    return getProcessCommandCallable(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/diskstores/{name}", params = "op=compact")
-  public Callable<ResponseEntity<String>> compactDiskStore(
-      @PathVariable("name") final String diskStoreNameId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.COMPACT_DISK_STORE);
-
-    command.addOption(CliStrings.COMPACT_DISK_STORE__NAME, decode(diskStoreNameId));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    return getProcessCommandCallable(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/diskstores")
-  @ResponseBody
-  public String createDiskStore(
-      @RequestParam(CliStrings.CREATE_DISK_STORE__NAME) final String diskStoreNameId,
-      @RequestParam(
-          value = CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE) final String[] directoryAndSizes,
-      @RequestParam(value = CliStrings.CREATE_DISK_STORE__ALLOW_FORCE_COMPACTION,
-          defaultValue = "false") final Boolean allowForceCompaction,
-      @RequestParam(value = CliStrings.CREATE_DISK_STORE__AUTO_COMPACT,
-          defaultValue = "true") final Boolean autoCompact,
-      @RequestParam(value = CliStrings.CREATE_DISK_STORE__COMPACTION_THRESHOLD,
-          defaultValue = "50") final Integer compactionThreshold,
-      @RequestParam(value = CliStrings.CREATE_DISK_STORE__MAX_OPLOG_SIZE,
-          defaultValue = "1024") final Integer maxOplogSize,
-      @RequestParam(value = CliStrings.CREATE_DISK_STORE__QUEUE_SIZE,
-          defaultValue = "0") final Integer queueSize,
-      @RequestParam(value = CliStrings.CREATE_DISK_STORE__TIME_INTERVAL,
-          defaultValue = "1000") final Long timeInterval,
-      @RequestParam(value = CliStrings.CREATE_DISK_STORE__WRITE_BUFFER_SIZE,
-          defaultValue = "32768") final Integer writeBufferSize,
-      @RequestParam(value = CliStrings.CREATE_DISK_STORE__DISK_USAGE_WARNING_PCT,
-          defaultValue = "90") final Float diskUsageWarningPercentage,
-      @RequestParam(value = CliStrings.CREATE_DISK_STORE__DISK_USAGE_CRITICAL_PCT,
-          defaultValue = "99") final Float diskUsageCriticalPercentage,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.CREATE_DISK_STORE);
-
-    command.addOption(CliStrings.CREATE_DISK_STORE__NAME, diskStoreNameId);
-    if (hasValue(directoryAndSizes)) {
-      command.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE,
-          StringUtils.join(directoryAndSizes, StringUtils.COMMA_DELIMITER));
-    }
-    command.addOption(CliStrings.CREATE_DISK_STORE__ALLOW_FORCE_COMPACTION,
-        String.valueOf(Boolean.TRUE.equals(allowForceCompaction)));
-    command.addOption(CliStrings.CREATE_DISK_STORE__AUTO_COMPACT,
-        String.valueOf(Boolean.TRUE.equals(autoCompact)));
-    command.addOption(CliStrings.CREATE_DISK_STORE__COMPACTION_THRESHOLD,
-        String.valueOf(compactionThreshold));
-    command.addOption(CliStrings.CREATE_DISK_STORE__MAX_OPLOG_SIZE, String.valueOf(maxOplogSize));
-    command.addOption(CliStrings.CREATE_DISK_STORE__QUEUE_SIZE, String.valueOf(queueSize));
-    command.addOption(CliStrings.CREATE_DISK_STORE__TIME_INTERVAL, String.valueOf(timeInterval));
-    command.addOption(CliStrings.CREATE_DISK_STORE__WRITE_BUFFER_SIZE,
-        String.valueOf(writeBufferSize));
-    command.addOption(CliStrings.CREATE_DISK_STORE__DISK_USAGE_WARNING_PCT,
-        String.valueOf(diskUsageWarningPercentage));
-    command.addOption(CliStrings.CREATE_DISK_STORE__DISK_USAGE_CRITICAL_PCT,
-        String.valueOf(diskUsageCriticalPercentage));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/diskstores/{name}")
-  @ResponseBody
-  public String describeDiskStore(@PathVariable("name") final String diskStoreNameId,
-      @RequestParam(CliStrings.MEMBER) final String memberNameId) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESCRIBE_DISK_STORE);
-    command.addOption(CliStrings.MEMBER, memberNameId);
-    command.addOption(CliStrings.DESCRIBE_DISK_STORE__NAME, decode(diskStoreNameId));
-    return processCommand(command.toString());
-  }
-
-  // TODO determine whether Async functionality is required
-  @RequestMapping(method = RequestMethod.DELETE, value = "/diskstores/{name}")
-  @ResponseBody
-  public String destroyDiskStore(@PathVariable("name") final String diskStoreNameId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESTROY_DISK_STORE);
-
-    command.addOption(CliStrings.DESTROY_DISK_STORE__NAME, decode(diskStoreNameId));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  // TODO determine whether Async functionality is required
-  @RequestMapping(method = RequestMethod.POST, value = "/diskstores/{id}", params = "op=revoke")
-  @ResponseBody
-  public String revokeMissingDiskStore(@PathVariable("id") final String diskStoreId) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.REVOKE_MISSING_DISK_STORE);
-    command.addOption(CliStrings.REVOKE_MISSING_DISK_STORE__ID, decode(diskStoreId));
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/diskstores/missing")
-  @ResponseBody
-  public String showMissingDiskStores() {
-    return processCommand(CliStrings.SHOW_MISSING_DISK_STORE);
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DurableClientCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DurableClientCommandsController.java
deleted file mode 100644
index 0e62e71..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DurableClientCommandsController.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.distributed.ConfigurationProperties;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The DurableClientCommandsController class implements GemFire Management REST API web service
- * endpoints for the durable client/CQs Gfsh commands.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.cli.commands.CloseDurableClientCommand
- * @see org.apache.geode.management.internal.cli.commands.CloseDurableCQsCommand
- * @see org.apache.geode.management.internal.cli.commands.CountDurableCQEventsCommand
- * @see org.apache.geode.management.internal.cli.commands.ListDurableClientCQsCommand
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("durableClientController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class DurableClientCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/durable-clients/{durable-client-id}/cqs")
-  @ResponseBody
-  public String listDurableClientContinuousQueries(
-      @PathVariable(ConfigurationProperties.DURABLE_CLIENT_ID) final String durableClientId,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.LIST_DURABLE_CQS);
-
-    command.addOption(CliStrings.LIST_DURABLE_CQS__DURABLECLIENTID, decode(durableClientId));
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET,
-      value = "/durable-clients/{durable-client-id}/cqs/events")
-  @ResponseBody
-  public String countDurableClientContinuousQueryEvents(
-      @PathVariable(ConfigurationProperties.DURABLE_CLIENT_ID) final String durableClientId,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    return internalCountDurableClientContinuousQueryEvents(decode(durableClientId), null,
-        memberNameId, groups);
-  }
-
-  @RequestMapping(method = RequestMethod.GET,
-      value = "/durable-clients/{durable-client-id}/cqs/{durable-cq-name}/events")
-  @ResponseBody
-  public String countDurableClientContinuousQueryEvents(
-      @PathVariable(ConfigurationProperties.DURABLE_CLIENT_ID) final String durableClientId,
-      @PathVariable("durable-cq-name") final String durableCqName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    return internalCountDurableClientContinuousQueryEvents(decode(durableClientId),
-        decode(durableCqName), memberNameId, groups);
-  }
-
-  protected String internalCountDurableClientContinuousQueryEvents(final String durableClientId,
-      final String cqName, final String memberNameId, final String[] groups) {
-    final CommandStringBuilder command =
-        new CommandStringBuilder(CliStrings.COUNT_DURABLE_CQ_EVENTS);
-
-    command.addOption(CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CLIENT__ID, durableClientId);
-
-    if (hasValue(cqName)) {
-      command.addOption(CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CQ__NAME, cqName);
-    }
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/durable-clients/{durable-client-id}",
-      params = "op=close")
-  @ResponseBody
-  public String closeDurableClient(
-      @PathVariable(ConfigurationProperties.DURABLE_CLIENT_ID) final String durableClientId,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.CLOSE_DURABLE_CLIENTS);
-
-    command.addOption(CliStrings.CLOSE_DURABLE_CLIENTS__CLIENT__ID, decode(durableClientId));
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST,
-      value = "/durable-clients/{durable-client-id}/cqs/{durable-cq-name}", params = "op=close")
-  @ResponseBody
-  public String closeDurableContinuousQuery(
-      @PathVariable(ConfigurationProperties.DURABLE_CLIENT_ID) final String durableClientId,
-      @PathVariable("durable-cq-name") final String durableCqName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.CLOSE_DURABLE_CQS);
-
-    command.addOption(CliStrings.CLOSE_DURABLE_CQS__DURABLE__CLIENT__ID, decode(durableClientId));
-    command.addOption(CliStrings.CLOSE_DURABLE_CQS__NAME, decode(durableCqName));
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java
deleted file mode 100644
index 87afc24..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.management.internal.web.controllers;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-import org.springframework.core.io.InputStreamResource;
-import org.springframework.http.HttpHeaders;
-import org.springframework.http.HttpStatus;
-import org.springframework.http.MediaType;
-import org.springframework.http.ResponseEntity;
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-
-import java.io.File;
-import java.io.FileInputStream;
-
-@Controller("exportLogController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-public class ExportLogController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/logs")
-  public ResponseEntity<InputStreamResource> exportLogs(
-      @RequestParam(value = CliStrings.EXPORT_LOGS__DIR, required = false) final String directory,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.EXPORT_LOGS__LOGLEVEL,
-          required = false) final String logLevel,
-      @RequestParam(value = CliStrings.EXPORT_LOGS__UPTO_LOGLEVEL,
-          defaultValue = "false") final Boolean onlyLogLevel,
-      @RequestParam(value = CliStrings.EXPORT_LOGS__MERGELOG,
-          defaultValue = "false") final Boolean mergeLog,
-      @RequestParam(value = CliStrings.EXPORT_LOGS__STARTTIME,
-          required = false) final String startTime,
-      @RequestParam(value = CliStrings.EXPORT_LOGS__ENDTIME, required = false) final String endTime,
-      @RequestParam(value = CliStrings.EXPORT_LOGS__LOGSONLY,
-          required = false) final boolean logsOnly,
-      @RequestParam(value = CliStrings.EXPORT_LOGS__STATSONLY,
-          required = false) final boolean statsOnly,
-      @RequestParam(value = CliStrings.EXPORT_LOGS__FILESIZELIMIT,
-          required = false) final String fileSizeLimit) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.EXPORT_LOGS);
-
-    command.addOption(CliStrings.EXPORT_LOGS__DIR, decode(directory));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(logLevel)) {
-      command.addOption(CliStrings.EXPORT_LOGS__LOGLEVEL, logLevel);
-    }
-
-    command.addOption(CliStrings.EXPORT_LOGS__UPTO_LOGLEVEL, String.valueOf(onlyLogLevel));
-    command.addOption(CliStrings.EXPORT_LOGS__MERGELOG, String.valueOf(mergeLog));
-    command.addOption(CliStrings.EXPORT_LOGS__LOGSONLY, String.valueOf(logsOnly));
-    command.addOption(CliStrings.EXPORT_LOGS__STATSONLY, String.valueOf(statsOnly));
-    command.addOption(CliStrings.EXPORT_LOGS__FILESIZELIMIT, fileSizeLimit);
-
-    if (hasValue(startTime)) {
-      command.addOption(CliStrings.EXPORT_LOGS__STARTTIME, startTime);
-    }
-
-    if (hasValue(endTime)) {
-      command.addOption(CliStrings.EXPORT_LOGS__ENDTIME, endTime);
-    }
-
-    String result = processCommand(command.toString());
-    return getResponse(result);
-
-  }
-
-  ResponseEntity<InputStreamResource> getResponse(String result) {
-    // the result is json string from CommandResult
-    Result commandResult = ResultBuilder.fromJson(result);
-    if (commandResult.getStatus().equals(Result.Status.OK)) {
-      return getOKResponse(commandResult);
-
-    } else {
-      return getErrorResponse(result);
-    }
-  }
-
-  private ResponseEntity<InputStreamResource> getErrorResponse(String result) {
-    HttpHeaders respHeaders = new HttpHeaders();
-    InputStreamResource isr;// if the command is successful, the output is the filepath,
-    // else we need to send the orignal result back so that the receiver will know to turn it
-    // into a Result object
-    try {
-      isr = new InputStreamResource(IOUtils.toInputStream(result, "UTF-8"));
-      respHeaders.set(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON_VALUE);
-      return new ResponseEntity<InputStreamResource>(isr, respHeaders, HttpStatus.OK);
-    } catch (Exception e) {
-      throw new RuntimeException("IO Error writing file to output stream", e);
-    }
-  }
-
-  private ResponseEntity<InputStreamResource> getOKResponse(Result commandResult) {
-    HttpHeaders respHeaders = new HttpHeaders();
-    InputStreamResource isr;// if the command is successful, the output is the filepath,
-    String filePath = commandResult.nextLine().trim();
-    File zipFile = new File(filePath);
-    try {
-      isr = new InputStreamResource(new FileInputStream(zipFile));
-      respHeaders.set(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_OCTET_STREAM_VALUE);
-      return new ResponseEntity<InputStreamResource>(isr, respHeaders, HttpStatus.OK);
-    } catch (Exception e) {
-      throw new RuntimeException("IO Error writing file to output stream", e);
-    } finally {
-      FileUtils.deleteQuietly(zipFile);
-    }
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/FunctionCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/FunctionCommandsController.java
deleted file mode 100644
index e8885da..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/FunctionCommandsController.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import java.util.concurrent.Callable;
-
-import org.springframework.http.ResponseEntity;
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The FunctionCommandsController class implements GemFire Management REST API web service endpoints
- * for the Gfsh Function Commands.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.cli.commands.DestroyFunctionCommand
- * @see org.apache.geode.management.internal.cli.commands.ExecuteFunctionCommand
- * @see org.apache.geode.management.internal.cli.commands.ListFunctionCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("functionController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class FunctionCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/functions")
-  @ResponseBody
-  public String listFunctions(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members,
-      @RequestParam(value = CliStrings.LIST_FUNCTION__MATCHES,
-          required = false) final String matches) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.LIST_FUNCTION);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(matches)) {
-      command.addOption(CliStrings.LIST_FUNCTION__MATCHES, matches);
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/functions/{id}")
-  public Callable<ResponseEntity<String>> executeFunction(
-      @PathVariable("id") final String functionId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String groupName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.EXECUTE_FUNCTION__ONREGION,
-          required = false) final String regionNamePath,
-      @RequestParam(value = CliStrings.EXECUTE_FUNCTION__ARGUMENTS,
-          required = false) final String[] arguments,
-      @RequestParam(value = CliStrings.EXECUTE_FUNCTION__FILTER,
-          required = false) final String filter,
-      @RequestParam(value = CliStrings.EXECUTE_FUNCTION__RESULTCOLLECTOR,
-          required = false) final String resultCollector) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.EXECUTE_FUNCTION);
-
-    command.addOption(CliStrings.EXECUTE_FUNCTION__ID, decode(functionId));
-
-    if (hasValue(groupName)) {
-      command.addOption(CliStrings.GROUP, groupName);
-    }
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(regionNamePath)) {
-      command.addOption(CliStrings.EXECUTE_FUNCTION__ONREGION, regionNamePath);
-    }
-
-    if (hasValue(arguments)) {
-      command.addOption(CliStrings.EXECUTE_FUNCTION__ARGUMENTS,
-          StringUtils.join(arguments, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(filter)) {
-      command.addOption(CliStrings.EXECUTE_FUNCTION__FILTER, filter);
-    }
-
-    if (hasValue(resultCollector)) {
-      command.addOption(CliStrings.EXECUTE_FUNCTION__RESULTCOLLECTOR, resultCollector);
-    }
-    return getProcessCommandCallable(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.DELETE, value = "/functions/{id}")
-  @ResponseBody
-  public String destroyFunction(@PathVariable("id") final String functionId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String groupName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESTROY_FUNCTION);
-
-    command.addOption(CliStrings.DESTROY_FUNCTION__ID, decode(functionId));
-
-    if (hasValue(groupName)) {
-      command.addOption(CliStrings.GROUP, groupName);
-    }
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    return processCommand(command.toString());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/IndexCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/IndexCommandsController.java
deleted file mode 100644
index 09d7f9a..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/IndexCommandsController.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The IndexCommandsController class implements the REST API calls for the Gfsh Index commands.
- * </p>
- * 
- * @see org.apache.geode.management.internal.cli.commands.ClearDefinedIndexesCommand
- * @see org.apache.geode.management.internal.cli.commands.CreateDefinedIndexesCommand
- * @see org.apache.geode.management.internal.cli.commands.CreateIndexCommand
- * @see org.apache.geode.management.internal.cli.commands.DefineIndexCommand
- * @see org.apache.geode.management.internal.cli.commands.DestroyIndexCommand
- * @see org.apache.geode.management.internal.cli.commands.ListIndexCommand
- * @see org.apache.geode.management.internal.cli.util.CommandStringBuilder
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("indexController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class IndexCommandsController extends AbstractCommandsController {
-
-  private static final String DEFAULT_INDEX_TYPE = "range";
-
-  @RequestMapping(method = RequestMethod.GET, value = "/indexes")
-  @ResponseBody
-  public String listIndex(@RequestParam(value = CliStrings.LIST_INDEX__STATS,
-      defaultValue = "false") final Boolean withStats) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.LIST_INDEX);
-    command.addOption(CliStrings.LIST_INDEX__STATS, String.valueOf(Boolean.TRUE.equals(withStats)));
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/indexes")
-  @ResponseBody
-  public String createIndex(@RequestParam(CliStrings.CREATE_INDEX__NAME) final String name,
-      @RequestParam(CliStrings.CREATE_INDEX__EXPRESSION) final String expression,
-      @RequestParam(CliStrings.CREATE_INDEX__REGION) final String regionNamePath,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String groupName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.CREATE_INDEX__TYPE,
-          defaultValue = DEFAULT_INDEX_TYPE) final String type) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.CREATE_INDEX);
-
-    command.addOption(CliStrings.CREATE_INDEX__NAME, name);
-    command.addOption(CliStrings.CREATE_INDEX__EXPRESSION, expression);
-    command.addOption(CliStrings.CREATE_INDEX__REGION, regionNamePath);
-    command.addOption(CliStrings.CREATE_INDEX__TYPE, type);
-
-    if (hasValue(groupName)) {
-      command.addOption(CliStrings.GROUP, groupName);
-    }
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/indexes", params = "op=create-defined")
-  @ResponseBody
-  public String createDefinedIndexes(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String groupName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.CREATE_DEFINED_INDEXES);
-
-    if (hasValue(groupName)) {
-      command.addOption(CliStrings.GROUP, groupName);
-    }
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.DELETE, value = "/indexes", params = "op=clear-defined")
-  @ResponseBody
-  public String clearDefinedIndexes() {
-    return processCommand(CliStrings.CLEAR_DEFINED_INDEXES);
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/indexes", params = "op=define")
-  @ResponseBody
-  public String defineIndex(@RequestParam(CliStrings.DEFINE_INDEX_NAME) final String name,
-      @RequestParam(CliStrings.DEFINE_INDEX__EXPRESSION) final String expression,
-      @RequestParam(CliStrings.DEFINE_INDEX__REGION) final String regionNamePath,
-      @RequestParam(value = CliStrings.DEFINE_INDEX__TYPE,
-          defaultValue = DEFAULT_INDEX_TYPE) final String type) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.DEFINE_INDEX);
-
-    command.addOption(CliStrings.DEFINE_INDEX_NAME, name);
-    command.addOption(CliStrings.DEFINE_INDEX__EXPRESSION, expression);
-    command.addOption(CliStrings.DEFINE_INDEX__REGION, regionNamePath);
-    command.addOption(CliStrings.DEFINE_INDEX__TYPE, type);
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.DELETE, value = "/indexes")
-  @ResponseBody
-  public String destroyIndexes(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String groupName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.DESTROY_INDEX__REGION,
-          required = false) final String regionNamePath) {
-    return internalDestroyIndex(null, groupName, memberNameId, regionNamePath);
-  }
-
-  @RequestMapping(method = RequestMethod.DELETE, value = "/indexes/{name}")
-  @ResponseBody
-  public String destroyIndex(@PathVariable("name") final String indexName,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String groupName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.DESTROY_INDEX__REGION,
-          required = false) final String regionNamePath) {
-    return internalDestroyIndex(decode(indexName), groupName, memberNameId, regionNamePath);
-  }
-
-  protected String internalDestroyIndex(final String indexName, final String groupName,
-      final String memberNameId, final String regionNamePath) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESTROY_INDEX);
-
-    if (hasValue(indexName)) {
-      command.addOption(CliStrings.DESTROY_INDEX__NAME, indexName);
-    }
-
-    if (hasValue(groupName)) {
-      command.addOption(CliStrings.GROUP, groupName);
-    }
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(regionNamePath)) {
-      command.addOption(CliStrings.DESTROY_INDEX__REGION, regionNamePath);
-    }
-
-    return processCommand(command.toString());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/LauncherLifecycleCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/LauncherLifecycleCommandsController.java
deleted file mode 100644
index 2e10842..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/LauncherLifecycleCommandsController.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-/**
- * The LauncherLifecycleCommandsController class implements REST API calls for the Gfsh Launcher
- * Lifecycle commands.
- * <p/>
- *
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("launcherLifecycleController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class LauncherLifecycleCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/members/{name}/locator")
-  @ResponseBody
-  public String statusLocator(@PathVariable("name") final String memberNameId) {
-    return getMemberMXBean(memberNameId).status();
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/members/{name}/server")
-  @ResponseBody
-  public String statusServer(@PathVariable("name") final String memberNameId) {
-    return getMemberMXBean(memberNameId).status();
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
deleted file mode 100644
index c9e7900..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.management.internal.cli.commands.ListMembersCommand;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The MemberCommandsController class implements GemFire Management REST API web service endpoints
- * for the Gfsh Member Commands.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.cli.commands.ListMembersCommand
- * @see org.apache.geode.management.internal.cli.commands.DescribeMemberCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("memberController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class MemberCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/members")
-  @ResponseBody
-  public String listMembers(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String groupName) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.LIST_MEMBER);
-    if (hasValue(groupName)) {
-      command.addOption(CliStrings.GROUP, groupName);
-    }
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/members/{name}")
-  @ResponseBody
-  public String describeMember(@PathVariable("name") final String memberNameId) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESCRIBE_MEMBER);
-    command.addOption(CliStrings.DESCRIBE_MEMBER__IDENTIFIER, decode(memberNameId));
-    return processCommand(command.toString());
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MiscellaneousCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MiscellaneousCommandsController.java
deleted file mode 100644
index 7bce3d0..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MiscellaneousCommandsController.java
+++ /dev/null
@@ -1,249 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The MiscellaneousCommandsController class implements GemFire Management REST API web service
- * endpoints for the Gfsh Miscellaneous Commands.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.cli.commands.ChangeLogLevelCommand
- * @see org.apache.geode.management.internal.cli.commands.ExportStackTraceCommand
- * @see org.apache.geode.management.internal.cli.commands.GCCommand
- * @see org.apache.geode.management.internal.cli.commands.NetstatCommand
- * @see org.apache.geode.management.internal.cli.commands.ShowDeadlockCommand
- * @see org.apache.geode.management.internal.cli.commands.ShowLogCommand
- * @see org.apache.geode.management.internal.cli.commands.ShowMetricsCommand
- * @see org.apache.geode.management.internal.cli.commands.ShutdownCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("miscellaneousController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class MiscellaneousCommandsController extends AbstractCommandsController {
-  // TODO determine whether Async functionality is required
-  @RequestMapping(method = RequestMethod.GET, value = "/stacktraces")
-  @ResponseBody
-  public String exportStackTraces(
-      @RequestParam(value = CliStrings.EXPORT_STACKTRACE__FILE, required = false) final String file,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String groupName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.EXPORT_STACKTRACE__FAIL__IF__FILE__PRESENT,
-          required = false) final boolean failIfFilePresent) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.EXPORT_STACKTRACE);
-
-    if (hasValue(file)) {
-      command.addOption(CliStrings.EXPORT_STACKTRACE__FILE, decode(file));
-    }
-
-    if (hasValue(groupName)) {
-      command.addOption(CliStrings.GROUP, groupName);
-    }
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(failIfFilePresent)) {
-      command.addOption(CliStrings.EXPORT_STACKTRACE__FAIL__IF__FILE__PRESENT,
-          String.valueOf(failIfFilePresent));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  // TODO add Async functionality
-  @RequestMapping(method = RequestMethod.POST, value = "/gc")
-  @ResponseBody
-  public String gc(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.GC);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  // TODO add Async functionality
-  @RequestMapping(method = RequestMethod.POST, value = "/members/{member}/gc")
-  @ResponseBody
-  public String gc(@PathVariable("member") final String memberNameId) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.GC);
-    command.addOption(CliStrings.MEMBER, decode(memberNameId));
-    return processCommand(command.toString());
-  }
-
-  // TODO add Async functionality
-  @RequestMapping(method = RequestMethod.GET, value = "/netstat")
-  @ResponseBody
-  public String netstat(
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String group,
-      @RequestParam(value = CliStrings.NETSTAT__FILE, required = false) final String file,
-      @RequestParam(value = CliStrings.NETSTAT__WITHLSOF,
-          defaultValue = "false") final Boolean withLsof) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.NETSTAT);
-
-    addCommandOption(null, command, CliStrings.MEMBER, members);
-    addCommandOption(null, command, CliStrings.GROUP, group);
-    addCommandOption(null, command, CliStrings.NETSTAT__FILE, file);
-    addCommandOption(null, command, CliStrings.NETSTAT__WITHLSOF, withLsof);
-
-    return processCommand(command.toString());
-  }
-
-  // TODO determine if Async functionality is required
-  @RequestMapping(method = RequestMethod.GET, value = "/deadlocks")
-  @ResponseBody
-  public String showDeadLock(
-      @RequestParam(CliStrings.SHOW_DEADLOCK__DEPENDENCIES__FILE) final String dependenciesFile) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.SHOW_DEADLOCK);
-    command.addOption(CliStrings.SHOW_DEADLOCK__DEPENDENCIES__FILE, decode(dependenciesFile));
-    return processCommand(command.toString());
-  }
-
-  // TODO determine if Async functionality is required
-  @RequestMapping(method = RequestMethod.GET, value = "/members/{member}/log")
-  @ResponseBody
-  public String showLog(@PathVariable("member") final String memberNameId,
-      @RequestParam(value = CliStrings.SHOW_LOG_LINE_NUM, defaultValue = "0") final Integer lines) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.SHOW_LOG);
-
-    command.addOption(CliStrings.MEMBER, decode(memberNameId));
-    command.addOption(CliStrings.SHOW_LOG_LINE_NUM, String.valueOf(lines));
-
-    return processCommand(command.toString());
-  }
-
-  // TODO determine if Async functionality is required
-  @RequestMapping(method = RequestMethod.GET, value = "/metrics")
-  @ResponseBody
-  public String showMetrics(
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId,
-      @RequestParam(value = CliStrings.SHOW_METRICS__REGION,
-          required = false) final String regionNamePath,
-      @RequestParam(value = CliStrings.SHOW_METRICS__FILE, required = false) final String file,
-      @RequestParam(value = CliStrings.SHOW_METRICS__CACHESERVER__PORT,
-          required = false) final String cacheServerPort,
-      @RequestParam(value = CliStrings.SHOW_METRICS__CATEGORY,
-          required = false) final String[] categories) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.SHOW_METRICS);
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    if (hasValue(regionNamePath)) {
-      command.addOption(CliStrings.SHOW_METRICS__REGION, regionNamePath);
-    }
-
-    if (hasValue(file)) {
-      command.addOption(CliStrings.SHOW_METRICS__FILE, file);
-    }
-
-    if (hasValue(cacheServerPort)) {
-      command.addOption(CliStrings.SHOW_METRICS__CACHESERVER__PORT, cacheServerPort);
-    }
-
-    if (hasValue(categories)) {
-      command.addOption(CliStrings.SHOW_METRICS__CATEGORY,
-          StringUtils.join(categories, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/shutdown")
-  @ResponseBody
-  public String shutdown(
-      @RequestParam(value = CliStrings.SHUTDOWN__TIMEOUT,
-          defaultValue = "-1") final Integer timeout,
-      @RequestParam(value = CliStrings.INCLUDE_LOCATORS,
-          defaultValue = "false") final boolean includeLocators) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.SHUTDOWN);
-    command.addOption(CliStrings.SHUTDOWN__TIMEOUT, String.valueOf(timeout));
-    command.addOption(CliStrings.INCLUDE_LOCATORS, String.valueOf(includeLocators));
-    return processCommand(command.toString());
-  }
-
-  // TODO determine whether the {groups} and {members} path variables corresponding to the --groups
-  // and --members
-  // command-line options in the 'change loglevel' Gfsh command actually accept multiple values,
-  // and...
-  // TODO if so, then change the groups and members method parameters to String[] types.
-  // TODO If not, then these options should be renamed!
-
-  @RequestMapping(method = RequestMethod.POST, value = "/groups/{groups}/loglevel")
-  @ResponseBody
-  public String changeLogLevelForGroups(@PathVariable("groups") final String groups,
-      @RequestParam(value = CliStrings.CHANGE_LOGLEVEL__LOGLEVEL) final String logLevel) {
-    return internalChangeLogLevel(groups, null, logLevel);
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/members/{members}/loglevel")
-  @ResponseBody
-  public String changeLogLevelForMembers(@PathVariable("members") final String members,
-      @RequestParam(value = CliStrings.CHANGE_LOGLEVEL__LOGLEVEL) final String logLevel) {
-    return internalChangeLogLevel(null, members, logLevel);
-  }
-
-  @RequestMapping(method = RequestMethod.POST,
-      value = "/members/{members}/groups/{groups}/loglevel")
-  @ResponseBody
-  public String changeLogLevelForMembersAndGroups(@PathVariable("members") final String members,
-      @PathVariable("groups") final String groups,
-      @RequestParam(value = CliStrings.CHANGE_LOGLEVEL__LOGLEVEL) final String logLevel) {
-    return internalChangeLogLevel(groups, members, logLevel);
-  }
-
-  // NOTE since "logLevel" is "required", then just set the option; no need to validate it's value.
-  private String internalChangeLogLevel(final String groups, final String members,
-      final String logLevel) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.CHANGE_LOGLEVEL);
-
-    command.addOption(CliStrings.CHANGE_LOGLEVEL__LOGLEVEL, decode(logLevel));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, decode(groups));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, decode(members));
-    }
-
-    return processCommand(command.toString());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/PdxCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/PdxCommandsController.java
deleted file mode 100644
index dd84444..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/PdxCommandsController.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The PdxCommandsController class implements GemFire Management REST API web service endpoints for
- * Gfsh PDX Commands.
- *
- * @see org.apache.geode.management.internal.cli.commands.PDXRenameCommand
- * @see org.apache.geode.management.internal.cli.commands.ConfigurePDXCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("pdxController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class PdxCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.POST, value = "/pdx")
-  @ResponseBody
-  public String configurePdx(
-      @RequestParam(value = CliStrings.CONFIGURE_PDX__READ__SERIALIZED,
-          required = false) final Boolean readSerialized,
-      @RequestParam(value = CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS,
-          required = false) final Boolean ignoreUnreadFields,
-      @RequestParam(value = CliStrings.CONFIGURE_PDX__DISKSTORE,
-          required = false) final String diskStore,
-      @RequestParam(value = CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES,
-          required = false) final String[] autoSerializerClasses,
-      @RequestParam(value = CliStrings.CONFIGURE_PDX__PORTABLE__AUTO__SERIALIZER__CLASSES,
-          required = false) final String[] portableAutoSerializerClasses) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.CONFIGURE_PDX);
-
-    if (Boolean.TRUE.equals(readSerialized)) {
-      command.addOption(CliStrings.CONFIGURE_PDX__READ__SERIALIZED, String.valueOf(readSerialized));
-    }
-
-    if (Boolean.TRUE.equals(ignoreUnreadFields)) {
-      command.addOption(CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS,
-          String.valueOf(ignoreUnreadFields));
-    }
-
-    if (hasValue(diskStore)) {
-      command.addOption(CliStrings.CONFIGURE_PDX__DISKSTORE, diskStore);
-    }
-
-    if (hasValue(autoSerializerClasses)) {
-      command.addOption(CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES,
-          StringUtils.join(autoSerializerClasses, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(portableAutoSerializerClasses)) {
-      command.addOption(CliStrings.CONFIGURE_PDX__PORTABLE__AUTO__SERIALIZER__CLASSES,
-          StringUtils.join(portableAutoSerializerClasses, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  // @RequestMapping(method = RequestMethod.DELETE, value = "/pdx/type/field")
-  // @ResponseBody
-  public String pdxDeleteField(@RequestParam(value = CliStrings.PDX_CLASS) final String className,
-      @RequestParam(value = CliStrings.PDX_FIELD) final String fieldName,
-      @RequestParam(value = CliStrings.PDX_DISKSTORE) final String diskStore,
-      @RequestParam(value = CliStrings.PDX_DISKDIR, required = false) final String[] diskDirs) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.PDX_DELETE_FIELD);
-
-    command.addOption(CliStrings.PDX_CLASS, className);
-    command.addOption(CliStrings.PDX_FIELD, fieldName);
-    command.addOption(CliStrings.PDX_DISKSTORE, diskStore);
-
-    if (hasValue(diskDirs)) {
-      command.addOption(CliStrings.PDX_DISKDIR,
-          StringUtils.join(diskDirs, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/pdx/type")
-  @ResponseBody
-  public String pdxRename(
-      @RequestParam(value = CliStrings.PDX_RENAME_OLD) final String oldClassName,
-      @RequestParam(value = CliStrings.PDX_RENAME_NEW) final String newClassName,
-      @RequestParam(value = CliStrings.PDX_DISKSTORE) final String diskStore,
-      @RequestParam(value = CliStrings.PDX_DISKDIR, required = false) final String[] diskDirs) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.PDX_RENAME);
-
-    command.addOption(CliStrings.PDX_RENAME_OLD, oldClassName);
-    command.addOption(CliStrings.PDX_RENAME_NEW, newClassName);
-    command.addOption(CliStrings.PDX_DISKSTORE, diskStore);
-
-    if (hasValue(diskDirs)) {
-      command.addOption(CliStrings.PDX_DISKDIR,
-          StringUtils.join(diskDirs, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/QueueCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/QueueCommandsController.java
deleted file mode 100644
index 9fea79a..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/QueueCommandsController.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The QueueCommandsController class implements GemFire Management REST API web service endpoints
- * for the Gfsh Queue Commands.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.cli.commands.CreateAsyncEventQueueCommand
- * @see org.apache.geode.management.internal.cli.commands.ListAsyncEventQueuesCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("queueController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class QueueCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.POST, value = "/async-event-queues")
-  @ResponseBody
-  public String createAsyncEventQueue(
-      @RequestParam(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ID) final String asyncEventQueueId,
-      @RequestParam(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER) final String listener,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE,
-          required = false) final String[] listenerParametersValues,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL,
-          defaultValue = "false") final Boolean parallel,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ENABLEBATCHCONFLATION,
-          defaultValue = "false") final Boolean enableBatchConflation,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCH_SIZE,
-          defaultValue = "100") final Integer batchSize,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCHTIMEINTERVAL,
-          defaultValue = "1000") final Integer batchTimeInterval,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT,
-          defaultValue = "false") final Boolean persistent,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISK_STORE,
-          required = false) final String diskStore,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS,
-          defaultValue = "true") final Boolean diskSynchronous,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY,
-          defaultValue = "false") final Boolean forwardExpirationDestroy,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY,
-          defaultValue = "100") final Integer maxQueueMemory,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS,
-          defaultValue = "1") final Integer dispatcherThreads,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY,
-          defaultValue = "KEY") final String orderPolicy,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER,
-          required = false) final String[] gatewayEventFilters,
-      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER,
-          required = false) final String gatewaySubstitutionFilter)
-
-  {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.CREATE_ASYNC_EVENT_QUEUE);
-
-    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ID, asyncEventQueueId);
-    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER, listener);
-
-    if (hasValue(listenerParametersValues)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE,
-          StringUtils.join(listenerParametersValues, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL,
-        String.valueOf(Boolean.TRUE.equals(parallel)));
-    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ENABLEBATCHCONFLATION,
-        String.valueOf(Boolean.TRUE.equals(enableBatchConflation)));
-    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY,
-        String.valueOf(forwardExpirationDestroy));
-
-    if (hasValue(batchSize)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCH_SIZE, String.valueOf(batchSize));
-    }
-
-    if (hasValue(batchTimeInterval)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCHTIMEINTERVAL,
-          String.valueOf(batchTimeInterval));
-    }
-
-    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT,
-        String.valueOf(Boolean.TRUE.equals(persistent)));
-
-    if (hasValue(diskStore)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISK_STORE, diskStore);
-    }
-
-    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS,
-        String.valueOf(Boolean.TRUE.equals(diskSynchronous)));
-
-    if (hasValue(maxQueueMemory)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY,
-          String.valueOf(maxQueueMemory));
-    }
-
-    if (hasValue(dispatcherThreads)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS,
-          String.valueOf(dispatcherThreads));
-    }
-
-    if (hasValue(orderPolicy)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY, orderPolicy);
-    }
-
-    if (hasValue(gatewayEventFilters)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER,
-          StringUtils.join(gatewayEventFilters, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(gatewaySubstitutionFilter)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER,
-          gatewaySubstitutionFilter);
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/async-event-queues")
-  @ResponseBody
-  public String listAsyncEventQueues() {
-    return processCommand(CliStrings.LIST_ASYNC_EVENT_QUEUES);
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/RegionCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/RegionCommandsController.java
deleted file mode 100644
index 9180414..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/RegionCommandsController.java
+++ /dev/null
@@ -1,401 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import org.apache.commons.lang.StringUtils;
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-import org.springframework.web.context.request.WebRequest;
-
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The ListRegionCommand and DescribeRegionCommand classes implement GemFire Management REST API web
- * service endpoints for the Gfsh Region Commands.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.cli.commands.ListRegionCommand
- * @see org.apache.geode.management.internal.cli.commands.DescribeRegionCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.PathVariable
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("regionController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class RegionCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/regions")
-  @ResponseBody
-  public String listRegions(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String groupName,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String memberNameId) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.LIST_REGION);
-
-    if (hasValue(groupName)) {
-      command.addOption(CliStrings.GROUP, groupName);
-    }
-
-    if (hasValue(memberNameId)) {
-      command.addOption(CliStrings.MEMBER, memberNameId);
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/regions/{name}")
-  @ResponseBody
-  public String describeRegion(@PathVariable("name") final String regionNamePath) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESCRIBE_REGION);
-    command.addOption(CliStrings.DESCRIBE_REGION__NAME, decode(regionNamePath));
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.PUT, value = "/regions/{name}")
-  @ResponseBody
-  public String alterRegion(final WebRequest request,
-      @PathVariable("name") final String regionNamePath,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIME,
-          required = false) final Integer entryIdleTimeExpiration,
-      @RequestParam(value = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIMEACTION,
-          required = false) final String entryIdleTimeExpirationAction,
-      @RequestParam(value = CliStrings.ALTER_REGION__ENTRYEXPIRATIONTIMETOLIVE,
-          required = false) final Integer entryTimeToLiveExpiration,
-      @RequestParam(value = CliStrings.ALTER_REGION__ENTRYEXPIRATIONTTLACTION,
-          required = false) final String entryTimeToLiveExpirationAction,
-      @RequestParam(value = CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIME,
-          required = false) final Integer regionIdleTimeExpiration,
-      @RequestParam(value = CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIMEACTION,
-          required = false) final String regionIdleTimeExpirationAction,
-      @RequestParam(value = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTL,
-          required = false) final Integer regionTimeToLiveExpiration,
-      @RequestParam(value = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTLACTION,
-          required = false) final String regionTimeToLiveExpirationAction,
-      @RequestParam(value = CliStrings.ALTER_REGION__CACHELISTENER,
-          required = false) final String[] cacheListeners,
-      @RequestParam(value = CliStrings.ALTER_REGION__CACHELOADER,
-          required = false) final String cacheLoader,
-      @RequestParam(value = CliStrings.ALTER_REGION__CACHEWRITER,
-          required = false) final String cacheWriter,
-      @RequestParam(value = CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID,
-          required = false) final String[] asyncEventQueueIds,
-      @RequestParam(value = CliStrings.ALTER_REGION__GATEWAYSENDERID,
-          required = false) final String[] gatewaySenderIds,
-      @RequestParam(value = CliStrings.ALTER_REGION__CLONINGENABLED,
-          required = false) final Boolean enableCloning,
-      @RequestParam(value = CliStrings.ALTER_REGION__EVICTIONMAX,
-          required = false) final Integer evictionMax) {
-
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.ALTER_REGION);
-
-    command.addOption(CliStrings.ALTER_REGION__REGION, decode(regionNamePath));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, ","));
-    }
-
-    addCommandOption(request, command, CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIME,
-        entryIdleTimeExpiration);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIMEACTION,
-        entryIdleTimeExpirationAction);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__ENTRYEXPIRATIONTIMETOLIVE,
-        entryTimeToLiveExpiration);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__ENTRYEXPIRATIONTTLACTION,
-        entryTimeToLiveExpirationAction);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIME,
-        regionIdleTimeExpiration);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIMEACTION,
-        regionIdleTimeExpirationAction);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__REGIONEXPIRATIONTTL,
-        regionTimeToLiveExpiration);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__REGIONEXPIRATIONTTLACTION,
-        regionTimeToLiveExpirationAction);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__CACHELISTENER, cacheListeners);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__CACHELOADER, cacheLoader);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__CACHEWRITER, cacheWriter);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID,
-        asyncEventQueueIds);
-    addCommandOption(request, command, CliStrings.ALTER_REGION__GATEWAYSENDERID, gatewaySenderIds);
-
-    if (Boolean.TRUE.equals(enableCloning)) {
-      command.addOption(CliStrings.ALTER_REGION__CLONINGENABLED, String.valueOf(enableCloning));
-    }
-
-    if (hasValue(evictionMax)) {
-      command.addOption(CliStrings.ALTER_REGION__EVICTIONMAX, String.valueOf(evictionMax));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/regions")
-  @ResponseBody
-  public String createRegion(@RequestParam("name") final String regionNamePath,
-      @RequestParam(value = CliStrings.CREATE_REGION__REGIONSHORTCUT,
-          required = false) final String regionType,
-      @RequestParam(value = CliStrings.CREATE_REGION__USEATTRIBUTESFROM,
-          required = false) final String regionTemplate,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.CREATE_REGION__SKIPIFEXISTS,
-          defaultValue = "true") final Boolean skipIfExists,
-      @RequestParam(value = CliStrings.CREATE_REGION__KEYCONSTRAINT,
-          required = false) final String keyConstraint,
-      @RequestParam(value = CliStrings.CREATE_REGION__VALUECONSTRAINT,
-          required = false) final String valueConstraint,
-      @RequestParam(value = CliStrings.CREATE_REGION__STATISTICSENABLED,
-          required = false) final Boolean enableStatistics,
-      @RequestParam(value = CliStrings.CREATE_REGION__ENTRYEXPIRATIONIDLETIME,
-          required = false) final Integer entryIdleTimeExpiration,
-      @RequestParam(value = CliStrings.CREATE_REGION__ENTRYEXPIRATIONIDLETIMEACTION,
-          required = false) final String entryIdleTimeExpirationAction,
-      @RequestParam(value = CliStrings.CREATE_REGION__ENTRYEXPIRATIONTIMETOLIVE,
-          required = false) final Integer entryTimeToLiveExpiration,
-      @RequestParam(value = CliStrings.CREATE_REGION__ENTRYEXPIRATIONTTLACTION,
-          required = false) final String entryTimeToLiveExpirationAction,
-      @RequestParam(value = CliStrings.CREATE_REGION__REGIONEXPIRATIONIDLETIME,
-          required = false) final Integer regionIdleTimeExpiration,
-      @RequestParam(value = CliStrings.CREATE_REGION__REGIONEXPIRATIONIDLETIMEACTION,
-          required = false) final String regionIdleTimeExpirationAction,
-      @RequestParam(value = CliStrings.CREATE_REGION__REGIONEXPIRATIONTTL,
-          required = false) final Integer regionTimeToLiveExpiration,
-      @RequestParam(value = CliStrings.CREATE_REGION__REGIONEXPIRATIONTTLACTION,
-          required = false) final String regionTimeToLiveExpirationAction,
-      @RequestParam(value = CliStrings.CREATE_REGION__DISKSTORE,
-          required = false) final String diskStore,
-      @RequestParam(value = CliStrings.CREATE_REGION__DISKSYNCHRONOUS,
-          required = false) final Boolean enableSynchronousDisk,
-      @RequestParam(value = CliStrings.CREATE_REGION__ENABLEASYNCCONFLATION,
-          required = false) final Boolean enableAsynchronousConflation,
-      @RequestParam(value = CliStrings.CREATE_REGION__ENABLESUBSCRIPTIONCONFLATION,
-          required = false) final Boolean enableSubscriptionConflation,
-      @RequestParam(value = CliStrings.CREATE_REGION__CACHELISTENER,
-          required = false) final String[] cacheListeners,
-      @RequestParam(value = CliStrings.CREATE_REGION__CACHELOADER,
-          required = false) final String cacheLoader,
-      @RequestParam(value = CliStrings.CREATE_REGION__CACHEWRITER,
-          required = false) final String cacheWriter,
-      @RequestParam(value = CliStrings.CREATE_REGION__ASYNCEVENTQUEUEID,
-          required = false) final String[] asyncEventQueueIds,
-      @RequestParam(value = CliStrings.CREATE_REGION__GATEWAYSENDERID,
-          required = false) final String[] gatewaySenderIds,
-      @RequestParam(value = CliStrings.CREATE_REGION__CONCURRENCYCHECKSENABLED,
-          required = false) final Boolean enableConcurrencyChecks,
-      @RequestParam(value = CliStrings.CREATE_REGION__CLONINGENABLED,
-          required = false) final Boolean enableCloning,
-      @RequestParam(value = CliStrings.CREATE_REGION__CONCURRENCYLEVEL,
-          required = false) final Integer concurrencyLevel,
-      @RequestParam(value = CliStrings.CREATE_REGION__COLOCATEDWITH,
-          required = false) final String colocatedWith,
-      @RequestParam(value = CliStrings.CREATE_REGION__LOCALMAXMEMORY,
-          required = false) final Integer localMaxMemory,
-      @RequestParam(value = CliStrings.CREATE_REGION__RECOVERYDELAY,
-          required = false) final Long recoveryDelay,
-      @RequestParam(value = CliStrings.CREATE_REGION__REDUNDANTCOPIES,
-          required = false) final Integer redundantCopies,
-      @RequestParam(value = CliStrings.CREATE_REGION__STARTUPRECOVERYDDELAY,
-          required = false) final Long startupRecoveryDelay,
-      @RequestParam(value = CliStrings.CREATE_REGION__TOTALMAXMEMORY,
-          required = false) final Long totalMaxMemory,
-      @RequestParam(value = CliStrings.CREATE_REGION__TOTALNUMBUCKETS,
-          required = false) final Integer totalNumBuckets,
-      @RequestParam(value = CliStrings.CREATE_REGION__COMPRESSOR,
-          required = false) final String compressor) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.CREATE_REGION);
-
-    assertArgument(hasValue(regionNamePath), "Region name/path is required!");
-
-    assertArgument(hasValue(regionType) || hasValue(regionTemplate),
-        "Either Region type or template-region must be specified!");
-
-    command.addOption(CliStrings.CREATE_REGION__REGION, regionNamePath);
-
-    if (hasValue(regionType)) {
-      command.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT, regionType);
-    } else {
-      command.addOption(CliStrings.CREATE_REGION__USEATTRIBUTESFROM, regionTemplate);
-    }
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, ","));
-    }
-
-    command.addOption(CliStrings.CREATE_REGION__SKIPIFEXISTS,
-        String.valueOf(Boolean.TRUE.equals(skipIfExists)));
-
-    if (hasValue(keyConstraint)) {
-      command.addOption(CliStrings.CREATE_REGION__KEYCONSTRAINT, keyConstraint);
-    }
-
-    if (hasValue(valueConstraint)) {
-      command.addOption(CliStrings.CREATE_REGION__VALUECONSTRAINT, valueConstraint);
-    }
-
-    if (Boolean.TRUE.equals(enableStatistics)) {
-      command.addOption(CliStrings.CREATE_REGION__STATISTICSENABLED,
-          String.valueOf(enableStatistics));
-    }
-
-    if (hasValue(entryIdleTimeExpiration)) {
-      command.addOption(CliStrings.CREATE_REGION__ENTRYEXPIRATIONIDLETIME,
-          String.valueOf(entryIdleTimeExpiration));
-    }
-
-    if (hasValue(entryIdleTimeExpirationAction)) {
-      command.addOption(CliStrings.CREATE_REGION__ENTRYEXPIRATIONIDLETIMEACTION,
-          entryIdleTimeExpirationAction);
-    }
-
-    if (hasValue(entryTimeToLiveExpiration)) {
-      command.addOption(CliStrings.CREATE_REGION__ENTRYEXPIRATIONTIMETOLIVE,
-          String.valueOf(entryTimeToLiveExpiration));
-    }
-
-    if (hasValue(entryTimeToLiveExpirationAction)) {
-      command.addOption(CliStrings.CREATE_REGION__ENTRYEXPIRATIONTTLACTION,
-          entryTimeToLiveExpirationAction);
-    }
-
-    if (hasValue(regionIdleTimeExpiration)) {
-      command.addOption(CliStrings.CREATE_REGION__REGIONEXPIRATIONIDLETIME,
-          String.valueOf(regionIdleTimeExpiration));
-    }
-
-    if (hasValue(regionIdleTimeExpirationAction)) {
-      command.addOption(CliStrings.CREATE_REGION__REGIONEXPIRATIONIDLETIMEACTION,
-          regionIdleTimeExpirationAction);
-    }
-
-    if (hasValue(regionTimeToLiveExpiration)) {
-      command.addOption(CliStrings.CREATE_REGION__REGIONEXPIRATIONTTL,
-          String.valueOf(regionTimeToLiveExpiration));
-    }
-
-    if (hasValue(regionTimeToLiveExpirationAction)) {
-      command.addOption(CliStrings.CREATE_REGION__REGIONEXPIRATIONTTLACTION,
-          regionTimeToLiveExpirationAction);
-    }
-
-    if (hasValue(diskStore)) {
-      command.addOption(CliStrings.CREATE_REGION__DISKSTORE, diskStore);
-    }
-
-    if (Boolean.TRUE.equals(enableSynchronousDisk)) {
-      command.addOption(CliStrings.CREATE_REGION__DISKSYNCHRONOUS,
-          String.valueOf(enableSynchronousDisk));
-    }
-
-    if (Boolean.TRUE.equals(enableAsynchronousConflation)) {
-      command.addOption(CliStrings.CREATE_REGION__ENABLEASYNCCONFLATION,
-          String.valueOf(enableAsynchronousConflation));
-    }
-
-    if (Boolean.TRUE.equals(enableSubscriptionConflation)) {
-      command.addOption(CliStrings.CREATE_REGION__ENABLESUBSCRIPTIONCONFLATION,
-          String.valueOf(enableSubscriptionConflation));
-    }
-
-    if (hasValue(cacheListeners)) {
-      command.addOption(CliStrings.CREATE_REGION__CACHELISTENER,
-          StringUtils.join(cacheListeners, ","));
-    }
-
-    if (hasValue(cacheLoader)) {
-      command.addOption(CliStrings.CREATE_REGION__CACHELOADER, cacheLoader);
-    }
-
-    if (hasValue(cacheWriter)) {
-      command.addOption(CliStrings.CREATE_REGION__CACHEWRITER, cacheWriter);
-    }
-
-    if (hasValue(asyncEventQueueIds)) {
-      command.addOption(CliStrings.CREATE_REGION__ASYNCEVENTQUEUEID,
-          StringUtils.join(asyncEventQueueIds, ","));
-    }
-
-    if (hasValue(gatewaySenderIds)) {
-      command.addOption(CliStrings.CREATE_REGION__GATEWAYSENDERID,
-          StringUtils.join(gatewaySenderIds, ","));
-    }
-
-    if (Boolean.TRUE.equals(enableConcurrencyChecks)) {
-      command.addOption(CliStrings.CREATE_REGION__CONCURRENCYCHECKSENABLED,
-          String.valueOf(enableConcurrencyChecks));
-    }
-
-    if (Boolean.TRUE.equals(enableCloning)) {
-      command.addOption(CliStrings.CREATE_REGION__CLONINGENABLED, String.valueOf(enableCloning));
-    }
-
-    if (hasValue(concurrencyLevel)) {
-      command.addOption(CliStrings.CREATE_REGION__CONCURRENCYLEVEL,
-          String.valueOf(concurrencyLevel));
-    }
-
-    if (hasValue(colocatedWith)) {
-      command.addOption(CliStrings.CREATE_REGION__COLOCATEDWITH, colocatedWith);
-    }
-
-    if (hasValue(localMaxMemory)) {
-      command.addOption(CliStrings.CREATE_REGION__LOCALMAXMEMORY, String.valueOf(localMaxMemory));
-    }
-
-    if (hasValue(recoveryDelay)) {
-      command.addOption(CliStrings.CREATE_REGION__RECOVERYDELAY, String.valueOf(recoveryDelay));
-    }
-
-    if (hasValue(redundantCopies)) {
-      command.addOption(CliStrings.CREATE_REGION__REDUNDANTCOPIES, String.valueOf(redundantCopies));
-    }
-
-    if (hasValue(startupRecoveryDelay)) {
-      command.addOption(CliStrings.CREATE_REGION__STARTUPRECOVERYDDELAY,
-          String.valueOf(startupRecoveryDelay));
-    }
-
-    if (hasValue(totalMaxMemory)) {
-      command.addOption(CliStrings.CREATE_REGION__TOTALMAXMEMORY, String.valueOf(totalMaxMemory));
-    }
-
-    if (hasValue(totalNumBuckets)) {
-      command.addOption(CliStrings.CREATE_REGION__TOTALNUMBUCKETS, String.valueOf(totalNumBuckets));
-    }
-
-    if (hasValue(compressor)) {
-      command.addOption(CliStrings.CREATE_REGION__COMPRESSOR, compressor);
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.DELETE, value = "/regions/{name}")
-  @ResponseBody
-  public String destroyRegion(@PathVariable("name") final String regionNamePath) {
-    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESTROY_REGION);
-    command.addOption(CliStrings.DESTROY_REGION__REGION, decode(regionNamePath));
-    return processCommand(command.toString());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ShellCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ShellCommandsController.java
index 63413b0..fa72cf1 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ShellCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ShellCommandsController.java
@@ -14,7 +14,11 @@
  */
 package org.apache.geode.management.internal.web.controllers;
 
+import static org.apache.geode.management.internal.web.controllers.AbstractMultiPartCommandsController.RESOURCES_REQUEST_PARAMETER;
+
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.Set;
 
 import javax.management.AttributeNotFoundException;
@@ -22,7 +26,10 @@ import javax.management.InstanceNotFoundException;
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.ArrayUtils;
+import org.springframework.core.io.InputStreamResource;
+import org.springframework.http.HttpHeaders;
 import org.springframework.http.HttpStatus;
 import org.springframework.http.MediaType;
 import org.springframework.http.ResponseEntity;
@@ -32,15 +39,16 @@ import org.springframework.web.bind.annotation.RequestMapping;
 import org.springframework.web.bind.annotation.RequestMethod;
 import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
+import org.springframework.web.multipart.MultipartFile;
 
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.lang.ObjectUtils;
 import org.apache.geode.internal.util.IOUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.management.internal.web.domain.LinkIndex;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.web.domain.QueryParameterSource;
-import org.apache.geode.management.internal.web.http.HttpMethod;
+import org.apache.geode.management.internal.web.util.ConvertUtils;
 
 /**
  * The ShellCommandsController class implements GemFire REST API calls for Gfsh Shell Commands.
@@ -65,10 +73,55 @@ public class ShellCommandsController extends AbstractCommandsController {
   protected static final String MBEAN_QUERY_LINK_RELATION = "mbean-query";
   protected static final String PING_LINK_RELATION = "ping";
 
-  @RequestMapping(method = RequestMethod.POST, value = "/management/commands", params = "cmd")
-  @ResponseBody
-  public String command(@RequestParam("cmd") final String command) {
-    return processCommand(decode(command));
+  @RequestMapping(method = {RequestMethod.GET, RequestMethod.POST}, value = "/management/commands")
+  public ResponseEntity<InputStreamResource> command(@RequestParam(value = "cmd") String command,
+      @RequestParam(value = RESOURCES_REQUEST_PARAMETER,
+          required = false) MultipartFile[] fileResource)
+      throws IOException {
+    String result =
+        processCommand(decode(command), getEnvironment(), ConvertUtils.convert(fileResource));
+    return getResponse(result);
+  }
+
+  ResponseEntity<InputStreamResource> getResponse(String result) {
+    // the result is json string from CommandResult
+    CommandResult commandResult = ResultBuilder.fromJson(result);
+
+    if (commandResult.getStatus().equals(Result.Status.OK) && commandResult.hasFileToDownload()) {
+      return getFileDownloadResponse(commandResult);
+    } else {
+      return getJsonResponse(result);
+    }
+  }
+
+  private ResponseEntity<InputStreamResource> getJsonResponse(String result) {
+    HttpHeaders respHeaders = new HttpHeaders();
+    InputStreamResource isr;// if the command is successful, the output is the filepath,
+    // else we need to send the orignal result back so that the receiver will know to turn it
+    // into a Result object
+    try {
+      isr = new InputStreamResource(org.apache.commons.io.IOUtils.toInputStream(result, "UTF-8"));
+      respHeaders.set(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON_VALUE);
+      return new ResponseEntity<>(isr, respHeaders, HttpStatus.OK);
+    } catch (Exception e) {
+      throw new RuntimeException("IO Error writing file to output stream", e);
+    }
+  }
+
+  private ResponseEntity<InputStreamResource> getFileDownloadResponse(CommandResult commandResult) {
+    HttpHeaders respHeaders = new HttpHeaders();
+    InputStreamResource isr;// if the command is successful, the output is the filepath,
+
+    Path filePath = commandResult.getFileToDownload();
+    try {
+      isr = new InputStreamResource(new FileInputStream(filePath.toFile()));
+      respHeaders.set(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_OCTET_STREAM_VALUE);
+      return new ResponseEntity<>(isr, respHeaders, HttpStatus.OK);
+    } catch (Exception e) {
+      throw new RuntimeException("IO Error writing file to output stream", e);
+    } finally {
+      FileUtils.deleteQuietly(filePath.toFile());
+    }
   }
 
   @RequestMapping(method = RequestMethod.GET, value = "/mbean/attribute")
@@ -122,172 +175,9 @@ public class ShellCommandsController extends AbstractCommandsController {
     }
   }
 
-  /**
-   * Gets a link index for the web service endpoints and REST API calls in GemFire for management
-   * and monitoring using GemFire shell (Gfsh).
-   * 
-   * @return a LinkIndex containing Links for all web service endpoints, REST API calls in GemFire.
-   * @see org.apache.geode.management.internal.cli.i18n.CliStrings
-   * @see AbstractCommandsController#toUri(String, String)
-   * @see org.apache.geode.management.internal.web.domain.Link
-   * @see org.apache.geode.management.internal.web.domain.LinkIndex
-   * @see org.apache.geode.management.internal.web.http.HttpMethod
-   */
-  @RequestMapping(method = RequestMethod.GET, value = "/index",
-      produces = MediaType.APPLICATION_XML_VALUE)
-  @ResponseBody
-  public LinkIndex index(@RequestParam(value = "scheme", required = false,
-      defaultValue = "http") final String scheme) {
-    return new LinkIndex()
-        // Cluster Commands
-        .add(new Link(CliStrings.STATUS_SHARED_CONFIG, toUri("/services/cluster-config", scheme)))
-        // Member Commands
-        .add(new Link(CliStrings.LIST_MEMBER, toUri("/members", scheme)))
-        .add(new Link(CliStrings.DESCRIBE_MEMBER, toUri("/members/{name}", scheme)))
-        // Region Commands
-        .add(new Link(CliStrings.LIST_REGION, toUri("/regions", scheme)))
-        .add(new Link(CliStrings.DESCRIBE_REGION, toUri("/regions/{name}", scheme)))
-        .add(new Link(CliStrings.ALTER_REGION, toUri("/regions/{name}", scheme), HttpMethod.PUT))
-        .add(new Link(CliStrings.CREATE_REGION, toUri("/regions", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.DESTROY_REGION, toUri("/regions/{name}", scheme),
-            HttpMethod.DELETE))
-        // Index Commands
-        .add(new Link(CliStrings.LIST_INDEX, toUri("/indexes", scheme)))
-        .add(new Link(CliStrings.CLEAR_DEFINED_INDEXES, toUri("/indexes?op=clear-defined", scheme),
-            HttpMethod.DELETE))
-        .add(new Link(CliStrings.CREATE_INDEX, toUri("/indexes", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.CREATE_DEFINED_INDEXES,
-            toUri("/indexes?op=create-defined", scheme), HttpMethod.POST))
-        .add(
-            new Link(CliStrings.DEFINE_INDEX, toUri("/indexes?op=define", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.DESTROY_INDEX, toUri("/indexes", scheme), HttpMethod.DELETE))
-        .add(
-            new Link(CliStrings.DESTROY_INDEX, toUri("/indexes/{name}", scheme), HttpMethod.DELETE))
-        // Data Commands
-        .add(new Link(CliStrings.GET, toUri("/regions/{region}/data", scheme), HttpMethod.GET))
-        .add(new Link(CliStrings.PUT, toUri("/regions/{region}/data", scheme), HttpMethod.PUT))
-        .add(
-            new Link(CliStrings.REMOVE, toUri("/regions/{region}/data", scheme), HttpMethod.DELETE))
-        .add(new Link(CliStrings.EXPORT_DATA,
-            toUri("/members/{member}/regions/{region}/data", scheme), HttpMethod.GET))
-        .add(new Link(CliStrings.IMPORT_DATA,
-            toUri("/members/{member}/regions/{region}/data", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.LOCATE_ENTRY, toUri("/regions/{region}/data/location", scheme),
-            HttpMethod.GET))
-        .add(new Link(CliStrings.QUERY, toUri("/regions/data/query", scheme), HttpMethod.GET))
-        .add(new Link(CliStrings.REBALANCE, toUri("/regions/data?op=rebalance", scheme),
-            HttpMethod.POST))
-        // Function Commands
-        .add(new Link(CliStrings.LIST_FUNCTION, toUri("/functions", scheme)))
-        .add(new Link(CliStrings.DESTROY_FUNCTION, toUri("/functions/{id}", scheme),
-            HttpMethod.DELETE))
-        .add(new Link(CliStrings.EXECUTE_FUNCTION, toUri("/functions/{id}", scheme),
-            HttpMethod.POST))
-        // Client Commands
-        .add(new Link(CliStrings.LIST_CLIENTS, toUri("/clients", scheme)))
-        .add(new Link(CliStrings.DESCRIBE_CLIENT, toUri("/clients/{clientID}", scheme)))
-        // Config Commands
-        .add(new Link(CliStrings.ALTER_RUNTIME_CONFIG, toUri("/config", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.DESCRIBE_CONFIG, toUri("/members/{member}/config", scheme)))
-        .add(new Link(CliStrings.EXPORT_CONFIG, toUri("/config", scheme)))
-        .add(new Link(CliStrings.EXPORT_SHARED_CONFIG, toUri("/config/cluster", scheme)))
-        .add(new Link(CliStrings.IMPORT_SHARED_CONFIG, toUri("/config/cluster", scheme),
-            HttpMethod.POST))
-        // Deploy Commands
-        .add(new Link(CliStrings.LIST_DEPLOYED, toUri("/deployed", scheme)))
-        .add(new Link(CliStrings.DEPLOY, toUri("/deployed", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.UNDEPLOY, toUri("/deployed", scheme), HttpMethod.DELETE))
-        // Disk Store Commands
-        .add(new Link(CliStrings.LIST_DISK_STORE, toUri("/diskstores", scheme)))
-        .add(new Link(CliStrings.BACKUP_DISK_STORE, toUri("/diskstores?op=backup", scheme),
-            HttpMethod.POST))
-        .add(new Link(CliStrings.COMPACT_DISK_STORE, toUri("/diskstores/{name}?op=compact", scheme),
-            HttpMethod.POST))
-        .add(new Link(CliStrings.CREATE_DISK_STORE, toUri("/diskstores", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.DESCRIBE_DISK_STORE, toUri("/diskstores/{name}", scheme)))
-        .add(new Link(CliStrings.DESTROY_DISK_STORE, toUri("/diskstores/{name}", scheme),
-            HttpMethod.DELETE))
-        .add(new Link(CliStrings.REVOKE_MISSING_DISK_STORE,
-            toUri("/diskstores/{id}?op=revoke", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.SHOW_MISSING_DISK_STORE, toUri("/diskstores/missing", scheme)))
-        // Durable Client Commands
-        .add(new Link(CliStrings.LIST_DURABLE_CQS,
-            toUri("/durable-clients/{durable-client-id}/cqs", scheme)))
-        .add(new Link(CliStrings.COUNT_DURABLE_CQ_EVENTS,
-            toUri("/durable-clients/{durable-client-id}/cqs/events", scheme)))
-        .add(new Link(CliStrings.COUNT_DURABLE_CQ_EVENTS,
-            toUri("/durable-clients/{durable-client-id}/cqs/{durable-cq-name}/events", scheme)))
-        .add(new Link(CliStrings.CLOSE_DURABLE_CLIENTS,
-            toUri("/durable-clients/{durable-client-id}?op=close", scheme), HttpMethod.POST))
-        .add(
-            new Link(CliStrings.CLOSE_DURABLE_CQS,
-                toUri("/durable-clients/{durable-client-id}/cqs/{durable-cq-name}?op=close",
-                    scheme),
-                HttpMethod.POST))
-        // Launcher Lifecycle Commands
-        .add(new Link(CliStrings.STATUS_LOCATOR, toUri("/members/{name}/locator", scheme)))
-        .add(new Link(CliStrings.STATUS_SERVER, toUri("/members/{name}/server", scheme)))
-        // Miscellaneous Commands
-        .add(new Link(CliStrings.CHANGE_LOGLEVEL, toUri("/groups/{groups}/loglevel", scheme),
-            HttpMethod.POST))
-        .add(new Link(CliStrings.CHANGE_LOGLEVEL, toUri("/members/{members}/loglevel", scheme),
-            HttpMethod.POST))
-        .add(new Link(CliStrings.CHANGE_LOGLEVEL,
-            toUri("/members/{members}/groups/{groups}/loglevel", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.EXPORT_LOGS, toUri("/logs", scheme)))
-        .add(new Link(CliStrings.EXPORT_STACKTRACE, toUri("/stacktraces", scheme)))
-        .add(new Link(CliStrings.GC, toUri("/gc", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.GC, toUri("/members/{member}/gc", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.NETSTAT, toUri("/netstat", scheme)))
-        .add(new Link(CliStrings.SHOW_DEADLOCK, toUri("/deadlocks", scheme)))
-        .add(new Link(CliStrings.SHOW_LOG, toUri("/members/{member}/log", scheme)))
-        .add(new Link(CliStrings.SHOW_METRICS, toUri("/metrics", scheme)))
-        .add(new Link(CliStrings.SHUTDOWN, toUri("/shutdown", scheme), HttpMethod.POST)) // verb!
-        // Queue Commands
-        .add(new Link(CliStrings.CREATE_ASYNC_EVENT_QUEUE, toUri("/async-event-queues", scheme),
-            HttpMethod.POST))
-        .add(new Link(CliStrings.LIST_ASYNC_EVENT_QUEUES, toUri("/async-event-queues", scheme)))
-        // PDX Commands
-        .add(new Link(CliStrings.CONFIGURE_PDX, toUri("/pdx", scheme), HttpMethod.POST))
-        // .add(new Link(CliStrings.PDX_DELETE_FIELD, toUri("/pdx/type/field"), HttpMethod.DELETE))
-        .add(new Link(CliStrings.PDX_RENAME, toUri("/pdx/type", scheme), HttpMethod.POST))
-        // Shell Commands
-        .add(new Link(MBEAN_ATTRIBUTE_LINK_RELATION, toUri("/mbean/attribute", scheme)))
-        .add(new Link(MBEAN_OPERATION_LINK_RELATION, toUri("/mbean/operation", scheme),
-            HttpMethod.POST))
-        .add(new Link(MBEAN_QUERY_LINK_RELATION, toUri("/mbean/query", scheme), HttpMethod.POST))
-        .add(new Link(PING_LINK_RELATION, toUri("/ping", scheme), HttpMethod.GET))
-        .add(new Link(CliStrings.VERSION, toUri("/version", scheme)))
-        // WAN Gateway Commands
-        .add(new Link(CliStrings.LIST_GATEWAY, toUri("/gateways", scheme)))
-        .add(new Link(CliStrings.CREATE_GATEWAYRECEIVER, toUri("/gateways/receivers", scheme),
-            HttpMethod.POST))
-        .add(new Link(CliStrings.CREATE_GATEWAYSENDER, toUri("/gateways/senders", scheme),
-            HttpMethod.POST))
-        .add(new Link(CliStrings.DESTROY_GATEWAYSENDER, toUri("/gateways/senders/{id}", scheme),
-            HttpMethod.DELETE))
-        .add(new Link(CliStrings.LOAD_BALANCE_GATEWAYSENDER,
-            toUri("/gateways/senders/{id}?op=load-balance", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.PAUSE_GATEWAYSENDER,
-            toUri("/gateways/senders/{id}?op=pause", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.RESUME_GATEWAYSENDER,
-            toUri("/gateways/senders/{id}?op=resume", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.START_GATEWAYRECEIVER,
-            toUri("/gateways/receivers?op=start", scheme), HttpMethod.POST))
-        .add(new Link(CliStrings.START_GATEWAYSENDER, toUri("/gateways/senders?op=start", scheme),
-            HttpMethod.POST))
-        .add(new Link(CliStrings.STATUS_GATEWAYRECEIVER, toUri("/gateways/receivers", scheme)))
-        .add(new Link(CliStrings.STATUS_GATEWAYSENDER, toUri("/gateways/senders/{id}", scheme)))
-        .add(new Link(CliStrings.STOP_GATEWAYRECEIVER, toUri("/gateways/receivers?op=stop", scheme),
-            HttpMethod.POST))
-        .add(new Link(CliStrings.STOP_GATEWAYSENDER,
-            toUri("/gateways/senders/{id}?op=stop", scheme), HttpMethod.POST));
-  }
-
   @RequestMapping(method = {RequestMethod.GET, RequestMethod.HEAD}, value = "/ping")
   public ResponseEntity<String> ping() {
-    return new ResponseEntity<>("<html><body><h1>Mischief Managed!</h1></body></html>",
-        HttpStatus.OK);
+    return new ResponseEntity<>("pong", HttpStatus.OK);
   }
 
   @RequestMapping(method = RequestMethod.GET, value = "/version")
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/WanCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/WanCommandsController.java
deleted file mode 100644
index 36b4d2e..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/WanCommandsController.java
+++ /dev/null
@@ -1,465 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.management.internal.web.controllers;
-
-import org.springframework.stereotype.Controller;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
-
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
-/**
- * The WanCommandsController class implements the GemFire Management REST API web service endpoints
- * for the Gfsh WAN Commands.
- *
- * @see org.apache.geode.management.internal.cli.commands.CreateGatewayReceiverCommand
- * @see org.apache.geode.management.internal.cli.commands.CreateGatewaySenderCommand
- * @see org.apache.geode.management.internal.cli.commands.DestroyGatewaySenderCommand
- * @see org.apache.geode.management.internal.cli.commands.GatewayCommandsUtils
- * @see org.apache.geode.management.internal.cli.commands.ListGatewayCommand
- * @see org.apache.geode.management.internal.cli.commands.LoadBalanceGatewaySenderCommand
- * @see org.apache.geode.management.internal.cli.commands.PauseGatewaySenderCommand
- * @see org.apache.geode.management.internal.cli.commands.ResumeGatewaySenderCommand
- * @see org.apache.geode.management.internal.cli.commands.StartGatewayReceiverCommand
- * @see org.apache.geode.management.internal.cli.commands.StartGatewaySenderCommand
- * @see org.apache.geode.management.internal.cli.commands.StatusGatewayReceiverCommand
- * @see org.apache.geode.management.internal.cli.commands.StatusGatewaySenderCommand
- * @see org.apache.geode.management.internal.cli.commands.StopGatewayReceiverCommand
- * @see org.apache.geode.management.internal.cli.commands.StopGatewaySenderCommand
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
- * @see org.springframework.stereotype.Controller
- * @see org.springframework.web.bind.annotation.RequestMapping
- * @see org.springframework.web.bind.annotation.RequestMethod
- * @see org.springframework.web.bind.annotation.RequestParam
- * @see org.springframework.web.bind.annotation.ResponseBody
- * @since GemFire 8.0
- */
-@Controller("wanController")
-@RequestMapping(AbstractCommandsController.REST_API_VERSION)
-@SuppressWarnings("unused")
-public class WanCommandsController extends AbstractCommandsController {
-
-  @RequestMapping(method = RequestMethod.GET, value = "/gateways")
-  @ResponseBody
-  public String listGateways(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.LIST_GATEWAY);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/gateways/receivers")
-  @ResponseBody
-  public String createGatewayReceiver(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYRECEIVER__MANUALSTART,
-          required = false) final Boolean manualStart,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYRECEIVER__STARTPORT,
-          required = false) final Integer startPort,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYRECEIVER__ENDPORT,
-          required = false) final Integer endPort,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYRECEIVER__BINDADDRESS,
-          required = false) final String bindAddress,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYRECEIVER__MAXTIMEBETWEENPINGS,
-          required = false) final Integer maximumTimeBetweenPings,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYRECEIVER__SOCKETBUFFERSIZE,
-          required = false) final Integer socketBufferSize,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYRECEIVER__GATEWAYTRANSPORTFILTER,
-          required = false) final String[] gatewayTransportFilters) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYRECEIVER);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(manualStart)) {
-      command.addOption(CliStrings.CREATE_GATEWAYRECEIVER__MANUALSTART,
-          String.valueOf(manualStart));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(startPort)) {
-      command.addOption(CliStrings.CREATE_GATEWAYRECEIVER__STARTPORT, String.valueOf(startPort));
-    }
-
-    if (hasValue(endPort)) {
-      command.addOption(CliStrings.CREATE_GATEWAYRECEIVER__ENDPORT, String.valueOf(endPort));
-    }
-
-    if (hasValue(bindAddress)) {
-      command.addOption(CliStrings.CREATE_GATEWAYRECEIVER__BINDADDRESS, bindAddress);
-    }
-
-    if (hasValue(maximumTimeBetweenPings)) {
-      command.addOption(CliStrings.CREATE_GATEWAYRECEIVER__MAXTIMEBETWEENPINGS,
-          String.valueOf(maximumTimeBetweenPings));
-    }
-
-    if (hasValue(socketBufferSize)) {
-      command.addOption(CliStrings.CREATE_GATEWAYRECEIVER__SOCKETBUFFERSIZE,
-          String.valueOf(socketBufferSize));
-    }
-
-    if (hasValue(gatewayTransportFilters)) {
-      command.addOption(CliStrings.CREATE_GATEWAYRECEIVER__GATEWAYTRANSPORTFILTER,
-          StringUtils.join(gatewayTransportFilters, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/gateways/senders")
-  @ResponseBody
-  public String createGatewaySender(
-      @RequestParam(CliStrings.CREATE_GATEWAYSENDER__ID) final String gatewaySenderId,
-      @RequestParam(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID) final Integer remoteDistributedSystemId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__PARALLEL,
-          required = false) final Boolean parallel,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__MANUALSTART,
-          required = false) final Boolean manualStart,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE,
-          required = false) final Integer socketBufferSize,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT,
-          required = false) final Integer socketReadTimeout,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION,
-          required = false) final Boolean enableBatchConflation,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE,
-          required = false) final Integer batchSize,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL,
-          required = false) final Integer batchTimeInterval,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE,
-          required = false) final Boolean enablePersistence,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__DISKSTORENAME,
-          required = false) final String diskStoreName,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS,
-          required = false) final Boolean diskSynchronous,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY,
-          required = false) final Integer maxQueueMemory,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD,
-          required = false) final Integer alertThreshold,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS,
-          required = false) final Integer dispatcherThreads,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY,
-          required = false) final String orderPolicy,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__GATEWAYEVENTFILTER,
-          required = false) final String[] gatewayEventFilters,
-      @RequestParam(value = CliStrings.CREATE_GATEWAYSENDER__GATEWAYTRANSPORTFILTER,
-          required = false) final String[] gatewayTransportFilters) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.CREATE_GATEWAYSENDER);
-
-    command.addOption(CliStrings.CREATE_GATEWAYSENDER__ID, gatewaySenderId);
-    command.addOption(CliStrings.CREATE_GATEWAYSENDER__REMOTEDISTRIBUTEDSYSTEMID,
-        String.valueOf(remoteDistributedSystemId));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(parallel)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__PARALLEL, String.valueOf(parallel));
-    }
-
-    if (hasValue(manualStart)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__MANUALSTART, String.valueOf(manualStart));
-    }
-
-    if (hasValue(socketBufferSize)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__SOCKETBUFFERSIZE,
-          String.valueOf(socketBufferSize));
-    }
-
-    if (hasValue(socketReadTimeout)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__SOCKETREADTIMEOUT,
-          String.valueOf(socketReadTimeout));
-    }
-
-    if (hasValue(enableBatchConflation)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__ENABLEBATCHCONFLATION,
-          String.valueOf(enableBatchConflation));
-    }
-
-    if (hasValue(batchSize)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__BATCHSIZE, String.valueOf(batchSize));
-    }
-
-    if (hasValue(batchTimeInterval)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__BATCHTIMEINTERVAL,
-          String.valueOf(batchTimeInterval));
-    }
-
-    if (hasValue(enablePersistence)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__ENABLEPERSISTENCE,
-          String.valueOf(enablePersistence));
-    }
-
-    if (hasValue(diskStoreName)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__DISKSTORENAME, diskStoreName);
-    }
-
-    if (hasValue(diskSynchronous)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__DISKSYNCHRONOUS,
-          String.valueOf(diskSynchronous));
-    }
-
-    if (hasValue(maxQueueMemory)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__MAXQUEUEMEMORY,
-          String.valueOf(maxQueueMemory));
-    }
-
-    if (hasValue(alertThreshold)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__ALERTTHRESHOLD,
-          String.valueOf(alertThreshold));
-    }
-
-    if (hasValue(dispatcherThreads)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__DISPATCHERTHREADS,
-          String.valueOf(dispatcherThreads));
-    }
-
-    if (hasValue(orderPolicy)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY, orderPolicy);
-    }
-
-    if (hasValue(gatewayEventFilters)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__GATEWAYEVENTFILTER,
-          StringUtils.join(gatewayEventFilters, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(gatewayTransportFilters)) {
-      command.addOption(CliStrings.CREATE_GATEWAYSENDER__GATEWAYTRANSPORTFILTER,
-          StringUtils.join(gatewayTransportFilters, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-
-  @RequestMapping(method = RequestMethod.DELETE, value = "/gateways/senders/{id}")
-  @ResponseBody
-  public String destroyGatewaySender(@PathVariable("id") final String gatewaySenderId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.DESTROY_GATEWAYSENDER);
-    command.addOption(CliStrings.CREATE_GATEWAYSENDER__ID, gatewaySenderId);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-    return processCommand(command.toString());
-  }
-
-
-  @RequestMapping(method = RequestMethod.POST, value = "/gateways/senders/{id}",
-      params = "op=load-balance")
-  @ResponseBody
-  public String loadBalanceGatewaySender(@PathVariable("id") final String gatewaySenderId) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.LOAD_BALANCE_GATEWAYSENDER);
-
-    command.addOption(CliStrings.LOAD_BALANCE_GATEWAYSENDER__ID, decode(gatewaySenderId));
-
-    return processCommand(command.toString());
-  }
-
-  // TODO determine whether Async functionality is required
-  @RequestMapping(method = RequestMethod.POST, value = "/gateways/senders/{id}",
-      params = "op=pause")
-  @ResponseBody
-  public String pauseGatewaySender(@PathVariable("id") final String gatewaySenderId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.PAUSE_GATEWAYSENDER);
-
-    command.addOption(CliStrings.PAUSE_GATEWAYSENDER__ID, decode(gatewaySenderId));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  // TODO determine whether Async functionality is required
-  @RequestMapping(method = RequestMethod.POST, value = "/gateways/senders/{id}",
-      params = "op=resume")
-  @ResponseBody
-  public String resumeGatewaySender(@PathVariable("id") final String gatewaySenderId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.RESUME_GATEWAYSENDER);
-
-    command.addOption(CliStrings.RESUME_GATEWAYSENDER__ID, decode(gatewaySenderId));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  // TODO determine whether Async functionality is required
-  @RequestMapping(method = RequestMethod.POST, value = "/gateways/receivers", params = "op=start")
-  @ResponseBody
-  public String startGatewayReceiver(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_GATEWAYRECEIVER);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  // TODO determine whether Async functionality is required
-  @RequestMapping(method = RequestMethod.POST, value = "/gateways/senders", params = "op=start")
-  @ResponseBody
-  public String startGatewaySender(
-      @RequestParam(CliStrings.START_GATEWAYSENDER__ID) final String gatewaySenderId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_GATEWAYSENDER);
-
-    command.addOption(CliStrings.START_GATEWAYSENDER__ID, gatewaySenderId);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/gateways/receivers")
-  @ResponseBody
-  public String statusGatewayReceivers(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.STATUS_GATEWAYRECEIVER);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.GET, value = "/gateways/senders/{id}")
-  @ResponseBody
-  public String statusGatewaySenders(@PathVariable("id") final String gatewaySenderId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.STATUS_GATEWAYSENDER);
-
-    command.addOption(CliStrings.STATUS_GATEWAYSENDER__ID, decode(gatewaySenderId));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/gateways/receivers", params = "op=stop")
-  @ResponseBody
-  public String stopGatewayReceiver(
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.STOP_GATEWAYRECEIVER);
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-  @RequestMapping(method = RequestMethod.POST, value = "/gateways/senders/{id}", params = "op=stop")
-  @ResponseBody
-  public String stopGatewaySender(@PathVariable("id") final String gatewaySenderId,
-      @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups,
-      @RequestParam(value = CliStrings.MEMBER, required = false) final String[] members) {
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.STOP_GATEWAYSENDER);
-
-    command.addOption(CliStrings.STOP_GATEWAYSENDER__ID, decode(gatewaySenderId));
-
-    if (hasValue(groups)) {
-      command.addOption(CliStrings.GROUP, StringUtils.join(groups, StringUtils.COMMA_DELIMITER));
-    }
-
-    if (hasValue(members)) {
-      command.addOption(CliStrings.MEMBER, StringUtils.join(members, StringUtils.COMMA_DELIMITER));
-    }
-
-    return processCommand(command.toString());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/domain/Link.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/domain/Link.java
deleted file mode 100644
index 9058eaa..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/domain/Link.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.domain;
-
-import java.io.Serializable;
-import java.net.URI;
-
-import javax.xml.bind.annotation.XmlAttribute;
-import javax.xml.bind.annotation.XmlType;
-
-import org.apache.geode.internal.lang.ObjectUtils;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.web.http.HttpMethod;
-import org.apache.geode.management.internal.web.util.UriUtils;
-
-/**
- * The Link class models hypermedia controls/link relations.
- * <p/>
- * 
- * @see java.lang.Comparable
- * @see java.io.Serializable
- * @see java.net.URI
- * @see javax.xml.bind.annotation.XmlAttribute
- * @see javax.xml.bind.annotation.XmlType
- * @see org.apache.geode.management.internal.web.domain.LinkIndex
- * @see org.apache.geode.management.internal.web.util.UriUtils
- * @see org.apache.geode.management.internal.web.http.HttpMethod
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-@XmlType(name = "link", propOrder = {"method", "href", "relation"})
-public class Link implements Comparable<Link>, Serializable {
-
-  protected static final HttpMethod DEFAULT_HTTP_METHOD = HttpMethod.GET;
-
-  protected static final String HREF_ATTRIBUTE_NAME = "href";
-  protected static final String LINK_ELEMENT_NAME = "link";
-  protected static final String METHOD_ATTRIBUTE_NAME = "method";
-  protected static final String RELATION_ATTRIBUTE_NAME = "rel";
-
-  // This enum type is used in place of Spring's org.springframework.http.HttpMethod enum due to
-  // classpath issues
-  // between the GemFire Locator/Manager and the embedded HTTP service using Tomcat with
-  // Java/Tomcat's class resolution
-  // delegation model.
-  private HttpMethod method;
-
-  private String relation;
-
-  private URI href;
-
-  public Link() {}
-
-  public Link(final String relation, final URI href) {
-    this(relation, href, DEFAULT_HTTP_METHOD);
-  }
-
-  public Link(final String relation, final URI href, final HttpMethod method) {
-    setRelation(relation);
-    setHref(href);
-    setMethod(method);
-  }
-
-  @XmlAttribute(name = HREF_ATTRIBUTE_NAME)
-  public URI getHref() {
-    return href;
-  }
-
-  public void setHref(final URI href) {
-    assert href != null : "The Link href URI cannot be null!";
-    this.href = href;
-  }
-
-  @XmlAttribute(name = METHOD_ATTRIBUTE_NAME)
-  public HttpMethod getMethod() {
-    return method;
-  }
-
-  public void setMethod(final HttpMethod method) {
-    this.method = ObjectUtils.defaultIfNull(method, DEFAULT_HTTP_METHOD);
-  }
-
-  @XmlAttribute(name = RELATION_ATTRIBUTE_NAME)
-  public String getRelation() {
-    return relation;
-  }
-
-  public void setRelation(final String relation) {
-    assert StringUtils.isNotBlank(relation) : "The Link relation (rel) must be specified!";
-    this.relation = relation;
-  }
-
-  @Override
-  public int compareTo(final Link link) {
-    int compareValue = getRelation().compareTo(link.getRelation());
-    compareValue = (compareValue != 0 ? compareValue : getHref().compareTo(link.getHref()));
-    return (compareValue != 0 ? compareValue : getMethod().compareTo(link.getMethod()));
-  }
-
-  @Override
-  public boolean equals(final Object obj) {
-    if (obj == this) {
-      return true;
-    }
-
-    if (!(obj instanceof Link)) {
-      return false;
-    }
-
-    final Link that = (Link) obj;
-
-    return ObjectUtils.equals(getHref(), that.getHref())
-        && ObjectUtils.equals(getMethod(), that.getMethod());
-  }
-
-  @Override
-  public int hashCode() {
-    int hashValue = 17;
-    hashValue = 37 * hashValue + ObjectUtils.hashCode(getHref());
-    hashValue = 37 * hashValue + ObjectUtils.hashCode(getMethod());
-    return hashValue;
-  }
-
-  /**
-   * The HTTP Request-Line begins with a method token, followed by the Request-URI and the protocol
-   * version, and ending with CRLF. However, this method just returns a String representation
-   * similar to the HTTP Request-Line based on values of the Link's properties, which only includes
-   * method and request URI.
-   * <p/>
-   * 
-   * @return a String representation of the HTTP request-line.
-   * @see java.net.URI
-   * @see org.apache.geode.management.internal.web.http.HttpMethod
-   * @see <a href=
-   *      "http://www.w3.org/Protocols/rfc2616/rfc2616-sec5.html">http://www.w3.org/Protocols/rfc2616/rfc2616-sec5.html</a>
-   */
-  public String toHttpRequestLine() {
-    return getMethod().name().concat(StringUtils.SPACE)
-        .concat(UriUtils.decode(getHref().toString()));
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder buffer = new StringBuilder();
-    buffer.append("{ class = ").append(getClass().getName());
-    buffer.append(", rel = ").append(getRelation());
-    buffer.append(", href = ").append(getHref());
-    buffer.append(", method = ").append(getMethod());
-    buffer.append(" }");
-    return buffer.toString();
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/domain/LinkIndex.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/domain/LinkIndex.java
deleted file mode 100644
index 2a99e82..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/domain/LinkIndex.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.domain;
-
-import org.apache.commons.lang.StringUtils;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * The LinkIndex class is abstraction for modeling an index of Links.
- * <p/>
- * 
- * @see javax.xml.bind.annotation.XmlRootElement
- * @see org.apache.geode.management.internal.web.domain.Link
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-@XmlRootElement(name = "link-index")
-public class LinkIndex implements Iterable<Link> {
-
-  @XmlElement(name = "link")
-  private final Set<Link> links = new TreeSet<Link>();
-
-  public LinkIndex add(final Link link) {
-    assert link != null : "The Link to add to the index cannot be null!";
-    links.add(link);
-    return this;
-  }
-
-  public LinkIndex addAll(final Link... links) {
-    assert links != null : "The array of Links to add to this index cannot be null!";
-    return addAll(Arrays.asList(links));
-  }
-
-  public LinkIndex addAll(final Iterable<Link> links) {
-    assert links != null : "The Iterable collection of Links to add to this index cannot be null!";
-    for (final Link link : links) {
-      add(link);
-    }
-    return this;
-  }
-
-  public Link find(final String relation) {
-    Link linkFound = null;
-
-    for (Link link : this) {
-      if (link.getRelation().equalsIgnoreCase(relation)) {
-        linkFound = link;
-        break;
-      }
-    }
-
-    return linkFound;
-  }
-
-  public Link[] findAll(final String relation) {
-    final List<Link> links = new ArrayList<Link>();
-
-    for (final Link link : this) {
-      if (link.getRelation().equalsIgnoreCase(relation)) {
-        links.add(link);
-      }
-    }
-
-    return links.toArray(new Link[links.size()]);
-  }
-
-  public boolean isEmpty() {
-    return links.isEmpty();
-  }
-
-  @Override
-  public Iterator<Link> iterator() {
-    return Collections.unmodifiableSet(links).iterator();
-  }
-
-  public int size() {
-    return links.size();
-  }
-
-  public List<Link> toList() {
-    return new ArrayList<Link>(links);
-  }
-
-  public Map<String, List<Link>> toMap() {
-    final Map<String, List<Link>> links = new TreeMap<String, List<Link>>();
-
-    for (final Link link : this) {
-      List<Link> linksByRelation = links.get(link.getRelation());
-
-      if (linksByRelation == null) {
-        linksByRelation = new ArrayList<Link>(size());
-        links.put(link.getRelation(), linksByRelation);
-      }
-
-      linksByRelation.add(link);
-    }
-
-    return links;
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder buffer = new StringBuilder("[");
-    int count = 0;
-
-    for (final Link link : this) {
-      buffer.append(count++ > 0 ? ", " : StringUtils.EMPTY).append(link);
-    }
-
-    buffer.append("]");
-
-    return buffer.toString();
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/http/ClientHttpRequest.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/http/ClientHttpRequest.java
deleted file mode 100644
index 7c60c9a..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/http/ClientHttpRequest.java
+++ /dev/null
@@ -1,497 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.http;
-
-import java.net.URI;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.springframework.http.HttpEntity;
-import org.springframework.http.HttpHeaders;
-import org.springframework.http.HttpMethod;
-import org.springframework.http.HttpRequest;
-import org.springframework.http.MediaType;
-import org.springframework.util.LinkedMultiValueMap;
-import org.springframework.util.MultiValueMap;
-import org.springframework.web.util.UriComponentsBuilder;
-import org.springframework.web.util.UriTemplate;
-
-import org.apache.geode.internal.lang.Filter;
-import org.apache.geode.internal.lang.ObjectUtils;
-import org.apache.geode.internal.util.CollectionUtils;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.management.internal.web.util.UriUtils;
-
-/**
- * The ClientHttpRequest class is an abstraction modeling an HTTP request sent by a client and
- * serves as the envelop encapsulating all the necessary information (headers, request parameters,
- * body, etc) to send the client's request using HTTP.
- * <p/>
- * The required information for an HTTP request comes from a combination of the Link class
- * containing the reference uniquely identifying the resource or location of where the request will
- * be sent, along with the HttpHeaders class capturing the headers for the request as well as the
- * generic container, HttpEntity to write the body of the request.
- * <p/>
- * This implementation of HttpRequest should not be confused with Spring's
- * org.springframework.http.client.ClientHttpRequest interface, which is often created by factory
- * using a specific HTTP client technology, like the Java HttpURLConnection or Apache's HTTP
- * components, and so on.
- * <p/>
- * 
- * @see java.net.URI
- * @see org.apache.geode.management.internal.web.http.HttpHeader
- * @see org.apache.geode.management.internal.web.http.HttpMethod
- * @see org.apache.geode.management.internal.web.domain.Link
- * @see org.springframework.http.HttpEntity
- * @see org.springframework.http.HttpHeaders
- * @see org.springframework.http.HttpMethod
- * @see org.springframework.http.HttpRequest
- * @see org.springframework.http.MediaType
- * @see org.springframework.util.MultiValueMap
- * @see org.springframework.web.util.UriComponentsBuilder
- * @see org.springframework.web.util.UriTemplate
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-public class ClientHttpRequest implements HttpRequest {
-
-  // the HTTP headers to be sent with the client's request message
-  private final HttpHeaders requestHeaders = new HttpHeaders();
-
-  // the Link referencing the URI and method used with HTTP for the client's request
-  private final Link link;
-
-  // the mapping of request parameter name and values encoded for HTTP and sent with/in the client's
-  // request message
-  private final MultiValueMap<String, Object> requestParameters =
-      new LinkedMultiValueMap<String, Object>();
-
-  // the content/media or payload for the body of the client's HTTP request
-  private Object content;
-
-  /**
-   * Constructs an instance of the ClientHttpRequest class initialized with the specified Link
-   * containing the URI and method for the client's HTTP request.
-   * <p/>
-   * 
-   * @param link the Link encapsulating the URI and method for the client's HTTP request.
-   * @see org.apache.geode.management.internal.web.domain.Link
-   */
-  public ClientHttpRequest(final Link link) {
-    assert link != null : "The Link containing the URI and method for the client's HTTP request cannot be null!";
-    this.link = link;
-  }
-
-  /**
-   * Gets the HTTP headers that will be sent in the client's HTTP request message.
-   * <p/>
-   * 
-   * @return the HTTP headers that will be sent in the client's HTTP request message.
-   * @see org.springframework.http.HttpHeaders
-   * @see org.springframework.http.HttpMessage#getHeaders()
-   */
-  @Override
-  public HttpHeaders getHeaders() {
-    return requestHeaders;
-  }
-
-  /**
-   * Gets the Link containing the URI and method used to send the client's HTTP request.
-   * <p/>
-   * 
-   * @return the Link encapsulating the URI and method for the client's HTTP request.
-   * @see org.apache.geode.management.internal.web.domain.Link
-   */
-  public Link getLink() {
-    return link;
-  }
-
-  /**
-   * Gets the HTTP method indicating the operation to perform on the resource identified in the
-   * client's HTTP request. This method converts GemFire's HttpMethod enumerated value from the Link
-   * into a corresponding Spring HttpMethod enumerated value.
-   * <p/>
-   * 
-   * @return a Spring HttpMethod enumerated value indicating the operation to perform on the
-   *         resource identified in the client's HTTP request.
-   * @see org.apache.geode.management.internal.web.http.HttpMethod
-   * @see org.apache.geode.management.internal.web.domain.Link#getMethod()
-   * @see org.springframework.http.HttpMethod
-   * @see org.springframework.http.HttpRequest#getMethod()
-   */
-  @Override
-  public HttpMethod getMethod() {
-    switch (getLink().getMethod()) {
-      case DELETE:
-        return HttpMethod.DELETE;
-      case HEAD:
-        return HttpMethod.HEAD;
-      case OPTIONS:
-        return HttpMethod.OPTIONS;
-      case POST:
-        return HttpMethod.POST;
-      case PUT:
-        return HttpMethod.PUT;
-      case TRACE:
-        return HttpMethod.TRACE;
-      case GET:
-      default:
-        return HttpMethod.GET;
-    }
-  }
-
-  /**
-   * Determines whether this is an HTTP DELETE request.
-   * <p/>
-   * 
-   * @return a boolean value indicating if the HTTP method is DELETE.
-   * @see #getMethod()
-   * @see org.springframework.http.HttpMethod#DELETE
-   */
-  public boolean isDelete() {
-    return HttpMethod.DELETE.equals(getMethod());
-  }
-
-  /**
-   * Determines whether this is an HTTP GET request.
-   * <p/>
-   * 
-   * @return a boolean value indicating if the HTTP method is GET.
-   * @see #getMethod()
-   * @see org.springframework.http.HttpMethod#GET
-   */
-  public boolean isGet() {
-    return HttpMethod.GET.equals(getMethod());
-  }
-
-  /**
-   * Determines whether this is an HTTP POST request.
-   * <p/>
-   * 
-   * @return a boolean value indicating if the HTTP method is POST.
-   * @see #getMethod()
-   * @see org.springframework.http.HttpMethod#POST
-   */
-  public boolean isPost() {
-    return HttpMethod.POST.equals(getMethod());
-  }
-
-  /**
-   * Determines whether this is an HTTP PUT request.
-   * <p/>
-   * 
-   * @return a boolean value indicating if the HTTP method is PUT.
-   * @see #getMethod()
-   * @see org.springframework.http.HttpMethod#PUT
-   */
-  public boolean isPut() {
-    return HttpMethod.PUT.equals(getMethod());
-  }
-
-  /**
-   * Gets the request parameters that will be sent in the client's HTTP request message.
-   * <p/>
-   * 
-   * @return a MultiValueMap of request parameters and values that will be sent in the client's HTTP
-   *         request message.
-   * @see org.springframework.util.MultiValueMap
-   */
-  public MultiValueMap<String, Object> getParameters() {
-    return requestParameters;
-  }
-
-  /**
-   * Gets the path variables in the URI template. Note, this would be better placed in the Link
-   * class, but Link cannot contain an Spring dependencies!
-   * <p/>
-   * 
-   * @return a List of Strings for each path variable in the URI template.
-   * @see #getURI()
-   * @see org.springframework.web.util.UriTemplate
-   */
-  protected List<String> getPathVariables() {
-    return Collections
-        .unmodifiableList(new UriTemplate(UriUtils.decode(getURI().toString())).getVariableNames());
-  }
-
-  /**
-   * Gets the URI for the client's HTTP request. The URI may actually be an encoded URI template
-   * containing path variables requiring expansion.
-   * <p/>
-   * 
-   * @return the URI of the resource targeted in the request by the client using HTTP.
-   * @see java.net.URI
-   * @see org.springframework.http.HttpRequest#getURI()
-   */
-  @Override
-  public URI getURI() {
-    return getLink().getHref();
-  }
-
-  /**
-   * Gets the URL for the client's HTTP request.
-   * <p/>
-   * 
-   * @return a URL as a URI referring to the location of the resource requested by the client via
-   *         HTTP.
-   * @see #getURL(java.util.Map)
-   * @see java.net.URI
-   */
-  public URI getURL() {
-    return getURL(Collections.<String, Object>emptyMap());
-  }
-
-  /**
-   * Gets the URL for the client's HTTP request.
-   * <p/>
-   * 
-   * @param uriVariables a Map of URI path variables to values in order to expand the URI template
-   *        into a URI.
-   * @return a URL as a URI referring to the location of the resource requested by the client via
-   *         HTTP.
-   * @see #getURI()
-   * @see java.net.URI
-   * @see org.springframework.web.util.UriComponents
-   * @see org.springframework.web.util.UriComponentsBuilder
-   */
-  public URI getURL(final Map<String, ?> uriVariables) {
-    final UriComponentsBuilder uriBuilder =
-        UriComponentsBuilder.fromUriString(UriUtils.decode(getURI().toString()));
-
-    if (isGet() || isDelete()) {
-      final List<String> pathVariables = getPathVariables();
-
-      // get query parameters to append to the URI/URL based on the request parameters that are not
-      // path variables...
-      final Map<String, List<Object>> queryParameters =
-          CollectionUtils.removeKeys(new LinkedMultiValueMap<String, Object>(getParameters()),
-              new Filter<Map.Entry<String, List<Object>>>() {
-                @Override
-                public boolean accept(final Map.Entry<String, List<Object>> entry) {
-                  return !pathVariables.contains(entry.getKey());
-                }
-              });
-
-      for (final String queryParameterName : queryParameters.keySet()) {
-        uriBuilder.queryParam(queryParameterName,
-            getParameters().get(queryParameterName).toArray());
-      }
-    }
-
-    return uriBuilder.build().expand(UriUtils.encode(new HashMap<String, Object>(uriVariables)))
-        .encode().toUri();
-  }
-
-  /**
-   * Gets the HTTP request entity encapsulating the headers and body of the HTTP message. The body
-   * of the HTTP request message will consist of an URL encoded application form (a mapping of
-   * key-value pairs) for POST/PUT HTTP requests.
-   * <p/>
-   * 
-   * @return an HttpEntity with the headers and body for the HTTP request message.
-   * @see #getParameters()
-   * @see org.springframework.http.HttpEntity
-   * @see org.springframework.http.HttpHeaders
-   */
-  public HttpEntity<?> createRequestEntity() {
-    if (isPost() || isPut()) {
-      // NOTE HTTP request parameters take precedence over HTTP message body content/media
-      if (!getParameters().isEmpty()) {
-        getHeaders().setContentType(determineContentType(MediaType.APPLICATION_FORM_URLENCODED));
-        return new HttpEntity<MultiValueMap<String, Object>>(getParameters(), getHeaders());
-      } else {
-        // NOTE the HTTP "Content-Type" header will be determined and set by the appropriate
-        // HttpMessageConverter
-        // based on the Class type of the "content".
-        return new HttpEntity<Object>(getContent(), getHeaders());
-      }
-    } else {
-      return new HttpEntity<Object>(getHeaders());
-    }
-  }
-
-  /**
-   * Tries to determine the content/media type of this HTTP request iff the HTTP "Content-Type"
-   * header was not explicitly set by the user, otherwise the user provided value is used. If the
-   * "Content-Type" HTTP header value is null, then the content/media/payload of this HTTP request
-   * is inspected to determine the content type.
-   * <p/>
-   * The simplest evaluation sets the content type to "application/x-www-form-urlencoded" if this is
-   * a POST or PUT HTTP request, unless any request parameter value is determined to have multiple
-   * parts, the the content type will be "multipart/form-data".
-   * <p/>
-   * 
-   * @param defaultContentType the default content/media type to use when the content type cannot be
-   *        determined from this HTTP request.
-   * @return a MediaType for the value of the HTTP Content-Type header as determined from this HTTP
-   *         request.
-   * @see #getHeaders()
-   * @see org.springframework.http.HttpHeaders#getContentType()
-   * @see org.springframework.http.MediaType
-   */
-  protected MediaType determineContentType(final MediaType defaultContentType) {
-    MediaType contentType = getHeaders().getContentType();
-
-    // if the content type HTTP header was not explicitly set, try to determine the media type from
-    // the content body
-    // of the HTTP request
-    if (contentType == null) {
-      if (isPost() || isPut()) {
-        OUT: for (final String name : getParameters().keySet()) {
-          for (final Object value : getParameters().get(name)) {
-            if (value != null && !(value instanceof String)) {
-              contentType = MediaType.MULTIPART_FORM_DATA;
-              break OUT;
-            }
-          }
-        }
-
-        // since this is a POST/PUT HTTP request, default the content/media type to
-        // "application/x-www-form-urlencoded"
-        contentType = ObjectUtils.defaultIfNull(contentType, MediaType.APPLICATION_FORM_URLENCODED);
-      } else {
-        // NOTE the "Content-Type" HTTP header is not applicable to GET/DELETE and other methods of
-        // HTTP requests
-        // since there is typically no content (media/payload/request body/etc) to send. Any request
-        // parameters
-        // are encoded in the URL as query parameters.
-      }
-    }
-
-    return ObjectUtils.defaultIfNull(contentType, defaultContentType);
-  }
-
-  public Object getContent() {
-    return content;
-  }
-
-  public void setContent(final Object content) {
-    this.content = content;
-  }
-
-  /**
-   * Adds 1 or more values for the specified HTTP header.
-   * <p/>
-   * 
-   * @param headerName a String specifying the name of the HTTP header.
-   * @param headerValues the array of values to set for the HTTP header.
-   * @see org.springframework.http.HttpHeaders#add(String, String)
-   */
-  public void addHeaderValues(final String headerName, final String... headerValues) {
-    if (headerValues != null) {
-      for (final String headerValue : headerValues) {
-        getHeaders().add(headerName, headerValue);
-      }
-    }
-  }
-
-  /**
-   * Gets the first value for the specified HTTP header or null if the HTTP header is not set.
-   * <p/>
-   * 
-   * @param headerName a String specifying the name of the HTTP header.
-   * @return the first value in the list of values for the HTTP header, or null if the HTTP header
-   *         is not set.
-   * @see org.springframework.http.HttpHeaders#getFirst(String)
-   */
-  public String getHeaderValue(final String headerName) {
-    return getHeaders().getFirst(headerName);
-  }
-
-  /**
-   * Gets all values for the specified HTTP header or an empty List if the HTTP header is not set.
-   * <p/>
-   * 
-   * @param headerName a String specifying the name of the HTTP header.
-   * @return a list of String values for the specified HTTP header.
-   * @see org.springframework.http.HttpHeaders#get(Object)
-   */
-  public List<String> getHeaderValues(final String headerName) {
-    return Collections.unmodifiableList(getHeaders().get(headerName));
-  }
-
-  /**
-   * Sets the specified HTTP header to the given value, overriding any previously set values for the
-   * HTTP header.
-   * <p/>
-   * 
-   * @param headerName a String specifying the name of the HTTP header.
-   * @param headerValue a String containing the value of the HTTP header.
-   * @see org.springframework.http.HttpHeaders#set(String, String)
-   */
-  public void setHeader(final String headerName, final String headerValue) {
-    getHeaders().set(headerName, headerValue);
-  }
-
-  /**
-   * Adds 1 or more parameter values to the HTTP request.
-   * <p/>
-   * 
-   * @param requestParameterName a String specifying the name of the HTTP request parameter.
-   * @param requestParameterValues the array of values to set for the HTTP request parameter.
-   * @see org.springframework.util.MultiValueMap#add(Object, Object)
-   */
-  public void addParameterValues(final String requestParameterName,
-      final Object... requestParameterValues) {
-    if (requestParameterValues != null) {
-      for (final Object requestParameterValue : requestParameterValues) {
-        getParameters().add(requestParameterName, requestParameterValue);
-      }
-    }
-  }
-
-  /**
-   * Gets the first value for the specified HTTP request parameter or null if the HTTP request
-   * parameter is not set.
-   * <p/>
-   * 
-   * @param requestParameterName a String specifying the name of the HTTP request parameter.
-   * @return the first value in the list of values for the HTTP request parameter, or null if the
-   *         HTTP request parameter is not set.
-   * @see org.springframework.util.MultiValueMap#getFirst(Object)
-   */
-  public Object getParameterValue(final String requestParameterName) {
-    return getParameters().getFirst(requestParameterName);
-  }
-
-  /**
-   * Gets all values for the specified HTTP request parameter or an empty List if the HTTP request
-   * parameter is not set.
-   * <p/>
-   * 
-   * @param requestParameterName a String specifying the name of the HTTP request parameter.
-   * @return a list of String values for the specified HTTP request parameter.
-   * @see org.springframework.util.MultiValueMap#get(Object)
-   */
-  public List<Object> getParameterValues(final String requestParameterName) {
-    return Collections.unmodifiableList(getParameters().get(requestParameterName));
-  }
-
-  /**
-   * Sets the specified HTTP request parameter to the given value, overriding any previously set
-   * values for the HTTP request parameter.
-   * <p/>
-   * 
-   * @param name a String specifying the name of the HTTP request parameter.
-   * @param value a String containing the value of the HTTP request parameter.
-   * @see org.springframework.util.MultiValueMap#set(Object, Object)
-   */
-  public void setParameter(final String name, final Object value) {
-    getParameters().set(name, value);
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/http/HttpHeader.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/http/HttpHeader.java
deleted file mode 100644
index 74836bc..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/http/HttpHeader.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.http;
-
-import org.apache.commons.lang.StringUtils;
-
-/**
- * The HttpHeader enum is an enumeration of all HTTP request/response header names.
- * <p/>
- * 
- * @see <a href=
- *      "http://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html">http://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html</a>
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-public enum HttpHeader {
-  ACCEPT("Accept"),
-  ACCEPT_CHARSET("Accept-Charset"),
-  ACCEPT_ENCODING("Accept-Encoding"),
-  ACCEPT_LANGUAGE("Accept-Language"),
-  ACCEPT_RANGES("Accept-Ranges"),
-  AGE("Age"),
-  ALLOW("Allow"),
-  AUTHORIZATION("Authorization"),
-  CACHE_CONTROL("Cache-Control"),
-  CONNECTION("Connection"),
-  CONTENT_ENCODING("Content-Encoding"),
-  CONTENT_LANGUAGE("Content-Language"),
-  CONTENT_LENGTH("Content-Length"),
-  CONTENT_LOCATION("Content-Location"),
-  CONTENT_MD5("Content-MD5"),
-  CONTENT_RANGE("Content-Range"),
-  CONTENT_TYPE("Content-Type"),
-  DATE("Date"),
-  ETAG("ETag"),
-  EXPECT("Expect"),
-  EXPIRES("Expires"),
-  FROM("From"),
-  HOST("Host"),
-  IF_MATCH("If-Match"),
-  IF_MODIFIED_SINCE("If-Modified-Since"),
-  IF_NONE_MATCH("If-None-Match"),
-  IF_RANGE("If-Range"),
-  IF_UNMODIFIED_SINCE("If-Unmodified-Since"),
-  LAST_MODIFIED("Last-Modified"),
-  LOCATION("Location"),
-  MAX_FORWARDS("Max-Forwards"),
-  PRAGMA("Pragma"),
-  PROXY_AUTHENTICATE("Proxy-Authenticate"),
-  PROXY_AUTHORIZATION("Proxy-Authorization"),
-  RANGE("Range"),
-  REFERER("Referer"),
-  RETRY_AFTER("Retry-After"),
-  SERVER("Server"),
-  TE("TE"),
-  TRAILER("Trailer"),
-  TRANSFER_ENCODING("Transfer-Encoding"),
-  UPGRADE("Upgrade"),
-  USER_AGENT("User-Agent"),
-  VARY("Vary"),
-  VIA("Via"),
-  WARNING("Warning"),
-  WWW_AUTHENTICATE("WWW-Authenticate");
-
-  // the name of the Http request or response header
-  private final String name;
-
-  HttpHeader(final String name) {
-    assert StringUtils.isNotBlank(name) : "The name of the HTTP request header must be specified!";
-    this.name = name;
-  }
-
-  public static HttpHeader valueOfName(final String name) {
-    for (final HttpHeader header : values()) {
-      if (header.getName().equalsIgnoreCase(name)) {
-        return header;
-      }
-    }
-
-    return null;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/http/support/SimpleHttpRequester.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/http/support/SimpleHttpRequester.java
index 89adc20..23c4482 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/http/support/SimpleHttpRequester.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/http/support/SimpleHttpRequester.java
@@ -17,12 +17,9 @@ package org.apache.geode.management.internal.web.http.support;
 
 import java.io.IOException;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.security.AuthenticationFailedException;
-import org.apache.geode.security.NotAuthorizedException;
-
 import org.springframework.http.HttpEntity;
 import org.springframework.http.HttpHeaders;
 import org.springframework.http.HttpMethod;
@@ -33,6 +30,10 @@ import org.springframework.http.client.SimpleClientHttpRequestFactory;
 import org.springframework.web.client.ResponseErrorHandler;
 import org.springframework.web.client.RestTemplate;
 
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.NotAuthorizedException;
+
 
 /**
  * The SimpleHttpRequester class is a Adapter/facade for the Spring RestTemplate class for
@@ -54,13 +55,9 @@ public class SimpleHttpRequester {
 
   private String pwd;
 
-  private Map<String, String> securityProperties;
+  private Properties securityProperties;
 
-  /**
-   * Default constructor to create an instance of the SimpleHttpRequester class using the default
-   * connection timeout of 30 seconds.
-   */
-  public SimpleHttpRequester(Gfsh gfsh, Map<String, String> securityProperties) {
+  public SimpleHttpRequester(final Gfsh gfsh, Properties securityProperties) {
     this(gfsh, DEFAULT_CONNECT_TIMEOUT, securityProperties);
   }
 
@@ -72,7 +69,7 @@ public class SimpleHttpRequester {
    *        establishing the HTTP connection to the HTTP server.
    */
   public SimpleHttpRequester(final Gfsh gfsh, final int connectTimeout,
-      Map<String, String> securityProperties) {
+      Properties securityProperties) {
     final SimpleClientHttpRequestFactory clientHttpRequestFactory =
         new SimpleClientHttpRequestFactory();
 
@@ -238,11 +235,9 @@ public class SimpleHttpRequester {
   protected HttpEntity<?> getRequestEntity() {
     HttpHeaders requestHeaders = new HttpHeaders();
     if (this.securityProperties != null) {
-      requestHeaders.setAll(securityProperties);
+      requestHeaders.setAll((Map) securityProperties);
     }
-
     HttpEntity<?> requestEntity = new HttpEntity(requestHeaders);
-
     return requestEntity;
 
   }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/AbstractHttpOperationInvoker.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/AbstractHttpOperationInvoker.java
deleted file mode 100644
index 63eb977..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/AbstractHttpOperationInvoker.java
+++ /dev/null
@@ -1,895 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.management.internal.web.shell;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.geode.internal.GemFireVersion;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.util.IOUtils;
-import org.apache.geode.management.DistributedSystemMXBean;
-import org.apache.geode.management.internal.MBeanJMXAdapter;
-import org.apache.geode.management.internal.ManagementConstants;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.management.internal.web.domain.QueryParameterSource;
-import org.apache.geode.management.internal.web.http.ClientHttpRequest;
-import org.apache.geode.management.internal.web.http.HttpHeader;
-import org.apache.geode.management.internal.web.http.HttpMethod;
-import org.apache.geode.management.internal.web.http.converter.SerializableObjectHttpMessageConverter;
-import org.apache.geode.management.internal.web.shell.support.HttpMBeanProxyFactory;
-import org.apache.geode.management.internal.web.util.UriUtils;
-import org.apache.geode.security.AuthenticationFailedException;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.logging.log4j.Logger;
-import org.springframework.http.HttpHeaders;
-import org.springframework.http.HttpStatus;
-import org.springframework.http.MediaType;
-import org.springframework.http.ResponseEntity;
-import org.springframework.http.client.ClientHttpResponse;
-import org.springframework.http.client.SimpleClientHttpRequestFactory;
-import org.springframework.http.converter.HttpMessageConverter;
-import org.springframework.http.converter.json.MappingJackson2HttpMessageConverter;
-import org.springframework.web.client.RequestCallback;
-import org.springframework.web.client.ResourceAccessException;
-import org.springframework.web.client.ResponseErrorHandler;
-import org.springframework.web.client.ResponseExtractor;
-import org.springframework.web.client.RestTemplate;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.URI;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import javax.management.ObjectName;
-import javax.management.QueryExp;
-
-/**
- * The AbstractHttpOperationInvoker class is an abstract base class encapsulating common
- * functionality for all HTTP-based OperationInvoker implementations.
- * 
- * @see org.apache.geode.management.internal.cli.shell.Gfsh
- * @see org.apache.geode.management.internal.cli.shell.OperationInvoker
- * @see org.apache.geode.management.internal.web.shell.HttpOperationInvoker
- * @see org.apache.geode.management.internal.web.shell.RestHttpOperationInvoker
- * @see org.apache.geode.management.internal.web.shell.SimpleHttpOperationInvoker
- * @see org.springframework.http.client.SimpleClientHttpRequestFactory
- * @see org.springframework.web.client.RestTemplate
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-public abstract class AbstractHttpOperationInvoker implements HttpOperationInvoker {
-
-  protected static final long DEFAULT_INITIAL_DELAY = TimeUnit.SECONDS.toMillis(1);
-  protected static final long DEFAULT_PERIOD = TimeUnit.MILLISECONDS.toMillis(2000);
-
-  protected static final String MBEAN_ATTRIBUTE_LINK_RELATION = "mbean-attribute";
-  protected static final String MBEAN_OPERATION_LINK_RELATION = "mbean-operation";
-  protected static final String MBEAN_QUERY_LINK_RELATION = "mbean-query";
-  protected static final String PING_LINK_RELATION = "ping";
-  protected static final String DEFAULT_ENCODING = UriUtils.DEFAULT_ENCODING;
-  protected static final String REST_API_BASE_URL = "http://localhost:8080";
-  protected static final String REST_API_VERSION = "/v1";
-  protected static final String REST_API_WEB_APP_CONTEXT = "/gemfire";
-  protected static final String REST_API_URL =
-      REST_API_BASE_URL + REST_API_WEB_APP_CONTEXT + REST_API_VERSION;
-  protected static final String USER_AGENT_HTTP_REQUEST_HEADER_VALUE =
-      "GemFire-Shell/v" + GemFireVersion.getGemFireVersion();
-
-  protected static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS;
-
-  // the ID of the GemFire distributed system (cluster)
-  private Integer clusterId = CLUSTER_ID_WHEN_NOT_CONNECTED;
-
-  // Executor for scheduling periodic Runnable task to assess the state of the Manager's HTTP
-  // service or Web Service
-  // hosting the M&M REST API (interface)
-  private final ScheduledExecutorService executorService;
-
-  // a reference to the GemFire shell (Gfsh) instance using this HTTP-based OperationInvoker for
-  // command execution
-  // and processing
-  private final Gfsh gfsh;
-
-  // a list of acceptable content/media types supported by Gfsh
-  private final List<MediaType> acceptableMediaTypes =
-      Arrays.asList(MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN);
-
-  // a Java Logger used to log severe, warning, informational and debug messages during the
-  // operation of this invoker
-  private final Logger logger = LogService.getLogger();
-
-  // the Spring RestTemplate used to send HTTP requests and make REST API calls
-  private volatile RestTemplate restTemplate;
-
-  // the base URL of the GemFire Manager's embedded HTTP service and REST API interface
-  private final String baseUrl;
-
-
-  protected Map<String, String> securityProperties;
-
-  /**
-   * Default, public, no-arg constructor to create an instance of the AbstractHttpOperationInvoker
-   * class for testing purposes.
-   */
-  AbstractHttpOperationInvoker(final String baseUrl) {
-    this.baseUrl = baseUrl;
-    this.executorService = null;
-    this.gfsh = null;
-    this.restTemplate = null;
-  }
-
-  /**
-   * Constructs an instance of the AbstractHttpOperationInvoker class with a reference to the
-   * GemFire shell (Gfsh) instance using this HTTP-based OperationInvoker to send commands to the
-   * GemFire Manager via HTTP for processing.
-   * 
-   * @param gfsh a reference to the instance of the GemFire shell (Gfsh) using this HTTP-based
-   *        OperationInvoker for command processing.
-   * @throws AssertionError if the reference to the Gfsh instance is null.
-   * @see #AbstractHttpOperationInvoker(org.apache.geode.management.internal.cli.shell.Gfsh, String,
-   *      Map)
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   */
-  public AbstractHttpOperationInvoker(final Gfsh gfsh, Map<String, String> securityProperties) {
-    this(gfsh, REST_API_URL, securityProperties);
-  }
-
-  /**
-   * Constructs an instance of the AbstractHttpOperationInvoker class with a reference to the
-   * GemFire shell (Gfsh) instance using this HTTP-based OperationInvoker to send commands to the
-   * GemFire Manager via HTTP for procsessing along with the base URL to the GemFire Manager's
-   * embedded HTTP service hosting the HTTP (REST) interface.
-   * 
-   * @param gfsh a reference to the instance of the GemFire shell (Gfsh) using this HTTP-based
-   *        OperationInvoker for command processing.
-   * @param baseUrl a String specifying the base URL to the GemFire Manager's embedded HTTP service
-   *        hosting the REST interface.
-   * @throws AssertionError if the reference to the Gfsh instance is null.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   */
-  public AbstractHttpOperationInvoker(final Gfsh gfsh, final String baseUrl,
-      Map<String, String> securityProperties) {
-    assertNotNull(gfsh, "The reference to the GemFire shell (Gfsh) cannot be null!");
-
-    this.gfsh = gfsh;
-    this.baseUrl = StringUtils.defaultIfBlank(baseUrl, REST_API_URL);
-    this.securityProperties = securityProperties;
-
-    // constructs an instance of a single-threaded, scheduled Executor to send periodic HTTP
-    // requests to the Manager's
-    // HTTP service or Web Service to assess the "alive" state
-    this.executorService = Executors.newSingleThreadScheduledExecutor();
-
-    // constructs an instance of the Spring RestTemplate for M&M REST API (interface) operations
-    this.restTemplate = new RestTemplate(new SimpleClientHttpRequestFactory());
-
-    // add our custom HttpMessageConverter for serializing DTO Objects into the HTTP request message
-    // body
-    // and de-serializing HTTP response message body content back into DTO Objects
-    List<HttpMessageConverter<?>> converters = this.restTemplate.getMessageConverters();
-    // remove the MappingJacksonHttpConverter
-    for (int i = converters.size() - 1; i >= 0; i--) {
-      HttpMessageConverter converter = converters.get(i);
-      if (converter instanceof MappingJackson2HttpMessageConverter) {
-        converters.remove(converter);
-      }
-    }
-    converters.add(new SerializableObjectHttpMessageConverter());
-
-    // set the ResponseErrorHandler handling any errors originating from our HTTP request
-    this.restTemplate.setErrorHandler(new ResponseErrorHandler() {
-      @Override
-      public boolean hasError(final ClientHttpResponse response) throws IOException {
-        final HttpStatus status = response.getStatusCode();
-
-        switch (status) {
-          case BAD_REQUEST: // 400 *
-          case UNAUTHORIZED: // 401
-          case FORBIDDEN: // 403
-          case NOT_FOUND: // 404 *
-          case METHOD_NOT_ALLOWED: // 405 *
-          case NOT_ACCEPTABLE: // 406 *
-          case REQUEST_TIMEOUT: // 408
-          case CONFLICT: // 409
-          case REQUEST_ENTITY_TOO_LARGE: // 413
-          case REQUEST_URI_TOO_LONG: // 414
-          case UNSUPPORTED_MEDIA_TYPE: // 415 *
-          case TOO_MANY_REQUESTS: // 429
-          case INTERNAL_SERVER_ERROR: // 500 *
-          case NOT_IMPLEMENTED: // 501
-          case BAD_GATEWAY: // 502 ?
-          case SERVICE_UNAVAILABLE: // 503
-            return true;
-          default:
-            return false;
-        }
-      }
-
-      @Override
-      public void handleError(final ClientHttpResponse response) throws IOException {
-        String body = readBody(response);
-        final String message = String.format("The HTTP request failed with: %1$d - %2$s.",
-            response.getRawStatusCode(), body);
-
-        if (gfsh.getDebug()) {
-          gfsh.logSevere(body, null);
-        }
-
-        if (response.getRawStatusCode() == 401) {
-          throw new AuthenticationFailedException(message);
-        } else if (response.getRawStatusCode() == 403) {
-          throw new NotAuthorizedException(message);
-        } else {
-          throw new RuntimeException(message);
-        }
-      }
-
-      private String readBody(final ClientHttpResponse response) throws IOException {
-        BufferedReader responseBodyReader = null;
-
-        try {
-          responseBodyReader = new BufferedReader(new InputStreamReader(response.getBody()));
-
-          final StringBuilder buffer = new StringBuilder();
-          String line;
-
-          while ((line = responseBodyReader.readLine()) != null) {
-            buffer.append(line).append(StringUtils.LINE_SEPARATOR);
-          }
-
-          return buffer.toString().trim();
-        } finally {
-          IOUtils.close(responseBodyReader);
-        }
-      }
-    });
-  }
-
-  /**
-   * Asserts the argument is valid, as determined by the caller passing the result of an evaluated
-   * expression to this assertion.
-   * 
-   * @param validArg a boolean value indicating the evaluation of the expression validating the
-   *        argument.
-   * @param message a String value used as the message when constructing an
-   *        IllegalArgumentException.
-   * @param args Object arguments used to populate placeholder's in the message.
-   * @throws IllegalArgumentException if the argument is not valid.
-   * @see java.lang.String#format(String, Object...)
-   */
-  protected static void assertArgument(final boolean validArg, final String message,
-      final Object... args) {
-    if (!validArg) {
-      throw new IllegalArgumentException(String.format(message, args));
-    }
-  }
-
-  /**
-   * Asserts the Object reference is not null!
-   * 
-   * @param obj the reference to the Object.
-   * @param message the String value used as the message when constructing and throwing a
-   *        NullPointerException.
-   * @param args Object arguments used to populate placeholder's in the message.
-   * @throws NullPointerException if the Object reference is null.
-   * @see java.lang.String#format(String, Object...)
-   */
-  protected static void assertNotNull(final Object obj, final String message,
-      final Object... args) {
-    if (obj == null) {
-      throw new NullPointerException(String.format(message, args));
-    }
-  }
-
-  /**
-   * Asserts whether state, based on the evaluation of a conditional expression, passed to this
-   * assertion is valid.
-   * 
-   * @param validState a boolean value indicating the evaluation of the expression from which the
-   *        conditional state is based. For example, a caller might use an expression of the form
-   *        (initableObj.isInitialized()).
-   * @param message a String values used as the message when constructing an IllegalStateException.
-   * @param args Object arguments used to populate placeholder's in the message.
-   * @throws IllegalStateException if the conditional state is not valid.
-   * @see java.lang.String#format(String, Object...)
-   */
-  protected static void assertState(final boolean validState, final String message,
-      final Object... args) {
-    if (!validState) {
-      throw new IllegalStateException(String.format(message, args));
-    }
-  }
-
-  /**
-   * Gets a list of acceptable content/media types supported by Gfsh.
-   * 
-   * @return a List of acceptable content/media types supported by Gfsh.
-   * @see org.springframework.http.MediaType
-   */
-  protected List<MediaType> getAcceptableMediaTypes() {
-    return acceptableMediaTypes;
-  }
-
-  /**
-   * Returns the base URL to GemFire's REST interface hosted in the GemFire Manager's embedded HTTP
-   * service (Tomcat server).
-   * 
-   * @return a String value specifying the base URL to the GemFire REST interface.
-   */
-  protected String getBaseUrl() {
-    return this.baseUrl;
-  }
-
-  /**
-   * Determines whether Gfsh is in debug mode (or whether the user enabled debugging in Gfsh).
-   * 
-   * @return a boolean value indicating if debugging has been turned on in Gfsh.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh#getDebug()
-   */
-  protected boolean isDebugEnabled() {
-    return getGfsh().getDebug();
-  }
-
-  /**
-   * Gets the ExecutorService used by this HTTP OperationInvoker to scheduled periodic or delayed
-   * tasks.
-   * 
-   * @return an instance of the ScheduledExecutorService for scheduling periodic or delayed tasks.
-   * @see java.util.concurrent.ScheduledExecutorService
-   */
-  protected ScheduledExecutorService getExecutorService() {
-    assertState(this.executorService != null,
-        "The ExecutorService for this HTTP OperationInvoker (%1$s) was not properly initialized!",
-        getClass().getName());
-    return this.executorService;
-  }
-
-  /**
-   * Returns the reference to the GemFire shell (Gfsh) instance using this HTTP-based
-   * OperationInvoker to send commands to the GemFire Manager for remote execution and processing.
-   * 
-   * @return a reference to the instance of the GemFire shell (Gfsh) using this HTTP-based
-   *         OperationInvoker to process commands.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   */
-  protected Gfsh getGfsh() {
-    return this.gfsh;
-  }
-
-  /**
-   * Returns a reference to the Spring RestTemplate used by this HTTP-based OperationInvoker to send
-   * HTTP requests to GemFire's REST interface, making REST API calls.
-   * 
-   * @return an instance of the Spring RestTemplate used to make REST API web service calls.
-   * @see org.springframework.web.client.RestTemplate
-   */
-  protected RestTemplate getRestTemplate() {
-    return this.restTemplate;
-  }
-
-  /**
-   * Creates an instance of a client HTTP request with the specified Link targeting the resource as
-   * well as the intended operation on the resource.
-   * 
-   * @param link a Link with the URI targeting and identifying the resource as well as the method of
-   *        operation on the resource.
-   * @return a client HTTP request with the details of the request.
-   * @see org.apache.geode.management.internal.web.http.ClientHttpRequest
-   * @see org.apache.geode.management.internal.web.domain.Link
-   */
-  protected ClientHttpRequest createHttpRequest(final Link link) {
-    final ClientHttpRequest request = new ClientHttpRequest(link);
-    request.addHeaderValues(HttpHeader.USER_AGENT.getName(), USER_AGENT_HTTP_REQUEST_HEADER_VALUE);
-    request.getHeaders().setAccept(getAcceptableMediaTypes());
-
-    if (this.securityProperties != null) {
-      Iterator<Entry<String, String>> it = this.securityProperties.entrySet().iterator();
-      while (it.hasNext()) {
-        Entry<String, String> entry = it.next();
-        request.addHeaderValues(entry.getKey(), entry.getValue());
-      }
-    }
-    return request;
-  }
-
-  /**
-   * Creates a Link with the specified relation and URI of the remote resource.
-   * 
-   * @param relation a String indicating the link relation, or relative state transition, operation.
-   * @param href the URI identifying the resource and it's location.
-   * @return a Link with the providing relation and URI.
-   * @see org.apache.geode.management.internal.web.domain.Link
-   * @see java.net.URI
-   */
-  protected Link createLink(final String relation, final URI href) {
-    return new Link(relation, href);
-  }
-
-  /**
-   * Creates a Link with the specified relation and URI of the remote resource along with the method
-   * of the operation.
-   * 
-   * @param relation a String indicating the link relation, or relative state transition, operation.
-   * @param href the URI identifying the resource and it's location.
-   * @param method the HTTP method for the operation of the request.
-   * @return a Link with the providing relation and URI.
-   * @see org.apache.geode.management.internal.web.http.HttpMethod
-   * @see org.apache.geode.management.internal.web.domain.Link
-   * @see java.net.URI
-   */
-  protected Link createLink(final String relation, final URI href, final HttpMethod method) {
-    return new Link(relation, href, method);
-  }
-
-  /**
-   * Decodes the encoded String value using the default encoding UTF-8. It is assumed the String
-   * value was encoded with the URLEncoder using the UTF-8 encoding. This method handles
-   * UnsupportedEncodingException by just returning the encodedValue.
-   * 
-   * @param encodedValue the encoded String value to decode.
-   * @return the decoded value of the String or encodedValue if the UTF-8 encoding is unsupported.
-   * @see org.apache.geode.management.internal.web.util.UriUtils#decode(String)
-   */
-  protected String decode(final String encodedValue) {
-    return UriUtils.decode(encodedValue);
-  }
-
-  /**
-   * Decodes the encoded String value using the specified encoding (such as UTF-8). It is assumed
-   * the String value was encoded with the URLEncoder using the specified encoding. This method
-   * handles UnsupportedEncodingException by just returning the encodedValue.
-   * 
-   * @param encodedValue a String value encoded in the encoding.
-   * @param encoding a String value specifying the encoding.
-   * @return the decoded value of the String or encodedValue if the specified encoding is
-   *         unsupported.
-   * @see org.apache.geode.management.internal.web.util.UriUtils#decode(String, String)
-   */
-  protected String decode(final String encodedValue, String encoding) {
-    return UriUtils.decode(encodedValue, encoding);
-  }
-
-  /**
-   * Encode the String value using the default encoding UTF-8.
-   * 
-   * @param value the String value to encode.
-   * @return an encoded value of the String using the default encoding UTF-8 or value if the UTF-8
-   *         encoding is unsupported.
-   * @see org.apache.geode.management.internal.web.util.UriUtils#encode(String)
-   */
-  protected String encode(final String value) {
-    return UriUtils.encode(value);
-  }
-
-  /**
-   * Encode the String value using the specified encoding (such as UTF-8).
-   * 
-   * @param value the String value to encode.
-   * @param encoding a String value indicating the encoding.
-   * @return an encoded value of the String using the specified encoding or value if the specified
-   *         encoding is unsupported.
-   * @see org.apache.geode.management.internal.web.util.UriUtils#encode(String, String)
-   */
-  protected String encode(final String value, final String encoding) {
-    return UriUtils.encode(value, encoding);
-  }
-
-  /**
-   * Finds a Link containing the HTTP request URI for the relational operation (state transition) on
-   * the resource.
-   * 
-   * @param relation a String describing the relational operation, or state transition on the
-   *        resource.
-   * @return an instance of Link containing the HTTP request URI used to perform the intended
-   *         operation on the resource.
-   * @see org.apache.geode.management.internal.web.domain.Link
-   */
-  protected Link findLink(final String relation) {
-    return null;
-  }
-
-  /**
-   * Handles resource access errors such as ConnectExceptions when the server-side process/service
-   * is not listening for client connections, or the connection to the server/service fails.
-   * 
-   * @param e the ResourceAccessException resulting in some sort of I/O error.
-   * @return a user-friendly String message describing the problem and appropriate action/response
-   *         by the user.
-   * @see #stop()
-   * @see org.springframework.web.client.ResourceAccessException
-   */
-  protected String handleResourceAccessException(final ResourceAccessException e) {
-    stop();
-
-    return String.format(
-        "The connection to the GemFire Manager's HTTP service @ %1$s failed with: %2$s. "
-            + "Please try reconnecting or see the GemFire Manager's log file for further details.",
-        getBaseUrl(), e.getMessage());
-  }
-
-  /**
-   * Displays the message inside GemFire shell at debug level.
-   * 
-   * @param message the String containing the message to display inside Gfsh.
-   * @see #isDebugEnabled()
-   * @see #printInfo(String, Object...)
-   */
-  protected void printDebug(final String message, final Object... args) {
-    if (isDebugEnabled()) {
-      printInfo(message, args);
-    }
-  }
-
-  /**
-   * Displays the message inside GemFire shell at info level.
-   * 
-   * @param message the String containing the message to display inside Gfsh.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh#printAsInfo(String)
-   */
-  protected void printInfo(final String message, final Object... args) {
-    getGfsh().printAsInfo(String.format(message, args));
-  }
-
-  /**
-   * Displays the message inside GemFire shell at warning level.
-   * 
-   * @param message the String containing the message to display inside Gfsh.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh#printAsWarning(String)
-   */
-  protected void printWarning(final String message, final Object... args) {
-    getGfsh().printAsWarning(String.format(message, args));
-  }
-
-  /**
-   * Displays the message inside GemFire shell at severe level.
-   * 
-   * @param message the String containing the message to display inside Gfsh.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh#printAsSevere(String)
-   */
-  protected void printSevere(final String message, final Object... args) {
-    getGfsh().printAsSevere(String.format(message, args));
-  }
-
-  /**
-   * Sends the HTTP request, using Spring's RestTemplate, to the GemFire REST API web service
-   * endpoint, expecting the specified response type from the server in return.
-   * 
-   * @param <T> the response type.
-   * @param request the client HTTP request to send.
-   * @param responseType the expected Class type of the return value in the server's response.
-   * @return a ResponseEntity encapsulating the details of the server's response to the client's
-   *         HTTP request.
-   * @see #send(org.apache.geode.management.internal.web.http.ClientHttpRequest, Class,
-   *      java.util.Map)
-   * @see org.apache.geode.management.internal.web.http.ClientHttpRequest
-   * @see org.springframework.http.ResponseEntity
-   */
-  protected <T> T send(final ClientHttpRequest request, final Class<T> responseType) {
-    return send(request, responseType, Collections.<String, Object>emptyMap());
-  }
-
-  /**
-   * Sends the HTTP request, using Spring's RestTemplate, to the GemFire REST API web service
-   * endpoint, expecting the specified response type from the server in return.
-   * 
-   * @param request the client HTTP request to send.
-   * @param responseType the expected Class type of the return value in the server's response.
-   * @param uriVariables a Mapping of URI template path variables to values.
-   * @return a ResponseEntity encapsulating the details of the server's response to the client's
-   *         HTTP request.
-   * @see java.net.URI
-   * @see org.apache.geode.management.internal.web.http.ClientHttpRequest
-   * @see org.springframework.http.ResponseEntity
-   * @see org.springframework.web.client.RestTemplate#exchange(java.net.URI,
-   *      org.springframework.http.HttpMethod, org.springframework.http.HttpEntity, Class)
-   */
-  protected <T> T send(final ClientHttpRequest request, final Class<T> responseType,
-      final Map<String, ?> uriVariables) {
-    final URI url = request.getURL(uriVariables);
-    if (isDebugEnabled()) {
-      printInfo("Link: %1$s", request.getLink().toHttpRequestLine());
-      printInfo("HTTP URL: %1$s", url);
-      printInfo("HTTP request headers: %1$s", request.getHeaders());
-      printInfo("HTTP request parameters: %1$s", request.getParameters());
-    }
-
-    final ResponseEntity<T> response = getRestTemplate().exchange(url, request.getMethod(),
-        request.createRequestEntity(), responseType);
-
-    if (isDebugEnabled()) {
-      printInfo("------------------------------------------------------------------------");
-      printInfo("HTTP response headers: %1$s", response.getHeaders());
-      printInfo("HTTP response status: %1$d - %2$s", response.getStatusCode().value(),
-          response.getStatusCode().getReasonPhrase());
-
-      printInfo("HTTP response body: ", response.getBody());
-    }
-
-    return response.getBody();
-  }
-
-  protected Object downloadResponseToTempFile(ClientHttpRequest request,
-      Map<String, ?> uriVariables) {
-    final URI url = request.getURL(uriVariables);
-
-    // Optional Accept header
-    RequestCallback requestCallback = r -> {
-      r.getHeaders().setAccept(Arrays.asList(MediaType.APPLICATION_OCTET_STREAM));
-      HttpHeaders header = request.getHeaders();
-      r.getHeaders().setAll(request.getHeaders().toSingleValueMap());
-    };
-
-    // Streams the response instead of loading it all in memory
-    ResponseExtractor<Object> responseExtractor = resp -> {
-      MediaType mediaType = resp.getHeaders().getContentType();
-      if (mediaType.equals(MediaType.APPLICATION_JSON)) {
-        return org.apache.commons.io.IOUtils.toString(resp.getBody(), "UTF-8");
-      } else {
-        Path tempFile = Files.createTempFile("fileDownload", "");
-        if (tempFile.toFile().exists()) {
-          FileUtils.deleteQuietly(tempFile.toFile());
-        }
-        Files.copy(resp.getBody(), tempFile);
-        return tempFile;
-      }
-    };
-    return getRestTemplate().execute(url, org.springframework.http.HttpMethod.GET, requestCallback,
-        responseExtractor);
-  }
-
-  /**
-   * Determines whether this HTTP-based OperationInvoker is successfully connected to the remote
-   * GemFire Manager's HTTP service in order to send commands for execution/processing.
-   * 
-   * @return a boolean value indicating the connection state of the HTTP-based OperationInvoker.
-   */
-  @Override
-  public boolean isConnected() {
-    return (getRestTemplate() != null);
-  }
-
-  /**
-   * Determines whether this HTTP-based OperationInvoker is ready to send commands to the GemFire
-   * Manager for remote execution/processing.
-   * 
-   * @return a boolean value indicating whether this HTTP-based OperationInvoker is ready for
-   *         command invocations.
-   * @see #isConnected()
-   */
-  @Override
-  public boolean isReady() {
-    return isConnected();
-  }
-
-  // TODO research the use of Jolokia instead
-
-  /**
-   * Read the attribute identified by name from a remote resource identified by name. The intent of
-   * this method is to return the value of an attribute on an MBean located in the remote
-   * MBeanServer.
-   * 
-   * @param resourceName name/url of the remote resource from which to fetch the attribute value.
-   * @param attributeName name of the attribute who's value will be fetched.
-   * @return the value of the named attribute for the named resource (typically an MBean).
-   * @throws MBeanAccessException if an MBean access error occurs.
-   * @throws RestApiCallForCommandNotFoundException if the REST API web service endpoint for
-   *         accessing an attribute on an MBean does not exists!
-   * @see #createHttpRequest(org.apache.geode.management.internal.web.domain.Link)
-   * @see #findLink(String)
-   * @see #send(org.apache.geode.management.internal.web.http.ClientHttpRequest, Class)
-   */
-  @Override
-  public Object getAttribute(final String resourceName, final String attributeName) {
-    final Link link = findLink(MBEAN_ATTRIBUTE_LINK_RELATION);
-
-    if (link != null) {
-      final ClientHttpRequest request = createHttpRequest(link);
-
-      request.addParameterValues("resourceName", resourceName);
-      request.addParameterValues("attributeName", attributeName);
-
-      try {
-        return IOUtils.deserializeObject(send(request, byte[].class));
-      } catch (IOException e) {
-        throw new MBeanAccessException(String.format(
-            "De-serializing the result of accessing attribute (%1$s) on MBean (%2$s) failed!",
-            resourceName, attributeName), e);
-      } catch (ClassNotFoundException e) {
-        throw new MBeanAccessException(String.format(
-            "The Class type of the result when accessing attribute (%1$s) on MBean (%2$s) was not found!",
-            resourceName, attributeName), e);
-      }
-    } else {
-      printSevere(
-          "Getting the value of attribute (%1$s) on MBean (%2$s) is currently an unsupported operation!",
-          attributeName, resourceName);
-      throw new RestApiCallForCommandNotFoundException(MBEAN_ATTRIBUTE_LINK_RELATION);
-    }
-  }
-
-  /**
-   * Gets the identifier of the GemFire cluster.
-   * 
-   * @return an integer value indicating the identifier of the GemFire cluster.
-   * @see #initClusterId()
-   */
-  @Override
-  public int getClusterId() {
-    return clusterId;
-  }
-
-  protected void initClusterId() {
-    if (isReady()) {
-      try {
-        clusterId = (Integer) getAttribute(ManagementConstants.OBJECTNAME__DISTRIBUTEDSYSTEM_MXBEAN,
-            "DistributedSystemId");
-        printDebug("Cluster ID (%1$s)", clusterId);
-      } catch (Exception ignore) {
-        printDebug("Failed to determine cluster ID: %1$s", ignore.getMessage());
-      }
-    }
-  }
-
-  /**
-   * Gets a proxy to the remote DistributedSystem MXBean to access attributes and invoke operations
-   * on the distributed system, or the GemFire cluster.
-   * 
-   * @return a proxy instance of the GemFire Manager's DistributedSystem MXBean.
-   * @see #getMBeanProxy(javax.management.ObjectName, Class)
-   * @see org.apache.geode.management.DistributedSystemMXBean
-   * @see org.apache.geode.management.internal.MBeanJMXAdapter#getDistributedSystemName()
-   */
-  public DistributedSystemMXBean getDistributedSystemMXBean() {
-    return getMBeanProxy(MBeanJMXAdapter.getDistributedSystemName(), DistributedSystemMXBean.class);
-  }
-
-  /**
-   * Gets a proxy to an MXBean on a remote MBeanServer using HTTP for remoting.
-   * 
-   * @param <T> the class type of the remote MXBean.
-   * @param objectName the JMX ObjectName uniquely identifying the remote MXBean.
-   * @param mbeanInterface the interface of the remote MXBean to proxy for attribute/operation
-   *        access.
-   * @return a proxy using HTTP remoting to access the specified, remote MXBean.
-   * @see javax.management.ObjectName
-   * @see org.apache.geode.management.internal.web.shell.support.HttpMBeanProxyFactory
-   */
-  public <T> T getMBeanProxy(final ObjectName objectName, final Class<T> mbeanInterface) {
-    return HttpMBeanProxyFactory.createMBeanProxy(this, objectName, mbeanInterface);
-  }
-
-  /**
-   * Invoke an operation identified by name on a remote resource identified by name with the given
-   * arguments. The intent of this method is to invoke an arbitrary operation on an MBean located in
-   * the remote MBeanServer.
-   * 
-   * @param resourceName name/url (object name) of the remote resource (MBea) on which operation is
-   *        to be invoked.
-   * @param operationName name of the operation to be invoked.
-   * @param params an array of arguments for the parameters to be set when the operation is invoked.
-   * @param signature an array containing the signature of the operation.
-   * @return result of the operation invocation.
-   * @throws MBeanAccessException if an MBean access error occurs.
-   * @throws RestApiCallForCommandNotFoundException if the REST API web service endpoint for
-   *         invoking an operation on an MBean does not exists!
-   * @see #createHttpRequest(org.apache.geode.management.internal.web.domain.Link)
-   * @see #findLink(String)
-   * @see #send(org.apache.geode.management.internal.web.http.ClientHttpRequest, Class)
-   */
-  // TODO research the use of Jolokia instead
-  @Override
-  public Object invoke(final String resourceName, final String operationName, final Object[] params,
-      final String[] signature) {
-    final Link link = findLink(MBEAN_OPERATION_LINK_RELATION);
-
-    if (link != null) {
-      final ClientHttpRequest request = createHttpRequest(link);
-
-      request.addParameterValues("resourceName", resourceName);
-      request.addParameterValues("operationName", operationName);
-      request.addParameterValues("signature", (Object[]) signature);
-      request.addParameterValues("parameters", params); // TODO may need to convert method parameter
-                                                        // arguments
-
-      try {
-        return IOUtils.deserializeObject(send(request, byte[].class));
-      } catch (IOException e) {
-        throw new MBeanAccessException(String.format(
-            "De-serializing the result from invoking operation (%1$s) on MBean (%2$s) failed!",
-            resourceName, operationName), e);
-      } catch (ClassNotFoundException e) {
-        throw new MBeanAccessException(String.format(
-            "The Class type of the result from invoking operation (%1$s) on MBean (%2$s) was not found!",
-            resourceName, operationName), e);
-      }
-    } else {
-      printSevere(
-          "Invoking operation (%1$s) on MBean (%2$s) is currently an unsupported operation!",
-          operationName, resourceName);
-      throw new RestApiCallForCommandNotFoundException(MBEAN_OPERATION_LINK_RELATION);
-    }
-  }
-
-  /**
-   * This method searches the MBean server, based on the OperationsInvoker's JMX-based or remoting
-   * capable MBean server connection, for MBeans matching a specific ObjectName or matching an
-   * ObjectName pattern along with satisfying criteria from the Query expression.
-   * 
-   * @param objectName the ObjectName or pattern for which matching MBeans in the target MBean
-   *        server will be returned.
-   * @param queryExpression the JMX-based query expression used to filter matching MBeans.
-   * @return a set of ObjectName's matching MBeans in the MBean server matching the ObjectName and
-   *         Query expression criteria.
-   * @see #createHttpRequest(org.apache.geode.management.internal.web.domain.Link)
-   * @see #findLink(String)
-   * @see #send(org.apache.geode.management.internal.web.http.ClientHttpRequest, Class)
-   * @see javax.management.ObjectName
-   * @see javax.management.QueryExp
-   */
-  @Override
-  @SuppressWarnings("unchecked")
-  public Set<ObjectName> queryNames(final ObjectName objectName, final QueryExp queryExpression) {
-    final Link link = findLink(MBEAN_QUERY_LINK_RELATION);
-
-    if (link != null) {
-      final ClientHttpRequest request = createHttpRequest(link);
-
-      request.setContent(new QueryParameterSource(objectName, queryExpression));
-      try {
-        return (Set<ObjectName>) IOUtils.deserializeObject(send(request, byte[].class));
-      } catch (Exception e) {
-        throw new MBeanAccessException(String.format(
-            "An error occurred while querying for MBean names using ObjectName pattern (%1$s) and Query expression (%2$s)!",
-            objectName, queryExpression), e);
-      }
-    } else {
-      printSevere(
-          "Running a query to get the ObjectNames of all MBeans matching the ObjectName pattern (%1$s) and Query expression (%2$s) is currently unsupported!",
-          objectName, queryExpression);
-      throw new RestApiCallForCommandNotFoundException(MBEAN_QUERY_LINK_RELATION);
-    }
-  }
-
-  /**
-   * Stops communication with and closes all connections to the remote HTTP server (service).
-   */
-  @Override
-  public void stop() {
-    if (executorService != null) {
-      executorService.shutdown();
-    }
-
-    restTemplate = null;
-  }
-
-  @Override
-  public String toString() {
-    return String.format("GemFire Manager HTTP service @ %1$s", getBaseUrl());
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/HttpOperationInvoker.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/HttpOperationInvoker.java
index a3ce554..a6f6c6f 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/HttpOperationInvoker.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/HttpOperationInvoker.java
@@ -14,16 +14,680 @@
  */
 package org.apache.geode.management.internal.web.shell;
 
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import javax.management.ObjectName;
+import javax.management.QueryExp;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.logging.log4j.Logger;
+import org.springframework.core.io.Resource;
+import org.springframework.http.HttpEntity;
+import org.springframework.http.HttpHeaders;
+import org.springframework.http.HttpMethod;
+import org.springframework.http.HttpStatus;
+import org.springframework.http.MediaType;
+import org.springframework.http.ResponseEntity;
+import org.springframework.http.client.ClientHttpResponse;
+import org.springframework.http.client.SimpleClientHttpRequestFactory;
+import org.springframework.http.converter.HttpMessageConverter;
+import org.springframework.http.converter.json.MappingJackson2HttpMessageConverter;
+import org.springframework.util.LinkedMultiValueMap;
+import org.springframework.util.MultiValueMap;
+import org.springframework.web.client.ResourceAccessException;
+import org.springframework.web.client.ResponseErrorHandler;
+import org.springframework.web.client.RestTemplate;
+import org.springframework.web.util.UriComponentsBuilder;
+
+import org.apache.geode.annotations.TestingOnly;
+import org.apache.geode.internal.GemFireVersion;
+import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.util.IOUtils;
+import org.apache.geode.management.DistributedSystemMXBean;
+import org.apache.geode.management.internal.MBeanJMXAdapter;
+import org.apache.geode.management.internal.ManagementConstants;
+import org.apache.geode.management.internal.cli.CommandRequest;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.cli.shell.OperationInvoker;
+import org.apache.geode.management.internal.web.domain.QueryParameterSource;
+import org.apache.geode.management.internal.web.http.converter.SerializableObjectHttpMessageConverter;
+import org.apache.geode.management.internal.web.http.support.SimpleHttpRequester;
+import org.apache.geode.management.internal.web.shell.support.HttpMBeanProxyFactory;
+import org.apache.geode.management.internal.web.util.ConvertUtils;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.NotAuthorizedException;
 
 /**
- * The HttpOperationInvoker interface is a marker interface for HTTP-based OperationInvokers.
- * <p/>
+ * The HttpOperationInvoker class is an abstract base class encapsulating common functionality for
+ * all HTTP-based OperationInvoker implementations.
  * 
+ * @see org.apache.geode.management.internal.cli.shell.Gfsh
  * @see org.apache.geode.management.internal.cli.shell.OperationInvoker
+ * @see org.apache.geode.management.internal.web.shell.HttpOperationInvoker
+ * @see org.springframework.http.client.SimpleClientHttpRequestFactory
+ * @see org.springframework.web.client.RestTemplate
  * @since GemFire 8.0
  */
 @SuppressWarnings("unused")
-public interface HttpOperationInvoker extends OperationInvoker {
+public class HttpOperationInvoker implements OperationInvoker {
+
+  protected static final long DEFAULT_INITIAL_DELAY = TimeUnit.SECONDS.toMillis(1);
+  protected static final long DEFAULT_PERIOD = TimeUnit.MILLISECONDS.toMillis(2000);
+
+  protected static final String REST_API_BASE_URL = "http://localhost:8080";
+  protected static final String REST_API_VERSION = "/v1";
+  protected static final String REST_API_WEB_APP_CONTEXT = "/geode-mgmt";
+  protected static final String REST_API_URL =
+      REST_API_BASE_URL + REST_API_WEB_APP_CONTEXT + REST_API_VERSION;
+  protected static final String USER_AGENT_HTTP_REQUEST_HEADER_VALUE =
+      "GemFire-Shell/v" + GemFireVersion.getGemFireVersion();
+
+  protected static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS;
+
+  // the ID of the GemFire distributed system (cluster)
+  private Integer clusterId = CLUSTER_ID_WHEN_NOT_CONNECTED;
+
+  protected static final String RESOURCES_REQUEST_PARAMETER = "resources";
+
+  // Executor for scheduling periodic Runnable task to assess the state of the Manager's HTTP
+  // service or Web Service
+  // hosting the M&M REST API (interface)
+  private final ScheduledExecutorService executorService;
+
+  // a reference to the GemFire shell (Gfsh) instance using this HTTP-based OperationInvoker for
+  // command execution
+  // and processing
+  private final Gfsh gfsh;
+
+  // a list of acceptable content/media types supported by Gfsh
+  private final List<MediaType> acceptableMediaTypes = Arrays.asList(MediaType.APPLICATION_JSON,
+      MediaType.TEXT_PLAIN, MediaType.APPLICATION_OCTET_STREAM);
+
+  // a Java Logger used to log severe, warning, informational and debug messages during the
+  // operation of this invoker
+  private final Logger logger = LogService.getLogger();
+
+  // the Spring RestTemplate used to executeRequest HTTP requests and make REST API calls
+  private volatile RestTemplate restTemplate;
+
+  // the base URL of the GemFire Manager's embedded HTTP service and REST API interface
+  private final String baseUrl;
+
+
+  protected Properties securityProperties;
+
+  /**
+   * Default no-arg constructor to create an instance of the SimpleHttpOperationInvoker class for
+   * testing purposes.
+   */
+  @TestingOnly
+  HttpOperationInvoker() {
+    this(REST_API_URL);
+  }
+
+  /**
+   * Default, public, no-arg constructor to create an instance of the HttpOperationInvoker class for
+   * testing purposes.
+   */
+  @TestingOnly
+  HttpOperationInvoker(final String baseUrl) {
+    this.baseUrl = baseUrl;
+    this.executorService = null;
+    this.gfsh = null;
+    this.restTemplate = null;
+  }
+
+  /**
+   * Constructs an instance of the HttpOperationInvoker class with a reference to the GemFire shell
+   * (Gfsh) instance using this HTTP-based OperationInvoker to send commands to the GemFire Manager
+   * via HTTP for procsessing along with the base URL to the GemFire Manager's embedded HTTP service
+   * hosting the HTTP (REST) interface.
+   *
+   * @param gfsh a reference to the instance of the GemFire shell (Gfsh) using this HTTP-based
+   *        OperationInvoker for command processing.
+   * @param baseUrl a String specifying the base URL to the GemFire Manager's embedded HTTP service
+   *        hosting the REST interface.
+   * @throws AssertionError if the reference to the Gfsh instance is null.
+   * @see org.apache.geode.management.internal.cli.shell.Gfsh
+   */
+  public HttpOperationInvoker(final Gfsh gfsh, final String baseUrl,
+      Properties securityProperties) {
+    this.gfsh = gfsh;
+    this.baseUrl = StringUtils.defaultIfBlank(baseUrl, REST_API_URL);
+    this.securityProperties = securityProperties;
+
+    // constructs an instance of a single-threaded, scheduled Executor to send periodic HTTP
+    // requests to the Manager's
+    // HTTP service or Web Service to assess the "alive" state
+    this.executorService = Executors.newSingleThreadScheduledExecutor();
+
+    // constructs an instance of the Spring RestTemplate for M&M REST API (interface) operations
+    this.restTemplate = new RestTemplate(new SimpleClientHttpRequestFactory());
+
+    // add our custom HttpMessageConverter for serializing DTO Objects into the HTTP request message
+    // body
+    // and de-serializing HTTP response message body content back into DTO Objects
+    List<HttpMessageConverter<?>> converters = this.restTemplate.getMessageConverters();
+    // remove the MappingJacksonHttpConverter
+    for (int i = converters.size() - 1; i >= 0; i--) {
+      HttpMessageConverter converter = converters.get(i);
+      if (converter instanceof MappingJackson2HttpMessageConverter) {
+        converters.remove(converter);
+      }
+    }
+    converters.add(new SerializableObjectHttpMessageConverter());
+
+    // set the ResponseErrorHandler handling any errors originating from our HTTP request
+    this.restTemplate.setErrorHandler(new ResponseErrorHandler() {
+      @Override
+      public boolean hasError(final ClientHttpResponse response) throws IOException {
+        final HttpStatus status = response.getStatusCode();
+
+        switch (status) {
+          case BAD_REQUEST: // 400 *
+          case UNAUTHORIZED: // 401
+          case FORBIDDEN: // 403
+          case NOT_FOUND: // 404 *
+          case METHOD_NOT_ALLOWED: // 405 *
+          case NOT_ACCEPTABLE: // 406 *
+          case REQUEST_TIMEOUT: // 408
+          case CONFLICT: // 409
+          case REQUEST_ENTITY_TOO_LARGE: // 413
+          case REQUEST_URI_TOO_LONG: // 414
+          case UNSUPPORTED_MEDIA_TYPE: // 415 *
+          case TOO_MANY_REQUESTS: // 429
+          case INTERNAL_SERVER_ERROR: // 500 *
+          case NOT_IMPLEMENTED: // 501
+          case BAD_GATEWAY: // 502 ?
+          case SERVICE_UNAVAILABLE: // 503
+            return true;
+          default:
+            return false;
+        }
+      }
+
+      @Override
+      public void handleError(final ClientHttpResponse response) throws IOException {
+        String body = readBody(response);
+        final String message = String.format("The HTTP request failed with: %1$d - %2$s.",
+            response.getRawStatusCode(), body);
+
+        if (gfsh.getDebug()) {
+          gfsh.logSevere(body, null);
+        }
+
+        if (response.getRawStatusCode() == 401) {
+          throw new AuthenticationFailedException(message);
+        } else if (response.getRawStatusCode() == 403) {
+          throw new NotAuthorizedException(message);
+        } else {
+          throw new RuntimeException(message);
+        }
+      }
+
+      private String readBody(final ClientHttpResponse response) throws IOException {
+        BufferedReader responseBodyReader = null;
+
+        try {
+          responseBodyReader = new BufferedReader(new InputStreamReader(response.getBody()));
+
+          final StringBuilder buffer = new StringBuilder();
+          String line;
+
+          while ((line = responseBodyReader.readLine()) != null) {
+            buffer.append(line).append(StringUtils.LINE_SEPARATOR);
+          }
+
+          return buffer.toString().trim();
+        } finally {
+          IOUtils.close(responseBodyReader);
+        }
+      }
+    });
+    setupBackgroundPingRequest();
+    initClusterId();
+  }
+
+  protected void setupBackgroundPingRequest() {
+    SimpleHttpRequester requester = new SimpleHttpRequester(gfsh, securityProperties);
+    getExecutorService().scheduleAtFixedRate(() -> {
+      try {
+        requester.exchange(baseUrl.concat("/ping"), String.class);
+      } catch (Exception e) {
+        printDebug("An error occurred while connecting to the Manager's HTTP service: %1$s: ",
+            e.getMessage());
+        getGfsh().notifyDisconnect(HttpOperationInvoker.this.toString());
+        stop();
+      }
+    }, DEFAULT_INITIAL_DELAY, DEFAULT_PERIOD, DEFAULT_TIME_UNIT);
+  }
+
+  /**
+   * Asserts whether state, based on the evaluation of a conditional expression, passed to this
+   * assertion is valid.
+   *
+   * @param validState a boolean value indicating the evaluation of the expression from which the
+   *        conditional state is based. For example, a caller might use an expression of the form
+   *        (initableObj.isInitialized()).
+   * @param message a String values used as the message when constructing an IllegalStateException.
+   * @param args Object arguments used to populate placeholder's in the message.
+   * @throws IllegalStateException if the conditional state is not valid.
+   * @see java.lang.String#format(String, Object...)
+   */
+  protected static void assertState(final boolean validState, final String message,
+      final Object... args) {
+    if (!validState) {
+      throw new IllegalStateException(String.format(message, args));
+    }
+  }
+
+  /**
+   * Determines whether Gfsh is in debug mode (or whether the user enabled debugging in Gfsh).
+   *
+   * @return a boolean value indicating if debugging has been turned on in Gfsh.
+   * @see org.apache.geode.management.internal.cli.shell.Gfsh#getDebug()
+   */
+  protected boolean isDebugEnabled() {
+    return getGfsh().getDebug();
+  }
+
+  /**
+   * Gets the ExecutorService used by this HTTP OperationInvoker to scheduled periodic or delayed
+   * tasks.
+   *
+   * @return an instance of the ScheduledExecutorService for scheduling periodic or delayed tasks.
+   * @see java.util.concurrent.ScheduledExecutorService
+   */
+  protected ScheduledExecutorService getExecutorService() {
+    return this.executorService;
+  }
+
+  /**
+   * Returns the reference to the GemFire shell (Gfsh) instance using this HTTP-based
+   * OperationInvoker to send commands to the GemFire Manager for remote execution and processing.
+   *
+   * @return a reference to the instance of the GemFire shell (Gfsh) using this HTTP-based
+   *         OperationInvoker to process commands.
+   * @see org.apache.geode.management.internal.cli.shell.Gfsh
+   */
+  protected Gfsh getGfsh() {
+    return this.gfsh;
+  }
+
+  /**
+   * Returns a reference to the Spring RestTemplate used by this HTTP-based OperationInvoker to send
+   * HTTP requests to GemFire's REST interface, making REST API calls.
+   *
+   * @return an instance of the Spring RestTemplate used to make REST API web service calls.
+   * @see org.springframework.web.client.RestTemplate
+   */
+  protected RestTemplate getRestTemplate() {
+    return this.restTemplate;
+  }
+
+  /**
+   * Handles resource access errors such as ConnectExceptions when the server-side process/service
+   * is not listening for client connections, or the connection to the server/service fails.
+   *
+   * @param e the ResourceAccessException resulting in some sort of I/O error.
+   * @return a user-friendly String message describing the problem and appropriate action/response
+   *         by the user.
+   * @see #stop()
+   * @see org.springframework.web.client.ResourceAccessException
+   */
+  protected String handleResourceAccessException(final ResourceAccessException e) {
+    stop();
+
+    return String.format(
+        "The connection to the GemFire Manager's HTTP service @ %1$s failed with: %2$s. "
+            + "Please try reconnecting or see the GemFire Manager's log file for further details.",
+        baseUrl, e.getMessage());
+  }
+
+  /**
+   * Displays the message inside GemFire shell at debug level.
+   *
+   * @param message the String containing the message to display inside Gfsh.
+   * @see #isDebugEnabled()
+   * @see #printInfo(String, Object...)
+   */
+  protected void printDebug(final String message, final Object... args) {
+    if (isDebugEnabled()) {
+      printInfo(message, args);
+    }
+  }
+
+  /**
+   * Displays the message inside GemFire shell at info level.
+   *
+   * @param message the String containing the message to display inside Gfsh.
+   * @see org.apache.geode.management.internal.cli.shell.Gfsh#printAsInfo(String)
+   */
+  protected void printInfo(final String message, final Object... args) {
+    getGfsh().printAsInfo(String.format(message, args));
+  }
+
+  /**
+   * Displays the message inside GemFire shell at warning level.
+   *
+   * @param message the String containing the message to display inside Gfsh.
+   * @see org.apache.geode.management.internal.cli.shell.Gfsh#printAsWarning(String)
+   */
+  protected void printWarning(final String message, final Object... args) {
+    getGfsh().printAsWarning(String.format(message, args));
+  }
+
+  /**
+   * Displays the message inside GemFire shell at severe level.
+   *
+   * @param message the String containing the message to display inside Gfsh.
+   * @see org.apache.geode.management.internal.cli.shell.Gfsh#printAsSevere(String)
+   */
+  protected void printSevere(final String message, final Object... args) {
+    getGfsh().printAsSevere(String.format(message, args));
+  }
+
+  private <T> T get(URI url, Class<T> responseType) {
+    return send(url, HttpMethod.GET, null, null, responseType);
+  }
+
+  private <T> T post(URI url, MediaType mediaType, Object content, Class<T> responseType) {
+    return send(url, HttpMethod.POST, mediaType, content, responseType);
+  }
+
+
+  private <T> T send(URI url, HttpMethod method, MediaType mediaType, Object content,
+      Class<T> responseType) {
+    HttpHeaders headers = new HttpHeaders();
+    headers.add(HttpHeaders.USER_AGENT, USER_AGENT_HTTP_REQUEST_HEADER_VALUE);
+    headers.setAccept(acceptableMediaTypes);
+    if (mediaType != null) {
+      headers.setContentType(mediaType);
+    }
+
+    if (this.securityProperties != null) {
+      for (String key : securityProperties.stringPropertyNames()) {
+        headers.add(key, securityProperties.getProperty(key));
+      }
+    }
+
+    HttpEntity<Object> httpEntity = new HttpEntity<>(content, headers);
+
+    final ResponseEntity<T> response =
+        getRestTemplate().exchange(url, method, httpEntity, responseType);
+    return response.getBody();
+  }
+
+  Object extractResponse(ClientHttpResponse response) throws IOException {
+    MediaType mediaType = response.getHeaders().getContentType();
+    if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+      return org.apache.commons.io.IOUtils.toString(response.getBody(), "UTF-8");
+    } else {
+      Path tempFile = Files.createTempFile("fileDownload", "");
+      if (tempFile.toFile().exists()) {
+        FileUtils.deleteQuietly(tempFile.toFile());
+      }
+      Files.copy(response.getBody(), tempFile);
+      return tempFile;
+    }
+  }
+
+  void addHeaderValues(org.springframework.http.client.ClientHttpRequest request)
+      throws IOException {
+    // update the headers
+    request.getHeaders().add(HttpHeaders.USER_AGENT, USER_AGENT_HTTP_REQUEST_HEADER_VALUE);
+    request.getHeaders().setAccept(acceptableMediaTypes);
+
+    if (this.securityProperties != null) {
+      for (String key : securityProperties.stringPropertyNames()) {
+        request.getHeaders().add(key, securityProperties.getProperty(key));
+      }
+    }
+  }
+
+  /**
+   * Determines whether this HTTP-based OperationInvoker is successfully connected to the remote
+   * GemFire Manager's HTTP service in order to send commands for execution/processing.
+   *
+   * @return a boolean value indicating the connection state of the HTTP-based OperationInvoker.
+   */
+  @Override
+  public boolean isConnected() {
+    return (getRestTemplate() != null);
+  }
+
+  /**
+   * Determines whether this HTTP-based OperationInvoker is ready to send commands to the GemFire
+   * Manager for remote execution/processing.
+   *
+   * @return a boolean value indicating whether this HTTP-based OperationInvoker is ready for
+   *         command invocations.
+   * @see #isConnected()
+   */
+  @Override
+  public boolean isReady() {
+    return isConnected();
+  }
+
+
+  private URI createURI(String path) {
+    try {
+      return new URI(baseUrl + path);
+    } catch (URISyntaxException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Read the attribute identified by name from a remote resource identified by name. The intent of
+   * this method is to return the value of an attribute on an MBean located in the remote
+   * MBeanServer.
+   *
+   * @param resourceName name/url of the remote resource from which to fetch the attribute value.
+   * @param attributeName name of the attribute who's value will be fetched.
+   * @return the value of the named attribute for the named resource (typically an MBean).
+   * @throws MBeanAccessException if an MBean access error occurs.
+   * @throws RestApiCallForCommandNotFoundException if the REST API web service endpoint for
+   *         accessing an attribute on an MBean does not exists!
+   */
+  @Override
+  public Object getAttribute(final String resourceName, final String attributeName) {
+    final URI link = UriComponentsBuilder.fromHttpUrl(baseUrl).path("/mbean/attribute")
+        .queryParam("resourceName", resourceName).queryParam("attributeName", attributeName).build()
+        .encode().toUri();
+
+    try {
+      return IOUtils.deserializeObject(get(link, byte[].class));
+    } catch (IOException e) {
+      throw new MBeanAccessException(String.format(
+          "De-serializing the result of accessing attribute (%1$s) on MBean (%2$s) failed!",
+          resourceName, attributeName), e);
+    } catch (ClassNotFoundException e) {
+      throw new MBeanAccessException(String.format(
+          "The Class type of the result when accessing attribute (%1$s) on MBean (%2$s) was not found!",
+          resourceName, attributeName), e);
+    }
+  }
+
+  /**
+   * Gets the identifier of the GemFire cluster.
+   *
+   * @return an integer value indicating the identifier of the GemFire cluster.
+   */
+  @Override
+  public int getClusterId() {
+    return clusterId;
+  }
+
+  protected void initClusterId() {
+    if (isReady()) {
+      try {
+        clusterId = (Integer) getAttribute(ManagementConstants.OBJECTNAME__DISTRIBUTEDSYSTEM_MXBEAN,
+            "DistributedSystemId");
+        printDebug("Cluster ID (%1$s)", clusterId);
+      } catch (Exception ignore) {
+        printDebug("Failed to determine cluster ID: %1$s", ignore.getMessage());
+      }
+    }
+  }
+
+  /**
+   * Gets a proxy to the remote DistributedSystem MXBean to access attributes and invoke operations
+   * on the distributed system, or the GemFire cluster.
+   *
+   * @return a proxy instance of the GemFire Manager's DistributedSystem MXBean.
+   * @see #getMBeanProxy(javax.management.ObjectName, Class)
+   * @see org.apache.geode.management.DistributedSystemMXBean
+   * @see org.apache.geode.management.internal.MBeanJMXAdapter#getDistributedSystemName()
+   */
+  public DistributedSystemMXBean getDistributedSystemMXBean() {
+    return getMBeanProxy(MBeanJMXAdapter.getDistributedSystemName(), DistributedSystemMXBean.class);
+  }
+
+  /**
+   * Gets a proxy to an MXBean on a remote MBeanServer using HTTP for remoting.
+   *
+   * @param <T> the class type of the remote MXBean.
+   * @param objectName the JMX ObjectName uniquely identifying the remote MXBean.
+   * @param mbeanInterface the interface of the remote MXBean to proxy for attribute/operation
+   *        access.
+   * @return a proxy using HTTP remoting to access the specified, remote MXBean.
+   * @see javax.management.ObjectName
+   * @see org.apache.geode.management.internal.web.shell.support.HttpMBeanProxyFactory
+   */
+  public <T> T getMBeanProxy(final ObjectName objectName, final Class<T> mbeanInterface) {
+    return HttpMBeanProxyFactory.createMBeanProxy(this, objectName, mbeanInterface);
+  }
+
+  /**
+   * Invoke an operation identified by name on a remote resource identified by name with the given
+   * arguments. The intent of this method is to invoke an arbitrary operation on an MBean located in
+   * the remote MBeanServer.
+   *
+   * @param resourceName name/url (object name) of the remote resource (MBea) on which operation is
+   *        to be invoked.
+   * @param operationName name of the operation to be invoked.
+   * @param params an array of arguments for the parameters to be set when the operation is invoked.
+   * @param signatures an array containing the signature of the operation.
+   * @return result of the operation invocation.
+   * @throws MBeanAccessException if an MBean access error occurs.
+   * @throws RestApiCallForCommandNotFoundException if the REST API web service endpoint for
+   *         invoking an operation on an MBean does not exists!
+   */
+  @Override
+  public Object invoke(final String resourceName, final String operationName, final Object[] params,
+      final String[] signatures) {
+    final URI link = createURI("/mbean/operation");
+
+    MultiValueMap<String, Object> content = new LinkedMultiValueMap<String, Object>();
+
+    content.add("resourceName", resourceName);
+    content.add("operationName", operationName);
+    if (params != null) {
+      for (Object param : params) {
+        content.add("parameters", param);
+      }
+    }
+    if (signatures != null) {
+      for (String signature : signatures) {
+        content.add("signature", signature);
+      }
+    }
+
+    try {
+      return IOUtils
+          .deserializeObject(post(link, MediaType.MULTIPART_FORM_DATA, content, byte[].class));
+    } catch (IOException e) {
+      throw new MBeanAccessException(String.format(
+          "De-serializing the result from invoking operation (%1$s) on MBean (%2$s) failed!",
+          resourceName, operationName), e);
+    } catch (ClassNotFoundException e) {
+      throw new MBeanAccessException(String.format(
+          "The Class type of the result from invoking operation (%1$s) on MBean (%2$s) was not found!",
+          resourceName, operationName), e);
+    }
+
+  }
+
+  /**
+   * This method searches the MBean server, based on the OperationsInvoker's JMX-based or remoting
+   * capable MBean server connection, for MBeans matching a specific ObjectName or matching an
+   * ObjectName pattern along with satisfying criteria from the Query expression.
+   *
+   * @param objectName the ObjectName or pattern for which matching MBeans in the target MBean
+   *        server will be returned.
+   * @param queryExpression the JMX-based query expression used to filter matching MBeans.
+   * @return a set of ObjectName's matching MBeans in the MBean server matching the ObjectName and
+   *         Query expression criteria.
+   */
+  @Override
+  @SuppressWarnings("unchecked")
+  public Set<ObjectName> queryNames(final ObjectName objectName, final QueryExp queryExpression) {
+    final URI link = createURI("/mbean/query");
+
+    Object content = new QueryParameterSource(objectName, queryExpression);
+    try {
+      return (Set<ObjectName>) IOUtils.deserializeObject(post(link, null, content, byte[].class));
+    } catch (Exception e) {
+      throw new MBeanAccessException(String.format(
+          "An error occurred while querying for MBean names using ObjectName pattern (%1$s) and Query expression (%2$s)!",
+          objectName, queryExpression), e);
+    }
+  }
+
+  /**
+   * Stops communication with and closes all connections to the remote HTTP server (service).
+   */
+  @Override
+  public void stop() {
+    if (executorService != null) {
+      executorService.shutdown();
+    }
+
+    restTemplate = null;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("GemFire Manager HTTP service @ %1$s", baseUrl);
+  }
+
+
+  /**
+   * Processes the requested command. Sends the command to the GemFire Manager for remote processing
+   * (execution).
+   *
+   * @param command the command requested/entered by the user to be processed.
+   * @return the result of the command execution.
+   */
+  @Override
+  public Object processCommand(final CommandRequest command) {
+    URI link = command.getHttpRequestUrl(baseUrl);
+    if (command.hasFileData()) {
+      MultiValueMap<String, Object> content = new LinkedMultiValueMap<String, Object>();
+
+      Resource[] resources = ConvertUtils.convert(command.getFileData());
+      for (Resource resource : resources) {
+        content.add(RESOURCES_REQUEST_PARAMETER, resource);
+      }
+      return post(link, MediaType.MULTIPART_FORM_DATA, content, String.class);
+    }
 
+    return getRestTemplate().execute(link, HttpMethod.POST, this::addHeaderValues,
+        this::extractResponse);
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/RestHttpOperationInvoker.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/RestHttpOperationInvoker.java
deleted file mode 100644
index 13fd42c..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/RestHttpOperationInvoker.java
+++ /dev/null
@@ -1,464 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.management.internal.web.shell;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.geode.internal.lang.Filter;
-import org.apache.geode.internal.lang.Initable;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.util.CollectionUtils;
-import org.apache.geode.management.internal.cli.CommandRequest;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.management.internal.web.domain.LinkIndex;
-import org.apache.geode.management.internal.web.http.ClientHttpRequest;
-import org.apache.geode.management.internal.web.http.HttpHeader;
-import org.apache.geode.management.internal.web.util.ConvertUtils;
-import org.apache.logging.log4j.Logger;
-import org.springframework.http.HttpMethod;
-import org.springframework.http.HttpStatus;
-import org.springframework.http.client.ClientHttpResponse;
-import org.springframework.web.client.ResourceAccessException;
-import org.springframework.web.util.UriTemplate;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-/**
- * The RestHttpOperationInvoker class is an implementation of the OperationInvoker interface that
- * translates (adapts) GemFire shell command invocations into HTTP requests to a corresponding REST
- * API call hosted by the GemFire Manager's HTTP service using the Spring RestTemplate.
- * 
- * @see org.apache.geode.internal.lang.Initable
- * @see org.apache.geode.management.internal.cli.shell.Gfsh
- * @see org.apache.geode.management.internal.cli.shell.OperationInvoker
- * @see org.apache.geode.management.internal.web.shell.AbstractHttpOperationInvoker
- * @see org.apache.geode.management.internal.web.shell.HttpOperationInvoker
- * @see org.apache.geode.management.internal.web.shell.SimpleHttpOperationInvoker
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-public class RestHttpOperationInvoker extends AbstractHttpOperationInvoker implements Initable {
-
-  private static final Logger logger = LogService.getLogger();
-
-  protected static final String ENVIRONMENT_VARIABLE_REQUEST_PARAMETER_PREFIX = "vf.gf.env.";
-  protected static final String RESOURCES_REQUEST_PARAMETER = "resources";
-
-  // the HttpOperationInvoker used when this RestHttpOperationInvoker is unable to resolve the
-  // correct REST API
-  // web service endpoint (URI) for a command
-  private final HttpOperationInvoker httpOperationInvoker;
-
-  // the LinkIndex containing Links to all GemFire REST API web service endpoints
-  private final LinkIndex linkIndex;
-
-  /**
-   * Constructs an instance of the RestHttpOperationInvoker class initialized with the given link
-   * index containing links referencing all REST API web service endpoints. This constructor should
-   * only be used for testing purposes.
-   * 
-   * @param linkIndex the LinkIndex containing Links to all REST API web service endpoints in
-   *        GemFire's REST interface.
-   * @see org.apache.geode.management.internal.web.domain.LinkIndex
-   */
-  RestHttpOperationInvoker(final LinkIndex linkIndex) {
-    super(REST_API_URL);
-    assertNotNull(linkIndex,
-        "The Link Index resolving commands to REST API web service endpoints cannot be null!");
-    this.linkIndex = linkIndex;
-    this.httpOperationInvoker = new SimpleHttpOperationInvoker();
-  }
-
-  /**
-   * Constructs an instance of the RestHttpOperationInvoker class initialized with the given link
-   * index containing links referencing all REST API web service endpoints. In addition, a reference
-   * to the instance of GemFire shell (Gfsh) using this RestHttpOperationInvoker to send command
-   * invocations to the GemFire Manager's HTTP service via HTTP for processing is required in order
-   * to interact with the shell and provide feedback to the user.
-   * 
-   * @param linkIndex the LinkIndex containing Links to all REST API web service endpoints in
-   *        GemFire' REST interface.
-   * @param gfsh a reference to the instance of the GemFire shell using this OperationInvoker to
-   *        process commands.
-   * @see #RestHttpOperationInvoker(org.apache.geode.management.internal.web.domain.LinkIndex,
-   *      org.apache.geode.management.internal.cli.shell.Gfsh, Map)
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   * @see org.apache.geode.management.internal.web.domain.LinkIndex
-   */
-  public RestHttpOperationInvoker(final LinkIndex linkIndex, final Gfsh gfsh,
-      Map<String, String> securityProperties) {
-    this(linkIndex, gfsh, CliStrings.CONNECT__DEFAULT_BASE_URL, securityProperties);
-  }
-
-  /**
-   * Constructs an instance of the RestHttpOperationInvoker class initialized with the given link
-   * index containing links referencing all REST API web service endpoints. In addition, a reference
-   * to the instance of GemFire shell (Gfsh) using this RestHttpOperationInvoker to send command
-   * invocations to the GemFire Manager's HTTP service via HTTP for processing is required in order
-   * to interact with the shell and provide feedback to the user. Finally, a URL to the HTTP service
-   * running in the GemFire Manager is specified as the base location for all HTTP requests.
-   * 
-   * @param linkIndex the LinkIndex containing Links to all REST API web service endpoints in
-   *        GemFire's REST interface.
-   * @param gfsh a reference to the instance of the GemFire shell using this OperationInvoker to
-   *        process commands.
-   * @param baseUrl the String specifying the base URL to the GemFire Manager's HTTP service, REST
-   *        interface.
-   * @see org.apache.geode.management.internal.web.domain.LinkIndex
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   */
-  public RestHttpOperationInvoker(final LinkIndex linkIndex, final Gfsh gfsh, final String baseUrl,
-      Map<String, String> securityProperties) {
-    super(gfsh, baseUrl, securityProperties);
-    assertNotNull(linkIndex,
-        "The Link Index resolving commands to REST API web service endpoints cannot be null!");
-    this.linkIndex = linkIndex;
-    this.httpOperationInvoker = new SimpleHttpOperationInvoker(gfsh, baseUrl, securityProperties);
-
-  }
-
-  /**
-   * Initializes the RestHttpOperationInvokers scheduled and periodic monitoring task to assess the
-   * availibity of the targeted GemFire Manager's HTTP service.
-   * 
-   * @see org.apache.geode.internal.lang.Initable#init()
-   * @see org.springframework.http.client.ClientHttpRequest
-   */
-  @SuppressWarnings("null")
-  public void init() {
-    final Link pingLink = getLinkIndex().find(PING_LINK_RELATION);
-
-    if (pingLink != null) {
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "Scheduling periodic HTTP ping requests to monitor the availability of the GemFire Manager HTTP service @ ({})",
-            getBaseUrl());
-      }
-
-      getExecutorService().scheduleAtFixedRate(new Runnable() {
-        public void run() {
-          try {
-            org.springframework.http.client.ClientHttpRequest httpRequest = getRestTemplate()
-                .getRequestFactory().createRequest(pingLink.getHref(), HttpMethod.HEAD);
-
-            httpRequest.getHeaders().set(HttpHeader.USER_AGENT.getName(),
-                USER_AGENT_HTTP_REQUEST_HEADER_VALUE);
-            httpRequest.getHeaders().setAccept(getAcceptableMediaTypes());
-            httpRequest.getHeaders().setContentLength(0l);
-
-            if (securityProperties != null) {
-              Iterator<Entry<String, String>> it = securityProperties.entrySet().iterator();
-              while (it.hasNext()) {
-                Entry<String, String> entry = it.next();
-                httpRequest.getHeaders().add(entry.getKey(), entry.getValue());
-              }
-            }
-
-            ClientHttpResponse httpResponse = httpRequest.execute();
-
-            if (HttpStatus.NOT_FOUND.equals(httpResponse.getStatusCode())) {
-              throw new IOException(String.format(
-                  "The HTTP service at URL (%1$s) could not be found!", pingLink.getHref()));
-            } else if (!HttpStatus.OK.equals(httpResponse.getStatusCode())) {
-              printDebug(
-                  "Received unexpected HTTP status code (%1$d - %2$s) for HTTP request (%3$s).",
-                  httpResponse.getRawStatusCode(), httpResponse.getStatusText(),
-                  pingLink.getHref());
-            }
-          } catch (IOException e) {
-            printDebug("An error occurred while connecting to the Manager's HTTP service: %1$s: ",
-                e.getMessage());
-            getGfsh().notifyDisconnect(RestHttpOperationInvoker.this.toString());
-            stop();
-          }
-        }
-      }, DEFAULT_INITIAL_DELAY, DEFAULT_PERIOD, DEFAULT_TIME_UNIT);
-    } else {
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "The Link to the GemFire Manager web service endpoint @ ({}) to monitor availability was not found!",
-            getBaseUrl());
-      }
-    }
-
-    initClusterId();
-  }
-
-  /**
-   * Returns a reference to an implementation of HttpOperationInvoker used as the fallback by this
-   * RestHttpOperationInvoker for processing commands via HTTP requests.
-   * 
-   * @return an instance of HttpOperationInvoker used by this RestHttpOperationInvoker as a fallback
-   *         to process commands via HTTP requests.
-   * @see org.apache.geode.management.internal.web.shell.HttpOperationInvoker
-   */
-  protected HttpOperationInvoker getHttpOperationInvoker() {
-    return httpOperationInvoker;
-  }
-
-  /**
-   * Returns the LinkIndex resolving Gfsh commands to GemFire REST API web service endpoints. The
-   * corresponding web service endpoint is a URI/URL uniquely identifying the resource on which the
-   * command was invoked.
-   * 
-   * @return the LinkIndex containing Links for all GemFire REST API web service endpoints.
-   * @see org.apache.geode.management.internal.web.domain.LinkIndex
-   */
-  protected LinkIndex getLinkIndex() {
-    return linkIndex;
-  }
-
-  /**
-   * Creates an HTTP request from the specified command invocation encapsulated by the
-   * CommandRequest object. The CommandRequest identifies the resource targeted by the command
-   * invocation along with any parameters to be sent as part of the HTTP request.
-   * 
-   * @param command the CommandRequest object encapsulating details of the command invocation.
-   * @return a client HTTP request detailing the operation to be performed on the remote resource
-   *         targeted by the command invocation.
-   * @see AbstractHttpOperationInvoker#createHttpRequest(org.apache.geode.management.internal.web.domain.Link)
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   * @see org.apache.geode.management.internal.web.http.ClientHttpRequest
-   * @see org.apache.geode.management.internal.web.util.ConvertUtils#convert(byte[][])
-   */
-  protected ClientHttpRequest createHttpRequest(final CommandRequest command) {
-    ClientHttpRequest request = createHttpRequest(findLink(command));
-
-    Map<String, String> commandParameters = command.getParameters();
-
-    for (Map.Entry<String, String> entry : commandParameters.entrySet()) {
-      if (NullValueFilter.INSTANCE.accept(entry)) {
-        request.addParameterValues(entry.getKey(), entry.getValue());
-      }
-    }
-
-    Map<String, String> environmentVariables = command.getEnvironment();
-
-    for (Map.Entry<String, String> entry : environmentVariables.entrySet()) {
-      if (EnvironmentVariableFilter.INSTANCE.accept(entry)) {
-        request.addParameterValues(ENVIRONMENT_VARIABLE_REQUEST_PARAMETER_PREFIX + entry.getKey(),
-            entry.getValue());
-      }
-    }
-
-    if (command.getFileData() != null) {
-      request.addParameterValues(RESOURCES_REQUEST_PARAMETER,
-          (Object[]) ConvertUtils.convert(command.getFileData()));
-    }
-
-    return request;
-  }
-
-  /**
-   * Finds a Link from the Link Index containing the HTTP request URI to the web service endpoint
-   * for the relative operation on the resource.
-   * 
-   * @param relation a String describing the relative operation (state transition) on the resource.
-   * @return an instance of Link containing the HTTP request URI used to perform the intended
-   *         operation on the resource.
-   * @see #getLinkIndex()
-   * @see org.apache.geode.management.internal.web.domain.Link
-   * @see org.apache.geode.management.internal.web.domain.LinkIndex#find(String)
-   */
-  @Override
-  protected Link findLink(final String relation) {
-    return getLinkIndex().find(relation);
-  }
-
-  /**
-   * Finds a Link from the Link Index corresponding to the command invocation. The CommandRequest
-   * indicates the intended function on the target resource so the proper Link based on it's
-   * relation (the state transition of the corresponding function), along with it's method of
-   * operation and corresponding REST API web service endpoint (URI), can be identified.
-   * 
-   * @param command the CommandRequest object encapsulating the details of the command invocation.
-   * @return a Link referencing the correct REST API web service endpoint (URI) and method for the
-   *         command invocation.
-   * @see #getLinkIndex()
-   * @see #resolveLink(org.apache.geode.management.internal.cli.CommandRequest, java.util.List)
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   * @see org.apache.geode.management.internal.web.domain.Link
-   * @see org.apache.geode.management.internal.web.domain.LinkIndex
-   */
-  protected Link findLink(final CommandRequest command) {
-    List<Link> linksFound = new ArrayList<>(getLinkIndex().size());
-
-    for (Link link : getLinkIndex()) {
-      if (command.getInput().startsWith(link.getRelation())) {
-        linksFound.add(link);
-      }
-    }
-
-    if (linksFound.isEmpty()) {
-      throw new RestApiCallForCommandNotFoundException(
-          String.format("No REST API call for command (%1$s) was found!", command.getInput()));
-    }
-
-    return (linksFound.size() > 1 ? resolveLink(command, linksFound) : linksFound.get(0));
-  }
-
-  /**
-   * Resolves one Link from a Collection of Links based on the command invocation matching multiple
-   * relations from the Link Index.
-   * 
-   * @param command the CommandRequest object encapsulating details of the command invocation.
-   * @param links a Collection of Links for the command matching the relation.
-   * @return the resolved Link matching the command exactly as entered by the user.
-   * @see #findLink(org.apache.geode.management.internal.cli.CommandRequest)
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   * @see org.apache.geode.management.internal.web.domain.Link
-   * @see org.springframework.web.util.UriTemplate
-   */
-  // Find and use the Link with the greatest number of path variables that can be expanded!
-  protected Link resolveLink(final CommandRequest command, final List<Link> links) {
-    // NOTE, Gfsh's ParseResult contains a Map entry for all command options whether or not the user
-    // set the option
-    // with a value on the command-line, argh!
-    Map<String, String> commandParametersCopy =
-        CollectionUtils.removeKeys(new HashMap<>(command.getParameters()), NoValueFilter.INSTANCE);
-
-    Link resolvedLink = null;
-
-    int pathVariableCount = 0;
-
-    for (Link link : links) {
-      final List<String> pathVariables =
-          new UriTemplate(decode(link.getHref().toString())).getVariableNames();
-
-      // first, all path variables in the URL/URI template must be resolvable/expandable for this
-      // Link
-      // to even be considered...
-      if (commandParametersCopy.keySet().containsAll(pathVariables)) {
-        // then, either we have not found a Link for the command yet, or the number of
-        // resolvable/expandable
-        // path variables in this Link has to be greater than the number of resolvable/expandable
-        // path variables
-        // for the last Link
-        if (resolvedLink == null || (pathVariables.size() > pathVariableCount)) {
-          resolvedLink = link;
-          pathVariableCount = pathVariables.size();
-        }
-      }
-    }
-
-    if (resolvedLink == null) {
-      throw new RestApiCallForCommandNotFoundException(
-          String.format("No REST API call for command (%1$s) was found!", command.getInput()));
-    }
-
-    return resolvedLink;
-  }
-
-  /**
-   * Processes the requested command. Sends the command to the GemFire Manager for remote processing
-   * (execution).
-   * 
-   * @param command the command requested/entered by the user to be processed.
-   * @return either a json string of the CommandResult or a Path to a temp file if the response is a
-   *         InputStream
-   * @see #createHttpRequest(org.apache.geode.management.internal.cli.CommandRequest)
-   * @see #handleResourceAccessException(org.springframework.web.client.ResourceAccessException)
-   * @see #isConnected()
-   * @see #send(org.apache.geode.management.internal.web.http.ClientHttpRequest, Class,
-   *      java.util.Map)
-   * @see #simpleProcessCommand(org.apache.geode.management.internal.cli.CommandRequest,
-   *      RestApiCallForCommandNotFoundException)
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   * @see org.springframework.http.ResponseEntity
-   */
-  @Override
-  public Object processCommand(final CommandRequest command) {
-    assertState(isConnected(),
-        "Gfsh must be connected to the GemFire Manager in order to process commands remotely!");
-
-    Object result = null;
-    try {
-      if (command.isDownloadFile()) {
-        result = downloadResponseToTempFile(createHttpRequest(command), command.getParameters());
-      } else {
-        result = send(createHttpRequest(command), String.class, command.getParameters());
-      }
-      return result;
-    } catch (RestApiCallForCommandNotFoundException e) {
-      return simpleProcessCommand(command, e);
-    } catch (ResourceAccessException e) {
-      return handleResourceAccessException(e);
-    }
-  }
-
-  /**
-   * A method to process the command by sending an HTTP request to the simple URL/URI web service
-   * endpoint, where all details of the request and command invocation are encoded in the URL/URI.
-   * 
-   * @param command the CommandRequest encapsulating the details of the command invocation.
-   * @param e the RestApiCallForCommandNotFoundException indicating the standard REST API web
-   *        service endpoint could not be found.
-   * @return the result of the command execution.
-   * @see #getHttpOperationInvoker()
-   * @see org.apache.geode.management.internal.web.shell.HttpOperationInvoker#processCommand(org.apache.geode.management.internal.cli.CommandRequest)
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   */
-  protected String simpleProcessCommand(final CommandRequest command,
-      final RestApiCallForCommandNotFoundException e) {
-    if (getHttpOperationInvoker() != null) {
-      printWarning(
-          "WARNING - No REST API web service endpoint (URI) exists for command (%1$s); using the non-RESTful, simple URI.",
-          command.getName());
-
-      return String.valueOf(getHttpOperationInvoker().processCommand(command));
-    }
-
-    throw e;
-  }
-
-  protected static class EnvironmentVariableFilter extends NoValueFilter {
-
-    protected static final EnvironmentVariableFilter INSTANCE = new EnvironmentVariableFilter();
-
-    @Override
-    public boolean accept(final Map.Entry<String, String> entry) {
-      return (!entry.getKey().startsWith("SYS") && super.accept(entry));
-    }
-  }
-
-  protected static class NoValueFilter implements Filter<Map.Entry<String, String>> {
-
-    protected static final NoValueFilter INSTANCE = new NoValueFilter();
-
-    @Override
-    public boolean accept(final Map.Entry<String, String> entry) {
-      return StringUtils.isNotBlank(entry.getValue());
-    }
-  }
-
-  protected static class NullValueFilter implements Filter<Map.Entry<String, ?>> {
-
-    protected static final NullValueFilter INSTANCE = new NullValueFilter();
-
-    @Override
-    public boolean accept(final Map.Entry<String, ?> entry) {
-      return (entry.getValue() != null);
-    }
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/SimpleHttpOperationInvoker.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/SimpleHttpOperationInvoker.java
deleted file mode 100644
index d11d824..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/SimpleHttpOperationInvoker.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.management.internal.web.shell;
-
-import org.apache.geode.management.internal.cli.CommandRequest;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.management.internal.web.http.ClientHttpRequest;
-import org.apache.geode.management.internal.web.http.HttpMethod;
-import org.springframework.web.client.ResourceAccessException;
-import org.springframework.web.util.UriComponentsBuilder;
-
-import java.net.URI;
-import java.util.Map;
-
-/**
- * The SimpleHttpOperationInvoker class is an implementation of the OperationInvoker interface that
- * issues commands to the GemFire Manager via HTTP. The SimpleHttpOperationInvoker uses a single URL
- * web service endpoint to process commands and return responses.
- * 
- * @see java.net.URI
- * @see org.apache.geode.management.internal.cli.shell.Gfsh
- * @see org.apache.geode.management.internal.cli.shell.OperationInvoker
- * @see org.apache.geode.management.internal.web.shell.AbstractHttpOperationInvoker
- * @see org.apache.geode.management.internal.web.shell.HttpOperationInvoker
- * @see org.apache.geode.management.internal.web.shell.RestHttpOperationInvoker
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-public class SimpleHttpOperationInvoker extends AbstractHttpOperationInvoker {
-
-  protected static final String CMD_QUERY_PARAMETER = "cmd";
-  protected static final String LINK_RELATION = "simple";
-  protected static final String REST_API_MANAGEMENT_COMMANDS_URI = "/management/commands";
-
-  /**
-   * Default no-arg constructor to create an instance of the SimpleHttpOperationInvoker class for
-   * testing purposes.
-   */
-  SimpleHttpOperationInvoker() {
-    super(REST_API_URL);
-  }
-
-  /**
-   * Constructs an instance of the SimpleHttpOperationInvoker class initialized with a reference to
-   * the GemFire shell (Gfsh) using this HTTP-based OperationInvoker to send command invocations to
-   * the GemFire Manager's HTTP service using HTTP processing.
-   * 
-   * @param gfsh a reference to the instance of the GemFire shell using this OperationInvoker to
-   *        process commands.
-   * @see #SimpleHttpOperationInvoker(org.apache.geode.management.internal.cli.shell.Gfsh, String,
-   *      Map)
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   */
-  public SimpleHttpOperationInvoker(final Gfsh gfsh, Map<String, String> securityProperties) {
-    this(gfsh, REST_API_URL, securityProperties);
-  }
-
-  /**
-   * Constructs an instance of the SimpleHttpOperationInvoker class initialized with a reference to
-   * the GemFire shell (Gfsh) using this HTTP-based OperationInvoker to send command invocations to
-   * the GemFire Manager's HTTP service using HTTP for processing. In addition, the base URL to the
-   * HTTP service running in the GemFire Manager is specified as the base location for all HTTP
-   * requests.
-   * 
-   * @param gfsh a reference to the instance of the GemFire shell using this OperationInvoker to
-   *        process commands.
-   * @param baseUrl the base URL to the GemFire Manager's HTTP service.
-   * @see org.apache.geode.management.internal.cli.shell.Gfsh
-   */
-  public SimpleHttpOperationInvoker(final Gfsh gfsh, final String baseUrl,
-      Map<String, String> securityProperties) {
-    super(gfsh, baseUrl, securityProperties);
-  }
-
-  /**
-   * Creates an HTTP request from a command invocation encapsulated in a CommandRequest object. The
-   * CommandRequest identifies the resource targeted by the command invocation along with any
-   * parameters to be sent as part of the HTTP request.
-   * 
-   * @param command a CommandRequest object encapsulating the details of the command invocation.
-   * @return a client HTTP request detailing the operation to be performed on the remote resource
-   *         targeted by the command invocation.
-   * @see #createLink(org.apache.geode.management.internal.cli.CommandRequest)
-   * @see AbstractHttpOperationInvoker#createHttpRequest(org.apache.geode.management.internal.web.domain.Link)
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   * @see org.apache.geode.management.internal.web.http.ClientHttpRequest
-   */
-  protected ClientHttpRequest createHttpRequest(final CommandRequest command) {
-    return createHttpRequest(createLink(command));
-  }
-
-  /**
-   * Creates a Link based on the resource being targeted by the command invocation. The Link will
-   * contain the URI uniquely identifying the resource along with the HTTP GET operation specifying
-   * the method of processing.
-   * 
-   * @param command a CommandRequest object encapsulating the details of the command invocation.
-   * @return a Link identifying the resource and the operation on the resource.
-   * @see AbstractHttpOperationInvoker#createLink(String, java.net.URI,
-   *      org.apache.geode.management.internal.web.http.HttpMethod)
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   * @see org.apache.geode.management.internal.web.domain.Link
-   */
-  protected Link createLink(final CommandRequest command) {
-    return createLink(LINK_RELATION, getHttpRequestUrl(command), HttpMethod.POST);
-  }
-
-  /**
-   * Gets HTTP request URL (URI) locating the proper resource along with details for the request.
-   * 
-   * @param command a CommandRequest object encapsulating the details of the command invocation.
-   * @return a URI identifying the resource, it's location as well as details of the HTTP request.
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   * @see java.net.URI
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   * @see org.springframework.web.util.UriComponentsBuilder
-   */
-  protected URI getHttpRequestUrl(final CommandRequest command) {
-    return UriComponentsBuilder.fromHttpUrl(getBaseUrl()).path(REST_API_MANAGEMENT_COMMANDS_URI)
-        .queryParam(CMD_QUERY_PARAMETER, command.getInput()).build().encode().toUri();
-  }
-
-  /**
-   * Processes the requested command. Sends the command to the GemFire Manager for remote processing
-   * (execution).
-   * 
-   * @param command the command requested/entered by the user to be processed.
-   * @return the result of the command execution.
-   * @see #isConnected()
-   * @see #createHttpRequest(org.apache.geode.management.internal.cli.CommandRequest)
-   * @see AbstractHttpOperationInvoker#handleResourceAccessException(org.springframework.web.client.ResourceAccessException)
-   * @see AbstractHttpOperationInvoker#send(org.apache.geode.management.internal.web.http.ClientHttpRequest,
-   *      Class)
-   * @see org.apache.geode.management.internal.cli.CommandRequest
-   * @see org.springframework.http.ResponseEntity
-   */
-  @Override
-  public String processCommand(final CommandRequest command) {
-    assertState(isConnected(),
-        "Gfsh must be connected to the GemFire Manager in order to process commands remotely!");
-
-    try {
-      return send(createHttpRequest(command), String.class);
-    } catch (ResourceAccessException e) {
-      return handleResourceAccessException(e);
-    }
-  }
-
-}
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/util/ConvertUtils.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/util/ConvertUtils.java
index 0b6fbe3..ffb56f2 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/util/ConvertUtils.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/util/ConvertUtils.java
@@ -14,17 +14,18 @@
  */
 package org.apache.geode.management.internal.web.util;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.commons.lang.StringUtils;
-import org.apache.geode.internal.util.IOUtils;
-import org.apache.geode.management.internal.cli.CliUtil;
-import org.apache.geode.management.internal.web.io.MultipartFileResourceAdapter;
 import org.springframework.core.io.ByteArrayResource;
 import org.springframework.core.io.Resource;
 import org.springframework.web.multipart.MultipartFile;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
+import org.apache.geode.internal.util.IOUtils;
+import org.apache.geode.management.internal.cli.CliUtil;
+import org.apache.geode.management.internal.web.io.MultipartFileResourceAdapter;
 
 /**
  * The ConvertUtils class is a support class for performing conversions used by the GemFire web
@@ -52,27 +53,27 @@ public abstract class ConvertUtils {
    * @see org.apache.geode.management.internal.cli.CliUtil#bytesToNames(byte[][])
    */
   public static Resource[] convert(final byte[][] fileData) {
-    if (fileData != null) {
-      final String[] fileNames = CliUtil.bytesToNames(fileData);
-      final byte[][] fileContent = CliUtil.bytesToData(fileData);
-
-      final List<Resource> resources = new ArrayList<Resource>(fileNames.length);
-
-      for (int index = 0; index < fileNames.length; index++) {
-        final String filename = fileNames[index];
-        resources.add(new ByteArrayResource(fileContent[index],
-            String.format("Contents of JAR file (%1$s).", filename)) {
-          @Override
-          public String getFilename() {
-            return filename;
-          }
-        });
-      }
+    if (fileData == null) {
+      return new Resource[0];
+    }
+
+    final String[] fileNames = CliUtil.bytesToNames(fileData);
+    final byte[][] fileContent = CliUtil.bytesToData(fileData);
 
-      return resources.toArray(new Resource[resources.size()]);
+    final List<Resource> resources = new ArrayList<Resource>(fileNames.length);
+
+    for (int index = 0; index < fileNames.length; index++) {
+      final String filename = fileNames[index];
+      resources.add(new ByteArrayResource(fileContent[index],
+          String.format("Contents of JAR file (%1$s).", filename)) {
+        @Override
+        public String getFilename() {
+          return filename;
+        }
+      });
     }
 
-    return new Resource[0];
+    return resources.toArray(new Resource[resources.size()]);
   }
 
   /**
@@ -89,17 +90,15 @@ public abstract class ConvertUtils {
    * @see org.springframework.web.multipart.MultipartFile
    */
   public static byte[][] convert(final MultipartFile... files) throws IOException {
-    if (files != null) {
-      final List<Resource> resources = new ArrayList<Resource>(files.length);
-
-      for (final MultipartFile file : files) {
-        resources.add(new MultipartFileResourceAdapter(file));
-      }
-
-      return convert(resources.toArray(new Resource[resources.size()]));
+    if (files == null) {
+      return new byte[0][];
     }
 
-    return new byte[0][];
+    final List<Resource> resources = new ArrayList<Resource>(files.length);
+    for (final MultipartFile file : files) {
+      resources.add(new MultipartFileResourceAdapter(file));
+    }
+    return convert(resources.toArray(new Resource[resources.size()]));
   }
 
   /**
@@ -116,23 +115,22 @@ public abstract class ConvertUtils {
    * @see org.springframework.core.io.Resource
    */
   public static byte[][] convert(final Resource... resources) throws IOException {
-    if (resources != null) {
-      final List<byte[]> fileData = new ArrayList<byte[]>(resources.length * 2);
+    if (resources == null) {
+      return new byte[0][];
+    }
 
-      for (final Resource resource : resources) {
-        if (StringUtils.isBlank(resource.getFilename())) {
-          throw new IllegalArgumentException(String.format(
-              "The filename of Resource (%1$s) must be specified!", resource.getDescription()));
-        }
+    final List<byte[]> fileData = new ArrayList<byte[]>(resources.length * 2);
 
-        fileData.add(resource.getFilename().getBytes());
-        fileData.add(IOUtils.toByteArray(resource.getInputStream()));
+    for (final Resource resource : resources) {
+      if (StringUtils.isBlank(resource.getFilename())) {
+        throw new IllegalArgumentException(String.format(
+            "The filename of Resource (%1$s) must be specified!", resource.getDescription()));
       }
 
-      return fileData.toArray(new byte[fileData.size()][]);
+      fileData.add(resource.getFilename().getBytes());
+      fileData.add(IOUtils.toByteArray(resource.getInputStream()));
     }
 
-    return new byte[0][];
+    return fileData.toArray(new byte[fileData.size()][]);
   }
-
 }
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/dunit/QueryParamsAuthorizationDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/dunit/QueryParamsAuthorizationDUnitTest.java
index d3d1e7a..d010772 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/dunit/QueryParamsAuthorizationDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/dunit/QueryParamsAuthorizationDUnitTest.java
@@ -14,13 +14,16 @@
  */
 package org.apache.geode.cache.query.dunit;
 
-import org.junit.experimental.categories.Category;
-import org.junit.Test;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_ACCESSOR;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTHENTICATOR;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_PREFIX;
+import static org.junit.Assert.assertTrue;
 
-import static org.junit.Assert.*;
-
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
@@ -38,11 +41,9 @@ import org.apache.geode.security.templates.DummyAuthenticator;
 import org.apache.geode.security.templates.UserPasswordAuthInit;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.NetworkUtils;
-import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.dunit.VM;
-import org.junit.Ignore;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
 
 /**
  * Test for accessing query bind parameters from authorization callbacks
@@ -65,48 +66,42 @@ public class QueryParamsAuthorizationDUnitTest extends JUnit4CacheTestCase {
     final VM server1 = host.getVM(0);
     final VM client = host.getVM(1);
     // create servers and regions
-    final int port = (Integer) server1.invoke(new SerializableCallable("Create Server1") {
-      @Override
-      public Object call() throws Exception {
-        CacheFactory cf = new CacheFactory().set(MCAST_PORT, "0")
-            .set(SECURITY_CLIENT_ACCESSOR,
-                "org.apache.geode.cache.query.dunit.QueryAuthorization.create")
-            .set(SECURITY_CLIENT_AUTHENTICATOR, DummyAuthenticator.class.getName() + ".create");
-        Cache cache = getCache(cf);
-        cache.createRegionFactory(RegionShortcut.REPLICATE).create(regName);
-        CacheServer server = cache.addCacheServer();
-        int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
-        server.setPort(port);
-        server.start();
-        return port;
-      }
+    final int port = server1.invoke("Create Server1", () -> {
+      CacheFactory cf = new CacheFactory().set(MCAST_PORT, "0")
+          .set(SECURITY_CLIENT_ACCESSOR,
+              "org.apache.geode.cache.query.dunit.QueryAuthorization.create")
+          .set(SECURITY_CLIENT_AUTHENTICATOR, DummyAuthenticator.class.getName() + ".create");
+      Cache cache = getCache(cf);
+      cache.createRegionFactory(RegionShortcut.REPLICATE).create(regName);
+      CacheServer server = cache.addCacheServer();
+      int serverPort = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
+      server.setPort(serverPort);
+      server.start();
+      return serverPort;
     });
 
     // create client
-    client.invoke(new SerializableCallable("Create client") {
-      @Override
-      public Object call() throws Exception {
-        ClientCacheFactory ccf = new ClientCacheFactory()
-            .addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port)
-            .set(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName() + ".create")
-            .set(SECURITY_PREFIX + "username", "root").set(SECURITY_PREFIX + "password", "root");
-
-        ClientCache cache = getClientCache(ccf);
-        Region r1 =
-            cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create(regName);
+    client.invoke("Create client", () -> {
+      ClientCacheFactory ccf = new ClientCacheFactory()
+          .addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port)
+          .set(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName() + ".create")
+          .set(SECURITY_PREFIX + "username", "root").set(SECURITY_PREFIX + "password", "root");
 
-        for (int i = 0; i < 20; i++) {
-          r1.put("key-" + i, new Portfolio(i));
-        }
+      ClientCache cache = getClientCache(ccf);
+      Region r1 =
+          cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create(regName);
 
-        QueryService qs = cache.getQueryService();
-        Object[] params = new Object[] {"active", 0};
-        SelectResults sr = (SelectResults) qs
-            .newQuery("select * from " + r1.getFullPath() + " where status = $1 and ID > $2 ")
-            .execute(params);
-        assertTrue("Result size should be greater than 0 ", sr.size() > 0);
-        return null;
+      for (int i = 0; i < 20; i++) {
+        r1.put("key-" + i, new Portfolio(i));
       }
+
+      QueryService qs = cache.getQueryService();
+      Object[] params = new Object[] {"active", 0};
+      SelectResults sr = (SelectResults) qs
+          .newQuery("select * from " + r1.getFullPath() + " where status = $1 and ID > $2 ")
+          .execute(params);
+      assertTrue("Result size should be greater than 0 ", sr.size() > 0);
+      return null;
     });
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/lang/InitializerJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/lang/InitializerJUnitTest.java
deleted file mode 100644
index 66e8ff1..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/lang/InitializerJUnitTest.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.lang;
-
-import static org.junit.Assert.*;
-
-import org.jmock.Expectations;
-import org.jmock.Mockery;
-import org.jmock.lib.concurrent.Synchroniser;
-import org.jmock.lib.legacy.ClassImposteriser;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.test.junit.categories.UnitTest;
-
-/**
- * The InitializerJUnitTest class is a test suite of test cases testing the contract and
- * functionality of the Initializer utility class.
- * <p/>
- * 
- * @see org.apache.geode.internal.lang.Initializer
- * @see org.jmock.Mockery
- * @see org.junit.Assert
- * @see org.junit.Test
- * @since GemFire 8.0
- */
-@Category(UnitTest.class)
-public class InitializerJUnitTest {
-
-  private Mockery mockContext;
-
-  @Before
-  public void setUp() {
-    mockContext = new Mockery();
-    mockContext.setImposteriser(ClassImposteriser.INSTANCE);
-    mockContext.setThreadingPolicy(new Synchroniser());
-  }
-
-  @After
-  public void tearDown() {
-    mockContext.assertIsSatisfied();
-    mockContext = null;
-  }
-
-  @Test
-  public void testInitWithInitableObject() {
-    final Initable initableObject =
-        mockContext.mock(Initable.class, "testInitWithInitableObject.Initable");
-
-    mockContext.checking(new Expectations() {
-      {
-        oneOf(initableObject).init();
-      }
-    });
-
-    assertTrue(Initializer.init(initableObject));
-  }
-
-  @Test
-  public void testInitWithNonInitiableObject() {
-    assertFalse(Initializer.init(new Object()));
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/util/ArgumentRedactorJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/util/ArgumentRedactorJUnitTest.java
index b40d485..e0d9bc6 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/util/ArgumentRedactorJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/util/ArgumentRedactorJUnitTest.java
@@ -16,18 +16,20 @@
 package org.apache.geode.internal.util;
 
 import static org.apache.geode.internal.util.ArgumentRedactor.redact;
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import org.apache.geode.test.junit.categories.UnitTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+
 /**
  * ArgumentRedactor Tester.
  */
@@ -122,4 +124,14 @@ public class ArgumentRedactorJUnitTest {
     arg = "-Dgemfire.security-properties=\"c:\\Program Files (x86)\\My Folder\"";
     assertEquals(arg, (redact(arg)));
   }
+
+  @Test
+  public void redactScriptLine() throws Exception {
+    assertThat(ArgumentRedactor.redactScriptLine("connect --password=test --user=test"))
+        .isEqualTo("connect --password=******** --user=test");
+
+    assertThat(
+        ArgumentRedactor.redactScriptLine("connect --test-password=test --product-password=test1"))
+            .isEqualTo("connect --test-password=******** --product-password=********");
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/CommandRequestTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/CommandRequestTest.java
index 0d6e7cc..79d1b6d 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/CommandRequestTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/CommandRequestTest.java
@@ -18,14 +18,15 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import org.apache.geode.management.internal.cli.commands.RebalanceCommand;
-import org.apache.geode.test.junit.categories.UnitTest;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.HashMap;
-import java.util.Map;
+import org.apache.geode.management.internal.cli.commands.RebalanceCommand;
+import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class CommandRequestTest {
@@ -57,7 +58,7 @@ public class CommandRequestTest {
 
     Map<String, String> parameters = this.commandRequest.getParameters();
     assertThat(parameters).containsKey(key);
-    assertThat(parameters.get(key)).isEqualTo(value.substring(1, value.length() - 1));
+    assertThat(parameters.get(key)).isEqualTo(value);
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserParsingTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserParsingTest.java
index 0357105..7e8ea4a 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserParsingTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserParsingTest.java
@@ -202,7 +202,7 @@ public class GfshParserParsingTest {
     Map<String, String> params = parseParams(input, "startLocator");
 
     assertThat(params.get("name")).isEqualTo("loc1");
-    assertThat(params.get("J")).isEqualTo("'-Dgemfire.http-service-port= 8080'");
+    assertThat(params.get("J")).isEqualTo("-Dgemfire.http-service-port= 8080");
   }
 
   @Test
@@ -321,6 +321,6 @@ public class GfshParserParsingTest {
   public void testValueOfJsonWithSpaceAndOuterQuotes() throws Exception {
     String command = "put --key=\"('name' : 'id')\" --value=456 --region=/test";
     GfshParseResult result = parser.parse(command);
-    assertThat(result.getParamValue("key")).isEqualTo("\"('name' : 'id')\"");
+    assertThat(result.getParamValue("key")).isEqualTo("('name' : 'id')");
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/HeadlessGfsh.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/HeadlessGfsh.java
index f636c89..340712c 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/HeadlessGfsh.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/HeadlessGfsh.java
@@ -14,14 +14,6 @@
  */
 package org.apache.geode.management.internal.cli;
 
-import jline.console.ConsoleReader;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.cli.shell.GfshConfig;
-import org.apache.geode.management.internal.cli.shell.jline.GfshUnsupportedTerminal;
-import org.springframework.shell.core.ExitShellRequest;
-import org.springframework.shell.event.ShellStatus.Status;
-
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileDescriptor;
@@ -37,6 +29,15 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.logging.Level;
 
+import jline.console.ConsoleReader;
+import org.springframework.shell.core.ExitShellRequest;
+import org.springframework.shell.event.ShellStatus.Status;
+
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+import org.apache.geode.management.internal.cli.shell.GfshConfig;
+import org.apache.geode.management.internal.cli.shell.jline.GfshUnsupportedTerminal;
+
 
 /**
  * This is headless shell which can be used to submit random commands and get command-result It is
@@ -154,6 +155,10 @@ public class HeadlessGfsh implements ResultHandler {
     shell.terminate();
   }
 
+  public Gfsh getGfsh() {
+    return shell;
+  }
+
   public boolean isConnectedAndReady() {
     return shell.isConnectedAndReady();
   }
@@ -286,6 +291,7 @@ public class HeadlessGfsh implements ResultHandler {
      */
     @Override
     public void logSevere(String message, Throwable t) {
+      t.printStackTrace();
       super.logSevere(message, t);
       errorString = message;
       hasError = true;
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigCommandsDUnitTest.java
index a4f523c..20fa400 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ConfigCommandsDUnitTest.java
@@ -71,7 +71,7 @@ import org.apache.geode.internal.logging.LogWriterImpl;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.cli.Result.Status;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.remote.CommandProcessor;
+import org.apache.geode.management.internal.cli.remote.OnlineCommandProcessor;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.test.dunit.Host;
@@ -157,8 +157,8 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
       config.setArchiveFileSizeLimit(1000);
 
       String command = CliStrings.DESCRIBE_CONFIG + " --member=" + controllerName;
-      CommandProcessor cmdProcessor = new CommandProcessor();
-      cmdProcessor.createCommandStatement(command, Collections.EMPTY_MAP).process();
+      OnlineCommandProcessor cmdProcessor = new OnlineCommandProcessor();
+      cmdProcessor.executeCommand(command);
 
       CommandResult cmdResult = executeCommand(command);
 
@@ -327,9 +327,8 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     assertEquals(true, config.getStatisticSamplingEnabled());
     assertEquals(10, config.getLogDiskSpaceLimit());
 
-    CommandProcessor commandProcessor = new CommandProcessor();
-    Result result =
-        commandProcessor.createCommandStatement("alter runtime", Collections.EMPTY_MAP).process();
+    OnlineCommandProcessor onlineCommandProcessor = new OnlineCommandProcessor();
+    Result result = onlineCommandProcessor.executeCommand("alter runtime", Collections.EMPTY_MAP);
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
index f6c7cae..894a773 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GfshCommandJUnitTest.java
@@ -77,10 +77,20 @@ public class GfshCommandJUnitTest {
 
   private Mockery mockContext;
 
-  private static class DefaultGfshCommmand implements GfshCommand {
+  private static <T extends Function> T register(T function) {
+    if (FunctionService.isRegistered(function.getId())) {
+      function = (T) FunctionService.getFunction(function.getId());
+    } else {
+      FunctionService.registerFunction(function);
+    }
+
+    return function;
+  }
+
+  private static class DefaultGfshCommand implements GfshCommand {
   }
 
-  private DefaultGfshCommmand defaultGfshCommmand;
+  private DefaultGfshCommand defaultGfshCommand;
 
   @Before
   public void setup() {
@@ -88,7 +98,7 @@ public class GfshCommandJUnitTest {
     mockContext.setImposteriser(ClassImposteriser.INSTANCE);
     mockContext.setThreadingPolicy(new Synchroniser());
 
-    defaultGfshCommmand = new DefaultGfshCommmand();
+    defaultGfshCommand = new DefaultGfshCommand();
   }
 
   @After
@@ -119,13 +129,13 @@ public class GfshCommandJUnitTest {
 
   @Test
   public void testConvertDefaultValue() {
-    assertNull(defaultGfshCommmand.convertDefaultValue(null, StringUtils.EMPTY));
+    assertNull(defaultGfshCommand.convertDefaultValue(null, StringUtils.EMPTY));
     assertEquals(StringUtils.EMPTY,
-        defaultGfshCommmand.convertDefaultValue(StringUtils.EMPTY, "test"));
+        defaultGfshCommand.convertDefaultValue(StringUtils.EMPTY, "test"));
     assertEquals(StringUtils.SPACE,
-        defaultGfshCommmand.convertDefaultValue(StringUtils.SPACE, "testing"));
+        defaultGfshCommand.convertDefaultValue(StringUtils.SPACE, "testing"));
     assertEquals("tested",
-        defaultGfshCommmand.convertDefaultValue(CliMetaData.ANNOTATION_DEFAULT_VALUE, "tested"));
+        defaultGfshCommand.convertDefaultValue(CliMetaData.ANNOTATION_DEFAULT_VALUE, "tested"));
   }
 
   @Test
@@ -324,7 +334,7 @@ public class GfshCommandJUnitTest {
           createAbstractCommandsSupport(mockContext.mock(InternalCache.class));
 
       assertFalse(FunctionService.isRegistered("testRegister"));
-      assertSame(mockFunction, commands.register(mockFunction));
+      assertSame(mockFunction, register(mockFunction));
       assertTrue(FunctionService.isRegistered("testRegister"));
     } finally {
       FunctionService.unregisterFunction("testRegister");
@@ -355,7 +365,7 @@ public class GfshCommandJUnitTest {
       FunctionService.registerFunction(registeredFunction);
 
       assertTrue(FunctionService.isRegistered("testRegisteredAlready"));
-      assertSame(registeredFunction, commands.register(unregisteredFunction));
+      assertSame(registeredFunction, register(unregisteredFunction));
       assertTrue(FunctionService.isRegistered("testRegisteredAlready"));
     } finally {
       FunctionService.unregisterFunction("testRegisteredAlready");
@@ -364,22 +374,22 @@ public class GfshCommandJUnitTest {
 
   @Test
   public void testToStringOnBoolean() {
-    assertEquals("false", defaultGfshCommmand.toString(null, null, null));
-    assertEquals("true", defaultGfshCommmand.toString(true, null, null));
-    assertEquals("true", defaultGfshCommmand.toString(Boolean.TRUE, null, null));
-    assertEquals("false", defaultGfshCommmand.toString(false, null, null));
-    assertEquals("false", defaultGfshCommmand.toString(Boolean.FALSE, null, null));
-    assertEquals("false", defaultGfshCommmand.toString(true, "false", "true"));
-    assertEquals("true", defaultGfshCommmand.toString(false, "false", "true"));
-    assertEquals("Yes", defaultGfshCommmand.toString(true, "Yes", "No"));
-    assertEquals("Yes", defaultGfshCommmand.toString(false, "No", "Yes"));
-    assertEquals("TRUE", defaultGfshCommmand.toString(Boolean.TRUE, "TRUE", "FALSE"));
-    assertEquals("FALSE", defaultGfshCommmand.toString(Boolean.FALSE, "TRUE", "FALSE"));
+    assertEquals("false", defaultGfshCommand.toString(null, null, null));
+    assertEquals("true", defaultGfshCommand.toString(true, null, null));
+    assertEquals("true", defaultGfshCommand.toString(Boolean.TRUE, null, null));
+    assertEquals("false", defaultGfshCommand.toString(false, null, null));
+    assertEquals("false", defaultGfshCommand.toString(Boolean.FALSE, null, null));
+    assertEquals("false", defaultGfshCommand.toString(true, "false", "true"));
+    assertEquals("true", defaultGfshCommand.toString(false, "false", "true"));
+    assertEquals("Yes", defaultGfshCommand.toString(true, "Yes", "No"));
+    assertEquals("Yes", defaultGfshCommand.toString(false, "No", "Yes"));
+    assertEquals("TRUE", defaultGfshCommand.toString(Boolean.TRUE, "TRUE", "FALSE"));
+    assertEquals("FALSE", defaultGfshCommand.toString(Boolean.FALSE, "TRUE", "FALSE"));
   }
 
   @Test
   public void testToStringOnThrowable() {
-    assertEquals("test", defaultGfshCommmand.toString(new Throwable("test"), false));
+    assertEquals("test", defaultGfshCommand.toString(new Throwable("test"), false));
   }
 
   @Test
@@ -389,7 +399,7 @@ public class GfshCommandJUnitTest {
 
     t.printStackTrace(new PrintWriter(writer));
 
-    assertEquals(writer.toString(), defaultGfshCommmand.toString(t, true));
+    assertEquals(writer.toString(), defaultGfshCommand.toString(t, true));
   }
 
   private static class TestCommands implements GfshCommand {
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShellCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShellCommandsDUnitTest.java
index 255013e..007be0a 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShellCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShellCommandsDUnitTest.java
@@ -19,10 +19,12 @@ import static org.apache.geode.test.dunit.Assert.assertNotNull;
 import static org.apache.geode.test.dunit.Assert.assertTrue;
 import static org.apache.geode.test.dunit.Assert.fail;
 import static org.apache.geode.test.dunit.LogWriterUtils.getLogWriter;
+import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -46,11 +48,22 @@ import org.apache.geode.test.junit.categories.FlakyTest;
 @SuppressWarnings("serial")
 public class ShellCommandsDUnitTest extends CliCommandTestBase {
 
+  private static final long serialVersionUID = 1L;
+
+  private transient LocatorLauncher locatorLauncher;
+
   @Override
   public final void postSetUpCliCommandTestBase() throws Exception {
     getDefaultShell();
   }
 
+  @After
+  public void after() throws Exception {
+    if (locatorLauncher != null) {
+      locatorLauncher.stop();
+    }
+  }
+
   protected CommandResult connectToLocator(final int locatorPort) {
     return executeCommand(new CommandStringBuilder(CliStrings.CONNECT)
         .addOption(CliStrings.CONNECT__LOCATOR, "localhost[" + locatorPort + "]").toString());
@@ -79,32 +92,27 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
 
     assertTrue(workingDirectory.isDirectory());
 
-    final LocatorLauncher locatorLauncher = new LocatorLauncher.Builder().setBindAddress(null)
-        .setForce(true).setMemberName(pathname).setPort(locatorPort)
+    locatorLauncher = new LocatorLauncher.Builder().setBindAddress(null).setForce(true)
+        .setMemberName(pathname).setPort(locatorPort)
         .setWorkingDirectory(IOUtils.tryGetCanonicalPathElseGetAbsolutePath(workingDirectory))
         .build();
 
     assertNotNull(locatorLauncher);
     assertEquals(locatorPort, locatorLauncher.getPort().intValue());
 
-    try {
-      // fix for bug 46729
-      locatorLauncher.start();
+    // fix for bug 46729
+    locatorLauncher.start();
 
-      final LocatorState locatorState =
-          locatorLauncher.waitOnStatusResponse(60, 10, TimeUnit.SECONDS);
+    final LocatorState locatorState =
+        locatorLauncher.waitOnStatusResponse(60, 10, TimeUnit.SECONDS);
 
-      assertNotNull(locatorState);
-      assertEquals(Status.ONLINE, locatorState.getStatus());
+    assertThat(locatorState).isNotNull();
+    assertThat(locatorState.getStatus()).isEqualTo(Status.ONLINE);
 
-      final Result result = connectToLocator(locatorPort);
+    final Result result = connectToLocator(locatorPort);
 
-      assertNotNull(result);
-      assertEquals(Result.Status.OK, result.getStatus());
-    } finally {
-      assertEquals(Status.STOPPED, locatorLauncher.stop().getStatus());
-      assertEquals(Status.NOT_RESPONDING, locatorLauncher.status().getStatus());
-    }
+    assertThat(result).isNotNull();
+    assertThat(result.getStatus()).as("Result is not OK: " + result).isEqualTo(Result.Status.OK);
   }
 
   @Test
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsDUnitTest.java
index b804099..428bbe0 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ShowMetricsDUnitTest.java
@@ -22,7 +22,6 @@ import static org.apache.geode.test.dunit.Wait.waitForCriterion;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Collections;
 import java.util.Properties;
 
 import javax.management.ObjectName;
@@ -46,7 +45,7 @@ import org.apache.geode.management.RegionMXBean;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.cli.Result.Status;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.remote.CommandProcessor;
+import org.apache.geode.management.internal.cli.remote.OnlineCommandProcessor;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.SerializableCallable;
@@ -100,9 +99,8 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
       public Object call() throws Exception {
         WaitCriterion wc = createMBeanWaitCriterion(1, "", null, 0);
         waitForCriterion(wc, 5000, 500, true);
-        CommandProcessor commandProcessor = new CommandProcessor();
-        Result result = commandProcessor
-            .createCommandStatement("show metrics", Collections.EMPTY_MAP).process();
+        OnlineCommandProcessor OnlineCommandProcessor = new OnlineCommandProcessor();
+        Result result = OnlineCommandProcessor.executeCommand("show metrics");
         String resultStr = commandResultToString((CommandResult) result);
         getLogWriter().info(resultStr);
         assertEquals(resultStr, true, result.getStatus().equals(Status.OK));
@@ -150,10 +148,8 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
       public Object call() throws Exception {
         WaitCriterion wc = createMBeanWaitCriterion(2, regionName, null, 0);
         waitForCriterion(wc, 5000, 500, true);
-        CommandProcessor commandProcessor = new CommandProcessor();
-        Result result = commandProcessor
-            .createCommandStatement("show metrics --region=REGION1", Collections.EMPTY_MAP)
-            .process();
+        OnlineCommandProcessor OnlineCommandProcessor = new OnlineCommandProcessor();
+        Result result = OnlineCommandProcessor.executeCommand("show metrics --region=REGION1");
         String resultAsString = commandResultToString((CommandResult) result);
         assertEquals(resultAsString, true, result.getStatus().equals(Status.OK));
         return resultAsString;
@@ -257,9 +253,8 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
             + distributedMember.getId() + " --" + CliStrings.SHOW_METRICS__CACHESERVER__PORT + "="
             + cacheServerPort + " --" + CliStrings.SHOW_METRICS__FILE + "=" + exportFileName;
 
-        CommandProcessor commandProcessor = new CommandProcessor();
-        Result result =
-            commandProcessor.createCommandStatement(command, Collections.EMPTY_MAP).process();
+        OnlineCommandProcessor OnlineCommandProcessor = new OnlineCommandProcessor();
+        Result result = OnlineCommandProcessor.executeCommand(command);
         String resultAsString = commandResultToString((CommandResult) result);
         assertEquals(resultAsString, true, result.getStatus().equals(Status.OK));
         assertTrue(result.hasIncomingFiles());
@@ -300,11 +295,9 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
 
         WaitCriterion wc = createMBeanWaitCriterion(4, regionName, distributedMember, 0);
         waitForCriterion(wc, 5000, 500, true);
-        CommandProcessor commandProcessor = new CommandProcessor();
-        Result result = commandProcessor
-            .createCommandStatement("show metrics --region=" + regionName + " --member="
-                + distributedMember.getName() + " --file=" + exportFileName, Collections.EMPTY_MAP)
-            .process();
+        OnlineCommandProcessor OnlineCommandProcessor = new OnlineCommandProcessor();
+        Result result = OnlineCommandProcessor.executeCommand("show metrics --region=" + regionName
+            + " --member=" + distributedMember.getName() + " --file=" + exportFileName);
         String resultAsString = commandResultToString((CommandResult) result);
         assertEquals(resultAsString, true, result.getStatus().equals(Status.OK));
         assertTrue(result.hasIncomingFiles());
@@ -343,11 +336,10 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
 
         WaitCriterion wc = createMBeanWaitCriterion(4, regionName, distributedMember, 0);
         waitForCriterion(wc, 5000, 500, true);
-        CommandProcessor commandProcessor = new CommandProcessor();
-        Result result = commandProcessor.createCommandStatement(
+        OnlineCommandProcessor OnlineCommandProcessor = new OnlineCommandProcessor();
+        Result result = OnlineCommandProcessor.executeCommand(
             "show metrics --region=" + regionName + " --member=" + distributedMember.getName()
-                + " --file=" + exportFileName + " --categories=region,eviction",
-            Collections.EMPTY_MAP).process();
+                + " --file=" + exportFileName + " --categories=region,eviction");
         String resultAsString = commandResultToString((CommandResult) result);
         assertEquals(resultAsString, true, result.getStatus().equals(Status.OK));
         assertTrue(result.hasIncomingFiles());
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/CommandExecutorTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/CommandExecutorTest.java
new file mode 100644
index 0000000..3020bc2
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/CommandExecutorTest.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.remote;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.Mockito.mock;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.springframework.shell.event.ParseResult;
+
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.security.NotAuthorizedException;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class CommandExecutorTest {
+  ParseResult parseResult;
+  CommandExecutor executor;
+  Result result;
+
+  @Before
+  public void before() {
+    parseResult = mock(ParseResult.class);
+    result = mock(Result.class);
+    executor = new CommandExecutor();
+  }
+
+
+  @Test
+  public void executeWhenGivenDummyParseResult() throws Exception {
+    Object result = executor.execute(parseResult);
+    assertThat(result).isInstanceOf(CommandResult.class);
+    assertThat(result.toString()).contains("Error while processing command");
+  }
+
+  @Test
+  public void returnsResultAsExpected() throws Exception {
+    executor = new CommandExecutor() {
+      protected Object invokeCommand(ParseResult parseResult) {
+        return result;
+      }
+    };
+    Object thisResult = executor.execute(parseResult);
+    assertThat(thisResult).isSameAs(result);
+  }
+
+  @Test
+  public void testNullResult() throws Exception {
+    executor = new CommandExecutor() {
+      protected Object invokeCommand(ParseResult parseResult) {
+        return null;
+      }
+    };
+    Object thisResult = executor.execute(parseResult);
+    assertThat(thisResult.toString()).contains("Command returned null");
+  }
+
+  @Test
+  public void anyRuntimeExceptionGetsCaught() throws Exception {
+    executor = new CommandExecutor() {
+      protected Object invokeCommand(ParseResult parseResult) {
+        throw new RuntimeException("my message here");
+      }
+    };
+    Object thisResult = executor.execute(parseResult);
+    assertThat(thisResult.toString()).contains("my message here");
+  }
+
+  @Test
+  public void notAuthorizedExceptionGetsThrown() throws Exception {
+    executor = new CommandExecutor() {
+      protected Object invokeCommand(ParseResult parseResult) {
+        throw new NotAuthorizedException("Not Authorized");
+      }
+    };
+    assertThatThrownBy(() -> executor.execute(parseResult))
+        .isInstanceOf(NotAuthorizedException.class);
+
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessorTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessorTest.java
new file mode 100644
index 0000000..df00cf1
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessorTest.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.remote;
+
+import static org.assertj.core.api.Java6Assertions.assertThat;
+import static org.assertj.core.api.Java6Assertions.assertThatThrownBy;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.internal.security.SecurityService;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.security.NotAuthorizedException;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class OnlineCommandProcessorTest {
+
+  Properties properties;
+  SecurityService securityService;
+  CommandExecutor executor;
+  OnlineCommandProcessor onlineCommandProcessor;
+  Result result;
+
+  @Before
+  public void before() {
+    properties = new Properties();
+    securityService = mock(SecurityService.class);
+    executor = mock(CommandExecutor.class);
+    result = mock(Result.class);
+    when(executor.execute(any())).thenReturn(result);
+
+    onlineCommandProcessor = new OnlineCommandProcessor(properties, securityService, executor);
+  }
+
+
+  @Test
+  public void executeWithNullThrowsNPE() throws Exception {
+    assertThatThrownBy(() -> onlineCommandProcessor.executeCommand(null))
+        .isInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void executeWithEmpty() throws Exception {
+    assertThat(onlineCommandProcessor.executeCommand("")).isNull();
+  }
+
+  @Test
+  public void executeStripsComments() throws Exception {
+    Result commandResult = onlineCommandProcessor.executeCommand("/*comment*/");
+    assertThat(commandResult).isNull();
+  }
+
+  @Test
+  public void executeReturnsExecutorResult() throws Exception {
+    Result commandResult = onlineCommandProcessor.executeCommand("start locator");
+    assertThat(commandResult).isSameAs(result);
+  }
+
+  @Test
+  public void handlesNotAuthorizedException() throws Exception {
+    when(executor.execute(any())).thenThrow(new NotAuthorizedException("not authorized"));
+    assertThatThrownBy(() -> onlineCommandProcessor.executeCommand("start locator"))
+        .isInstanceOf(NotAuthorizedException.class);
+  }
+
+  @Test
+  public void handlesParsingError() throws Exception {
+    Result commandResult = onlineCommandProcessor.executeCommand("foo --bar");
+    assertThat(commandResult).isInstanceOf(CommandResult.class);
+    assertThat(commandResult.toString()).contains("Could not parse command string. foo --bar");
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/result/CommandResultTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/result/CommandResultTest.java
new file mode 100644
index 0000000..6f9a865
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/result/CommandResultTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal.cli.result;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class CommandResultTest {
+
+  @Test
+  public void emptyResultHasOneEmptyLine() {
+    CommandResult commandResult = new CommandResult(new InfoResultData());
+
+    assertThat(commandResult.nextLine()).isEqualTo("");
+    assertThat(commandResult.hasNextLine()).isFalse();
+  }
+
+  @Test
+  public void resultWithOneLineHasOneLine() {
+    CommandResult commandResult = new CommandResult(new InfoResultData("oneLine"));
+
+    assertThat(commandResult.nextLine()).isEqualTo("oneLine\n");
+    assertThat(commandResult.hasNextLine()).isFalse();
+  }
+
+  @Test
+  public void resultWithTwoLinesHasTwoLines() {
+    InfoResultData resultData = new InfoResultData();
+    resultData.addLine("lineOne");
+    resultData.addLine("lineTwo");
+    CommandResult commandResult = new CommandResult(resultData);
+
+    assertThat(commandResult.nextLine()).isEqualTo("lineOne\nlineTwo\n");
+    assertThat(commandResult.hasNextLine()).isFalse();
+  }
+
+  @Test
+  public void emptyResultDoesNotHaveFileToDownload() {
+    CommandResult commandResult = new CommandResult(new InfoResultData());
+
+    assertThat(commandResult.hasFileToDownload()).isFalse();
+  }
+
+  @Test
+  public void resultWithFileDoesHaveFileToDownload() {
+    Path fileToDownload = Paths.get(".").toAbsolutePath();
+    CommandResult commandResult = new CommandResult(fileToDownload);
+
+    assertThat(commandResult.hasFileToDownload()).isTrue();
+    assertThat(commandResult.nextLine()).isEqualTo(fileToDownload.toString() + '\n');
+    assertThat(commandResult.getFileToDownload()).isEqualTo(fileToDownload);
+  }
+
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/web/domain/LinkTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/result/DownloadFileResultTest.java
similarity index 50%
rename from geode-core/src/test/java/org/apache/geode/management/internal/web/domain/LinkTest.java
rename to geode-core/src/test/java/org/apache/geode/management/internal/cli/result/DownloadFileResultTest.java
index ff98b7e..e47118a 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/web/domain/LinkTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/result/DownloadFileResultTest.java
@@ -12,34 +12,37 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.management.internal.web.domain;
 
-import static org.mockito.Mockito.*;
+package org.apache.geode.management.internal.cli.result;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
-import org.apache.geode.test.junit.categories.UnitTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.net.URI;
-
-import org.apache.geode.management.internal.web.http.HttpMethod;
+import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
-public class LinkTest {
-
+public class DownloadFileResultTest {
   @Test
-  public void shouldBeMockable() throws Exception {
-    Link mockLink = mock(Link.class);
-    URI href = null;
-    HttpMethod method = HttpMethod.CONNECT;
-    String relation = "";
+  public void resultAlwaysHasFreshLineOnCreation() throws Exception {
+    DownloadFileResult result = new DownloadFileResult("/vol/log.zip");
+    assertThat(result.hasNextLine()).isTrue();
+    assertThat(result.nextLine()).isEqualTo("/vol/log.zip");
+  }
 
-    mockLink.setHref(href);
-    mockLink.setMethod(method);
-    mockLink.setRelation(relation);
+  @Test
+  public void resultOnlyHasOneLine() throws Exception {
+    DownloadFileResult result = new DownloadFileResult("/vol/log.zip");
+    result.nextLine();
+    assertThat(result.hasNextLine()).isFalse();
+  }
 
-    verify(mockLink, times(1)).setHref(href);
-    verify(mockLink, times(1)).setMethod(method);
-    verify(mockLink, times(1)).setRelation(relation);
+  @Test
+  public void consecutiveNextLine() throws Exception {
+    DownloadFileResult result = new DownloadFileResult("/vol/log.zip");
+    result.nextLine();
+    assertThatThrownBy(() -> result.nextLine()).isInstanceOf(IndexOutOfBoundsException.class);
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyTest.java
similarity index 98%
rename from geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyJUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyTest.java
index ece0c7e..bc5362b 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyTest.java
@@ -19,6 +19,11 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.springframework.shell.core.CommandMarker;
+
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.CommandRequest;
@@ -27,16 +32,12 @@ import org.apache.geode.management.internal.cli.GfshParseResult;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.test.junit.categories.UnitTest;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.springframework.shell.core.CommandMarker;
 
 /**
  * GfshExecutionStrategyTest - Includes tests to for GfshExecutionStrategyTest
  */
 @Category(UnitTest.class)
-public class GfshExecutionStrategyJUnitTest {
+public class GfshExecutionStrategyTest {
   private static final String COMMAND1_SUCESS = "Command1 Executed successfully";
   private static final String COMMAND2_SUCESS = "Command2 Executed successfully";
 
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/DeployCommandsSecurityTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/DeployCommandsSecurityTest.java
index 9dd4d99..05f9121 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/DeployCommandsSecurityTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/DeployCommandsSecurityTest.java
@@ -14,9 +14,8 @@
  */
 package org.apache.geode.management.internal.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 
@@ -44,8 +43,7 @@ public class DeployCommandsSecurityTest {
 
   @ClassRule
   public static ServerStarterRule server = new ServerStarterRule()
-      .withProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName()).withJMXManager()
-      .withAutoStart();
+      .withSecurityManager(SimpleTestSecurityManager.class).withJMXManager().withAutoStart();
 
   @ClassRule
   public static TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -94,7 +92,8 @@ public class DeployCommandsSecurityTest {
   @ConnectionConfiguration(user = "cluster,data", password = "cluster,data")
   public void testPowerAccess1() {
     String result = bean.processCommand(deployCommand);
-    assertTrue(result.contains("File does not contain valid JAR content: functions.jar"));
+
+    assertThat(result).contains("can not be executed only from server side");
   }
 
   @Test // only power user can deploy
@@ -102,9 +101,6 @@ public class DeployCommandsSecurityTest {
       password = "clusterManage,clusterWrite,dataManage,dataWrite")
   public void testPowerAccess2() {
     String result = bean.processCommand(deployCommand);
-    assertTrue(result.contains("File does not contain valid JAR content: functions.jar"));
+    assertThat(result).contains("can not be executed only from server side");
   }
-
-
-
 }
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserDUnitTest.java
index e3fe173..6d46090 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/MultiUserDUnitTest.java
@@ -14,7 +14,6 @@
  */
 package org.apache.geode.management.internal.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.NAME;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
@@ -36,9 +35,8 @@ import org.apache.geode.management.cli.Result.Status;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.result.ErrorResultData;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.security.TestSecurityManager;
+import org.apache.geode.security.SimpleTestSecurityManager;
 import org.apache.geode.test.dunit.AsyncInvocation;
-import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
@@ -60,10 +58,7 @@ public class MultiUserDUnitTest {
   @Before
   public void setup() throws Exception {
     Properties properties = new Properties();
-    properties.put(NAME, MultiUserDUnitTest.class.getSimpleName());
-    properties.put(SECURITY_MANAGER, TestSecurityManager.class.getName());
-    properties.put("security-json",
-        "org/apache/geode/management/internal/security/cacheServer.json");
+    properties.put(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName());
     server = lsRule.startServerAsJmxManager(0, properties);
   }
 
@@ -76,21 +71,21 @@ public class MultiUserDUnitTest {
 
     // set up vm_1 as a gfsh vm, data-reader will login and log out constantly in this vm until the
     // test is done.
-    VM vm1 = Host.getHost(0).getVM(1);
+    VM vm1 = lsRule.getVM(1);
     AsyncInvocation vm1Invoke = vm1.invokeAsync("run as data-reader", () -> {
       GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
-      gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManger, "data-reader", "1234567");
+      gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManger, "dataRead", "dataRead");
 
       Awaitility.waitAtMost(5, TimeUnit.MILLISECONDS);
       gfsh.close();
     });
 
-    VM vm2 = Host.getHost(0).getVM(2);
+    VM vm2 = lsRule.getVM(2);
     // set up vm_2 as a gfsh vm, and then connect as "stranger" and try to execute the commands and
     // assert errors comes back are NotAuthorized
     AsyncInvocation vm2Invoke = vm2.invokeAsync("run as guest", () -> {
       GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
-      gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManger, "stranger", "1234567");
+      gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManger, "guest", "guest");
 
       List<TestCommand> allCommands = TestCommand.getOnlineCommands();
       for (TestCommand command : allCommands) {
@@ -110,14 +105,13 @@ public class MultiUserDUnitTest {
         assertEquals("Not an expected result: " + result.toString(),
             ResultBuilder.ERRORCODE_UNAUTHORIZED,
             ((ErrorResultData) result.getResultData()).getErrorCode());
-
       }
       gfsh.close();
       LogService.getLogger().info("vm 2 done!");
     });
 
 
-    VM vm3 = Host.getHost(0).getVM(3);
+    VM vm3 = lsRule.getVM(3);
     IgnoredException
         .addIgnoredException("java.lang.IllegalArgumentException: Region doesnt exist: {0}", vm3);
     IgnoredException.addIgnoredException("java.lang.ClassNotFoundException: myApp.myListener", vm3);
@@ -126,7 +120,7 @@ public class MultiUserDUnitTest {
     // commands and assert we don't get a NotAuthorized Exception
     AsyncInvocation vm3Invoke = vm3.invokeAsync("run as superUser", () -> {
       GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
-      gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManger, "super-user", "1234567");
+      gfsh.secureConnectAndVerify(jmxPort, PortType.jmxManger, "data,cluster", "data,cluster");
 
       List<TestCommand> allCommands = TestCommand.getOnlineCommands();
       for (TestCommand command : allCommands) {
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ExportLogControllerTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ExportLogControllerTest.java
deleted file mode 100644
index bee7db2..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ExportLogControllerTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.management.internal.web.controllers;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.geode.management.internal.cli.CommandResponseBuilder;
-import org.apache.geode.management.internal.cli.result.CommandResult;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.test.junit.categories.UnitTest;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.springframework.core.io.InputStreamResource;
-import org.springframework.http.HttpHeaders;
-import org.springframework.http.MediaType;
-import org.springframework.http.ResponseEntity;
-
-@Category(UnitTest.class)
-public class ExportLogControllerTest {
-  private ExportLogController controller;
-
-  @Before
-  public void before() throws Exception {
-    controller = new ExportLogController();
-  }
-
-  @Test
-  public void testErrorResponse() throws Exception {
-    String message = "No Members Found";
-    CommandResult result = (CommandResult) ResultBuilder.createUserErrorResult(message);
-    String responseJson = CommandResponseBuilder.createCommandResponseJson("memberName", result);
-
-    ResponseEntity<InputStreamResource> resp = controller.getResponse(responseJson);
-    HttpHeaders headers = resp.getHeaders();
-    assertThat(headers.get(HttpHeaders.CONTENT_TYPE).get(0))
-        .isEqualTo(MediaType.APPLICATION_JSON_VALUE);
-
-    InputStreamResource body = resp.getBody();
-    assertThat(IOUtils.toString(body.getInputStream(), "utf-8")).isEqualTo(responseJson);
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ShellCommandsControllerProcessCommandTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ShellCommandsControllerProcessCommandTest.java
new file mode 100644
index 0000000..3c4d8ff
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ShellCommandsControllerProcessCommandTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.web.controllers;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.springframework.core.io.InputStreamResource;
+import org.springframework.http.HttpHeaders;
+import org.springframework.http.MediaType;
+import org.springframework.http.ResponseEntity;
+
+import org.apache.geode.management.internal.cli.CommandResponseBuilder;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.management.internal.cli.result.ErrorResultData;
+import org.apache.geode.management.internal.cli.result.InfoResultData;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class ShellCommandsControllerProcessCommandTest {
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private ShellCommandsController controller;
+  private CommandResult fakeResult;
+
+  @Before
+  public void setup() {
+
+    controller = new ShellCommandsController() {
+      @Override
+      protected String processCommand(String command, final Map<String, String> environment,
+          byte[][] fileData) {
+        return CommandResponseBuilder.createCommandResponseJson("someMember", fakeResult);
+      }
+    };
+  }
+
+  @Test
+  public void infoOkResult() throws IOException {
+    fakeResult = new CommandResult(new InfoResultData("Some info message"));
+
+    ResponseEntity<InputStreamResource> responseJsonStream = controller.command("xyz", null);
+    assertThatContentTypeEquals(responseJsonStream, MediaType.APPLICATION_JSON);
+
+    String responseJson = toString(responseJsonStream);
+    CommandResult result = ResultBuilder.fromJson(responseJson);
+
+    assertThat(result.nextLine()).isEqualTo(fakeResult.nextLine());
+  }
+
+  @Test
+  public void errorResult() throws IOException {
+    ErrorResultData errorResultData = new ErrorResultData("Some error message");
+    fakeResult = new CommandResult(errorResultData);
+
+    ResponseEntity<InputStreamResource> responseJsonStream = controller.command("xyz", null);
+    assertThatContentTypeEquals(responseJsonStream, MediaType.APPLICATION_JSON);
+
+    String responseJson = toString(responseJsonStream);
+    CommandResult result = ResultBuilder.fromJson(responseJson);
+
+    assertThat(result.nextLine()).isEqualTo(fakeResult.nextLine());
+  }
+
+  @Test
+  public void resultWithFile() throws IOException {
+    File tempFile = temporaryFolder.newFile();
+    FileUtils.writeStringToFile(tempFile, "some file contents", "UTF-8");
+
+    fakeResult = new CommandResult(tempFile.toPath());
+
+    ResponseEntity<InputStreamResource> responseFileStream = controller.command("xyz", null);
+
+    assertThatContentTypeEquals(responseFileStream, MediaType.APPLICATION_OCTET_STREAM);
+
+    String fileContents = toFileContents(responseFileStream);
+    assertThat(fileContents).isEqualTo("some file contents");
+  }
+
+  private String toFileContents(ResponseEntity<InputStreamResource> response) throws IOException {
+    return IOUtils.toString(response.getBody().getInputStream(), "UTF-8");
+  }
+
+  private String toString(ResponseEntity<InputStreamResource> response) throws IOException {
+    return IOUtils.toString(response.getBody().getInputStream(), "UTF-8");
+  }
+
+  private void assertThatContentTypeEquals(ResponseEntity<InputStreamResource> response,
+      MediaType mediaType) {
+    assertThat(response.getHeaders().get(HttpHeaders.CONTENT_TYPE))
+        .containsExactly(mediaType.toString());
+
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/WanCommandsControllerJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
deleted file mode 100755
index 59d6ff4..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.*;
-import static junitparams.JUnitParamsRunner.*;
-import static org.assertj.core.api.Assertions.*;
-
-import org.apache.geode.test.junit.categories.UnitTest;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
-
-/**
- * Unit tests for WanCommandsController.
- * 
- * Introduced for GEODE-213 "JMX -http manager treats "start gateway-sender" as "start
- * gateway-receiver"
- */
-@SuppressWarnings("unused")
-@Category(UnitTest.class)
-@RunWith(JUnitParamsRunner.class)
-public class WanCommandsControllerJUnitTest {
-
-  private TestableWanCommandsController wanCommandsController;
-
-  @Before
-  public void setUp() {
-    this.wanCommandsController = new TestableWanCommandsController();
-  }
-
-  @Test
-  public void shouldDefineStartGatewayReceiverCommandWithNulls() {
-    this.wanCommandsController.startGatewaySender(null, null, null);
-
-    assertThat(this.wanCommandsController.testableCommand)
-        .contains("--" + START_GATEWAYSENDER__ID + "=" + null);
-    assertThat(this.wanCommandsController.testableCommand).contains(START_GATEWAYSENDER);
-    assertThat(this.wanCommandsController.testableCommand).doesNotContain(GROUP);
-    assertThat(this.wanCommandsController.testableCommand).doesNotContain(MEMBER);
-  }
-
-  @Test
-  @Parameters(method = "getParametersWithGroupsAndMembers")
-  public void shouldDefineStartGatewayReceiverCommandWithGroupsAndMembers(
-      final String gatewaySenderId, final String[] groups, final String[] members,
-      final boolean containsGroups, final boolean containsMembers) {
-    this.wanCommandsController.startGatewaySender(gatewaySenderId, groups, members);
-
-    assertThat(this.wanCommandsController.testableCommand).contains(START_GATEWAYSENDER);
-    assertThat(this.wanCommandsController.testableCommand)
-        .contains("--" + START_GATEWAYSENDER__ID + "=" + gatewaySenderId);
-    assertThat(this.wanCommandsController.testableCommand.contains(GROUP))
-        .isEqualTo(containsGroups);
-    assertThat(this.wanCommandsController.testableCommand.contains(MEMBER))
-        .isEqualTo(containsMembers);
-    if (containsGroups) {
-      assertThat(this.wanCommandsController.testableCommand)
-          .contains("--" + GROUP + "=" + groups[0]);
-    }
-    if (containsMembers) {
-      assertThat(this.wanCommandsController.testableCommand)
-          .contains("--" + MEMBER + "=" + members[0]);
-    }
-  }
-
-  /**
-   * Tests null gatewaySenderId.
-   * 
-   * This test verifies that the class under test allows null value which would only be rejected at
-   * runtime by the overall framework and is tested within an integration test.
-   *
-   * see WanCommandGatewaySenderStartDUnitTest
-   */
-  @Test
-  public void shouldDefineStartGatewayReceiverCommandWithNullGatewaySenderId() {
-    this.wanCommandsController.startGatewaySender(null, null, null);
-
-    assertThat(this.wanCommandsController.testableCommand).contains(START_GATEWAYSENDER);
-    assertThat(this.wanCommandsController.testableCommand)
-        .contains("--" + START_GATEWAYSENDER__ID + "=" + null);
-  }
-
-  /**
-   * Tests empty gatewaySenderId.
-   * 
-   * This test verifies that the class under test allows empty value which would only be rejected at
-   * runtime by the overall framework and is tested within an integration test.
-   *
-   * see WanCommandGatewaySenderStartDUnitTest
-   */
-  @Test
-  public void shouldDefineStartGatewayReceiverCommandWithEmptyGatewaySenderId() {
-    this.wanCommandsController.startGatewaySender("", null, null);
-
-    assertThat(this.wanCommandsController.testableCommand).contains(START_GATEWAYSENDER);
-    assertThat(this.wanCommandsController.testableCommand)
-        .contains("--" + START_GATEWAYSENDER__ID + "=" + "");
-  }
-
-  private static Object[] getParametersWithGroupsAndMembers() {
-    return $(new Object[] {"sender1", new String[] {}, new String[] {}, false, false},
-        new Object[] {"sender2", new String[] {"group1"}, new String[] {}, true, false},
-        new Object[] {"sender3", new String[] {"group1", "group2"}, new String[] {}, true, false},
-        new Object[] {"sender4", new String[] {}, new String[] {"member1"}, false, true},
-        new Object[] {"sender5", new String[] {}, new String[] {"member1", "member2"}, false, true},
-        new Object[] {"sender6", new String[] {"group1"}, new String[] {"member1"}, true, true},
-        new Object[] {"sender7", new String[] {"group1", "group2"},
-            new String[] {"member1", "member2"}, true, true});
-  }
-
-  /**
-   * It would be ideal to refactor AbstractCommandsController such that defining of command strings
-   * and submitting them for execution are performed by two different classes. Then we could mock
-   * the executor class while testing just the command string definition class.
-   */
-  private static class TestableWanCommandsController extends WanCommandsController {
-
-    protected String testableCommand;
-
-    @Override
-    protected String processCommand(final String command) {
-      this.testableCommand = command;
-      return null; // do nothing
-    }
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/web/shell/HttpOperationInvokerTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/web/shell/HttpOperationInvokerTest.java
new file mode 100644
index 0000000..58f88ab
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/web/shell/HttpOperationInvokerTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.web.shell;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.nio.file.Path;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.springframework.http.HttpHeaders;
+import org.springframework.http.HttpStatus;
+import org.springframework.http.MediaType;
+import org.springframework.http.client.ClientHttpResponse;
+import org.springframework.mock.http.client.MockClientHttpResponse;
+
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class HttpOperationInvokerTest {
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private HttpOperationInvoker invoker;
+  private ClientHttpResponse response;
+
+  @Before
+  public void setup() {}
+
+  @Test
+  public void extractResponseOfJsonString() throws Exception {
+    String responseString = "my response";
+    invoker = new HttpOperationInvoker();
+    response =
+        new MockClientHttpResponse(IOUtils.toInputStream(responseString, "UTF-8"), HttpStatus.OK);
+
+    response.getHeaders().add(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON_VALUE);
+    Object result = invoker.extractResponse(response);
+    assertThat(result).isEqualTo(responseString);
+  }
+
+  @Test
+  public void extractResponseOfFileDownload() throws Exception {
+    File responseFile = temporaryFolder.newFile();
+    FileUtils.writeStringToFile(responseFile, "some file contents", "UTF-8");
+    invoker = new HttpOperationInvoker();
+    response = new MockClientHttpResponse(new FileInputStream(responseFile), HttpStatus.OK);
+    response.getHeaders().add(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_OCTET_STREAM_VALUE);
+    Object result = invoker.extractResponse(response);
+    Path fileResult = (Path) result;
+    assertThat(fileResult).hasSameContentAs(responseFile.toPath());
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
index f8b8590..da5e917 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
@@ -17,6 +17,7 @@ package org.apache.geode.test.dunit.rules;
 import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
 import static org.assertj.core.api.Assertions.assertThat;
 
+import java.io.File;
 import java.util.function.Supplier;
 
 import org.apache.commons.lang.StringUtils;
@@ -28,6 +29,7 @@ import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.HeadlessGfsh;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.junit.rules.DescribedExternalResource;
@@ -62,6 +64,8 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
   private boolean connected = false;
   private IgnoredException ignoredException;
   private TemporaryFolder temporaryFolder = new TemporaryFolder();
+  private File workingDir;
+
 
   public GfshShellConnectionRule() {
     try {
@@ -79,8 +83,8 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
 
   @Override
   protected void before(Description description) throws Throwable {
-    this.gfsh = new HeadlessGfsh(getClass().getName(), 30,
-        temporaryFolder.newFolder("gfsh_files").getAbsolutePath());
+    workingDir = temporaryFolder.newFolder("gfsh_files");
+    this.gfsh = new HeadlessGfsh(getClass().getName(), 30, workingDir.getAbsolutePath());
     ignoredException =
         addIgnoredException("java.rmi.NoSuchObjectException: no such object in table");
 
@@ -138,7 +142,7 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
       endpoint = "localhost[" + port + "]";
       connectCommand.addOption(CliStrings.CONNECT__LOCATOR, endpoint);
     } else if (type == PortType.http) {
-      endpoint = "http://localhost:" + port + "/gemfire/v1";
+      endpoint = "http://localhost:" + port + "/geode-mgmt/v1";
       connectCommand.addOption(CliStrings.CONNECT__USE_HTTP, Boolean.TRUE.toString());
       connectCommand.addOption(CliStrings.CONNECT__URL, endpoint);
     } else {
@@ -156,7 +160,7 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
     // when we connect too soon, we would get "Failed to retrieve RMIServer stub:
     // javax.naming.CommunicationException [Root exception is java.rmi.NoSuchObjectException: no
     // such object in table]" Exception.
-    // can not use Awaitility here because it starts another thead, but the Gfsh instance is in a
+    // can not use Awaitility here because it starts another thread, but the Gfsh instance is in a
     // threadLocal variable, See Gfsh.getExistingInstance()
     CommandResult result = null;
     for (int i = 0; i < 50; i++) {
@@ -194,10 +198,14 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
     gfsh = null;
   }
 
-  public HeadlessGfsh getGfsh() {
+  public HeadlessGfsh getHeadlessGfsh() {
     return gfsh;
   }
 
+  public Gfsh getGfsh() {
+    return gfsh.getGfsh();
+  }
+
   public CommandResult executeCommand(String command) throws Exception {
     gfsh.executeCommand(command);
     CommandResult result = (CommandResult) gfsh.getResult();
@@ -230,9 +238,7 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
-    assertThat(result.getStatus())
-        .describedAs("Failure in command: " + command + "\n Result " + result)
-        .isEqualTo(Result.Status.OK);
+    assertThat(result.getStatus()).isEqualTo(Result.Status.OK);
     return result;
   }
 
@@ -245,6 +251,10 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
     return connected;
   }
 
+  public File getWorkingDir() {
+    return workingDir;
+  }
+
   public enum PortType {
     locator, jmxManger, http
   }
diff --git a/geode-junit/build.gradle b/geode-junit/build.gradle
index ccfbb24..b0a53ce 100755
--- a/geode-junit/build.gradle
+++ b/geode-junit/build.gradle
@@ -23,6 +23,7 @@ dependencies {
   compile 'commons-io:commons-io:' + project.'commons-io.version'
   compile 'commons-lang:commons-lang:' + project.'commons-lang.version'
   compile 'com.google.guava:guava:' + project.'guava.version'
+  compile 'org.mockito:mockito-core:' + project.'mockito-core.version'
   compile 'org.awaitility:awaitility:' + project.'awaitility.version'
 
 
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/web/http/ClientHttpRequestTest.java b/geode-junit/src/main/java/org/apache/geode/test/junit/ResultCaptor.java
similarity index 53%
rename from geode-core/src/test/java/org/apache/geode/management/internal/web/http/ClientHttpRequestTest.java
rename to geode-junit/src/main/java/org/apache/geode/test/junit/ResultCaptor.java
index 4a5a0d4..6ce3a61 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/web/http/ClientHttpRequestTest.java
+++ b/geode-junit/src/main/java/org/apache/geode/test/junit/ResultCaptor.java
@@ -12,26 +12,23 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.management.internal.web.http;
 
-import static org.assertj.core.api.Assertions.*;
-import static org.mockito.Mockito.*;
+package org.apache.geode.test.junit;
 
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.test.junit.categories.UnitTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
-@Category(UnitTest.class)
-public class ClientHttpRequestTest {
+public class ResultCaptor<T> implements Answer {
+  private T result = null;
 
-  @Test
-  public void shouldBeMockable() throws Exception {
-    ClientHttpRequest mockClientHttpRequest = mock(ClientHttpRequest.class);
-    Link mockLink = mock(Link.class);
-
-    when(mockClientHttpRequest.getLink()).thenReturn(mockLink);
+  public T getResult() {
+    return result;
+  }
 
-    assertThat(mockClientHttpRequest.getLink()).isSameAs(mockLink);
+  @Override
+  public T answer(InvocationOnMock invocationOnMock) throws Throwable {
+    result = (T) invocationOnMock.callRealMethod();
+    return result;
   }
+
 }
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/CommandOverHttpTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/CommandOverHttpTest.java
new file mode 100644
index 0000000..3ba1c56
--- /dev/null
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/CommandOverHttpTest.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.geode.internal.ClassBuilder;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class CommandOverHttpTest {
+
+  @ClassRule
+  public static ServerStarterRule server =
+      new ServerStarterRule().withWorkingDir().withLogFile().withJMXManager().withAutoStart();
+
+  @Rule
+  public GfshShellConnectionRule gfshRule = new GfshShellConnectionRule();
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void before() throws Exception {
+    gfshRule.connectAndVerify(server.getHttpPort(), GfshShellConnectionRule.PortType.http);
+  }
+
+  @Test
+  public void testListClient() throws Exception {
+    CommandResult result = gfshRule.executeCommand("list clients");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.toString()).contains("No clients were retrieved for cache-servers");
+  }
+
+  @Test
+  public void testDescribeClient() throws Exception {
+    CommandResult result = gfshRule.executeCommand("describe client --clientID=xyz");
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.toString()).contains("Specified Client ID xyz not present");
+  }
+
+  @Test
+  public void exportLogs() throws Exception {
+    CommandResult result = gfshRule.executeAndVerifyCommand("export logs");
+    assertThat(result.getContent().toString()).contains("Logs exported to:");
+  }
+
+  @Test
+  public void deployJar() throws Exception {
+    String className = "DeployCommandFunction";
+    String jarName = "deployCommand.jar";
+    File jar = temporaryFolder.newFile(jarName);
+    new ClassBuilder().writeJarFromName(className, jar);
+    gfshRule.executeAndVerifyCommand("deploy --jar=" + jar);
+  }
+
+  @Test
+  public void exportConfig() throws Exception {
+    String dir = temporaryFolder.getRoot().getAbsolutePath();
+    gfshRule.executeAndVerifyCommand("export config --dir=" + dir);
+    String result = gfshRule.getGfshOutput();
+    assertThat(result).contains("Downloading Cache XML file: " + dir + "/server-cache.xml");
+    assertThat(result).contains("Downloading properties file: " + dir + "/server-gf.properties");
+  }
+}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
deleted file mode 100644
index d8ef3a8..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
+++ /dev/null
@@ -1,349 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.cli.commands;
-
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_CIPHERS;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_ENABLED;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_KEYSTORE;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_KEYSTORE_TYPE;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_PROTOCOLS;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_TRUSTSTORE;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT__KEY_STORE;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT__KEY_STORE_PASSWORD;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT__SSL_CIPHERS;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT__SSL_PROTOCOLS;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT__TRUST_STORE;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT__TRUST_STORE_PASSWORD;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT__URL;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT__USE_HTTP;
-import static org.apache.geode.management.internal.cli.i18n.CliStrings.CONNECT__USE_SSL;
-import static org.apache.geode.util.test.TestUtil.getResourcePath;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Properties;
-
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.HttpsURLConnection;
-import javax.net.ssl.SSLSession;
-
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.management.internal.cli.HeadlessGfsh;
-import org.apache.geode.management.internal.cli.result.CommandResult;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.FlakyTest;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-/**
- * @since GemFire 8.1
- */
-@Category({DistributedTest.class, SecurityTest.class, FlakyTest.class}) // GEODE-3530
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-@SuppressWarnings("serial")
-public class ConnectCommandWithHttpAndSSLDUnitTest extends CliCommandTestBase {
-
-  private static final ThreadLocal<Properties> sslInfoHolder = new ThreadLocal<>();
-
-  private File jks;
-
-  @Parameterized.Parameter
-  public String urlContext;
-
-  @Parameterized.Parameters
-  public static Collection<String> data() {
-    return Arrays.asList("/geode-mgmt", "/gemfire");
-  }
-
-  @Override
-  public final void postSetUpCliCommandTestBase() throws Exception {
-    this.jks = new File(getResourcePath(getClass(), "/ssl/trusted.keystore"));
-  }
-
-  @Override
-  protected final void preTearDownCliCommandTestBase() throws Exception {
-    destroyDefaultSetup();
-  }
-
-  @Override
-  public final void postTearDownCacheTestCase() throws Exception {
-    sslInfoHolder.set(null);
-  }
-
-  @Test
-  public void testMutualAuthentication() throws Exception {
-    Properties serverProps = new Properties();
-    serverProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE, "JKS");
-    serverProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS, "SSL");
-    serverProps.setProperty(HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION, "true");
-    serverProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE, jks.getCanonicalPath());
-    serverProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD, "password");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__KEY_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__KEY_STORE_PASSWORD, "password");
-    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "SSL");
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(serverProps);
-  }
-
-  @Test
-  public void testSimpleSSL() throws Exception {
-    Properties serverProps = new Properties();
-    serverProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE, "JKS");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(serverProps);
-  }
-
-  @Test
-  public void testSSLWithoutKeyStoreType() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithSSLProtocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS, "SSL");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithTLSProtocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS, "TLS");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithTLSv11Protocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS, "TLSv1.1");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithTLSv12Protocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS, "TLSv1.2");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testWithMultipleProtocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS, "SSL,TLSv1.2");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Ignore("TODO: disabled for unknown reason")
-  @Test
-  public void testSSLWithCipherSuite() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS, "TLSv1.2");
-
-    // Its bad to hard code here. But using SocketFactory.getDefaultCiphers() somehow is not working
-    // with the option
-    // "https.cipherSuites" which is required to restrict cipher suite with HttpsURLConnection
-    // Keeping the below code for further investigation on different Java versions ( 7 & 8) @TODO
-
-    /*
-     * SSLContext sslContext = SSLContext.getInstance("TLSv1.2");
-     * 
-     * sslContext.init(null, null, new java.security.SecureRandom()); String[] cipherSuites =
-     * sslContext.getSocketFactory().getSupportedCipherSuites();
-     */
-
-    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS, "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    clientProps.setProperty(CONNECT__SSL_CIPHERS, "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
-    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Ignore("TODO: disabled for unknown reason")
-  @Test
-  public void testSSLWithMultipleCipherSuite() throws Exception {
-    System.setProperty("javax.net.debug", "ssl,handshake,failure");
-
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS, "TLSv1.2");
-    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS,
-        "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256,TLS_EMPTY_RENEGOTIATION_INFO_SCSV");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Override
-  protected void connect(final String host, final int jmxPort, final int httpPort,
-      final HeadlessGfsh shell) {
-    assertNotNull(host);
-    assertNotNull(shell);
-
-    final CommandStringBuilder command = new CommandStringBuilder(CONNECT);
-    String endpoint;
-
-    // This is for testing purpose only. If we remove this piece of code we will
-    // get a java.security.cert.CertificateException
-    // as matching hostname can not be obtained in all test environment.
-    HttpsURLConnection.setDefaultHostnameVerifier(new HostnameVerifier() {
-      @Override
-      public boolean verify(String string, SSLSession ssls) {
-        return true;
-      }
-    });
-
-    endpoint = "https://" + host + ":" + httpPort + urlContext + "/v1";
-
-    command.addOption(CONNECT__USE_HTTP, Boolean.TRUE.toString());
-    command.addOption(CONNECT__URL, endpoint);
-    command.addOption(CONNECT__USE_SSL, Boolean.TRUE.toString());
-
-    if (sslInfoHolder.get().getProperty(CONNECT__KEY_STORE) != null) {
-      command.addOption(CONNECT__KEY_STORE, sslInfoHolder.get().getProperty(CONNECT__KEY_STORE));
-    }
-    if (sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD) != null) {
-      command.addOption(CONNECT__KEY_STORE_PASSWORD,
-          sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD));
-    }
-    if (sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE) != null) {
-      command.addOption(CONNECT__TRUST_STORE,
-          sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE));
-    }
-    if (sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD) != null) {
-      command.addOption(CONNECT__TRUST_STORE_PASSWORD,
-          sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD));
-    }
-    if (sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS) != null) {
-      command.addOption(CONNECT__SSL_PROTOCOLS,
-          sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS));
-    }
-    if (sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS) != null) {
-      command.addOption(CONNECT__SSL_CIPHERS,
-          sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS));
-    }
-
-    CommandResult result = executeCommand(shell, command.toString());
-
-    if (!shell.isConnectedAndReady()) {
-      fail("Connect command failed to connect to manager " + endpoint + " result="
-          + commandResultToString(result));
-    }
-
-    info("Successfully connected to managing node using HTTPS");
-    assertEquals(true, shell.isConnectedAndReady());
-  }
-
-}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSecurityTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSecurityTest.java
index 838924a..f268932 100644
--- a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSecurityTest.java
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandWithSecurityTest.java
@@ -37,19 +37,22 @@ public class ConnectCommandWithSecurityTest {
 
   @Test
   public void connectToLocator() throws Exception {
-    gfsh.secureConnectAndVerify(locator.getPort(), GfshShellConnectionRule.PortType.locator, "test",
-        "test");
+    gfsh.secureConnectAndVerify(locator.getPort(), GfshShellConnectionRule.PortType.locator,
+        "clusterRead", "clusterRead");
+    gfsh.executeAndVerifyCommand("list members");
   }
 
   @Test
   public void connectOverJmx() throws Exception {
     gfsh.secureConnectAndVerify(locator.getJmxPort(), GfshShellConnectionRule.PortType.jmxManger,
-        "test", "test");
+        "clusterRead", "clusterRead");
+    gfsh.executeAndVerifyCommand("list members");
   }
 
   @Test
   public void connectOverHttp() throws Exception {
     gfsh.secureConnectAndVerify(locator.getHttpPort(), GfshShellConnectionRule.PortType.http,
-        "test", "test");
+        "clusterRead", "clusterRead");
+    gfsh.executeAndVerifyCommand("list members");
   }
 }
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/QueryNamesOverHttpDUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/QueryNamesOverHttpDUnitTest.java
deleted file mode 100644
index dbd29b5..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/QueryNamesOverHttpDUnitTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.management.internal.cli.commands;
-
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.management.internal.web.domain.LinkIndex;
-import org.apache.geode.management.internal.web.http.HttpMethod;
-import org.apache.geode.management.internal.web.shell.RestHttpOperationInvoker;
-import org.apache.geode.test.dunit.rules.LocatorStarterRule;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.Set;
-import javax.management.ObjectName;
-import javax.management.Query;
-import javax.management.QueryExp;
-
-@Category(IntegrationTest.class)
-public class QueryNamesOverHttpDUnitTest {
-  @Rule
-  public LocatorStarterRule locatorRule = new LocatorStarterRule().withAutoStart();
-
-  @Test
-  public void testQueryNameOverHttp() throws Exception {
-
-    LinkIndex links = new LinkIndex();
-    links.add(new Link("mbean-query",
-        new URI("http://localhost:" + locatorRule.getHttpPort() + "/gemfire/v1/mbean/query"),
-        HttpMethod.POST));
-    RestHttpOperationInvoker invoker =
-        new RestHttpOperationInvoker(links, mock(Gfsh.class), new HashMap<>());
-
-    ObjectName objectName = ObjectName.getInstance("GemFire:type=Member,*");
-    QueryExp query = Query.eq(Query.attr("Name"), Query.value("mock"));
-
-    Set<ObjectName> names = invoker.queryNames(objectName, query);
-    assertTrue(names.isEmpty());
-  }
-}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/security/LogNoPasswordTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/security/LogNoPasswordTest.java
index 20359a9..204436b 100644
--- a/geode-web/src/test/java/org/apache/geode/management/internal/security/LogNoPasswordTest.java
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/security/LogNoPasswordTest.java
@@ -16,12 +16,14 @@
 package org.apache.geode.management.internal.security;
 
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
 import java.util.Properties;
 import java.util.Scanner;
 
+import org.apache.commons.lang.ArrayUtils;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -29,7 +31,8 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.security.AuthenticationFailedException;
 import org.apache.geode.security.SecurityManager;
 import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
-import org.apache.geode.test.dunit.rules.LocatorStarterRule;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
 @Category(IntegrationTest.class)
@@ -37,26 +40,34 @@ public class LogNoPasswordTest {
 
   private static String PASSWORD = "abcdefghijklmn";
   @Rule
-  public LocatorStarterRule locator = new LocatorStarterRule().withProperty(LOG_LEVEL, "DEBUG")
-      .withWorkingDir().withLogFile().withSecurityManager(MySecurityManager.class);
+  public LocatorServerStartupRule lsRule = new LocatorServerStartupRule().withLogFile();
 
   @Rule
   public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
 
   @Test
   public void testPasswordInLogs() throws Exception {
-    locator.startLocator();
+    Properties properties = new Properties();
+    properties.setProperty(LOG_LEVEL, "debug");
+    properties.setProperty(SECURITY_MANAGER, MySecurityManager.class.getName());
+    MemberVM locator = lsRule.startLocatorVM(0, properties);
     gfsh.secureConnectAndVerify(locator.getHttpPort(), GfshShellConnectionRule.PortType.http, "any",
         PASSWORD);
     gfsh.executeAndVerifyCommand("list members");
 
-    // scan all log files to find any occurrences of password
-    File[] logFiles = locator.getWorkingDir().listFiles(file -> file.toString().endsWith(".log"));
+    // scan all locator log files to find any occurrences of password
+    File[] serverLogFiles =
+        locator.getWorkingDir().listFiles(file -> file.toString().endsWith(".log"));
+    File[] gfshLogFiles = gfsh.getWorkingDir().listFiles(file -> file.toString().endsWith(".log"));
+
+    File[] logFiles = (File[]) ArrayUtils.addAll(serverLogFiles, gfshLogFiles);
+
     for (File logFile : logFiles) {
       Scanner scanner = new Scanner(logFile);
       while (scanner.hasNextLine()) {
         String line = scanner.nextLine();
-        assertThat(line).doesNotContain(PASSWORD);
+        assertThat(line).describedAs("File: %s, Line: %s", logFile.getAbsolutePath(), line)
+            .doesNotContain(PASSWORD);
       }
     }
   }
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/AbstractWebTestCase.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/AbstractWebTestCase.java
deleted file mode 100644
index 73be12a..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/web/AbstractWebTestCase.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.geode.management.internal.web.domain.Link;
-
-import java.io.UnsupportedEncodingException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URLDecoder;
-import java.net.URLEncoder;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * The AbstractWebDomainTests class is abstract base class containing functionality common to a test
- * suite classes in the org.apache.geode.management.internal.web.domain package.
- * <p/>
- * 
- * @see java.net.URI
- * @see java.net.URLDecoder
- * @see java.net.URLEncoder
- * @see org.apache.geode.management.internal.web.domain.Link
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-public abstract class AbstractWebTestCase {
-
-  protected <E> E[] createArray(final E... array) {
-    return array;
-  }
-
-  protected <K, V> Map<K, V> createMap(final K[] keys, final V[] values) {
-    assert keys != null : "The Keys for the Map cannot be null!";
-    assert values != null : "The Values for the Map cannot be null!";
-    assert keys.length == values.length;
-
-    final Map<K, V> map = new HashMap<>(keys.length);
-    int index = 0;
-
-    for (final K key : keys) {
-      map.put(key, values[index++]);
-    }
-
-    return map;
-  }
-
-  protected String decode(final String encodedValue) throws UnsupportedEncodingException {
-    return URLDecoder.decode(encodedValue, "UTF-8");
-  }
-
-  protected String encode(final String value) throws UnsupportedEncodingException {
-    return URLEncoder.encode(value, "UTF-8");
-  }
-
-  protected String toString(final Link... links) throws UnsupportedEncodingException {
-    final StringBuilder buffer = new StringBuilder("[");
-    int count = 0;
-
-    for (final Link link : links) {
-      buffer.append(count++ > 0 ? ", " : StringUtils.EMPTY).append(toString(link));
-
-    }
-
-    buffer.append("]");
-
-    return buffer.toString();
-  }
-
-  protected String toString(final Link link) throws UnsupportedEncodingException {
-    return link.toHttpRequestLine();
-  }
-
-  protected String toString(final URI uri) throws UnsupportedEncodingException {
-    return decode(uri.toString());
-  }
-
-  protected URI toUri(final String uriString)
-      throws UnsupportedEncodingException, URISyntaxException {
-    return new URI(encode(uriString));
-  }
-
-}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/controllers/ShellCommandsControllerJUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/controllers/ShellCommandsControllerJUnitTest.java
deleted file mode 100644
index 2f9214c..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/web/controllers/ShellCommandsControllerJUnitTest.java
+++ /dev/null
@@ -1,253 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.controllers;
-
-import static java.util.stream.Collectors.toSet;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import io.github.lukehutch.fastclasspathscanner.FastClasspathScanner;
-import org.apache.commons.lang.StringUtils;
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.internal.cli.util.ClasspathScanLoadHelper;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.management.internal.web.domain.LinkIndex;
-import org.apache.geode.management.internal.web.util.UriUtils;
-import org.apache.geode.test.junit.categories.UnitTest;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.springframework.mock.web.MockHttpServletRequest;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.context.request.RequestAttributes;
-import org.springframework.web.context.request.RequestContextHolder;
-import org.springframework.web.context.request.ServletRequestAttributes;
-
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import javax.servlet.http.HttpServletRequest;
-
-/**
- * The ShellCommandsControllerJUnitTest class is a test suite of test cases testing the contract and
- * functionality of the ShellCommandsController class, and specifically ensuring that all GemFire
- * Gfsh commands have a corresponding Management REST API call and web service endpoint in the
- * GemFire Management REST Interface.
- * <p/>
- * 
- * @see org.junit.Test
- * @see org.apache.geode.management.internal.web.controllers.ShellCommandsController
- * @since GemFire 8.0
- */
-@Category(UnitTest.class)
-public class ShellCommandsControllerJUnitTest {
-
-  private static ShellCommandsController controller;
-
-  @BeforeClass
-  public static void setupBeforeClass() {
-    controller = new ShellCommandsController();
-    MockHttpServletRequest request = new MockHttpServletRequest();
-    request.setContextPath("gemfire");
-    RequestContextHolder.setRequestAttributes(new ServletRequestAttributes(request));
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() {
-    controller = null;
-  }
-
-  private List<String> getCliCommands() {
-    try {
-      Set<Class<?>> commandClasses = ClasspathScanLoadHelper.scanPackageForClassesImplementing(
-          "org.apache.geode.management.internal.cli.commands", CommandMarker.class);
-
-      List<String> commands = new ArrayList<>(commandClasses.size());
-
-      for (Class<?> commandClass : commandClasses) {
-        for (Method method : commandClass.getMethods()) {
-          if (method.isAnnotationPresent(CliCommand.class)) {
-            if (!(method.isAnnotationPresent(CliMetaData.class)
-                && method.getAnnotation(CliMetaData.class).shellOnly())) {
-              CliCommand commandAnnotation = method.getAnnotation(CliCommand.class);
-              commands.addAll(Arrays.asList(commandAnnotation.value()));
-            }
-          }
-        }
-      }
-
-      return commands;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  private List<String> getControllerWebServiceEndpoints() {
-    RequestAttributes requestAttrs = RequestContextHolder.getRequestAttributes();
-    HttpServletRequest servletRequest = ((ServletRequestAttributes) requestAttrs).getRequest();
-    String scheme = servletRequest.getScheme();
-
-    try {
-      Set<Class<?>> controllerClasses = scanPackageForClassesExtending(
-          "org.apache.geode.management.internal.web.controllers", AbstractCommandsController.class);
-
-      List<String> controllerWebServiceEndpoints = new ArrayList<>(controllerClasses.size());
-
-      for (Class<?> controllerClass : controllerClasses) {
-        if (!AbstractCommandsController.class.equals(controllerClass)) {
-          for (Method method : controllerClass.getMethods()) {
-            if (method.isAnnotationPresent(RequestMapping.class)) {
-              RequestMapping requestMappingAnnotation = method.getAnnotation(RequestMapping.class);
-
-              String webServiceEndpoint =
-                  String.format("%1$s %2$s", requestMappingAnnotation.method()[0], UriUtils.decode(
-                      controller.toUri(requestMappingAnnotation.value()[0], scheme).toString()));
-
-              String[] requestParameters = requestMappingAnnotation.params();
-
-              if (requestParameters.length > 0) {
-                webServiceEndpoint += "?".concat(StringUtils.join(requestParameters, "&amp;"));
-              }
-
-              controllerWebServiceEndpoints.add(webServiceEndpoint);
-            }
-          }
-        }
-      }
-
-      return controllerWebServiceEndpoints;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Test
-  public void testUniqueIndex() {
-    LinkIndex linkIndex = controller.index("https");
-
-    List<String> conflicts = new ArrayList<>();
-    Map<String, String> uriRelationMapping = new HashMap<>(linkIndex.size());
-
-    for (Link link : linkIndex) {
-      if (uriRelationMapping.containsKey(link.toHttpRequestLine())) {
-        conflicts.add(String.format(
-            "REST API endpoint (%1$s) for (%2$s) conflicts with the REST API endpoint for (%3$s)",
-            link.toHttpRequestLine(), link.getRelation(),
-            uriRelationMapping.get(link.toHttpRequestLine())));
-      } else {
-        uriRelationMapping.put(link.toHttpRequestLine(), link.getRelation());
-      }
-    }
-
-    assertTrue(String.format("Conflicts: %1$s!", conflicts), conflicts.isEmpty());
-  }
-
-  @Test
-  public void testIndex() {
-    List<String> commands = getCliCommands();
-
-    assertNotNull(commands);
-    assertFalse(commands.isEmpty());
-
-    LinkIndex linkIndex = controller.index("https");
-
-    assertNotNull(linkIndex);
-    assertFalse(linkIndex.isEmpty());
-
-    List<String> linkCommands = new ArrayList<>(linkIndex.size());
-
-    for (Link link : linkIndex) {
-      linkCommands.add(link.getRelation());
-    }
-
-    assertEquals(linkIndex.size(), linkCommands.size());
-
-    List<String> missingLinkCommands = new ArrayList<>(commands);
-
-    missingLinkCommands.removeAll(linkCommands);
-
-    assertTrue(String.format(
-        "The GemFire Management REST API Link Index is missing Link(s) for the following command(s): %1$s",
-        missingLinkCommands), missingLinkCommands.isEmpty());
-  }
-
-  @Test
-  public void testCommandHasRestApiControllerWebServiceEndpoint() {
-    List<String> controllerWebServiceEndpoints = getControllerWebServiceEndpoints();
-
-    assertNotNull(controllerWebServiceEndpoints);
-    assertFalse(controllerWebServiceEndpoints.isEmpty());
-
-    LinkIndex linkIndex = controller.index("http");
-
-    assertNotNull(linkIndex);
-    assertFalse(linkIndex.isEmpty());
-
-    List<String> linkWebServiceEndpoints = new ArrayList<>(linkIndex.size());
-
-    for (Link link : linkIndex) {
-      linkWebServiceEndpoints.add(link.toHttpRequestLine());
-    }
-
-    assertEquals(linkIndex.size(), linkWebServiceEndpoints.size());
-
-    List<String> missingControllerWebServiceEndpoints = new ArrayList<>(linkWebServiceEndpoints);
-
-    missingControllerWebServiceEndpoints.removeAll(controllerWebServiceEndpoints);
-
-    assertTrue(
-        String.format(
-            "The Management REST API Web Service Controllers in (%1$s) are missing the following REST API Web Service Endpoint(s): %2$s!",
-            getClass().getPackage().getName(), missingControllerWebServiceEndpoints),
-        missingControllerWebServiceEndpoints.isEmpty());
-  }
-
-  @Test
-  public void testIndexUrisHaveCorrectScheme() {
-    String versionCmd = "version";
-    List<String> controllerWebServiceEndpoints = getControllerWebServiceEndpoints();
-
-    assertNotNull(controllerWebServiceEndpoints);
-    assertFalse(controllerWebServiceEndpoints.isEmpty());
-
-    String testScheme = "xyz";
-    LinkIndex linkIndex = controller.index(testScheme);
-
-    assertNotNull(linkIndex);
-    assertFalse(linkIndex.isEmpty());
-
-    assertTrue(String.format("Link does not have correct scheme %1$s", linkIndex.find(versionCmd)),
-        testScheme.equals(linkIndex.find(versionCmd).getHref().getScheme()));
-  }
-
-  public static Set<Class<?>> scanPackageForClassesExtending(String packageToScan,
-      Class<?> superclass) {
-    Set<Class<?>> classesImplementing = new HashSet<>();
-    new FastClasspathScanner(packageToScan).matchSubclassesOf(superclass, classesImplementing::add)
-        .scan();
-
-    return classesImplementing.stream().collect(toSet());
-  }
-}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/domain/LinkIndexJUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/domain/LinkIndexJUnitTest.java
deleted file mode 100644
index 7308b31..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/web/domain/LinkIndexJUnitTest.java
+++ /dev/null
@@ -1,275 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.domain;
-
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.geode.internal.util.CollectionUtils;
-import org.apache.geode.management.internal.web.AbstractWebTestCase;
-import org.apache.geode.management.internal.web.http.HttpMethod;
-import org.apache.geode.test.junit.categories.UnitTest;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * The LinkIndexJUnitTest class is a test suite of test cases testing the contract and functionality
- * of the LinkIndex class.
- * <p/>
- * 
- * @see java.net.URI
- * @see org.apache.geode.management.internal.web.AbstractWebTestCase
- * @see org.apache.geode.management.internal.web.domain.LinkIndex
- * @see org.junit.Assert
- * @see org.junit.Test
- * @since GemFire 8.0
- */
-@Category(UnitTest.class)
-public class LinkIndexJUnitTest extends AbstractWebTestCase {
-
-  @Test
-  public void testAdd() throws Exception {
-    final Link link =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{id}"));
-
-    final LinkIndex linkIndex = new LinkIndex();
-
-    assertTrue(linkIndex.isEmpty());
-    assertEquals(0, linkIndex.size());
-    assertEquals(linkIndex, linkIndex.add(link));
-    assertFalse(linkIndex.isEmpty());
-    assertEquals(1, linkIndex.size());
-    assertEquals(linkIndex, linkIndex.add(link)); // test duplicate addition
-    assertFalse(linkIndex.isEmpty());
-    assertEquals(1, linkIndex.size());
-  }
-
-  @Test(expected = AssertionError.class)
-  public void testAddNullLink() {
-    final LinkIndex linkIndex = new LinkIndex();
-
-    assertTrue(linkIndex.isEmpty());
-
-    try {
-      linkIndex.add(null);
-    } finally {
-      assertTrue(linkIndex.isEmpty());
-    }
-  }
-
-  @Test
-  public void testAddAll() throws Exception {
-    final Link create = new Link("create-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.POST);
-    final Link retrieve =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{id}"));
-    final Link update = new Link("update-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.PUT);
-    final Link delete = new Link("delete-resource",
-        toUri("http://host.domain.com:port/service/v1/resources/{id}"), HttpMethod.DELETE);
-
-    final LinkIndex linkIndex = new LinkIndex();
-
-    assertTrue(linkIndex.isEmpty());
-    assertEquals(linkIndex, linkIndex.addAll(create, retrieve, update, delete));
-    assertFalse(linkIndex.isEmpty());
-    assertEquals(4, linkIndex.size());
-  }
-
-  @Test(expected = AssertionError.class)
-  public void testAddAllWithNullLinks() {
-    final LinkIndex linkIndex = new LinkIndex();
-
-    assertTrue(linkIndex.isEmpty());
-
-    try {
-      linkIndex.addAll((Iterable<Link>) null);
-    } finally {
-      assertTrue(linkIndex.isEmpty());
-    }
-  }
-
-  @Test
-  public void testFind() throws Exception {
-    final Link list =
-        new Link("get-resources", toUri("http://host.domain.com:port/service/v1/resources"));
-    final Link create = new Link("create-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.POST);
-    final Link retrieve =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{id}"));
-    final Link update = new Link("update-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.PUT);
-    final Link delete = new Link("delete-resource",
-        toUri("http://host.domain.com:port/service/v1/resources/{id}"), HttpMethod.DELETE);
-
-    final LinkIndex linkIndex = new LinkIndex();
-
-    assertTrue(linkIndex.isEmpty());
-    assertEquals(linkIndex, linkIndex.addAll(list, create, retrieve, update, delete));
-    assertFalse(linkIndex.isEmpty());
-    assertEquals(5, linkIndex.size());
-    assertEquals(list, linkIndex.find("get-resources"));
-    assertEquals(retrieve, linkIndex.find("get-resource"));
-    assertEquals(update, linkIndex.find("UPDATE-RESOURCE"));
-    assertEquals(delete, linkIndex.find("Delete-Resource"));
-    assertNull(linkIndex.find("destroy-resource"));
-    assertNull(linkIndex.find("save-resource"));
-  }
-
-  @Test
-  public void testFindAll() throws Exception {
-    final Link list =
-        new Link("get-resources", toUri("http://host.domain.com:port/service/v1/resources"));
-    final Link create = new Link("create-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.POST);
-    final Link retrieveById =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{id}"));
-    final Link retrieveByName =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{name}"));
-    final Link update = new Link("update-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.PUT);
-    final Link delete = new Link("delete-resource",
-        toUri("http://host.domain.com:port/service/v1/resources/{id}"), HttpMethod.DELETE);
-
-    final LinkIndex linkIndex = new LinkIndex();
-
-    assertTrue(linkIndex.isEmpty());
-    assertEquals(linkIndex,
-        linkIndex.addAll(list, create, retrieveById, retrieveByName, update, delete));
-    assertFalse(linkIndex.isEmpty());
-    assertEquals(6, linkIndex.size());
-
-    final Link[] retrieveLinks = linkIndex.findAll("get-resource");
-
-    assertNotNull(retrieveLinks);
-    assertEquals(2, retrieveLinks.length);
-    assertTrue(
-        Arrays.asList(retrieveLinks).containsAll(Arrays.asList(retrieveById, retrieveByName)));
-
-    final Link[] saveLinks = linkIndex.findAll("save-resource");
-
-    assertNotNull(saveLinks);
-    assertEquals(0, saveLinks.length);
-  }
-
-  @Test
-  public void testIterator() throws Exception {
-    final Link list =
-        new Link("get-resources", toUri("http://host.domain.com:port/service/v1/resources"));
-    final Link create = new Link("create-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.POST);
-    final Link retrieveById =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{id}"));
-    final Link retrieveByName =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{name}"));
-    final Link update = new Link("update-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.PUT);
-    final Link delete = new Link("delete-resource",
-        toUri("http://host.domain.com:port/service/v1/resources/{id}"), HttpMethod.DELETE);
-
-    final LinkIndex linkIndex = new LinkIndex();
-
-    assertTrue(linkIndex.isEmpty());
-    assertEquals(linkIndex,
-        linkIndex.addAll(list, create, retrieveById, retrieveByName, update, delete));
-    assertFalse(linkIndex.isEmpty());
-    assertEquals(6, linkIndex.size());
-
-    final Collection<Link> expectedLinks =
-        Arrays.asList(list, create, retrieveById, retrieveByName, update, delete);
-
-    final Collection<Link> actualLinks = new ArrayList<Link>(linkIndex.size());
-
-    for (final Link link : linkIndex) {
-      actualLinks.add(link);
-    }
-
-    assertTrue(actualLinks.containsAll(expectedLinks));
-  }
-
-  @Test
-  public void testToList() throws Exception {
-    final Link list =
-        new Link("get-resources", toUri("http://host.domain.com:port/service/v1/resources"));
-    final Link create = new Link("create-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.POST);
-    final Link retrieveById =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{id}"));
-    final Link retrieveByName =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{name}"));
-    final Link update = new Link("update-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.PUT);
-    final Link delete = new Link("delete-resource",
-        toUri("http://host.domain.com:port/service/v1/resources/{id}"), HttpMethod.DELETE);
-
-    final LinkIndex linkIndex = new LinkIndex();
-
-    assertTrue(linkIndex.isEmpty());
-    assertEquals(linkIndex,
-        linkIndex.addAll(list, create, retrieveById, retrieveByName, update, delete));
-    assertFalse(linkIndex.isEmpty());
-    assertEquals(6, linkIndex.size());
-
-    final List<Link> expectedList =
-        CollectionUtils.asList(list, create, retrieveById, retrieveByName, update, delete);
-
-    Collections.sort(expectedList);
-
-    assertEquals(expectedList, linkIndex.toList());
-  }
-
-  @Test
-  public void testToMap() throws Exception {
-    final Link list =
-        new Link("get-resources", toUri("http://host.domain.com:port/service/v1/resources"));
-    final Link create = new Link("create-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.POST);
-    final Link retrieveById =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{id}"));
-    final Link retrieveByName =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{name}"));
-    final Link update = new Link("update-resource",
-        toUri("http://host.domain.com:port/service/v1/resources"), HttpMethod.PUT);
-    final Link delete = new Link("delete-resource",
-        toUri("http://host.domain.com:port/service/v1/resources/{id}"), HttpMethod.DELETE);
-
-    final LinkIndex linkIndex = new LinkIndex();
-
-    assertTrue(linkIndex.isEmpty());
-    assertEquals(linkIndex,
-        linkIndex.addAll(list, create, retrieveById, retrieveByName, update, delete));
-    assertFalse(linkIndex.isEmpty());
-    assertEquals(6, linkIndex.size());
-
-    final Map<String, List<Link>> expectedMap = new HashMap<String, List<Link>>(5);
-
-    expectedMap.put("get-resources", Arrays.asList(list));
-    expectedMap.put("create-resource", Arrays.asList(create));
-    expectedMap.put("get-resource", Arrays.asList(retrieveById, retrieveByName));
-    expectedMap.put("update-resource", Arrays.asList(update));
-    expectedMap.put("delete-resource", Arrays.asList(delete));
-
-    assertEquals(expectedMap, linkIndex.toMap());
-  }
-
-}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/domain/LinkJUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/domain/LinkJUnitTest.java
deleted file mode 100644
index 2f3f2dc..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/web/domain/LinkJUnitTest.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.domain;
-
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.geode.internal.util.CollectionUtils;
-import org.apache.geode.management.internal.web.AbstractWebTestCase;
-import org.apache.geode.management.internal.web.http.HttpMethod;
-import org.apache.geode.test.junit.categories.UnitTest;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * The LinkJUnitTest class is a test suite of test cases testing the contract and functionality of
- * the Link class.
- * <p/>
- * 
- * @see java.net.URI
- * @see org.apache.geode.management.internal.web.AbstractWebTestCase
- * @see org.apache.geode.management.internal.web.domain.Link
- * @see org.junit.Assert
- * @see org.junit.Test
- * @since GemFire 8.0
- */
-@Category(UnitTest.class)
-public class LinkJUnitTest extends AbstractWebTestCase {
-
-  @Test
-  public void testConstructDefaultLink() {
-    final Link link = new Link();
-
-    assertNotNull(link);
-    assertNull(link.getHref());
-    assertNull(link.getMethod());
-    assertNull(link.getRelation());
-  }
-
-  @Test
-  public void testConstructLinkWithRelationAndHref() throws Exception {
-    final Link link = new Link("get-resource", toUri("http://host:port/service/v1/resources/{id}"));
-
-    assertNotNull(link);
-    assertEquals("http://host:port/service/v1/resources/{id}", toString(link.getHref()));
-    assertEquals(HttpMethod.GET, link.getMethod());
-    assertEquals("get-resource", link.getRelation());
-  }
-
-  @Test
-  public void testConstructLinkWithRelationHrefAndMethod() throws Exception {
-    final Link link = new Link("create-resource", toUri("http://host:port/service/v1/resources"),
-        HttpMethod.POST);
-
-    assertNotNull(link);
-    assertEquals("http://host:port/service/v1/resources", toString(link.getHref()));
-    assertEquals(HttpMethod.POST, link.getMethod());
-    assertEquals("create-resource", link.getRelation());
-  }
-
-  @Test
-  public void testSetAndGetMethod() {
-    final Link link = new Link();
-
-    assertNotNull(link);
-    assertNull(link.getMethod());
-
-    link.setMethod(HttpMethod.POST);
-
-    assertEquals(HttpMethod.POST, link.getMethod());
-
-    link.setMethod(null);
-
-    assertEquals(HttpMethod.GET, link.getMethod());
-  }
-
-  @Test
-  public void testCompareTo() throws Exception {
-    final Link link0 = new Link("resources", toUri("http://host:port/service/v1/resources"));
-    final Link link1 =
-        new Link("resource", toUri("http://host:port/service/v1/resources"), HttpMethod.POST);
-    final Link link2 = new Link("resource", toUri("http://host:port/service/v1/resources/{id}"));
-    final Link link3 = new Link("resource", toUri("http://host:port/service/v1/resources/{name}"));
-    final Link link4 = new Link("resource", toUri("http://host:port/service/v1/resources/{id}"),
-        HttpMethod.DELETE);
-
-    final List<Link> expectedList =
-        new ArrayList<Link>(Arrays.asList(link1, link4, link2, link3, link0));
-
-    final List<Link> actualList = CollectionUtils.asList(link0, link1, link2, link3, link4);
-
-    Collections.sort(actualList);
-
-    System.out.println(toString(expectedList.toArray(new Link[expectedList.size()])));
-    System.out.println(toString(actualList.toArray(new Link[actualList.size()])));
-
-    assertEquals(expectedList, actualList);
-  }
-
-  @Test
-  public void testToHttpRequestLine() throws Exception {
-    final Link link =
-        new Link("get-resource", toUri("http://host.domain.com:port/service/v1/resources/{id}"));
-
-    assertNotNull(link);
-    assertEquals(HttpMethod.GET, link.getMethod());
-    assertEquals("http://host.domain.com:port/service/v1/resources/{id}", toString(link.getHref()));
-    assertEquals("GET ".concat("http://host.domain.com:port/service/v1/resources/{id}"),
-        link.toHttpRequestLine());
-  }
-
-}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/http/ClientHttpRequestJUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/http/ClientHttpRequestJUnitTest.java
deleted file mode 100644
index a95a58e..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/web/http/ClientHttpRequestJUnitTest.java
+++ /dev/null
@@ -1,556 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.http;
-
-import static org.junit.Assert.*;
-
-import java.net.URI;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.jmock.Mockery;
-import org.jmock.lib.concurrent.Synchroniser;
-import org.jmock.lib.legacy.ClassImposteriser;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.springframework.http.HttpEntity;
-import org.springframework.http.MediaType;
-import org.springframework.util.LinkedMultiValueMap;
-import org.springframework.util.MultiValueMap;
-
-import org.apache.geode.management.internal.web.AbstractWebTestCase;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.test.junit.categories.UnitTest;
-
-/**
- * The ClientHttpRequestJUnitTest class is a test suite of test cases testing the contract and
- * functionality of the ClientHttpRequest class.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.web.AbstractWebTestCase
- * @see org.apache.geode.management.internal.web.http.ClientHttpRequest
- * @see org.jmock.Mockery
- * @see org.junit.Assert
- * @see org.junit.Test
- * @since GemFire 8.0
- */
-@Category(UnitTest.class)
-public class ClientHttpRequestJUnitTest extends AbstractWebTestCase {
-
-  private Mockery mockContext;
-
-  @Before
-  public void setUp() {
-    mockContext = new Mockery();
-    mockContext.setImposteriser(ClassImposteriser.INSTANCE);
-    mockContext.setThreadingPolicy(new Synchroniser());
-  }
-
-  @After
-  public void tearDown() {
-    mockContext.assertIsSatisfied();
-    mockContext = null;
-  }
-
-  @Test
-  public void testCreateClientHttpRequest() throws Exception {
-    final Link expectedLink = new Link("test", toUri("http://host.domain.com:8080/app/service"));
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertNotNull(request);
-    assertEquals(expectedLink, request.getLink());
-  }
-
-  @Test(expected = AssertionError.class)
-  public void testCreateClientHttpRequestWithNullLink() {
-    new ClientHttpRequest(null);
-  }
-
-  @Test
-  public void testGetMethod() throws Exception {
-    Link link = new Link("get-resource", toUri("http://host.domain.com:8080/app/resources/{id}"));
-    ClientHttpRequest request = new ClientHttpRequest(link);
-
-    assertEquals(link, request.getLink());
-    assertEquals(org.springframework.http.HttpMethod.GET, request.getMethod());
-
-    link = new Link("delete-resource", toUri("http://host.domain.com:8080/app/resources/{id}"),
-        HttpMethod.DELETE);
-    request = new ClientHttpRequest(link);
-
-    assertEquals(link, request.getLink());
-    assertEquals(org.springframework.http.HttpMethod.DELETE, request.getMethod());
-
-    link = new Link("delete-resource", toUri("http://host.domain.com:8080/app/service"),
-        HttpMethod.HEAD);
-    request = new ClientHttpRequest(link);
-
-    assertEquals(link, request.getLink());
-    assertEquals(org.springframework.http.HttpMethod.HEAD, request.getMethod());
-
-    link = new Link("delete-resource", toUri("http://host.domain.com:8080/app/service"),
-        HttpMethod.OPTIONS);
-    request = new ClientHttpRequest(link);
-
-    assertEquals(link, request.getLink());
-    assertEquals(org.springframework.http.HttpMethod.OPTIONS, request.getMethod());
-
-    link = new Link("delete-resource", toUri("http://host.domain.com:8080/app/resources"),
-        HttpMethod.POST);
-    request = new ClientHttpRequest(link);
-
-    assertEquals(link, request.getLink());
-    assertEquals(org.springframework.http.HttpMethod.POST, request.getMethod());
-
-    link = new Link("delete-resource", toUri("http://host.domain.com:8080/app/resources"),
-        HttpMethod.PUT);
-    request = new ClientHttpRequest(link);
-
-    assertEquals(link, request.getLink());
-    assertEquals(org.springframework.http.HttpMethod.PUT, request.getMethod());
-
-    link = new Link("delete-resource", toUri("http://host.domain.com:8080/app"), HttpMethod.TRACE);
-    request = new ClientHttpRequest(link);
-
-    assertEquals(link, request.getLink());
-    assertEquals(org.springframework.http.HttpMethod.TRACE, request.getMethod());
-  }
-
-  @Test
-  public void testIsDelete() throws Exception {
-    final Link expectedLink = new Link("delete",
-        toUri("http://host.domain.com:8080/app/resources/{id}"), HttpMethod.DELETE);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertTrue(request.isDelete());
-    assertFalse(request.isGet());
-    assertFalse(request.isPost());
-    assertFalse(request.isPut());
-  }
-
-  @Test
-  public void testIsGet() throws Exception {
-    final Link expectedLink =
-        new Link("get", toUri("http://host.domain.com:8080/app/resources/{id}"), HttpMethod.GET);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertFalse(request.isDelete());
-    assertTrue(request.isGet());
-    assertFalse(request.isPost());
-    assertFalse(request.isPut());
-  }
-
-  @Test
-  public void testIsPost() throws Exception {
-    final Link expectedLink =
-        new Link("post", toUri("http://host.domain.com:8080/app/resources"), HttpMethod.POST);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertFalse(request.isDelete());
-    assertFalse(request.isGet());
-    assertTrue(request.isPost());
-    assertFalse(request.isPut());
-  }
-
-  @Test
-  public void testIsPut() throws Exception {
-    final Link expectedLink =
-        new Link("put", toUri("http://host.domain.com:8080/app/resources/{id}"), HttpMethod.PUT);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertFalse(request.isDelete());
-    assertFalse(request.isGet());
-    assertFalse(request.isPost());
-    assertTrue(request.isPut());
-  }
-
-  @Test
-  public void testGetPathVariables() throws Exception {
-    final Link expectedLink = new Link("test",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books/{author}/{title}"));
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertEquals(Arrays.asList("name", "author", "title"), request.getPathVariables());
-  }
-
-  @Test
-  public void testGetPathVariablesWithUriHavingNoPathVariables() throws Exception {
-    final Link expectedLink = new Link("test", toUri("http://host.domain.com:8080/app/service"));
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-
-    final List<String> actualPathVariables = request.getPathVariables();
-
-    assertNotNull(actualPathVariables);
-    assertTrue(actualPathVariables.isEmpty());
-  }
-
-  @Test
-  public void testGetURI() throws Exception {
-    final URI expectedURI = toUri("http://host.domain.com:8080/app/service");
-    final Link expectedLink = new Link("test", expectedURI);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertEquals(expectedURI, request.getURI());
-  }
-
-  @Test
-  public void testGetURLForGet() throws Exception {
-    final Link expectedLink = new Link("find",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books"), HttpMethod.GET);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    request.addParameterValues("author", "Rowling");
-    request.addParameterValues("category", "science-fiction");
-
-    assertEquals(expectedLink, request.getLink());
-    assertEquals(
-        "http://host.domain.com:8080/app/libraries/amazon/books?author=Rowling&category=science-fiction",
-        toString(request.getURL(Collections.singletonMap("name", "amazon"))));
-  }
-
-  @Test
-  public void testGetURLForGetEncoded() throws Exception {
-    final Link expectedLink = new Link("readValue4Key",
-        toUri("http://host.domain.com:8080/app/regions/{region}/keys/{key}"), HttpMethod.GET);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    final Map<String, Object> uriVariables = new HashMap<String, Object>(4);
-
-    uriVariables.put("region", "Customers/Accounts/Orders");
-    uriVariables.put("key", "123");
-    uriVariables.put("item", "456");
-
-    assertEquals(expectedLink, request.getLink());
-    assertEquals("http://host.domain.com:8080/app/regions/Customers%2FAccounts%2FOrders/keys/123",
-        toString(request.getURL(uriVariables)));
-  }
-
-  @Test
-  public void testGetURLForGetWithQueryParametersNoBody() throws Exception {
-    final Link expectedLink = new Link("find",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books/{author}"), HttpMethod.GET);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    request.addParameterValues("author", "Rowling");
-    request.addParameterValues("category", "science-fiction");
-    request.addParameterValues("name", "Boston");
-    request.addParameterValues("year", "2007");
-
-    final Map<String, Object> uriVariables = new HashMap<String, Object>(4);
-
-    uriVariables.put("author", "Rowling");
-    uriVariables.put("category", "mystery");
-    uriVariables.put("isbn", "0-123456789");
-    uriVariables.put("name", "Amazon");
-
-    assertEquals(expectedLink, request.getLink());
-    assertEquals(
-        "http://host.domain.com:8080/app/libraries/Amazon/books/Rowling?category=science-fiction&year=2007",
-        toString(request.getURL(uriVariables)));
-  }
-
-  @Test
-  public void testGetURLForDelete() throws Exception {
-    final Link expectedLink = new Link("delete-all",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books"), HttpMethod.DELETE);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    request.addParameterValues("category", "romance");
-
-    assertEquals(expectedLink, request.getLink());
-    assertEquals("http://host.domain.com:8080/app/libraries/congress/books?category=romance",
-        toString(request.getURL(Collections.singletonMap("name", "congress"))));
-  }
-
-  @Test
-  public void testGetURLForPost() throws Exception {
-    final Link expectedLink = new Link("post",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books"), HttpMethod.POST);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    request.addParameterValues("author", "Douglas Adams");
-    request.addParameterValues("title", "The Hitchhiker's Guide to the Galaxy");
-    request.addParameterValues("year", "1979");
-    request.addParameterValues("isbn", "0345453743");
-
-    assertEquals(expectedLink, request.getLink());
-    assertEquals("http://host.domain.com:8080/app/libraries/royal/books",
-        toString(request.getURL(Collections.singletonMap("name", "royal"))));
-  }
-
-  @Test
-  public void testGetURLForPut() throws Exception {
-    final Link expectedLink = new Link("put",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books/{isbn}"), HttpMethod.PUT);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    request.addParameterValues("year", "1983");
-
-    final Map<String, String> uriVariables = new HashMap<String, String>(2);
-
-    uriVariables.put("name", "royal");
-    uriVariables.put("isbn", "0345453743");
-
-    assertEquals(expectedLink, request.getLink());
-    assertEquals("http://host.domain.com:8080/app/libraries/royal/books/0345453743",
-        toString(request.getURL(uriVariables)));
-  }
-
-  @Test
-  public void testCreateRequestEntityForGet() throws Exception {
-    final Link expectedLink = new Link("find",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books"), HttpMethod.GET);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-
-    request.addHeaderValues(HttpHeader.CONTENT_TYPE.getName(), MediaType.TEXT_PLAIN_VALUE);
-    request.addParameterValues("author", "Rowling");
-    request.addParameterValues("category", "science-fiction");
-
-    final HttpEntity<?> requestEntity = request.createRequestEntity();
-
-    assertNotNull(requestEntity);
-    assertNotNull(requestEntity.getHeaders());
-    assertEquals(MediaType.TEXT_PLAIN, requestEntity.getHeaders().getContentType());
-    assertNull(requestEntity.getBody());
-  }
-
-  @Test
-  public void testCreateRequestEntityForDelete() throws Exception {
-    final Link expectedLink = new Link("delete-all",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books"), HttpMethod.DELETE);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-
-    request.addHeaderValues(HttpHeader.ACCEPT.getName(), MediaType.APPLICATION_JSON_VALUE);
-    request.addParameterValues("category", "romance");
-
-    final HttpEntity<?> requestEntity = request.createRequestEntity();
-
-    assertNotNull(requestEntity);
-    assertNotNull(requestEntity.getHeaders());
-    assertEquals(Collections.singletonList(MediaType.APPLICATION_JSON),
-        requestEntity.getHeaders().getAccept());
-    assertNull(requestEntity.getBody());
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testCreateRequestEntityForPost() throws Exception {
-    final Link expectedLink = new Link("post",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books"), HttpMethod.POST);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-
-    final MultiValueMap<String, Object> expectedRequestParameters =
-        new LinkedMultiValueMap<String, Object>(4);
-
-    expectedRequestParameters.add("author", "Douglas Adams");
-    expectedRequestParameters.add("title", "The Hitchhiker's Guide to the Galaxy");
-    expectedRequestParameters.add("year", "1979");
-    expectedRequestParameters.add("isbn", "0345453743");
-
-    request.addHeaderValues(HttpHeader.CONTENT_TYPE.getName(),
-        MediaType.APPLICATION_FORM_URLENCODED_VALUE);
-    request.addParameterValues("author", expectedRequestParameters.getFirst("author"));
-    request.addParameterValues("title", expectedRequestParameters.getFirst("title"));
-    request.addParameterValues("year", expectedRequestParameters.getFirst("year"));
-    request.addParameterValues("isbn", expectedRequestParameters.getFirst("isbn"));
-
-    final HttpEntity<MultiValueMap<String, Object>> requestEntity =
-        (HttpEntity<MultiValueMap<String, Object>>) request.createRequestEntity();
-
-    assertNotNull(requestEntity);
-    assertNotNull(requestEntity.getHeaders());
-    assertEquals(MediaType.APPLICATION_FORM_URLENCODED,
-        requestEntity.getHeaders().getContentType());
-    assertEquals(expectedRequestParameters, requestEntity.getBody());
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testCreateRequestEntityForPut() throws Exception {
-    final Link expectedLink = new Link("put",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books/{isbn}"), HttpMethod.PUT);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-
-    final MultiValueMap<String, Object> expectedRequestParameters =
-        new LinkedMultiValueMap<String, Object>(4);
-
-    expectedRequestParameters.add("year", "1979");
-
-    request.addHeaderValues(HttpHeader.ACCEPT.getName(), MediaType.TEXT_XML_VALUE);
-    request.addHeaderValues(HttpHeader.CONTENT_TYPE.getName(),
-        MediaType.APPLICATION_FORM_URLENCODED_VALUE);
-    request.addParameterValues("year", expectedRequestParameters.getFirst("year"));
-
-    final HttpEntity<MultiValueMap<String, Object>> requestEntity =
-        (HttpEntity<MultiValueMap<String, Object>>) request.createRequestEntity();
-
-    assertNotNull(requestEntity);
-    assertNotNull(requestEntity.getHeaders());
-    assertEquals(Collections.singletonList(MediaType.TEXT_XML),
-        requestEntity.getHeaders().getAccept());
-    assertEquals(MediaType.APPLICATION_FORM_URLENCODED,
-        requestEntity.getHeaders().getContentType());
-    assertEquals(expectedRequestParameters, requestEntity.getBody());
-  }
-
-  @Test
-  public void testCreateRequestEntityOnPost() throws Exception {
-    final Library mockLibrary =
-        mockContext.mock(Library.class, "testCreateRequestEntityOnPost.Library");
-    final Link expectedLink =
-        new Link("post", toUri("http://host.domain.com:8080/app/libraries"), HttpMethod.POST);
-
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertTrue(request.isPost());
-    assertNull(request.getContent());
-
-    request.setContent(mockLibrary);
-
-    assertSame(mockLibrary, request.getContent());
-
-    final HttpEntity<?> requestEntity = request.createRequestEntity();
-
-    assertNotNull(requestEntity);
-    assertTrue(requestEntity.getBody() instanceof Library);
-  }
-
-  @Test
-  public void testCreateRequestEntityOnPut() throws Exception {
-    final Book mockBook = mockContext.mock(Book.class, "testCreateRequestEntityOnPut.Book");
-    final Link expectedLink = new Link("put",
-        toUri("http://host.domain.com:8080/app/libraries/{name}/books/{id}"), HttpMethod.PUT);
-
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertTrue(request.isPut());
-    assertNull(request.getContent());
-
-    request.setContent(mockBook);
-    request.addParameterValues("isbn", "0-123456789");
-    request.addParameterValues("category", "science-fiction", "sci-fi", "fiction");
-
-    assertSame(mockBook, request.getContent());
-    assertEquals("0-123456789", request.getParameterValue("isbn"));
-    assertTrue(request.getParameterValues("category")
-        .containsAll(Arrays.asList("science-fiction", "sci-fi", "fiction")));
-
-    final HttpEntity<?> requestEntity = request.createRequestEntity();
-
-    assertNotNull(requestEntity);
-    assertTrue(requestEntity.getBody() instanceof MultiValueMap);
-    assertEquals(MediaType.APPLICATION_FORM_URLENCODED,
-        requestEntity.getHeaders().getContentType());
-  }
-
-  @Test
-  public void testSetAndGetHeaderValues() throws Exception {
-    final Link expectedLink =
-        new Link("put", toUri("http://host.domain.com:8080/app/libraries"), HttpMethod.PUT);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertTrue(request.getHeaders().isEmpty());
-
-    request.addHeaderValues(HttpHeader.CONTENT_TYPE.getName(), MediaType.APPLICATION_JSON_VALUE);
-    request.addHeaderValues(HttpHeader.ACCEPT.getName(), MediaType.APPLICATION_JSON_VALUE,
-        MediaType.APPLICATION_XML_VALUE, MediaType.TEXT_PLAIN_VALUE);
-
-    assertEquals(MediaType.APPLICATION_JSON_VALUE,
-        request.getHeaderValue(HttpHeader.CONTENT_TYPE.getName()));
-    assertEquals(1, request.getHeaderValues(HttpHeader.CONTENT_TYPE.getName()).size());
-    assertEquals(MediaType.APPLICATION_JSON_VALUE,
-        request.getHeaderValue(HttpHeader.ACCEPT.getName()));
-    assertEquals(3, request.getHeaderValues(HttpHeader.ACCEPT.getName()).size());
-    assertTrue(request.getHeaderValues(HttpHeader.ACCEPT.getName())
-        .containsAll(Arrays.asList(MediaType.APPLICATION_JSON_VALUE,
-            MediaType.APPLICATION_XML_VALUE, MediaType.TEXT_PLAIN_VALUE)));
-
-    request.setHeader(HttpHeader.ACCEPT.getName(), MediaType.APPLICATION_OCTET_STREAM_VALUE);
-
-    assertEquals(MediaType.APPLICATION_OCTET_STREAM_VALUE,
-        request.getHeaderValue(HttpHeader.ACCEPT.getName()));
-    assertEquals(1, request.getHeaderValues(HttpHeader.ACCEPT.getName()).size());
-    assertTrue(request.getHeaderValues(HttpHeader.ACCEPT.getName())
-        .containsAll(Arrays.asList(MediaType.APPLICATION_OCTET_STREAM_VALUE)));
-  }
-
-  @Test
-  public void testSetAndGetParameterValues() throws Exception {
-    final Link expectedLink =
-        new Link("put", toUri("http://host.domain.com:8080/app/libraries"), HttpMethod.PUT);
-    final ClientHttpRequest request = new ClientHttpRequest(expectedLink);
-
-    assertEquals(expectedLink, request.getLink());
-    assertTrue(request.getParameters().isEmpty());
-
-    request.addParameterValues("parameterOne", "value");
-    request.addParameterValues("parameterTwo", "test", "testing", "tested");
-
-    assertEquals("value", request.getParameterValue("parameterOne"));
-    assertEquals(1, request.getParameterValues("parameterOne").size());
-    assertEquals("test", request.getParameterValue("parameterTwo"));
-    assertEquals(3, request.getParameterValues("parameterTwo").size());
-    assertTrue(request.getParameterValues("parameterTwo")
-        .containsAll(Arrays.asList("test", "testing", "tested")));
-
-    request.setParameter("parameterTwo", "development");
-
-    assertEquals("development", request.getParameterValue("parameterTwo"));
-    assertEquals(1, request.getParameterValues("parameterTwo").size());
-    assertTrue(
-        request.getParameterValues("parameterTwo").containsAll(Arrays.asList("development")));
-  }
-
-  @SuppressWarnings("unused")
-  private static interface Library {
-    public String getName();
-  }
-
-  @SuppressWarnings("unused")
-  private static interface Book {
-    public String getAuthor();
-
-    public String getIsbn();
-
-    public String getTitle();
-
-    public Integer getYear();
-  }
-
-}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/HttpOperationInvokerMBeanOperationTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/HttpOperationInvokerMBeanOperationTest.java
new file mode 100644
index 0000000..7f2842f
--- /dev/null
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/HttpOperationInvokerMBeanOperationTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.web.shell;
+
+import static org.apache.geode.distributed.ConfigurationProperties.DISTRIBUTED_SYSTEM_ID;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Set;
+
+import javax.management.ObjectName;
+import javax.management.Query;
+import javax.management.QueryExp;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.management.DistributedSystemMXBean;
+import org.apache.geode.management.internal.ManagementConstants;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class HttpOperationInvokerMBeanOperationTest {
+
+  @ClassRule
+  public static LocatorStarterRule locator =
+      new LocatorStarterRule().withProperty(DISTRIBUTED_SYSTEM_ID, "100").withAutoStart();
+  private HttpOperationInvoker invoker;
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @Before
+  public void before() throws Exception {
+    gfsh.connectAndVerify(locator.getHttpPort(), GfshShellConnectionRule.PortType.http);
+    invoker = (HttpOperationInvoker) gfsh.getGfsh().getOperationInvoker();
+  }
+
+  @Test
+  public void getAttribute() throws Exception {
+    Integer distributedSystemId =
+        (Integer) invoker.getAttribute(ManagementConstants.OBJECTNAME__DISTRIBUTEDSYSTEM_MXBEAN,
+            "DistributedSystemId");
+    assertThat(distributedSystemId).isEqualTo(100);
+  }
+
+  @Test
+  public void invoke() throws Exception {
+    String[] gatewayReceivers =
+        (String[]) invoker.invoke(ManagementConstants.OBJECTNAME__DISTRIBUTEDSYSTEM_MXBEAN,
+            "listGatewayReceivers", new Object[0], new String[0]);
+
+    assertThat(gatewayReceivers).isEmpty();
+  }
+
+  @Test
+  public void queryName() throws Exception {
+    ObjectName objectName = ObjectName.getInstance("GemFire:type=Member,*");
+    QueryExp query = Query.eq(Query.attr("Name"), Query.value("mock"));
+
+    Set<ObjectName> names = invoker.queryNames(objectName, query);
+    assertTrue(names.isEmpty());
+  }
+
+  @Test
+  public void getClusterId() throws Exception {
+    assertThat(invoker.getClusterId()).isEqualTo(100);
+  }
+
+  @Test
+  public void getDistributedSystemMbean() throws Exception {
+    DistributedSystemMXBean bean = invoker.getDistributedSystemMXBean();
+    assertThat(bean).isInstanceOf(DistributedSystemMXBean.class);
+  }
+}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/HttpOperationInvokerSecurityTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/HttpOperationInvokerSecurityTest.java
new file mode 100644
index 0000000..5c27323
--- /dev/null
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/HttpOperationInvokerSecurityTest.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.web.shell;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.Set;
+
+import javax.management.ObjectName;
+import javax.management.Query;
+import javax.management.QueryExp;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.management.DistributedSystemMXBean;
+import org.apache.geode.management.internal.ManagementConstants;
+import org.apache.geode.management.internal.cli.CommandRequest;
+import org.apache.geode.security.NotAuthorizedException;
+import org.apache.geode.security.SimpleTestSecurityManager;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class HttpOperationInvokerSecurityTest {
+
+  @ClassRule
+  public static LocatorStarterRule locator =
+      new LocatorStarterRule().withSecurityManager(SimpleTestSecurityManager.class).withAutoStart();
+
+  @ClassRule
+  public static GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  private static HttpOperationInvoker invoker;
+  private static CommandRequest request;
+
+  @Test
+  public void performBeanOperationsNoAuthorizationCheck() throws Exception {
+    gfsh.secureConnectAndVerify(locator.getHttpPort(), GfshShellConnectionRule.PortType.http,
+        "test", "test");
+    invoker = (HttpOperationInvoker) gfsh.getGfsh().getOperationInvoker();
+
+    Integer distributedSystemId =
+        (Integer) invoker.getAttribute(ManagementConstants.OBJECTNAME__DISTRIBUTEDSYSTEM_MXBEAN,
+            "DistributedSystemId");
+    assertThat(distributedSystemId).isEqualTo(-1);
+
+    assertThat(invoker.getClusterId()).isEqualTo(-1);
+
+    DistributedSystemMXBean bean = invoker.getDistributedSystemMXBean();
+    assertThat(bean).isInstanceOf(DistributedSystemMXBean.class);
+
+    String[] gatewayReceivers =
+        (String[]) invoker.invoke(ManagementConstants.OBJECTNAME__DISTRIBUTEDSYSTEM_MXBEAN,
+            "listGatewayReceivers", new Object[0], new String[0]);
+
+    assertThat(gatewayReceivers).isEmpty();
+    ObjectName objectName = ObjectName.getInstance("GemFire:type=Member,*");
+    QueryExp query = Query.eq(Query.attr("Name"), Query.value("mock"));
+
+    Set<ObjectName> names = invoker.queryNames(objectName, query);
+    assertTrue(names.isEmpty());
+    gfsh.disconnect();
+  }
+
+  @Test
+  public void processCommandHasAuthorizationCheck() throws Exception {
+    gfsh.secureConnectAndVerify(locator.getHttpPort(), GfshShellConnectionRule.PortType.http,
+        "test", "test");
+
+    invoker = (HttpOperationInvoker) gfsh.getGfsh().getOperationInvoker();
+
+    request = mock(CommandRequest.class);
+    when(request.getHttpRequestUrl(anyString())).thenCallRealMethod();
+    when(request.getUserInput()).thenReturn("list members");
+
+
+    assertThatThrownBy(() -> invoker.processCommand(request))
+        .isInstanceOf(NotAuthorizedException.class);
+    gfsh.disconnect();
+  }
+}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/RestHttpOperationInvokerJUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/RestHttpOperationInvokerJUnitTest.java
deleted file mode 100644
index 2bebd2e..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/RestHttpOperationInvokerJUnitTest.java
+++ /dev/null
@@ -1,470 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.shell;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.geode.management.internal.cli.CommandRequest;
-import org.apache.geode.management.internal.web.AbstractWebTestCase;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.management.internal.web.domain.LinkIndex;
-import org.apache.geode.management.internal.web.http.ClientHttpRequest;
-import org.apache.geode.management.internal.web.http.HttpMethod;
-import org.apache.geode.test.junit.categories.UnitTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.springframework.core.io.Resource;
-import org.springframework.web.client.ResourceAccessException;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-/**
- * The RestHttpOperationInvokerJUnitTest class is a test suite of test cases testing the contract
- * and functionality of the RestHttpOperationInvoker class.
- * <p/>
- * 
- * @see java.net.URI
- * @see org.apache.geode.management.internal.cli.CommandRequest
- * @see org.apache.geode.management.internal.web.http.HttpMethod
- * @see org.apache.geode.management.internal.web.domain.Link
- * @see org.apache.geode.management.internal.web.domain.LinkIndex
- * @see org.apache.geode.management.internal.web.shell.RestHttpOperationInvoker
- * @see org.junit.Assert
- * @see org.junit.Test
- * @since GemFire 8.0
- */
-@SuppressWarnings("unused")
-@Category(UnitTest.class)
-public class RestHttpOperationInvokerJUnitTest extends AbstractWebTestCase {
-
-  private LinkIndex linkIndex;
-
-  private RestHttpOperationInvoker operationInvoker;
-
-  @Before
-  public void setUp() throws Exception {
-    final Link listLibraries =
-        new Link("list-libraries", toUri("http://host.domain.com/service/v1/libraries"));
-    final Link getLibrary =
-        new Link("get-library", toUri("http://host.domain.com/service/v1/libraries/{name}"));
-    final Link listBooks =
-        new Link("list-books", toUri("http://host.domain.com/service/v1/libraries/{name}/books"));
-    final Link listBooksByAuthor = new Link("list-books",
-        toUri("http://host.domain.com/service/v1/libraries/{name}/books/{author}"));
-    final Link listBooksByAuthorAndCategory = new Link("list-books",
-        toUri("http://host.domain.com/service/v1/libraries/{name}/books/{author}/{category}"));
-    final Link listBooksByAuthorAndYear = new Link("list-books",
-        toUri("http://host.domain.com/service/v1/libraries/{name}/books/{author}/{year}"));
-    final Link listBooksByAuthorCategoryAndYear = new Link("list-books", toUri(
-        "http://host.domain.com/service/v1/libraries/{name}/books/{author}/{category}/{year}"));
-    final Link addBook = new Link("add-book",
-        toUri("http://host.domain.com/service/v1/libraries/{name}/books"), HttpMethod.POST);
-    final Link getBookByIsbn = new Link("get-book",
-        toUri("http://host.domain.com/service/v1/libraries/{name}/books/{isbn}"));
-    final Link getBookByTitle = new Link("get-book",
-        toUri("http://host.domain.com/service/v1/libraries/{name}/books/{title}"));
-    final Link removeBook = new Link("remove-book",
-        toUri("http://host.domain.com/service/v1/libraries/{name}/books/{isbn}"),
-        HttpMethod.DELETE);
-
-    linkIndex = new LinkIndex();
-
-    linkIndex.addAll(listLibraries, getLibrary, listBooks, listBooksByAuthor,
-        listBooksByAuthorAndCategory, listBooksByAuthorAndYear, listBooksByAuthorCategoryAndYear,
-        addBook, getBookByIsbn, getBookByTitle, removeBook);
-
-    assertEquals(11, linkIndex.size());
-
-    operationInvoker = new RestHttpOperationInvoker(linkIndex);
-
-    assertSame(linkIndex, operationInvoker.getLinkIndex());
-  }
-
-  @After
-  public void tearDown() {
-    operationInvoker.stop();
-    operationInvoker = null;
-  }
-
-  private CommandRequest createCommandRequest(final String command,
-      final Map<String, String> options) {
-    return new TestCommandRequest(command, options, Collections.emptyMap(), null);
-  }
-
-  private CommandRequest createCommandRequest(final String command,
-      final Map<String, String> options, final Map<String, String> environment) {
-    return new TestCommandRequest(command, options, environment, null);
-  }
-
-  private CommandRequest createCommandRequest(final String command,
-      final Map<String, String> options, final byte[][] fileData) {
-    return new TestCommandRequest(command, options, Collections.emptyMap(), fileData);
-  }
-
-  private CommandRequest createCommandRequest(final String command,
-      final Map<String, String> options, final Map<String, String> environment,
-      final byte[][] fileData) {
-    return new TestCommandRequest(command, options, environment, fileData);
-  }
-
-  private LinkIndex getLinkIndex() {
-    assertTrue("The LinkIndex was not property initialized!", linkIndex != null);
-    return linkIndex;
-  }
-
-  private RestHttpOperationInvoker getOperationInvoker() {
-    assertTrue("The RestHttpOperationInvoker was not properly initialized!",
-        operationInvoker != null);
-    return operationInvoker;
-  }
-
-  @Test
-  public void testCreateHttpRequest() {
-    final Map<String, String> commandOptions = new HashMap<>();
-
-    commandOptions.put("author", "Adams");
-    commandOptions.put("blankOption", "  ");
-    commandOptions.put("category", "sci-fi");
-    commandOptions.put("emptyOption", StringUtils.EMPTY);
-    commandOptions.put("isbn", "0-123456789");
-    commandOptions.put("nullOption", null);
-    commandOptions.put("title", "Hitch Hiker's Guide to the Galaxy");
-    commandOptions.put("year", "1983");
-
-    final CommandRequest command = createCommandRequest("add-book", commandOptions);
-
-    final ClientHttpRequest request = getOperationInvoker().createHttpRequest(command);
-
-    assertNotNull(request);
-    assertEquals("POST http://host.domain.com/service/v1/libraries/{name}/books",
-        request.getLink().toHttpRequestLine());
-    assertEquals("Adams", request.getParameterValue("author"));
-    assertEquals("sci-fi", request.getParameterValue("category"));
-    assertEquals("0-123456789", request.getParameterValue("isbn"));
-    assertEquals("Hitch Hiker's Guide to the Galaxy", request.getParameterValue("title"));
-    assertEquals("1983", request.getParameterValue("year"));
-    assertTrue(request.getParameters().containsKey("blankOption"));
-    assertTrue(request.getParameters().containsKey("emptyOption"));
-    assertFalse(request.getParameters().containsKey("nullOption"));
-
-    for (String requestParameter : request.getParameters().keySet()) {
-      assertFalse(requestParameter
-          .startsWith(RestHttpOperationInvoker.ENVIRONMENT_VARIABLE_REQUEST_PARAMETER_PREFIX));
-    }
-
-    assertNull(request.getParameterValue(RestHttpOperationInvoker.RESOURCES_REQUEST_PARAMETER));
-  }
-
-  @Test
-  public void testCreateHttpRequestWithEnvironmentVariables() {
-    final Map<String, String> commandOptions = new HashMap<>(2);
-
-    commandOptions.put("name", "ElLibreDeCongress");
-    commandOptions.put("isbn", "${ISBN}");
-
-    final Map<String, String> environment = new HashMap<>(2);
-
-    environment.put("ISBN", "0-987654321");
-    environment.put("VAR", "test");
-
-    final CommandRequest command = createCommandRequest("get-book", commandOptions, environment);
-
-    final ClientHttpRequest request = getOperationInvoker().createHttpRequest(command);
-
-    assertNotNull(request);
-    assertEquals("GET http://host.domain.com/service/v1/libraries/{name}/books/{isbn}",
-        request.getLink().toHttpRequestLine());
-    assertEquals("${ISBN}", request.getParameterValue("isbn"));
-    assertFalse(request.getParameters().containsKey("ISBN"));
-    assertEquals("0-987654321", request.getParameterValue(
-        RestHttpOperationInvoker.ENVIRONMENT_VARIABLE_REQUEST_PARAMETER_PREFIX + "ISBN"));
-    assertFalse(request.getParameters().containsKey("VAR"));
-    assertEquals("test", request.getParameterValue(
-        RestHttpOperationInvoker.ENVIRONMENT_VARIABLE_REQUEST_PARAMETER_PREFIX + "VAR"));
-  }
-
-  @Test
-  public void testCreateHttpRequestWithFileData() {
-    final Map<String, String> commandOptions = Collections.singletonMap("isbn", "0-123456789");
-
-    final byte[][] fileData = {"/path/to/book/content.txt".getBytes(),
-        "Once upon a time in a galaxy far, far away...".getBytes()};
-
-    final CommandRequest command = createCommandRequest("add-book", commandOptions, fileData);
-
-    final ClientHttpRequest request = getOperationInvoker().createHttpRequest(command);
-
-    assertNotNull(request);
-    assertEquals("POST http://host.domain.com/service/v1/libraries/{name}/books",
-        request.getLink().toHttpRequestLine());
-    assertEquals("0-123456789", request.getParameterValue("isbn"));
-    assertTrue(
-        request.getParameters().containsKey(RestHttpOperationInvoker.RESOURCES_REQUEST_PARAMETER));
-    assertTrue(request.getParameterValue(
-        RestHttpOperationInvoker.RESOURCES_REQUEST_PARAMETER) instanceof Resource);
-
-    final List<Object> resources =
-        request.getParameterValues(RestHttpOperationInvoker.RESOURCES_REQUEST_PARAMETER);
-
-    assertNotNull(resources);
-    assertFalse(resources.isEmpty());
-    assertEquals(1, resources.size());
-  }
-
-  @Test
-  public void testFindAndResolveLink() throws Exception {
-    final Map<String, String> commandOptions = new HashMap<>();
-
-    commandOptions.put("name", "BarnesN'Noble");
-
-    Link link =
-        getOperationInvoker().findLink(createCommandRequest("list-libraries", commandOptions));
-
-    assertNotNull(link);
-    assertEquals("http://host.domain.com/service/v1/libraries", toString(link.getHref()));
-
-    link = getOperationInvoker().findLink(createCommandRequest("get-library", commandOptions));
-
-    assertNotNull(link);
-    assertEquals("http://host.domain.com/service/v1/libraries/{name}", toString(link.getHref()));
-
-    commandOptions.put("author", "J.K.Rowling");
-
-    link = getOperationInvoker().findLink(createCommandRequest("list-books", commandOptions));
-
-    assertNotNull(link);
-    assertEquals("http://host.domain.com/service/v1/libraries/{name}/books/{author}",
-        toString(link.getHref()));
-
-    commandOptions.put("category", "sci-fi");
-    commandOptions.put("year", "1998");
-    commandOptions.put("bogus", "data");
-
-    link = getOperationInvoker().findLink(createCommandRequest("list-books", commandOptions));
-
-    assertNotNull(link);
-    assertEquals(
-        "http://host.domain.com/service/v1/libraries/{name}/books/{author}/{category}/{year}",
-        toString(link.getHref()));
-
-    commandOptions.remove("category");
-
-    link = getOperationInvoker().findLink(createCommandRequest("list-books", commandOptions));
-
-    assertNotNull(link);
-    assertEquals("http://host.domain.com/service/v1/libraries/{name}/books/{author}/{year}",
-        toString(link.getHref()));
-
-    commandOptions.put("category", "fantasy");
-    commandOptions.put("isbn", "0-123456789");
-    commandOptions.put("title", "Harry Potter");
-
-    link = getOperationInvoker().findLink(createCommandRequest("add-book", commandOptions));
-
-    assertNotNull(link);
-    assertEquals("http://host.domain.com/service/v1/libraries/{name}/books",
-        toString(link.getHref()));
-
-    commandOptions.remove("isbn");
-
-    link = getOperationInvoker().findLink(createCommandRequest("get-book", commandOptions));
-
-    assertNotNull(link);
-    assertEquals("http://host.domain.com/service/v1/libraries/{name}/books/{title}",
-        toString(link.getHref()));
-
-    link = getOperationInvoker().findLink(createCommandRequest("remove-book", commandOptions));
-
-    assertNotNull(link);
-    assertEquals("http://host.domain.com/service/v1/libraries/{name}/books/{isbn}",
-        toString(link.getHref()));
-  }
-
-  @Test
-  public void testProcessCommand() {
-    final String expectedResult = "{\"libraries\":[{\"library-of\":\"Congress\"}]"; // JSON
-
-    final RestHttpOperationInvoker operationInvoker = new RestHttpOperationInvoker(getLinkIndex()) {
-      @Override
-      public boolean isConnected() {
-        return true;
-      }
-
-      @Override
-      @SuppressWarnings("unchecked")
-      protected <T> T send(final ClientHttpRequest request, final Class<T> responseType,
-          final Map<String, ?> uriVariables) {
-        return (T) expectedResult;
-      }
-    };
-
-    final Object actualResult = operationInvoker
-        .processCommand(createCommandRequest("list-libraries", Collections.emptyMap()));
-
-    assertEquals(expectedResult, actualResult);
-  }
-
-  @Test
-  public void testProcessCommandDelegatesToSimpleProcessCommand() {
-    final String expectedResult = "<resources>test</resources>";
-
-    final RestHttpOperationInvoker operationInvoker = new RestHttpOperationInvoker(getLinkIndex()) {
-      @Override
-      public boolean isConnected() {
-        return true;
-      }
-
-      @Override
-      protected HttpOperationInvoker getHttpOperationInvoker() {
-        return new AbstractHttpOperationInvoker(AbstractHttpOperationInvoker.REST_API_URL) {
-          @Override
-          public Object processCommand(final CommandRequest command) {
-            return expectedResult;
-          }
-        };
-      }
-
-      @Override
-      protected void printWarning(final String message, final Object... args) {}
-    };
-
-    final Object actualResult = operationInvoker
-        .processCommand(createCommandRequest("get resource", Collections.emptyMap()));
-
-    assertEquals(expectedResult, actualResult);
-  }
-
-  @Test
-  public void testProcessCommandHandlesResourceAccessException() {
-    final RestHttpOperationInvoker operationInvoker = new RestHttpOperationInvoker(getLinkIndex()) {
-      private boolean connected = true;
-
-      @Override
-      public boolean isConnected() {
-        return connected;
-      }
-
-      @Override
-      protected void printWarning(final String message, final Object... args) {}
-
-      @Override
-      protected <T> T send(final ClientHttpRequest request, final Class<T> responseType,
-          final Map<String, ?> uriVariables) {
-        throw new ResourceAccessException("test");
-      }
-
-      @Override
-      public void stop() {
-        this.connected = false;
-      }
-    };
-
-    assertTrue(operationInvoker.isConnected());
-
-    final String expectedResult = String.format(
-        "The connection to the GemFire Manager's HTTP service @ %1$s failed with: %2$s. "
-            + "Please try reconnecting or see the GemFire Manager's log file for further details.",
-        operationInvoker.getBaseUrl(), "test");
-
-    final Object actualResult = operationInvoker
-        .processCommand(createCommandRequest("list-libraries", Collections.emptyMap()));
-
-    assertFalse(operationInvoker.isConnected());
-    assertEquals(expectedResult, actualResult);
-  }
-
-  @Test(expected = RestApiCallForCommandNotFoundException.class)
-  public void testProcessCommandThrowsRestApiCallForCommandNotFoundException() {
-    final RestHttpOperationInvoker operationInvoker = new RestHttpOperationInvoker(getLinkIndex()) {
-      @Override
-      public boolean isConnected() {
-        return true;
-      }
-
-      @Override
-      protected HttpOperationInvoker getHttpOperationInvoker() {
-        return null;
-      }
-    };
-
-    try {
-      operationInvoker.processCommand(createCommandRequest("get resource", Collections.emptyMap()));
-    } catch (RestApiCallForCommandNotFoundException e) {
-      assertEquals("No REST API call for command (get resource) was found!", e.getMessage());
-      throw e;
-    }
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testProcessCommandWhenNotConnected() {
-    try {
-      getOperationInvoker()
-          .processCommand(createCommandRequest("get-book", Collections.emptyMap()));
-    } catch (IllegalStateException e) {
-      assertEquals(
-          "Gfsh must be connected to the GemFire Manager in order to process commands remotely!",
-          e.getMessage());
-      throw e;
-    }
-  }
-
-  private static class TestCommandRequest extends CommandRequest {
-
-    private final Map<String, String> commandParameters = new TreeMap<>();
-
-    private final String command;
-
-    protected TestCommandRequest(final String command, final Map<String, String> commandParameters,
-        final Map<String, String> environment, final byte[][] fileData) {
-      super(environment, fileData);
-
-      assert command != null : "The command cannot be null!";
-
-      this.command = command;
-
-      if (commandParameters != null) {
-        this.commandParameters.putAll(commandParameters);
-      }
-    }
-
-    @Override
-    public String getInput() {
-      return command;
-    }
-
-    @Override
-    public String getName() {
-      return command;
-    }
-
-    @Override
-    public Map<String, String> getParameters() {
-      return Collections.unmodifiableMap(commandParameters);
-    }
-  }
-
-}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/SimpleHttpOperationInvokerJUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/SimpleHttpOperationInvokerJUnitTest.java
deleted file mode 100644
index 4571f2c..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/web/shell/SimpleHttpOperationInvokerJUnitTest.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.shell;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.geode.management.internal.cli.CommandRequest;
-import org.apache.geode.management.internal.web.AbstractWebTestCase;
-import org.apache.geode.management.internal.web.domain.Link;
-import org.apache.geode.management.internal.web.http.ClientHttpRequest;
-import org.apache.geode.management.internal.web.http.HttpHeader;
-import org.apache.geode.management.internal.web.http.HttpMethod;
-import org.apache.geode.test.junit.categories.UnitTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.springframework.web.client.ResourceAccessException;
-
-import java.util.Collections;
-
-/**
- * The SimpleHttpOperationInvokerJUnitTest class is a test suite of test cases testing the contract
- * and functionality of the SimpleHttpOperationInvoker class.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.web.AbstractWebTestCase
- * @see org.apache.geode.management.internal.web.shell.SimpleHttpOperationInvoker
- * @see org.junit.Assert
- * @see org.junit.After
- * @see org.junit.Before
- * @see org.junit.Test
- * @since GemFire 8.0
- */
-@Category(UnitTest.class)
-public class SimpleHttpOperationInvokerJUnitTest extends AbstractWebTestCase {
-
-  private SimpleHttpOperationInvoker operationInvoker;
-
-  @Before
-  public void setUp() {
-    operationInvoker = new SimpleHttpOperationInvoker();
-  }
-
-  @After
-  public void tearDown() {
-    operationInvoker.stop();
-    operationInvoker = null;
-  }
-
-  private CommandRequest createCommandRequest(final String command) {
-    return new TestCommandRequest(command);
-  }
-
-  private String getExpectedHttpRequestUrl(final CommandRequest command) {
-    return SimpleHttpOperationInvoker.REST_API_URL
-        .concat(SimpleHttpOperationInvoker.REST_API_MANAGEMENT_COMMANDS_URI).concat("?")
-        .concat(SimpleHttpOperationInvoker.CMD_QUERY_PARAMETER).concat("=")
-        .concat(command.getInput());
-  }
-
-  private SimpleHttpOperationInvoker getOperationInvoker() {
-    return operationInvoker;
-  }
-
-  @Test
-  public void testCreateHttpRequest() throws Exception {
-    final CommandRequest command =
-        createCommandRequest("save resource --path=/path/to/file --size=1024KB");
-
-    final ClientHttpRequest request = getOperationInvoker().createHttpRequest(command);
-
-    assertNotNull(request);
-    assertEquals(SimpleHttpOperationInvoker.USER_AGENT_HTTP_REQUEST_HEADER_VALUE,
-        request.getHeaderValue(HttpHeader.USER_AGENT.getName()));
-
-    final Link requestLink = request.getLink();
-
-    assertNotNull(requestLink);
-    assertTrue(toString(requestLink).startsWith("POST"));
-    assertTrue(toString(requestLink).endsWith(command.getInput()));
-  }
-
-  @Test
-  public void testCreateLink() throws Exception {
-    final CommandRequest command = createCommandRequest("delete resource --id=1");
-
-    final Link actualLink = getOperationInvoker().createLink(command);
-
-    assertNotNull(actualLink);
-    assertEquals(SimpleHttpOperationInvoker.LINK_RELATION, actualLink.getRelation());
-    assertEquals(HttpMethod.POST, actualLink.getMethod());
-    assertTrue(toString(actualLink.getHref()).endsWith(command.getInput()));
-  }
-
-  @Test
-  public void testGetHttpRequestUrl() throws Exception {
-    final CommandRequest command = createCommandRequest("get resource --option=value");
-
-    assertEquals(getExpectedHttpRequestUrl(command),
-        toString(getOperationInvoker().getHttpRequestUrl(command)));
-  }
-
-  @Test
-  public void testProcessCommand() {
-    final String expectedResult = "<resource>test</resource>"; // XML
-
-    final SimpleHttpOperationInvoker operationInvoker = new SimpleHttpOperationInvoker() {
-      @Override
-      public boolean isConnected() {
-        return true;
-      }
-
-      @Override
-      @SuppressWarnings("unchecked")
-      protected <T> T send(final ClientHttpRequest request, final Class<T> responseType) {
-        return (T) expectedResult;
-      }
-    };
-
-    final String actualResult =
-        operationInvoker.processCommand(createCommandRequest("get resource --id=1"));
-
-    assertEquals(expectedResult, actualResult);
-  }
-
-  @Test
-  public void testProcessCommandHandlesResourceAccessException() {
-    final SimpleHttpOperationInvoker operationInvoker = new SimpleHttpOperationInvoker() {
-      private boolean connected = true;
-
-      @Override
-      public boolean isConnected() {
-        return connected;
-      }
-
-      @Override
-      protected <T> T send(final ClientHttpRequest request, final Class<T> responseType) {
-        throw new ResourceAccessException("test");
-      }
-
-      @Override
-      public void stop() {
-        this.connected = false;
-      }
-    };
-
-    assertTrue(operationInvoker.isConnected());
-
-    final String expectedResult = String.format(
-        "The connection to the GemFire Manager's HTTP service @ %1$s failed with: %2$s. "
-            + "Please try reconnecting or see the GemFire Manager's log file for further details.",
-        operationInvoker.getBaseUrl(), "test");
-
-    final String actualResult =
-        operationInvoker.processCommand(createCommandRequest("get resource --id=1"));
-
-    assertFalse(operationInvoker.isConnected());
-    assertEquals(expectedResult, actualResult);
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testProcessCommandWhenNotConnected() {
-    try {
-      getOperationInvoker().processCommand(createCommandRequest("get resource"));
-    } catch (IllegalStateException e) {
-      assertEquals(
-          "Gfsh must be connected to the GemFire Manager in order to process commands remotely!",
-          e.getMessage());
-      throw e;
-    }
-  }
-
-  private static class TestCommandRequest extends CommandRequest {
-
-    private final String command;
-
-    protected TestCommandRequest(final String command) {
-      super(Collections.<String, String>emptyMap());
-      assert command != null : "The command cannot be null!";
-      this.command = command;
-    }
-
-    @Override
-    public String getInput() {
-      return command;
-    }
-  }
-
-}
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/web/util/UriUtilsJUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/web/util/UriUtilsJUnitTest.java
deleted file mode 100644
index 9d4dd46..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/web/util/UriUtilsJUnitTest.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.management.internal.web.util;
-
-import static org.junit.Assert.*;
-
-import java.util.Map;
-
-import org.apache.geode.management.internal.web.AbstractWebTestCase;
-import org.apache.geode.test.junit.categories.UnitTest;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * The UriUtilsJUnitTest class is a test suite of test cases testing the contract and functionality
- * of the UriUtils class.
- * <p/>
- * 
- * @see org.apache.geode.management.internal.web.AbstractWebTestCase
- * @see org.apache.geode.management.internal.web.util.UriUtils
- * @see org.junit.Assert
- * @see org.junit.Test
- * @since GemFire 8.0
- */
-@Category(UnitTest.class)
-public class UriUtilsJUnitTest extends AbstractWebTestCase {
-
-  @Test
-  public void testDecodeNull() {
-    assertNull(UriUtils.decode((String) null));
-  }
-
-  @Test
-  public void testDecodeStringArray() throws Exception {
-    final String[] encodedValues = {null, "123", "test", encode("Path/Subpath"),
-        encode(encode(encode("/Customers/Accounts/Orders/Items")))};
-
-    final String[] decodedValues = UriUtils.decode(encodedValues);
-
-    assertSame(encodedValues, decodedValues);
-    assertEquals(5, decodedValues.length);
-    assertNull(decodedValues[0]);
-    assertEquals("123", decodedValues[1]);
-    assertEquals("test", decodedValues[2]);
-    assertEquals("Path/Subpath", decodedValues[3]);
-    assertEquals("/Customers/Accounts/Orders/Items", decodedValues[4]);
-  }
-
-  @Test
-  public void testDecodeMap() throws Exception {
-    final Map<String, Object> encodedForm =
-        createMap(createArray("0", "1", "2", "3", "4"), (Object[]) createArray(null, "123", "test",
-            encode("Path/Subpath"), encode(encode(encode("/Customers/Accounts/Orders/Items")))));
-
-    final Map<String, Object> decodedForm = UriUtils.decode(encodedForm);
-
-    assertSame(encodedForm, decodedForm);
-    assertEquals(5, decodedForm.size());
-    assertNull(decodedForm.get("0"));
-    assertEquals("123", decodedForm.get("1"));
-    assertEquals("test", decodedForm.get("2"));
-    assertEquals("Path/Subpath", decodedForm.get("3"));
-    assertEquals("/Customers/Accounts/Orders/Items", decodedForm.get("4"));
-  }
-
-  @Test
-  public void testEncodeNull() {
-    assertNull(UriUtils.encode((String) null));
-  }
-
-  @Test
-  public void testEncodeStringArray() throws Exception {
-    final String[] values =
-        {null, "123", "test", "Path/Subpath", "/Customers/Accounts/Orders/Items"};
-    final String[] encodedValues = UriUtils.encode(values);
-
-    assertSame(values, encodedValues);
-    assertEquals(5, encodedValues.length);
-    assertNull(encodedValues[0]);
-    assertEquals("123", encodedValues[1]);
-    assertEquals("test", encodedValues[2]);
-    assertEquals(encode("Path/Subpath"), encodedValues[3]);
-    assertEquals(encode("/Customers/Accounts/Orders/Items"), encodedValues[4]);
-  }
-
-  @Test
-  public void testEncodeMap() throws Exception {
-    final Map<String, Object> form =
-        createMap(createArray("0", "1", "2", "3", "4"), (Object[]) createArray(null, "123", "test",
-            "Path/Subpath", "/Customers/Accounts/Orders/Items"));
-
-    final Map<String, Object> encodedForm = UriUtils.encode(form);
-
-    assertSame(form, encodedForm);
-    assertEquals(5, encodedForm.size());
-    assertNull(encodedForm.get("0"));
-    assertEquals("123", encodedForm.get("1"));
-    assertEquals("test", encodedForm.get("2"));
-    assertEquals(encode("Path/Subpath"), encodedForm.get("3"));
-    assertEquals(encode("/Customers/Accounts/Orders/Items"), encodedForm.get("4"));
-  }
-
-}

-- 
To stop receiving notification emails like this one, please contact
['"commits@geode.apache.org" <co...@geode.apache.org>'].