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/04/27 20:18:11 UTC

[1/9] geode git commit: GEODE-1597: use Spring shell's parser and delete our own parsing code

Repository: geode
Updated Branches:
  refs/heads/develop bee0b7d57 -> 1fc0f0ca7


http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyJUnitTest.java
----------------------------------------------------------------------
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/GfshExecutionStrategyJUnitTest.java
index 54c7cf7..ece0c7e 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/GfshExecutionStrategyJUnitTest.java
@@ -14,125 +14,94 @@
  */
 package org.apache.geode.management.internal.cli.shell;
 
-import static org.junit.Assert.*;
-
-import java.util.List;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.ConverterHint;
 import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.CommandManager;
-import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.annotation.CliArgument;
+import org.apache.geode.management.internal.cli.CommandRequest;
+import org.apache.geode.management.internal.cli.CommandResponseBuilder;
+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.management.internal.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
 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.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.shell.event.ParseResult;
 
 /**
  * GfshExecutionStrategyTest - Includes tests to for GfshExecutionStrategyTest
  */
 @Category(UnitTest.class)
 public class GfshExecutionStrategyJUnitTest {
-
-  private static final String COMMAND1_NAME = "command1";
-  private static final String COMMAND1_NAME_ALIAS = "command1_alias";
-  private static final String COMMAND2_NAME = "command2";
   private static final String COMMAND1_SUCESS = "Command1 Executed successfully";
   private static final String COMMAND2_SUCESS = "Command2 Executed successfully";
-  private static final String COMMAND1_HELP = "help for " + COMMAND1_NAME;
 
-  @After
-  public void tearDown() {
-    CommandManager.clearInstance();
+  private Gfsh gfsh;
+  private GfshParseResult parsedCommand;
+  private GfshExecutionStrategy gfshExecutionStrategy;
+
+  @Before
+  public void before() {
+    gfsh = mock(Gfsh.class);
+    parsedCommand = mock(GfshParseResult.class);
+    gfshExecutionStrategy = new GfshExecutionStrategy(gfsh);
   }
 
   /**
-   * tests execute method by executing dummy method command1
+   * tests execute offline command
    */
   @Test
-  public void testGfshExecutionStartegyExecute() throws Exception {
-    CommandManager commandManager = CommandManager.getInstance();
-    assertNotNull("CommandManager should not be null.", commandManager);
-    commandManager.add(Commands.class.newInstance());
-    GfshParser parser = new GfshParser(commandManager);
-    String[] command1Names =
-        ((CliCommand) Commands.class.getMethod(COMMAND1_NAME).getAnnotation(CliCommand.class))
-            .value();
-    String input = command1Names[0];
-    ParseResult parseResult = null;
-    parseResult = parser.parse(input);
-    String[] args = new String[] {command1Names[0]};
-    Gfsh gfsh = Gfsh.getInstance(false, args, new GfshConfig());
-    GfshExecutionStrategy gfshExecutionStrategy = new GfshExecutionStrategy(gfsh);
-    Result resultObject = (Result) gfshExecutionStrategy.execute(parseResult);
-    String str = resultObject.nextLine();
-    assertTrue(str.trim().equals(COMMAND1_SUCESS));
+  public void testOfflineCommand() throws Exception {
+    when(parsedCommand.getMethod()).thenReturn(Commands.class.getDeclaredMethod("offlineCommand"));
+    when(parsedCommand.getInstance()).thenReturn(new Commands());
+    Result result = (Result) gfshExecutionStrategy.execute(parsedCommand);
+    assertThat(result.nextLine().trim()).isEqualTo(COMMAND1_SUCESS);
   }
 
   /**
-   * tests isReadyForCommnads method by executing dummy method command1. TODO: this method is hard
-   * coded in source which may change in future. So this test should also be accordingly changed
+   * tests execute online command
    */
   @Test
-  public void testGfshExecutionStartegyIsReadyForCommands() throws Exception {
-    CommandManager commandManager = CommandManager.getInstance();
-    assertNotNull("CommandManager should not be null.", commandManager);
-    commandManager.add(Commands.class.newInstance());
-    String[] command1Names =
-        ((CliCommand) Commands.class.getMethod(COMMAND1_NAME).getAnnotation(CliCommand.class))
-            .value();
-    String[] args = new String[] {command1Names[0]};
-    Gfsh gfsh = Gfsh.getInstance(false, args, new GfshConfig());
-    GfshExecutionStrategy gfshExecutionStrategy = new GfshExecutionStrategy(gfsh);
-    boolean ready = gfshExecutionStrategy.isReadyForCommands();
-    assertTrue(ready);
+  public void testOnLineCommandWhenGfshisOffLine() throws Exception {
+    when(parsedCommand.getMethod()).thenReturn(Commands.class.getDeclaredMethod("onlineCommand"));
+    when(parsedCommand.getInstance()).thenReturn(new Commands());
+    when(gfsh.isConnectedAndReady()).thenReturn(false);
+    Result result = (Result) gfshExecutionStrategy.execute(parsedCommand);
+    assertThat(result).isNull();
+  }
+
+  @Test
+  public void testOnLineCommandWhenGfshisOnLine() throws Exception {
+    when(parsedCommand.getMethod()).thenReturn(Commands.class.getDeclaredMethod("onlineCommand"));
+    when(parsedCommand.getInstance()).thenReturn(new Commands());
+    when(gfsh.isConnectedAndReady()).thenReturn(true);
+    OperationInvoker invoker = mock(OperationInvoker.class);
+
+    Result offLineResult = new Commands().onlineCommand();
+    String jsonResult = CommandResponseBuilder.createCommandResponseJson("memberName",
+        (CommandResult) offLineResult);
+    when(invoker.processCommand(any(CommandRequest.class))).thenReturn(jsonResult);
+    when(gfsh.getOperationInvoker()).thenReturn(invoker);
+    Result result = (Result) gfshExecutionStrategy.execute(parsedCommand);
+    assertThat(result.nextLine().trim()).isEqualTo(COMMAND2_SUCESS);
   }
 
   /**
    * represents class for dummy methods
    */
   public static class Commands implements CommandMarker {
-
-    @CliCommand(value = {COMMAND1_NAME, COMMAND1_NAME_ALIAS}, help = COMMAND1_HELP)
     @CliMetaData(shellOnly = true)
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public static Result command1() {
+    public Result offlineCommand() {
       return ResultBuilder.createInfoResult(COMMAND1_SUCESS);
     }
 
-    @CliCommand(value = {COMMAND2_NAME})
     @CliMetaData(shellOnly = false)
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public static Result command2() {
+    public Result onlineCommand() {
       return ResultBuilder.createInfoResult(COMMAND2_SUCESS);
     }
-
-    @CliCommand(value = {"testParamConcat"})
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public static Result testParamConcat(@CliOption(key = {"string"}) String string,
-        @CliOption(key = {"stringArray"}) @CliMetaData(valueSeparator = ",") String[] stringArray,
-        @CliOption(key = {"stringList"}, optionContext = ConverterHint.STRING_LIST) @CliMetaData(
-            valueSeparator = ",") List<String> stringList,
-        @CliOption(key = {"integer"}) Integer integer,
-        @CliOption(key = {"colonArray"}) @CliMetaData(valueSeparator = ":") String[] colonArray) {
-      return null;
-    }
-
-    @CliCommand(value = {"testMultiWordArg"})
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public static Result testMultiWordArg(@CliArgument(name = "arg1") String arg1,
-        @CliArgument(name = "arg2") String arg2) {
-      return null;
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshHistoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshHistoryJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshHistoryJUnitTest.java
index 58453b7..a563c65 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshHistoryJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshHistoryJUnitTest.java
@@ -14,13 +14,10 @@
  */
 package org.apache.geode.management.internal.cli.shell;
 
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.lang.reflect.Field;
-import java.nio.file.Files;
-import java.util.List;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 
+import org.apache.geode.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -28,7 +25,10 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 
-import org.apache.geode.test.junit.categories.IntegrationTest;
+import java.io.File;
+import java.lang.reflect.Field;
+import java.nio.file.Files;
+import java.util.List;
 
 @Category(IntegrationTest.class)
 public class GfshHistoryJUnitTest {
@@ -65,29 +65,26 @@ public class GfshHistoryJUnitTest {
   @Test
   public void testHistoryFileIsCreated() throws Exception {
     Gfsh gfsh = Gfsh.getInstance(false, new String[] {}, gfshConfig);
-    gfsh.executeScriptLine("connect --fake-param=foo");
+    gfsh.executeScriptLine("connect");
 
     List<String> lines = Files.readAllLines(gfshHistoryFile.toPath());
     assertEquals(2, lines.size());
-    assertEquals(lines.get(1), "connect --fake-param=foo");
+    assertEquals(lines.get(1), "connect");
   }
 
   @Test
   public void testHistoryFileDoesNotContainPasswords() throws Exception {
     Gfsh gfsh = Gfsh.getInstance(false, new String[] {}, gfshConfig);
-    gfsh.executeScriptLine(
-        "connect --password=foo --password = foo --password= goo --password =goo --password-param=blah --other-password-param=    gah");
+    gfsh.executeScriptLine("connect --password=foo");
 
     List<String> lines = Files.readAllLines(gfshHistoryFile.toPath());
-    assertEquals(
-        "connect --password=***** --password = ***** --password= ***** --password =***** --password-param=***** --other-password-param= *****",
-        lines.get(1));
+    assertEquals("connect --password=*****", lines.get(1));
   }
 
   @Test
   public void testClearHistory() throws Exception {
     Gfsh gfsh = Gfsh.getInstance(false, new String[] {}, gfshConfig);
-    gfsh.executeScriptLine("connect --fake-param=foo");
+    gfsh.executeScriptLine("connect");
     List<String> lines = Files.readAllLines(gfshHistoryFile.toPath());
     assertEquals(2, lines.size());
 

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshJunitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshJunitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshJunitTest.java
new file mode 100644
index 0000000..2a9c0d0
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/shell/GfshJunitTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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 static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class GfshJunitTest {
+  private String testString;
+
+  @Before
+  public void before() {
+    testString = "This is a test string.";
+  }
+
+  @Test
+  public void testWrapTest() {
+    assertThat(Gfsh.wrapText(testString, 0, -1)).isEqualTo(testString);
+    assertThat(Gfsh.wrapText(testString, 0, 0)).isEqualTo(testString);
+    assertThat(Gfsh.wrapText(testString, 0, 1)).isEqualTo(testString);
+    assertThat(Gfsh.wrapText(testString, 0, 10)).isEqualTo("This is a\ntest\nstring.");
+    assertThat(Gfsh.wrapText(testString, 1, 100)).isEqualTo(Gfsh.LINE_INDENT + testString);
+    assertThat(Gfsh.wrapText(testString, 2, 100))
+        .isEqualTo(Gfsh.LINE_INDENT + Gfsh.LINE_INDENT + testString);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java
index abbc5c0..5dc77aa 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java
@@ -86,7 +86,7 @@ public class ClusterConfigDistributionDUnitTest {
 
 
     String asyncEventQueueJarPath = createAsyncEventQueueJar();
-    gfshConnector.executeAndVerifyCommand("deploy jar --jar=" + asyncEventQueueJarPath);
+    gfshConnector.executeAndVerifyCommand("deploy --jar=" + asyncEventQueueJarPath);
 
 
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_ASYNC_EVENT_QUEUE);
@@ -166,7 +166,6 @@ public class ClusterConfigDistributionDUnitTest {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CONFIGURE_PDX);
     csb.addOptionWithValueCheck(CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES, "com.foo.*");
     csb.addOptionWithValueCheck(CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS, "true");
-    csb.addOptionWithValueCheck(CliStrings.CONFIGURE_PDX__PERSISTENT, "true");
     csb.addOptionWithValueCheck(CliStrings.CONFIGURE_PDX__READ__SERIALIZED, "true");
 
     String message = gfshConnector.execute(csb.getCommandString());

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
index ffe6a28..3f8f4d9 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
@@ -14,12 +14,12 @@
  */
 package org.apache.geode.management.internal.security;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.geode.security.ResourcePermission;
 import org.apache.shiro.authz.Permission;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class TestCommand {
 
   public static ResourcePermission none = null;
@@ -142,7 +142,7 @@ public class TestCommand {
 
     // FunctionCommands
     // createTestCommand("destroy function --id=InterestCalculations", dataManage);
-    createTestCommand("execute function --id=InterestCalculations --group=Group1", dataWrite);
+    createTestCommand("execute function --id=InterestCalculations --groups=Group1", dataWrite);
     createTestCommand("list functions", clusterRead);
 
     // GfshHelpCommands
@@ -178,7 +178,7 @@ public class TestCommand {
     createTestCommand("list members", clusterRead);
 
     // Misc Commands
-    createTestCommand("change loglevel --loglevel=severe --member=server1", clusterWrite);
+    createTestCommand("change loglevel --loglevel=severe --members=server1", clusterWrite);
     createTestCommand("export logs --dir=data/logs", clusterRead);
     createTestCommand("export stack-traces --file=stack.txt", clusterRead);
     createTestCommand("gc", clusterManage);

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
index 4bfa868..3756b81 100755
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
@@ -17,12 +17,18 @@ package org.apache.geode.cache.lucene.internal.cli;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.execute.*;
-import org.apache.geode.cache.lucene.internal.cli.functions.*;
+import org.apache.geode.cache.execute.Execution;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionInvocationTargetException;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.cache.lucene.internal.cli.functions.LuceneCreateIndexFunction;
+import org.apache.geode.cache.lucene.internal.cli.functions.LuceneDescribeIndexFunction;
+import org.apache.geode.cache.lucene.internal.cli.functions.LuceneDestroyIndexFunction;
+import org.apache.geode.cache.lucene.internal.cli.functions.LuceneListIndexFunction;
+import org.apache.geode.cache.lucene.internal.cli.functions.LuceneSearchIndexFunction;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
 import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.security.IntegratedSecurityService;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.management.cli.CliMetaData;
@@ -170,13 +176,11 @@ public class LuceneIndexCommands extends AbstractCommandsSupport {
           help = LuceneCliStrings.LUCENE_CREATE_INDEX__REGION_HELP) final String regionPath,
 
       @CliOption(key = LuceneCliStrings.LUCENE_CREATE_INDEX__FIELD, mandatory = true,
-          help = LuceneCliStrings.LUCENE_CREATE_INDEX__FIELD_HELP) @CliMetaData(
-              valueSeparator = ",") final String[] fields,
+          help = LuceneCliStrings.LUCENE_CREATE_INDEX__FIELD_HELP) final String[] fields,
 
       @CliOption(key = LuceneCliStrings.LUCENE_CREATE_INDEX__ANALYZER, mandatory = false,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = LuceneCliStrings.LUCENE_CREATE_INDEX__ANALYZER_HELP) @CliMetaData(
-              valueSeparator = ",") final String[] analyzers) {
+          help = LuceneCliStrings.LUCENE_CREATE_INDEX__ANALYZER_HELP) final String[] analyzers) {
 
     Result result = null;
     XmlEntity xmlEntity = null;

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java
index ad734e8..423fc59 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java
@@ -14,20 +14,21 @@
  */
 package org.apache.geode.cache.lucene;
 
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME;
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.lucene.internal.cli.LuceneCliStrings;
-import org.apache.geode.cache.lucene.internal.cli.LuceneIndexCommands;
 import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.CommandManager;
 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.ErrorResultData;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.security.SimpleTestSecurityManager;
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
 import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
 import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
@@ -41,16 +42,12 @@ import org.junit.runner.RunWith;
 
 import java.io.Serializable;
 import java.util.Properties;
-
-import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME;
-import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
 
 @Category({DistributedTest.class, SecurityTest.class})
 @RunWith(JUnitParamsRunner.class)
-public class LuceneCommandsSecurityDUnitTest extends JUnit4CacheTestCase {
+public class LuceneCommandsSecurityDUnitTest {
 
   @Rule
   public LocatorServerStartupRule locatorServer = new LocatorServerStartupRule();
@@ -190,7 +187,6 @@ public class LuceneCommandsSecurityDUnitTest extends JUnit4CacheTestCase {
   }
 
   private String getCreateIndexCommand() throws Exception {
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
@@ -207,7 +203,6 @@ public class LuceneCommandsSecurityDUnitTest extends JUnit4CacheTestCase {
   }
 
   private String getSearchIndexCommand() throws Exception {
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_SEARCH_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
@@ -229,7 +224,6 @@ public class LuceneCommandsSecurityDUnitTest extends JUnit4CacheTestCase {
   }
 
   private String getDestroyIndexCommand() throws Exception {
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESTROY_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
index 7f203ce..9e7b152 100755
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommandsDUnitTest.java
@@ -14,8 +14,16 @@
  */
 package org.apache.geode.cache.lucene.internal.cli;
 
-import junitparams.Parameters;
-import org.apache.geode.cache.*;
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME;
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
+import static org.apache.geode.test.dunit.Assert.assertArrayEquals;
+import static org.apache.geode.test.dunit.Assert.assertEquals;
+import static org.apache.geode.test.dunit.Assert.assertFalse;
+import static org.apache.geode.test.dunit.Assert.assertTrue;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.lucene.LuceneIndex;
 import org.apache.geode.cache.lucene.LuceneQuery;
 import org.apache.geode.cache.lucene.LuceneService;
@@ -25,28 +33,23 @@ import org.apache.geode.cache.lucene.internal.LuceneIndexImpl;
 import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.management.cli.Result.Status;
-import org.apache.geode.management.internal.cli.CommandManager;
 import org.apache.geode.management.internal.cli.commands.CliCommandTestBase;
 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.TabularResultData;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-import org.apache.geode.test.dunit.*;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.junit.categories.DistributedTest;
-import org.awaitility.Awaitility;
-
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.awaitility.Awaitility;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 
-import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.*;
-import static org.apache.geode.test.dunit.Assert.*;
-
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -54,8 +57,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
-
 import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
 
 @Category(DistributedTest.class)
 @RunWith(JUnitParamsRunner.class)
@@ -73,7 +76,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     final VM vm1 = Host.getHost(0).getVM(1);
 
     createIndex(vm1);
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
 
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE_LIST_INDEX__STATS, "true");
@@ -87,7 +89,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     final VM vm1 = Host.getHost(0).getVM(1);
 
     createIndex(vm1);
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
 
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
     String resultAsString = executeCommandAndLogResult(csb);
@@ -99,8 +100,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
   public void listIndexWhenNoExistingIndexShouldReturnNoIndex() throws Exception {
     final VM vm1 = Host.getHost(0).getVM(1);
 
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
-
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
     String resultAsString = executeCommandAndLogResult(csb);
     assertTrue(resultAsString.contains("No lucene indexes found"));
@@ -111,7 +110,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     final VM vm1 = Host.getHost(0).getVM(1);
 
     createIndexWithoutRegion(vm1);
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
 
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE_LIST_INDEX__STATS, "true");
@@ -152,8 +150,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
       getCache();
     });
 
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
-
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
@@ -176,8 +172,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
       getCache();
     });
 
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
-
     List<String> analyzerNames = new ArrayList<>();
     analyzerNames.add(StandardAnalyzer.class.getCanonicalName());
     analyzerNames.add(KeywordAnalyzer.class.getCanonicalName());
@@ -210,8 +204,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
       getCache();
     });
 
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
-
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
@@ -235,7 +227,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
       getCache();
     });
 
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     String analyzerList = StandardAnalyzer.class.getCanonicalName() + ",null,"
         + KeywordAnalyzer.class.getCanonicalName();
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
@@ -265,7 +256,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     final VM vm1 = Host.getHost(0).getVM(1);
 
     createIndex(vm1);
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
 
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESCRIBE_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
@@ -279,7 +269,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
     final VM vm1 = Host.getHost(0).getVM(1);
 
     createIndex(vm1);
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
 
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESCRIBE_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, "notAnIndex");
@@ -560,7 +549,6 @@ public class LuceneIndexCommandsDUnitTest extends CliCommandTestBase {
 
   private CommandResult createAndExecuteDestroyIndexCommand(String indexName, String regionPath)
       throws Exception {
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESTROY_INDEX);
     if (indexName != null) {
       csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, indexName);

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
index a0ac52f..7acff1b 100755
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/configuration/LuceneClusterConfigurationDUnitTest.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.geode.cache.RegionShortcut;
@@ -29,11 +28,9 @@ import org.apache.geode.cache.lucene.LuceneIndex;
 import org.apache.geode.cache.lucene.LuceneService;
 import org.apache.geode.cache.lucene.LuceneServiceProvider;
 import org.apache.geode.cache.lucene.internal.cli.LuceneCliStrings;
-import org.apache.geode.cache.lucene.internal.cli.LuceneIndexCommands;
 import org.apache.geode.cache.lucene.internal.xml.LuceneXmlConstants;
 import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.management.internal.cli.CommandManager;
 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.TabularResultData;
@@ -259,7 +256,6 @@ public class LuceneClusterConfigurationDUnitTest {
 
   private void createLuceneIndexUsingGfsh(String indexName) throws Exception {
     // Execute Gfsh command to create lucene index.
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, indexName);
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
@@ -269,7 +265,6 @@ public class LuceneClusterConfigurationDUnitTest {
 
   private void createLuceneIndexWithAnalyzerUsingGfsh(boolean addGroup) throws Exception {
     // Gfsh command to create lucene index.
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
     csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
     csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
@@ -285,7 +280,6 @@ public class LuceneClusterConfigurationDUnitTest {
 
   private void destroyLuceneIndexUsingGfsh(String indexName) throws Exception {
     // Execute Gfsh command to destroy lucene index.
-    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
     CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESTROY_INDEX);
     if (indexName != null) {
       csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, indexName);


[9/9] geode git commit: GEODE-1597: use Spring shell's parser and delete our own parsing code

Posted by ji...@apache.org.
GEODE-1597: use Spring shell's parser and delete our own parsing code

* Use Spring's SimpleParser as a basis for command parsing
* reworked help/hint
* removing singleton CommandManager


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

Branch: refs/heads/develop
Commit: 1fc0f0ca72705f5ea84ea67f0507bfcad515aacf
Parents: bee0b7d
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue Apr 25 13:51:42 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Apr 27 13:16:24 2017 -0700

----------------------------------------------------------------------
 .../geode/management/cli/CliMetaData.java       |   22 +-
 .../geode/management/cli/ConverterHint.java     |    6 +-
 .../management/internal/cli/CommandManager.java |  514 +-----
 .../internal/cli/CommandResponseBuilder.java    |    5 +-
 .../internal/cli/GfshParseResult.java           |   79 +-
 .../management/internal/cli/GfshParser.java     | 1613 +++---------------
 .../geode/management/internal/cli/Launcher.java |   67 +-
 .../internal/cli/annotation/CliArgument.java    |   81 -
 .../internal/cli/commands/ClientCommands.java   |   25 +-
 .../internal/cli/commands/ConfigCommands.java   |    6 +-
 .../CreateAlterDestroyRegionCommands.java       |   24 +-
 .../internal/cli/commands/DataCommands.java     |   48 +-
 .../internal/cli/commands/DeployCommands.java   |   12 +-
 .../cli/commands/DiskStoreCommands.java         |   43 +-
 .../internal/cli/commands/FunctionCommands.java |    6 +-
 .../internal/cli/commands/GfshHelpCommands.java |   72 +-
 .../cli/commands/LauncherLifecycleCommands.java |   23 +-
 .../cli/commands/MiscellaneousCommands.java     |   16 +-
 .../internal/cli/commands/PDXCommands.java      |   21 +-
 .../internal/cli/commands/QueueCommands.java    |    7 +-
 .../internal/cli/commands/ShellCommands.java    |  332 ++--
 .../internal/cli/commands/WanCommands.java      |   82 +-
 .../cli/converters/BooleanConverter.java        |   54 -
 .../internal/cli/converters/DirConverter.java   |  158 --
 .../internal/cli/converters/EnumConverter.java  |   64 -
 .../internal/cli/converters/HelpConverter.java  |   68 -
 .../cli/converters/HintTopicConverter.java      |   71 -
 .../cli/converters/LogLevelConverter.java       |    2 +-
 .../cli/converters/StringArrayConverter.java    |   53 -
 .../cli/converters/StringListConverter.java     |   56 -
 .../cli/exceptions/CliCommandException.java     |   66 -
 .../exceptions/CliCommandInvalidException.java  |   39 -
 .../CliCommandMultiModeOptionException.java     |   49 -
 .../CliCommandNotAvailableException.java        |   36 -
 .../exceptions/CliCommandOptionException.java   |   65 -
 ...CommandOptionHasMultipleValuesException.java |   47 -
 .../CliCommandOptionInvalidException.java       |   37 -
 .../CliCommandOptionMissingException.java       |   45 -
 .../CliCommandOptionNotApplicableException.java |   46 -
 ...liCommandOptionValueConversionException.java |   38 -
 .../CliCommandOptionValueException.java         |   49 -
 .../CliCommandOptionValueMissingException.java  |   46 -
 .../cli/exceptions/ExceptionGenerator.java      |   48 -
 .../cli/exceptions/ExceptionHandler.java        |   92 -
 .../management/internal/cli/help/CliTopic.java  |  132 --
 .../management/internal/cli/help/HelpBlock.java |   86 +
 .../management/internal/cli/help/Helper.java    |  345 ++++
 .../management/internal/cli/help/Topic.java     |   58 +
 .../internal/cli/help/format/Block.java         |   42 -
 .../internal/cli/help/format/DataNode.java      |   48 -
 .../internal/cli/help/format/Help.java          |   44 -
 .../internal/cli/help/format/NewHelp.java       |   52 -
 .../internal/cli/help/format/Row.java           |   28 -
 .../internal/cli/help/utils/FormatOutput.java   |   33 -
 .../internal/cli/help/utils/HelpUtils.java      |  401 -----
 .../internal/cli/i18n/CliStrings.java           |    7 +-
 .../cli/multistep/CLIMultiStepHelper.java       |   11 +-
 .../internal/cli/parser/Argument.java           |   71 -
 .../internal/cli/parser/AvailabilityTarget.java |  106 --
 .../internal/cli/parser/CommandTarget.java      |  176 --
 .../internal/cli/parser/GfshMethodTarget.java   |  121 --
 .../internal/cli/parser/GfshOptionParser.java   |   37 -
 .../internal/cli/parser/MethodParameter.java    |   39 -
 .../management/internal/cli/parser/Option.java  |  217 ---
 .../internal/cli/parser/OptionSet.java          |  128 --
 .../internal/cli/parser/Parameter.java          |  116 --
 .../internal/cli/parser/ParserUtils.java        |  186 --
 .../internal/cli/parser/SyntaxConstants.java    |   34 -
 .../cli/parser/jopt/JoptOptionParser.java       |  302 ----
 .../preprocessor/EnclosingCharacters.java       |   32 -
 .../cli/parser/preprocessor/Preprocessor.java   |  151 --
 .../parser/preprocessor/PreprocessorUtils.java  |  327 ----
 .../internal/cli/parser/preprocessor/Stack.java |   52 -
 .../cli/parser/preprocessor/TrimmedInput.java   |   44 -
 .../internal/cli/remote/CommandProcessor.java   |   23 +-
 .../management/internal/cli/shell/Gfsh.java     |  516 +++---
 .../cli/shell/GfshExecutionStrategy.java        |    5 +-
 .../internal/cli/shell/MultiCommandHelper.java  |   10 +-
 .../internal/cli/shell/jline/GfshHistory.java   |   21 +-
 .../internal/cli/util/CommandStringBuilder.java |   32 +-
 .../web/controllers/DataCommandsController.java |    6 +-
 .../web/controllers/ExportLogController.java    |    2 +-
 .../internal/cli/CommandManagerJUnitTest.java   |  163 +-
 .../internal/cli/GfshParserIntegrationTest.java |  421 ++++-
 .../internal/cli/GfshParserJUnitTest.java       |  881 +---------
 .../internal/cli/JoptOptionParserTest.java      |  527 ------
 .../cli/annotations/CliArgumentJUnitTest.java   |  154 --
 .../cli/commands/CliCommandTestBase.java        |   85 +-
 ...eateAlterDestroyRegionCommandsDUnitTest.java |    6 +-
 .../cli/commands/DeployCommandsDUnitTest.java   |   13 +-
 .../commands/DiskStoreCommandsDUnitTest.java    |    4 +-
 .../commands/GemfireDataCommandsDUnitTest.java  |   55 +-
 .../commands/HelpCommandsIntegrationTest.java   |  141 --
 .../cli/commands/QueueCommandsDUnitTest.java    |    4 +-
 .../internal/cli/help/HelpBlockUnitTest.java    |   76 +
 .../internal/cli/help/HelperUnitTest.java       |  171 ++
 .../cli/parser/ParserUtilsJUnitTest.java        |   81 -
 .../preprocessor/PreprocessorJUnitTest.java     |  296 ----
 .../PreprocessorUtilsJUnitTest.java             |  121 --
 .../shell/GfshExecutionStrategyJUnitTest.java   |  129 +-
 .../cli/shell/GfshHistoryJUnitTest.java         |   27 +-
 .../internal/cli/shell/GfshJunitTest.java       |   45 +
 .../ClusterConfigDistributionDUnitTest.java     |    3 +-
 .../internal/security/TestCommand.java          |   10 +-
 .../internal/cli/LuceneIndexCommands.java       |   18 +-
 .../lucene/LuceneCommandsSecurityDUnitTest.java |   24 +-
 .../cli/LuceneIndexCommandsDUnitTest.java       |   40 +-
 .../LuceneClusterConfigurationDUnitTest.java    |    6 -
 108 files changed, 2354 insertions(+), 9521 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/cli/CliMetaData.java
----------------------------------------------------------------------
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 e69d78a..2e6dc39 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
@@ -68,29 +68,19 @@ public @interface CliMetaData {
 
   /**
    * String used as a separator when multiple values for a command are specified
+   * 
+   * @deprecated since 1.2, Command methods may override both the delimiter and the escape through
+   *             spring shell's {@code splittingRegex} option context
    */
   String valueSeparator() default org.apache.geode.management.cli.CliMetaData.ANNOTATION_NULL_VALUE;
 
-
-  // TODO - Abhishek - refactor to group this
-  // /**
-  // *
-  // * @since GemFire 8.0
-  // */
-  // @Retention(RetentionPolicy.RUNTIME)
-  // @Target({ ElementType.PARAMETER })
-  // public @interface ParameterMetadata {
-  // /**
-  // * String used as a separator when multiple values for a command are specified
-  // */
-  // String valueSeparator() default CliMetaData.ANNOTATION_NULL_VALUE;
-  // }
-
   /**
    * An annotation to define additional meta-data for availability of commands.
-   * 
+   *
    *
    * @since GemFire 8.0
+   *
+   * @deprecated since Geode1.2, not used at all
    */
   @Retention(RetentionPolicy.RUNTIME)
   @Target({ElementType.METHOD})

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/cli/ConverterHint.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/cli/ConverterHint.java b/geode-core/src/main/java/org/apache/geode/management/cli/ConverterHint.java
index f45abc4..a4b30be 100644
--- a/geode-core/src/main/java/org/apache/geode/management/cli/ConverterHint.java
+++ b/geode-core/src/main/java/org/apache/geode/management/cli/ConverterHint.java
@@ -23,7 +23,6 @@ import org.springframework.shell.core.annotation.CliOption;
  * @since GemFire 8.0
  */
 public interface ConverterHint {
-  public static final String DIRS = "converter.hint.dirs";
   public static final String DIR_PATHSTRING = "converter.hint.dir.path.string";
   public static final String DISKSTORE_ALL = "converter.hint.cluster.diskstore";
   public static final String FILE = "converter.hint.file";
@@ -40,10 +39,7 @@ public interface ConverterHint {
   public static final String LOCATOR_DISCOVERY_CONFIG = "converter.hint.locators.discovery.config";
   public static final String REGIONPATH = "converter.hint.region.path";
   public static final String INDEX_TYPE = "converter.hint.index.type";
-  public static final String STRING_LIST = "converter.hint.list.string";
   public static final String GATEWAY_SENDER_ID = "converter.hint.gateway.senderid";
   public static final String GATEWAY_RECEIVER_ID = "converter.hint.gateway.receiverid";
-  public static final String LOG_LEVEL = "converter.hint.log.levels";
-
-  public static final String STRING_DISABLER = "converter.hint.disable-string-converter";
+  public static final String LOG_LEVEL = "converter.hint.log.levels:disable-string-converter";
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandManager.java
----------------------------------------------------------------------
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 4400445..24b07fa 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
@@ -14,68 +14,81 @@
  */
 package org.apache.geode.management.internal.cli;
 
+import static org.apache.geode.distributed.ConfigurationProperties.USER_COMMAND_PACKAGES;
+
 import org.apache.geode.distributed.ConfigurationProperties;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.internal.cli.annotation.CliArgument;
-import org.apache.geode.management.internal.cli.help.CliTopic;
-import org.apache.geode.management.internal.cli.parser.*;
-import org.apache.geode.management.internal.cli.parser.jopt.JoptOptionParser;
+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.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.springframework.shell.core.annotation.CliOption;
 
 import java.io.IOException;
-import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
-import java.util.*;
-import java.util.Map.Entry;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.ServiceConfigurationError;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.StringTokenizer;
 
 /**
+ *
+ * this only takes care of loading all available command markers and converters from the application
  * 
  * @since GemFire 7.0
  */
 public class CommandManager {
-  // 1. Load Commands, availability indicators - Take from GfshParser
-  // 2. Load Converters - Take from GfshParser
-  // 3. Load Result Converters - Add
-
-  private static final Object INSTANCE_LOCK = new Object();
-  private static CommandManager INSTANCE = null;
   public static final String USER_CMD_PACKAGES_PROPERTY =
       DistributionConfig.GEMFIRE_PREFIX + USER_COMMAND_PACKAGES;
   public static final String USER_CMD_PACKAGES_ENV_VARIABLE = "GEMFIRE_USER_COMMAND_PACKAGES";
+  private static final Object INSTANCE_LOCK = new Object();
 
-  private Properties cacheProperties;
+  private final Helper helper = new Helper();
+
+  private final List<Converter<?>> converters = new ArrayList<Converter<?>>();
+  private final List<CommandMarker> commandMarkers = new ArrayList<>();
 
+  private Properties cacheProperties;
   private LogWrapper logWrapper;
 
-  private CommandManager(final boolean loadDefaultCommands, final Properties cacheProperties)
-      throws ClassNotFoundException, IOException {
+  /**
+   * this constructor is used from Gfsh VM. We are getting the user-command-package from system
+   * environment. used by Gfsh.
+   */
+  public CommandManager() {
+    this(null);
+  }
+
+  /**
+   * this is used when getting the instance in a cache server. We are getting the
+   * user-command-package from distribution properties. used by CommandProcessor.
+   */
+  public CommandManager(final Properties cacheProperties) {
     if (cacheProperties != null) {
       this.cacheProperties = cacheProperties;
     }
-
     logWrapper = LogWrapper.getInstance();
-    if (loadDefaultCommands) {
-      loadCommands();
-
-      if (logWrapper.fineEnabled()) {
-        logWrapper.fine("Commands Loaded: " + commands.keySet());
-        logWrapper
-            .fine("Command Availability Indicators Loaded: " + availabilityIndicators.keySet());
-        logWrapper.fine("Converters Loaded: " + converters);
-      }
+    loadCommands();
+  }
+
+  private static void raiseExceptionIfEmpty(Set<Class<?>> foundClasses, String errorFor)
+      throws IllegalStateException {
+    if (foundClasses == null || foundClasses.isEmpty()) {
+      throw new IllegalStateException(
+          "Required " + errorFor + " classes were not loaded. Check logs for errors.");
     }
   }
 
-  private void loadUserCommands() throws ClassNotFoundException, IOException {
+  private void loadUserCommands() {
     final Set<String> userCommandPackages = new HashSet<String>();
 
     // Find by packages specified by the system property
@@ -122,12 +135,8 @@ public class CommandManager {
           }
         }
         raiseExceptionIfEmpty(foundClasses, "User Command");
-      } catch (ClassNotFoundException e) {
+      } catch (ClassNotFoundException | IOException e) {
         logWrapper.warning("Could not load User Commands due to " + e.getLocalizedMessage());
-        throw e;
-      } catch (IOException e) {
-        logWrapper.warning("Could not load User Commands due to " + e.getLocalizedMessage());
-        throw e;
       } catch (IllegalStateException e) {
         logWrapper.warning(e.getMessage(), e);
         throw e;
@@ -137,7 +146,7 @@ public class CommandManager {
 
   /**
    * Loads commands via {@link ServiceLoader} from {@link ClassPathLoader}.
-   * 
+   *
    * @since GemFire 8.1
    */
   private void loadPluginCommands() {
@@ -158,7 +167,7 @@ public class CommandManager {
     }
   }
 
-  private void loadCommands() throws ClassNotFoundException, IOException {
+  private void loadCommands() {
     loadUserCommands();
 
     loadPluginCommands();
@@ -166,6 +175,7 @@ public class CommandManager {
     // CommandMarkers
     Set<Class<?>> foundClasses = null;
     try {
+      // geode's commands
       foundClasses = ClasspathScanLoadHelper.loadAndGet(
           "org.apache.geode.management.internal.cli.commands", CommandMarker.class, true);
       for (Class<?> klass : foundClasses) {
@@ -177,12 +187,13 @@ public class CommandManager {
         }
       }
       raiseExceptionIfEmpty(foundClasses, "Commands");
+
+      // do not add Spring shell's commands for now. When we add it, we need to tell the parser that
+      // these are offline commands.
     } catch (ClassNotFoundException e) {
       logWrapper.warning("Could not load Commands due to " + e.getLocalizedMessage());
-      throw e;
     } catch (IOException e) {
       logWrapper.warning("Could not load Commands due to " + e.getLocalizedMessage());
-      throw e;
     } catch (IllegalStateException e) {
       logWrapper.warning(e.getMessage(), e);
       throw e;
@@ -201,26 +212,13 @@ public class CommandManager {
         }
       }
       raiseExceptionIfEmpty(foundClasses, "Converters");
-    } catch (ClassNotFoundException e) {
-      logWrapper.warning("Could not load Converters due to " + e.getLocalizedMessage());
-      throw e;
-    } catch (IOException e) {
-      logWrapper.warning("Could not load Converters due to " + e.getLocalizedMessage());
-      throw e;
-    } catch (IllegalStateException e) {
-      logWrapper.warning(e.getMessage(), e);
-      throw e;
-    }
 
-    // Roo's Converters
-    try {
+      // Spring shell's converters
       foundClasses = ClasspathScanLoadHelper.loadAndGet("org.springframework.shell.converters",
           Converter.class, true);
       for (Class<?> klass : foundClasses) {
         try {
-          if (!SHL_CONVERTERS_TOSKIP.contains(klass.getName())) {
-            add((Converter<?>) klass.newInstance());
-          }
+          add((Converter<?>) klass.newInstance());
         } catch (Exception e) {
           logWrapper.warning(
               "Could not load Converter from: " + klass + " due to " + e.getLocalizedMessage()); // continue
@@ -228,417 +226,71 @@ public class CommandManager {
       }
       raiseExceptionIfEmpty(foundClasses, "Basic Converters");
     } catch (ClassNotFoundException e) {
-      logWrapper.warning("Could not load Default Converters due to " + e.getLocalizedMessage());// TODO
-                                                                                                // -
-                                                                                                // Abhishek:
-                                                                                                // Should
-                                                                                                // these
-                                                                                                // converters
-                                                                                                // be
-                                                                                                // moved
-                                                                                                // in
-                                                                                                // GemFire?
-      throw e;
+      logWrapper.warning("Could not load Converters due to " + e.getLocalizedMessage());
     } catch (IOException e) {
-      logWrapper.warning("Could not load Default Converters due to " + e.getLocalizedMessage());// TODO
-                                                                                                // -
-                                                                                                // Abhishek:
-                                                                                                // Should
-                                                                                                // these
-                                                                                                // converters
-                                                                                                // be
-                                                                                                // moved
-                                                                                                // in
-                                                                                                // GemFire?
-      throw e;
+      logWrapper.warning("Could not load Converters due to " + e.getLocalizedMessage());
     } catch (IllegalStateException e) {
       logWrapper.warning(e.getMessage(), e);
       throw e;
     }
   }
 
-  private static void raiseExceptionIfEmpty(Set<Class<?>> foundClasses, String errorFor)
-      throws IllegalStateException {
-    if (foundClasses == null || foundClasses.isEmpty()) {
-      throw new IllegalStateException(
-          "Required " + errorFor + " classes were not loaded. Check logs for errors.");
-    }
-  }
-
-  public static CommandManager getInstance() throws ClassNotFoundException, IOException {
-    return getInstance(true);
-  }
-
-  public static CommandManager getInstance(Properties cacheProperties)
-      throws ClassNotFoundException, IOException {
-    return getInstance(true, cacheProperties);
-  }
-
-  // For testing.
-  public static void clearInstance() {
-    synchronized (INSTANCE_LOCK) {
-      INSTANCE = null;
-    }
-  }
-
-  // This method exists for test code use only ...
-  /* package */static CommandManager getInstance(boolean loadDefaultCommands)
-      throws ClassNotFoundException, IOException {
-    return getInstance(loadDefaultCommands, null);
-  }
-
-  private static CommandManager getInstance(boolean loadDefaultCommands, Properties cacheProperties)
-      throws ClassNotFoundException, IOException {
-    synchronized (INSTANCE_LOCK) {
-      if (INSTANCE == null) {
-        INSTANCE = new CommandManager(loadDefaultCommands, cacheProperties);
-      }
-      return INSTANCE;
-    }
+  public List<Converter<?>> getConverters() {
+    return converters;
   }
 
-  public static CommandManager getExisting() {
-    // if (INSTANCE == null) {
-    // throw new IllegalStateException("CommandManager doesn't exist.");
-    // }
-    return INSTANCE;
+  public List<CommandMarker> getCommandMarkers() {
+    return commandMarkers;
   }
 
-  /** Skip some of the Converters from Spring Shell for our customization */
-  private static List<String> SHL_CONVERTERS_TOSKIP = new ArrayList<String>();
-  static {
-    // Over-ridden by cggm.internal.cli.converters.BooleanConverter
-    SHL_CONVERTERS_TOSKIP.add("org.springframework.shell.converters.BooleanConverter");
-    // Over-ridden by cggm.internal.cli.converters.EnumConverter
-    SHL_CONVERTERS_TOSKIP.add("org.springframework.shell.converters.EnumConverter");
-  }
-
-  /**
-   * List of converters which should be populated first before any command can be added
-   */
-  private final List<Converter<?>> converters = new ArrayList<Converter<?>>();
-
-  /**
-   * Map of command string and actual CommandTarget object
-   * 
-   * This map can also be implemented as a trie to support command abbreviation
-   */
-  private final Map<String, CommandTarget> commands = new TreeMap<String, CommandTarget>();
-
-  /**
-   * This method will store the all the availabilityIndicators
-   */
-  private final Map<String, AvailabilityTarget> availabilityIndicators =
-      new HashMap<String, AvailabilityTarget>();
-
-  /**
-   */
-  private final Map<String, CliTopic> topics = new TreeMap<String, CliTopic>();
-
   /**
    * Method to add new Converter
-   * 
+   *
    * @param converter
    */
-  public void add(Converter<?> converter) {
+  void add(Converter<?> converter) {
     converters.add(converter);
   }
 
   /**
    * Method to add new Commands to the parser
-   * 
+   *
    * @param commandMarker
    */
-  public void add(CommandMarker commandMarker) {
-    // First we need to find out all the methods marked with
-    // Command annotation
-    Method[] methods = commandMarker.getClass().getMethods();
-    for (Method method : methods) {
-      if (method.getAnnotation(CliCommand.class) != null) {
-
-        //
-        // First Build the option parser
-        //
-
-        // Create the empty LinkedLists for storing the argument and
-        // options
-        LinkedList<Argument> arguments = new LinkedList<Argument>();
-        LinkedList<Option> options = new LinkedList<Option>();
-        // Also we need to create the OptionParser for each command
-        GfshOptionParser optionParser = getOptionParser();
-        // Now get all the parameters annotations of the method
-        Annotation[][] parameterAnnotations = method.getParameterAnnotations();
-        // Also get the parameter Types
-        Class<?>[] parameterTypes = method.getParameterTypes();
-
-        int parameterNo = 0;
-
-        for (int i = 0; i < parameterAnnotations.length; i++) {
-          // Get all the annotations for this specific parameter
-          Annotation[] annotations = parameterAnnotations[i];
-          // Also get the parameter type for this parameter
-          Class<?> parameterType = parameterTypes[i];
-
-          boolean paramFound = false;
-          String valueSeparator = CliMetaData.ANNOTATION_NULL_VALUE;
-          for (Annotation annotation : annotations) {
-            if (annotation instanceof CliArgument) {
-              // Here we need to create the argument Object
-              Argument argumentToAdd =
-                  createArgument((CliArgument) annotation, parameterType, parameterNo);
-              arguments.add(argumentToAdd);
-              parameterNo++;
-            } else if (annotation instanceof CliOption) {
-              Option createdOption =
-                  createOption((CliOption) annotation, parameterType, parameterNo);
-              if (!CliMetaData.ANNOTATION_NULL_VALUE.equals(valueSeparator)) { // CliMetaData was
-                                                                               // found earlier
-                createdOption.setValueSeparator(valueSeparator);
-
-                // reset valueSeparator back to null
-                valueSeparator = CliMetaData.ANNOTATION_NULL_VALUE;
-              } else { // CliMetaData is yet to be found
-                paramFound = true;
-              }
-              options.add(createdOption);
-              parameterNo++;
-            } else if (annotation instanceof CliMetaData) {
-              valueSeparator = ((CliMetaData) annotation).valueSeparator();
-              if (!CliMetaData.ANNOTATION_NULL_VALUE.equals(valueSeparator)) {
-                if (paramFound) { // CliOption was detected earlier
-                  Option lastAddedOption = options.getLast();
-                  lastAddedOption.setValueSeparator(valueSeparator);
-                  // reset valueSeparator back to null
-                  valueSeparator = CliMetaData.ANNOTATION_NULL_VALUE;
-                } // param not found yet, store valueSeparator value
-              } else {
-                // reset valueSeparator back to null
-                valueSeparator = CliMetaData.ANNOTATION_NULL_VALUE;
-              }
-            }
-          }
-        }
-        optionParser.setArguments(arguments);
-        optionParser.setOptions(options);
-
-        //
-        // Now build the commandTarget
-        //
-
-        // First build the MethodTarget for the command Method
-        GfshMethodTarget gfshMethodTarget = new GfshMethodTarget(method, commandMarker);
-
-        // Fetch the value array from the cliCommand annotation
-        CliCommand cliCommand = method.getAnnotation(CliCommand.class);
-        String[] values = cliCommand.value();
-
-        // First string will point to the command
-        // rest of them will act as synonyms
-        String commandName = null;
-        String[] synonyms = null;
-        if (values.length > 1) {
-          synonyms = new String[values.length - 1];
-        }
-
-        commandName = values[0];
-
-        for (int j = 1; j < values.length; j++) {
-          synonyms[j - 1] = values[j];
-        }
-
-        // Create the commandTarget object
-        CommandTarget commandTarget = new CommandTarget(commandName, synonyms, gfshMethodTarget,
-            optionParser, null, cliCommand.help());
-
-        // Now for each string in values put an entry in the commands
-        // map
-        for (String string : values) {
-          if (commands.get(string) == null) {
-            commands.put(string, commandTarget);
-          } else {
-            // TODO Handle collision
-            logWrapper.info("Multiple commands configured with the same name: " + string);
-          }
-        }
-
-        if (CliUtil.isGfshVM()) {
-          CliMetaData commandMetaData = method.getAnnotation(CliMetaData.class);
-          if (commandMetaData != null) {
-            String[] relatedTopics = commandMetaData.relatedTopic();
-            // System.out.println("relatedTopic :: "+Arrays.toString(relatedTopics));
-            for (String topicName : relatedTopics) {
-              CliTopic topic = topics.get(topicName);
-              if (topic == null) {
-                topic = new CliTopic(topicName);
-                topics.put(topicName, topic);
-              }
-              topic.addCommandTarget(commandTarget);
-            }
-          }
-        }
-
-      } else if (method.getAnnotation(CliAvailabilityIndicator.class) != null) {
-        // Now add this availability Indicator to the list of
-        // availability Indicators
-        CliAvailabilityIndicator cliAvailabilityIndicator =
-            method.getAnnotation(CliAvailabilityIndicator.class);
-
-        // Create a AvailabilityTarget for this availability Indicator
-        AvailabilityTarget availabilityIndicator = new AvailabilityTarget(commandMarker, method);
-
-        String[] value = cliAvailabilityIndicator.value();
-        for (String string : value) {
-          availabilityIndicators.put(string, availabilityIndicator);
-        }
-
+  void add(CommandMarker commandMarker) {
+    commandMarkers.add(commandMarker);
+    for (Method method : commandMarker.getClass().getMethods()) {
+      CliCommand cliCommand = method.getAnnotation(CliCommand.class);
+      CliAvailabilityIndicator availability = method.getAnnotation(CliAvailabilityIndicator.class);
+      if (cliCommand == null && availability == null) {
+        continue;
       }
-    }
-    // Now we must update all the existing CommandTargets to add
-    // this availability Indicator if it applies to them
-    updateAvailabilityIndicators();
-  }
-
 
-  /**
-   * Will update all the references to availability Indicators for commands
-   * 
-   */
-  public void updateAvailabilityIndicators() {
-    for (String string : availabilityIndicators.keySet()) {
-      CommandTarget commandTarget = commands.get(string);
-      if (commandTarget != null) {
-        commandTarget.setAvailabilityIndicator(availabilityIndicators.get(string));
+      if (cliCommand != null) {
+        helper.addCommand(cliCommand, method);
       }
-    }
-  }
 
-  /**
-   * Creates a new {@link Option} instance
-   * 
-   * @param cliOption
-   * @param parameterType
-   * @param parameterNo
-   * @return Option
-   */
-  public Option createOption(CliOption cliOption, Class<?> parameterType, int parameterNo) {
-    Option option = new Option();
-
-    // First set the Option identifiers
-    List<String> synonyms = new ArrayList<String>();
-    for (String string : cliOption.key()) {
-      if (!option.setLongOption(string)) {
-        synonyms.add(string);
+      if (availability != null) {
+        helper.addAvailabilityIndicator(availability, new MethodTarget(method, commandMarker));
       }
     }
-    option.setSynonyms(synonyms);
-    if (!(option.getAggregate().size() > 0)) {
-      logWrapper.warning("Option should have a name");
-    }
-    // Set the option Help
-    option.setHelp(cliOption.help());
-
-    // Set whether the option is required or not
-    option.setRequired(cliOption.mandatory());
-
-    // Set the fields related to option value
-    option.setSystemProvided(cliOption.systemProvided());
-    option.setSpecifiedDefaultValue(cliOption.specifiedDefaultValue());
-    option.setUnspecifiedDefaultValue(cliOption.unspecifiedDefaultValue());
-
-    // Set the things which are useful for value conversion and
-    // auto-completion
-    option.setContext(cliOption.optionContext());
-    // Find the matching Converter<?> for this option
-    option.setConverter(getConverter(parameterType, option.getContext()));
-
-    option.setDataType(parameterType);
-    option.setParameterNo(parameterNo);
-    return option;
   }
 
-  /**
-   * Creates a new {@link Argument} instance
-   * 
-   * @param cliArgument
-   * @param parameterType
-   * @param parameterNo
-   * @return Argument
-   */
-  public Argument createArgument(CliArgument cliArgument, Class<?> parameterType, int parameterNo) {
-    Argument argument = new Argument();
-    argument.setArgumentName(cliArgument.name());
-    argument.setContext(cliArgument.argumentContext());
-    argument.setConverter(getConverter(parameterType, argument.getContext()));
-    argument.setHelp(cliArgument.help());
-    argument.setRequired(cliArgument.mandatory());
-    argument.setDataType(parameterType);
-    argument.setParameterNo(parameterNo);
-    argument.setUnspecifiedDefaultValue(cliArgument.unspecifiedDefaultValue());
-    argument.setSystemProvided(cliArgument.systemProvided());
-    return argument;
+  public Helper getHelper() {
+    return helper;
   }
 
-  /**
-   * Looks for a matching {@link Converter}
-   * 
-   * @param parameterType
-   * @param context
-   * @return {@link Converter}
-   */
-  public Converter<?> getConverter(Class<?> parameterType, String context) {
-    for (Converter<?> converter : converters) {
-      if (converter.supports(parameterType, context)) {
-        return converter;
-      }
+  public String obtainHelp(String buffer) {
+    int terminalWidth = -1;
+    Gfsh gfsh = Gfsh.getCurrentInstance();
+    if (gfsh != null) {
+      terminalWidth = gfsh.getTerminalWidth();
     }
-    return null;
+    return helper.getHelp(buffer, terminalWidth);
   }
 
-  /**
-   * For the time being this method returns a {@link JoptOptionParser} object but in the future we
-   * can change which optionParser should be returned.
-   * 
-   * @return {@link GfshOptionParser}
-   */
-  private GfshOptionParser getOptionParser() {
-    return new JoptOptionParser();
+  public String obtainHint(String topic) {
+    return helper.getHint(topic);
   }
 
-  /**
-   * @return the commands
-   */
-  public Map<String, CommandTarget> getCommands() {
-    return Collections.unmodifiableMap(commands);
-  }
-
-  AvailabilityTarget getAvailabilityIndicator(Object key) {
-    return availabilityIndicators.get(key);
-  }
-
-  public Set<String> getTopicNames() {
-    Set<String> topicsNames = topics.keySet();
-    return Collections.unmodifiableSet(topicsNames);
-  }
-
-  public List<CliTopic> getTopics() {
-    List<CliTopic> topicsList = new ArrayList<CliTopic>(topics.values());
-    return Collections.unmodifiableList(topicsList);
-  }
-
-  public CliTopic getTopic(String topicName) {
-    CliTopic foundTopic = topics.get(topicName);
-
-    if (foundTopic == null) {
-      Set<Entry<String, CliTopic>> entries = topics.entrySet();
-
-      for (Entry<String, CliTopic> entry : entries) {
-        if (entry.getKey().equalsIgnoreCase(topicName)) {
-          foundTopic = entry.getValue();
-          break;
-        }
-      }
-    }
-
-    return foundTopic;
-  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandResponseBuilder.java
----------------------------------------------------------------------
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 bda030d..3f8f20d 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
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.management.internal.cli;
 
+import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.internal.cli.json.GfJsonException;
 import org.apache.geode.management.internal.cli.json.GfJsonObject;
 import org.apache.geode.management.internal.cli.remote.CommandExecutionContext;
@@ -24,8 +25,6 @@ import org.apache.geode.management.internal.cli.result.CommandResult;
  * @since GemFire 7.0
  */
 public class CommandResponseBuilder {
-  // Command Response Constants
-  private static final String NO_TOKEN_ACCESSOR = "__NULL__";
 
   public static CommandResponse prepareCommandResponse(String memberName, CommandResult result) {
     GfJsonObject content = null;
@@ -42,7 +41,7 @@ public class CommandResponseBuilder {
         getType(result), // contentType
         result.getStatus().getCode(), // status code
         "1/1", // page --- TODO - Abhishek - define a scrollable ResultData
-        NO_TOKEN_ACCESSOR, // tokenAccessor for next results
+        CliMetaData.ANNOTATION_NULL_VALUE, // tokenAccessor for next results
         getDebugInfo(result), // debugData
         result.getHeader(), // header
         content, // content

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParseResult.java
----------------------------------------------------------------------
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 d879e2d..e91e365 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
@@ -14,17 +14,20 @@
  */
 package org.apache.geode.management.internal.cli;
 
+import org.apache.commons.lang.StringUtils;
+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;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+import org.springframework.shell.event.ParseResult;
+
+import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.springframework.shell.event.ParseResult;
-
-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;
-
 /**
  * Immutable representation of the outcome of parsing a given shell line. * Extends
  * {@link ParseResult} to add a field to specify the command string that was input by the user.
@@ -41,7 +44,7 @@ import org.apache.geode.management.internal.cli.shell.OperationInvoker;
 public class GfshParseResult extends ParseResult {
   private String userInput;
   private String commandName;
-  private Map<String, String> paramValueStringMap;
+  private Map<String, String> paramValueStringMap = new HashMap<>();
 
   /**
    * Creates a GfshParseResult instance to represent parsing outcome.
@@ -52,12 +55,40 @@ public class GfshParseResult extends ParseResult {
    * @param userInput user specified commands string
    */
   protected GfshParseResult(final Method method, final Object instance, final Object[] arguments,
-      final String userInput, final String commandName,
-      final Map<String, String> parametersAsString) {
+      final String userInput) {
     super(method, instance, arguments);
-    this.userInput = userInput;
-    this.commandName = commandName;
-    this.paramValueStringMap = new HashMap<String, String>(parametersAsString);
+    this.userInput = userInput.trim();
+
+    CliCommand cliCommand = method.getAnnotation(CliCommand.class);
+    commandName = cliCommand.value()[0];
+
+    Annotation[][] parameterAnnotations = method.getParameterAnnotations();
+    if (arguments == null) {
+      return;
+    }
+
+    for (int i = 0; i < arguments.length; i++) {
+      Object argument = arguments[i];
+      if (argument == null) {
+        continue;
+      }
+
+      CliOption cliOption = getCliOption(parameterAnnotations, i);
+
+      String argumentAsString;
+      if (argument instanceof Object[]) {
+        argumentAsString = StringUtils.join((Object[]) argument, ",");
+      } 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(" ")) {
+        argumentAsString = "'" + argumentAsString + "'";
+      }
+      paramValueStringMap.put(cliOption.key()[0], argumentAsString);
+    }
   }
 
   /**
@@ -67,6 +98,9 @@ public class GfshParseResult extends ParseResult {
     return userInput;
   }
 
+  public String getParamValue(String param) {
+    return paramValueStringMap.get(param);
+  }
 
   /**
    * @return the unmodifiable paramValueStringMap
@@ -75,18 +109,17 @@ public class GfshParseResult extends ParseResult {
     return Collections.unmodifiableMap(paramValueStringMap);
   }
 
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append(GfshParseResult.class.getSimpleName());
-    builder.append(" [method=").append(getMethod());
-    builder.append(", instance=").append(getInstance());
-    builder.append(", arguments=").append(CliUtil.arrayToString(getArguments()));
-    builder.append("]");
-    return builder.toString();
-  }
-
   public String getCommandName() {
     return commandName;
   }
+
+  private CliOption getCliOption(Annotation[][] parameterAnnotations, int index) {
+    Annotation[] annotations = parameterAnnotations[index];
+    for (Annotation annotation : annotations) {
+      if (annotation instanceof CliOption) {
+        return (CliOption) annotation;
+      }
+    }
+    return null;
+  }
 }


[8/9] geode git commit: GEODE-1597: use Spring shell's parser and delete our own parsing code

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParser.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParser.java
index a1d03e4..20ae022 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParser.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/GfshParser.java
@@ -15,49 +15,20 @@
 package org.apache.geode.management.internal.cli;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.geode.management.cli.CommandProcessingException;
-import org.apache.geode.management.internal.cli.exceptions.CliCommandMultiModeOptionException;
-import org.apache.geode.management.internal.cli.exceptions.CliCommandOptionException;
-import org.apache.geode.management.internal.cli.exceptions.CliException;
-import org.apache.geode.management.internal.cli.exceptions.ExceptionHandler;
-import org.apache.geode.management.internal.cli.help.format.NewHelp;
-import org.apache.geode.management.internal.cli.help.utils.HelpUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.modes.CommandModes;
-import org.apache.geode.management.internal.cli.modes.CommandModes.CommandMode;
-import org.apache.geode.management.internal.cli.parser.Argument;
-import org.apache.geode.management.internal.cli.parser.AvailabilityTarget;
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.GfshMethodTarget;
-import org.apache.geode.management.internal.cli.parser.MethodParameter;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-import org.apache.geode.management.internal.cli.parser.Parameter;
-import org.apache.geode.management.internal.cli.parser.ParserUtils;
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
-import org.apache.geode.management.internal.cli.parser.preprocessor.PreprocessorUtils;
-import org.apache.geode.management.internal.cli.parser.preprocessor.TrimmedInput;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.cli.util.CLIConsoleBufferUtil;
-import org.springframework.shell.core.AbstractShell;
+import org.springframework.shell.converters.ArrayConverter;
+import org.springframework.shell.core.CommandMarker;
 import org.springframework.shell.core.Completion;
 import org.springframework.shell.core.Converter;
-import org.springframework.shell.core.MethodTarget;
 import org.springframework.shell.core.Parser;
+import org.springframework.shell.core.SimpleParser;
 import org.springframework.shell.event.ParseResult;
 
-import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
+import java.util.Properties;
 import java.util.logging.Logger;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 /**
@@ -65,1469 +36,299 @@ import java.util.regex.Pattern;
  *
  * @since GemFire 7.0
  */
-public class GfshParser implements Parser {
+public class GfshParser extends SimpleParser {
 
   public static final String LINE_SEPARATOR = System.getProperty("line.separator");
-
-  // Constants used while finding command targets for help
-  private final static Short EXACT_TARGET = (short) 0;
-  private final static Short MATCHING_TARGETS = (short) 1;
+  public static final String OPTION_VALUE_SPECIFIER = "=";
+  public static final String OPTION_SEPARATOR = " ";
+  public static final String SHORT_OPTION_SPECIFIER = "-";
+  public static final String LONG_OPTION_SPECIFIER = "--";
+  public static final String COMMAND_DELIMITER = ";";
+  public static final String CONTINUATION_CHARACTER = "\\";
 
   // Make use of LogWrapper
   private static final LogWrapper logWrapper = LogWrapper.getInstance();
 
-  // private CliStringResourceBundle cliStringBundle;
-  private CommandManager commandManager;
-
+  // pattern used to split the user input with whitespaces except those in quotes (single or double)
+  private static Pattern PATTERN =
+      Pattern.compile("\\s*([^\\s']*)'([^']*)'\\s+|\\s*([^\\s\"]*)\"([^\"]*)\"\\s+|\\S+");
   /**
    * Used for warning messages
    */
-  // TODO Investigating using GemFire logging.
-  private final Logger consoleLogger;
+  private Logger consoleLogger;
+
+  private CommandManager commandManager = null;
+
+  public GfshParser() {
+    this(null);
+  }
 
-  public GfshParser(CommandManager commandManager) {
-    // cliStringBundle = new
-    // CliStringResourceBundle("org/apache/geode/management/internal/cli/i18n/CliStringResourceBundle");
-    this.commandManager = commandManager;
+  public GfshParser(Properties cacheProperties) {
     if (CliUtil.isGfshVM()) {
       consoleLogger = Logger.getLogger(this.getClass().getCanonicalName());
     } else {
       consoleLogger = logWrapper.getLogger();
     }
-  }
-
-  // ///////////////// Parser interface Methods Start //////////////////////////
-  // ////////////////////// Implemented Methods ////////////////////////////////
 
-  /**
-   * Populates a list of completion candidates. See {@link Parser#complete(String, int, List)} for
-   * details.
-   *
-   * @param buffer
-   * @param cursor
-   * @param completionCandidates
-   *
-   * @return new cursor position
-   */
-  public int complete(String buffer, int cursor, List<String> completionCandidates) {
-    final List<Completion> candidates = new ArrayList<Completion>();
-    final int result = completeAdvanced(buffer, cursor, candidates);
-    for (final Completion completion : candidates) {
-      completionCandidates.add(completion.getValue());
-    }
-    return result;
-  }
-
-  /**
-   * Populates a list of completion candidates.
-   *
-   * @param buffer
-   * @param cursor
-   * @param completionCandidates
-   *
-   * @return new cursor position
-   */
-  public int completeAdvanced(String buffer, int cursor, List<Completion> completionCandidates) {
-    // Currently, support for auto-completion
-    // in between is not supported, only if the
-    // cursor is at the end
+    this.commandManager = new CommandManager(cacheProperties);
 
-    if (cursor <= buffer.length() - 1
-        && !PreprocessorUtils.containsOnlyWhiteSpaces(buffer.substring(cursor))
-        || (ParserUtils.contains(buffer, SyntaxConstants.COMMAND_DELIMITER))) {
-      return cursor;
+    for (CommandMarker command : commandManager.getCommandMarkers()) {
+      add(command);
     }
 
-    int desiredCursorPosition = 0;
-
-    try {
-      TrimmedInput simpleTrim = PreprocessorUtils.simpleTrim(buffer);
-      desiredCursorPosition += simpleTrim.getNoOfSpacesRemoved();
-      List<CommandTarget> targets = locateTargets(simpleTrim.getString());
-      if (targets.size() > 1) {
-        String padding =
-            desiredCursorPosition != 0 ? ParserUtils.getPadding(desiredCursorPosition) : "";
-        // This means that what the user has entered matches
-        // the beginning of many commands
-        for (CommandTarget commandTarget : targets) {
-          completionCandidates
-              .add(new Completion(padding + commandTarget.getGfshMethodTarget().getKey()));
-        }
-      } else {
-        if (targets.size() == 1) {
-          CommandTarget commandTarget = targets.get(0);
-          // Only one command matches but we still have to check
-          // whether the user has properly entered it or not
-          if (simpleTrim.getString().length() >= commandTarget.getGfshMethodTarget().getKey()
-              .length()) {
-            /* int position = */
-            return completeParameters(commandTarget,
-                desiredCursorPosition + commandTarget.getGfshMethodTarget().getKey().length(),
-                commandTarget.getGfshMethodTarget().getRemainingBuffer(), cursor,
-                completionCandidates);
-            /*
-             * updateCompletionCandidates(completionCandidates, buffer, position); return 0;
-             */
-          } else {
-            String padding =
-                desiredCursorPosition != 0 ? ParserUtils.getPadding(desiredCursorPosition) : "";
-            // User has still not entered the command name properly,
-            // we need to populate the completionCandidates list
-            completionCandidates
-                .add(new Completion(padding + commandTarget.getGfshMethodTarget().getKey()));
-          }
-        }
+    List<Converter<?>> converters = commandManager.getConverters();
+    for (Converter<?> converter : converters) {
+      if (converter.getClass().isAssignableFrom(ArrayConverter.class)) {
+        ArrayConverter arrayConverter = (ArrayConverter) converter;
+        arrayConverter.setConverters(new HashSet<>(converters));
       }
-
-    } catch (IllegalArgumentException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    } catch (IllegalAccessException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    } catch (InvocationTargetException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    } catch (RuntimeException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
+      add(converter);
     }
-    // Returning 0 for exceptions too. This will break the completors' loop in
-    // jline.ConsoleReader.complete() & will return false
-    return 0;
   }
 
-  @SuppressWarnings("unused")
-  private void updateCompletionCandidates(List<Completion> completionCandidates, String buffer,
-      int position) {
-    List<Completion> temp = new ArrayList<Completion>();
-    while (completionCandidates.size() > 0) {
-      temp.add(completionCandidates.remove(0));
-    }
-    for (Completion completion : temp) {
-      completionCandidates.add(new Completion(buffer.substring(0, position) + completion.getValue(),
-          completion.getFormattedValue(), completion.getHeading(), completion.getOrder()));
-    }
+  public CommandManager getCommandManager() {
+    return commandManager;
   }
 
-  private int completeParameters(CommandTarget commandTarget, int cursorStart,
-      String remainingBuffer, int cursor, List<Completion> completionCandidates) {
-    int desiredCursorPosition = cursorStart;
-    // Factor for remainingBuffer
-    boolean sizeReduced = false;
-    // We need to work modify the flow according to the CliException
-    // generated. For that we will need a reference to the Exception
-    // CliException reference
-    CliCommandOptionException coe = null;
-    OptionSet userOptionSet = null;
-    try {
-      // We need to remove the space which separates command from the
-      // parameters
-      if (remainingBuffer.length() > 0) {
-        remainingBuffer = remainingBuffer.substring(1);
-        sizeReduced = true;
-      }
-
-      userOptionSet = commandTarget.getOptionParser().parse(remainingBuffer);
-    } catch (CliException ce) {
-      if (ce instanceof CliCommandOptionException) {
-        coe = (CliCommandOptionException) ce;
-        coe.setCommandTarget(commandTarget);
-        userOptionSet = coe.getOptionSet();
-      }
-    }
-
-    // Contains mandatory options which have not been specified
-    List<Option> mandatoryOptions = new ArrayList<Option>();
-    // Contains non-mandatory options which have not been specified
-    List<Option> unspecifiedOptions = new ArrayList<Option>();
-    // First we need a list to create a list of all the options specified
-    Map<String, Option> optionsPresentMap = new LinkedHashMap<String, Option>();
-    if (userOptionSet != null) {
-
-      // Start with the arguments
-      String argumentSeparator = " ";
-      for (Argument argument : commandTarget.getOptionParser().getArguments()) {
-        if (completionCandidates.size() == 0) {
-          boolean warning = false;
-          if (userOptionSet.hasArgument(argument)) {
-            boolean incrementCursor = true;
-            // Here we need to get all the possible values for this
-            // argument
-            if (getAllPossibleValuesForParameter(completionCandidates, argument,
-                userOptionSet.getValue(argument), commandTarget.getGfshMethodTarget())) {
-              // Check whether the list of completionCandidates is
-              // not empty
-              if (completionCandidates.size() > 0) {
-                // First check whether the argument value
-                // matches with any
-                // of the completionCandidates
-                if (perfectMatch(completionCandidates, userOptionSet.getValue(argument))) {
-                  // Remove all the completionCandidates
-                  completionCandidates.clear();
-                } else {
-                  modifyCompletionCandidates(completionCandidates, argumentSeparator,
-                      userOptionSet.getValue(argument));
-                  // For this case also we should not
-                  // increment the
-                  // cursorPosition
-                  if (completionCandidates.size() > 0) {
-                    incrementCursor = false;
-                  }
-                }
-              }
-            } else {
-              // The completion candidates should be cleared if the Converter has
-              // populated it with some values
-              completionCandidates.clear();
-            }
-            if (incrementCursor) {
-              desiredCursorPosition +=
-                  userOptionSet.getValue(argument).length() + argumentSeparator.length();
-            }
-          } else {
-            if (argument.isRequired()) {
-              // Here the converter will come in handy
-              // to get suggestion for arguments
-              if (getAllPossibleValuesForParameter(completionCandidates, argument, null,
-                  commandTarget.getGfshMethodTarget())) {
-                if (completionCandidates.size() == 0) {
-                  // Enable warning if nothing is returned
-                  warning = true;
-                }
-              } else {
-                // The completion candidates should be cleared if the Converter has
-                // populated it with some values
-                completionCandidates.clear();
-                warning = true;
-              }
-            } else {
-              boolean checkForPossibleValues = true;
-              // This means that the argument is not mandatory
-              // Now here we need to check whether user wants to
-              // enter an option.
-              if (endsWithOptionSpecifiers(userOptionSet.getUserInput())
-                  || hasOptionSpecified(userOptionSet.getUserInput())) {
-                // This means options have started, and we
-                // cannot have arguments after options
-                // So, we just skip checking for possible
-                // values
-                checkForPossibleValues = false;
-              }
-              // Just try getting the PossibleValues without
-              // aiming
-              if (checkForPossibleValues) {
-                getAllPossibleValuesForParameter(completionCandidates, argument, null,
-                    commandTarget.getGfshMethodTarget());
-              }
-            }
-            if (completionCandidates.size() > 0) {
-              modifyCompletionCandidates(completionCandidates, argumentSeparator, (String[]) null);
-            }
-          }
-          if (warning) {
-            String argMessage = argument.getArgumentName()
-                + ((argument.getHelp() != null && !argument.getHelp().equals(""))
-                    ? ": " + argument.getHelp() : "");
-            logWarning(
-                CliStrings.format(CliStrings.GFSHPARSER__MSG__REQUIRED_ARGUMENT_0, argMessage));
-            return desiredCursorPosition + userOptionSet.getNoOfSpacesRemoved();
-          }
-        }
-        argumentSeparator = SyntaxConstants.ARGUMENT_SEPARATOR;
-      }
-      if (completionCandidates.size() > 0) {
-        return desiredCursorPosition + userOptionSet.getNoOfSpacesRemoved();
-      }
-
-      // Now process options
-
-      boolean warningValueRequired = false;
-      Option warningOption = null;
-
-
-
-      for (Option option : commandTarget.getOptionParser().getOptions()) {
-        if (userOptionSet.hasOption(option)) {
-          // We are supporting option synonyms,
-          // so we need to check that here
-          for (String string : userOptionSet.getSplit()) {
-            if (string.startsWith(SyntaxConstants.LONG_OPTION_SPECIFIER)) {
-              // Remove option prefix
-              string = StringUtils.removeStart(string, SyntaxConstants.LONG_OPTION_SPECIFIER);
-              // Remove value specifier
-              string = StringUtils.removeEnd(string, SyntaxConstants.OPTION_VALUE_SPECIFIER);
-              if (!string.equals("")) {
-                if (option.getLongOption().equals(string)) {
-                  // This means that user has entered the whole option and
-                  // Increment desiredCursorPostion by the length of the
-                  // option and the option specifier, including space
-                  desiredCursorPosition +=
-                      /* space */1 + SyntaxConstants.LONG_OPTION_SPECIFIER.length()
-                          + option.getLongOption().length();
-                  break;
-
-                } else {
-                  // This is only possible if the user has
-                  // entered one of the synonyms of the options
-                  // which wasn't displayed initially
-                  for (String optionSynonym : option.getSynonyms()) {
-                    if (optionSynonym.equals(string)) {
-                      // This means that what the user has
-                      // entered is actually a
-                      // synonym for the option
-                      desiredCursorPosition += /* space */1
-                          + SyntaxConstants.LONG_OPTION_SPECIFIER.length() + optionSynonym.length();
-                      break;
-                    }
-                  }
-                }
-              }
-            }
-          }
-
-          optionsPresentMap.put(option.getLongOption(), option);
-
-          // For option value
-
-          if (userOptionSet.hasValue(option)) {
-            String value = userOptionSet.getValue(option);
-            boolean valueActuallySpecified = false;
-
-            String valueSeparator = SyntaxConstants.VALUE_SEPARATOR;
-            if (option.getValueSeparator() != null) {
-              valueSeparator = option.getValueSeparator();
-            }
-
-            // JOpt doesn't maintain trailing comma (separator), hence reading it from buffer.
-            boolean bufferEndsWithValueSeparator = remainingBuffer.endsWith(valueSeparator);
-            // Check whether the value assigned to the option is
-            // actually part of the split array or has been
-            // assigned using the specifiedDefaultValue attribute
-
-            // userOptionElement can be option name or value of that option.
-            // E.g. "--opt=val" has elements "opt" & "val"
-            for (String userOptionElement : userOptionSet.getSplit()) {
-              if (userOptionElement.equals(value)
-                  || (userOptionElement).equals(value + valueSeparator)) {
-                valueActuallySpecified = true;
-              }
-            }
-            if (!valueActuallySpecified) {
-              continue;
-            }
-            boolean incrementCursor = true;
-            boolean considerLastValue = false;
-            int lengthToBeAdded = 0;
-            int lastIndexOf = 0;
-
-            // This should only be invoked if we don't have any
-            // completionCandidates beforeHand
-            if (completionCandidates.size() == 0) {
-              // Here also we might need to invoke converter to
-              // get values apt for the option
-              if (!endsWithOptionSpecifiers(userOptionSet.getUserInput())
-                  && getAllPossibleValuesForParameter(completionCandidates, option, value,
-                      commandTarget.getGfshMethodTarget())) {
-
-                // If the value returned by getAllPossibleValues
-                // is the same as that entered by the
-                // user we need to remove it from the
-                // completionCandidates and move forward
-
-                String prefix = "";
-                String[] split = ParserUtils.splitValues(value, valueSeparator);
-
-                if (completionCandidates.size() > 0) {
-                  if (PreprocessorUtils.isSyntaxValid(value) && bufferEndsWithValueSeparator) {
-                    // This means that the user wants to
-                    // enter more values,
-                    prefix = valueSeparator;
-                  } else if (perfectMatch(completionCandidates, split)) {
-                    // If the user does not want to enter
-                    // more values, and it matches one
-                    // of the values then we do not
-                    // need to suggest anything for
-                    // this option
-                    completionCandidates.clear();
-                    considerLastValue = true;
-                  } else if (ParserUtils.contains(value, valueSeparator)) {
-                    prefix = valueSeparator;
-                  } else {
-                    incrementCursor = false;
-                    if (value.startsWith(" ")) {
-                      prefix = "  ";
-                    } else if (value.startsWith("\n")) {
-                      prefix = "\n";
-                    } else {
-                      prefix = SyntaxConstants.OPTION_VALUE_SPECIFIER;
-                    }
-                  }
-                }
-                modifyCompletionCandidates(completionCandidates, prefix,
-                    bufferEndsWithValueSeparator, split);
-                if (completionCandidates.size() == 0) {
-                  incrementCursor = true;
-                  considerLastValue = true;
-                }
-              } else {
-                // The completion candidates should be cleared if the Converter has
-                // populated it with some values
-                completionCandidates.clear();
-                considerLastValue = true;
-              }
-            } else {
-              // Make everything true
-              considerLastValue = true;
-            }
-            // FIX for: 46265
-            // if bufferEndsWithValueSeparator, append a valueSeparator to get the real lastIndexOd
-            // e.g. Let's say remainingBuffer is: cmd --opt1=val1,val2,
-            // value would be: cmd --opt1=val1,val2 ---> not there's no comma in the end.
-            // This doesn't give us the real last index of valueSeparator, hence add extra
-            // valueSeparator.
-            lastIndexOf = ParserUtils.lastIndexOf(
-                bufferEndsWithValueSeparator ? value + valueSeparator : value, valueSeparator);
-            lengthToBeAdded =
-                value.substring(0, (lastIndexOf > 0 ? lastIndexOf : value.length())).length();
-            // Increment desiredCursorPosition
-            if (incrementCursor) {
-              desiredCursorPosition +=
-                  /* value specifier length */SyntaxConstants.OPTION_VALUE_SPECIFIER.length()
-                      + lengthToBeAdded
-                      + ((considerLastValue) ? value.length() - lengthToBeAdded : 0);
-              if (value.endsWith(" ") && considerLastValue) {
-                desiredCursorPosition--;
-              }
-            }
-            if (completionCandidates.size() == 0) {
-              if (!PreprocessorUtils.isSyntaxValid(value)) {
-                return desiredCursorPosition + userOptionSet.getNoOfSpacesRemoved();
-              } else {
-                // Check whether the value ends with
-                // VALUE_SEPARATOR,
-                // if yes then we need to return
-                if (value.endsWith(valueSeparator)) {
-                  return desiredCursorPosition + userOptionSet.getNoOfSpacesRemoved();
-                }
-              }
-            }
-          } else {
-            // Here the converter is useful to invoke
-            // auto-suggestion, get Values from Converter
-            if (completionCandidates.size() == 0) {
-              if (getAllPossibleValuesForParameter(completionCandidates, option, null,
-                  commandTarget.getGfshMethodTarget())) {
-                if (completionCandidates.size() == 0) {
-                  warningValueRequired = true;
-                } else {
-                  modifyCompletionCandidates(completionCandidates,
-                      SyntaxConstants.OPTION_VALUE_SPECIFIER, new String[] {null});
-                }
-              } else {
-                // The completion candidates should be cleared if the Converter
-                // has populated it with some values
-                completionCandidates.clear();
-                warningValueRequired = true;
-              }
-            }
-          }
-        } else {
-
-          // As we have reached here, the OptionParser was not able to
-          // detect anything which proves that the option is present
-          // So, we check with what the user provided and add only
-          // that to the list of options to prompt for
-
-          for (String userOptString : userOptionSet.getSplit()) {
-            // Now to determine whether what the user specified was
-            // an option, we need to check whether it starts
-            // with an option specifier
-            if (userOptString.startsWith(SyntaxConstants.LONG_OPTION_SPECIFIER)) {
-              // Now remove the option specifier part
-              userOptString =
-                  StringUtils.removeStart(userOptString, SyntaxConstants.LONG_OPTION_SPECIFIER);
-              if (option.getLongOption().startsWith(userOptString) && !userOptString.equals("")
-                  && !option.getLongOption().equals(userOptString)
-                  && !optionsPresentMap.containsKey(userOptString)) {
-
-                completionCandidates.add(new Completion(
-                    " " + SyntaxConstants.LONG_OPTION_SPECIFIER + option.getLongOption(),
-                    option.getLongOption(), "", 0));
-              } else {
-                for (String optionSynonym : option.getSynonyms()) {
-                  if (optionSynonym.startsWith(userOptString) && !userOptString.equals("")
-                      && !optionSynonym.equals(userOptString)) {
-                    completionCandidates.add(
-                        new Completion(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + optionSynonym,
-                            optionSynonym, "", 0));
-                    break;
-                  }
-                }
-              }
-            }
-          }
-
-          if (completionCandidates.size() == 0) {
-            if (option.isRequired()) {
-              mandatoryOptions.add(option);
-            } else {
-              unspecifiedOptions.add(option);
-            }
-          }
-        }
-        if (warningValueRequired/* || warningMultipleValuesNotSupported */) {
-          warningOption = option;
-          warningValueRequired = false;
-        }
-      }
-
-      // Display warning if something not specified
-      if (warningOption != null) {
-        String optionMsg = warningOption.getLongOption()
-            + ((warningOption.getHelp() != null && !warningOption.getHelp().equals(""))
-                ? ": " + warningOption.getHelp() : "");
-        logWarning(
-            CliStrings.format(CliStrings.GFSHPARSER__MSG__VALUE_REQUIRED_FOR_OPTION_0, optionMsg));
-
-        desiredCursorPosition += userOptionSet.getNoOfSpacesRemoved();
-        completionCandidates
-            .add(new Completion(SyntaxConstants.OPTION_VALUE_SPECIFIER, "", null, 0));
-        return desiredCursorPosition;
-      }
-
-    }
-
-    // Calculate the cursor position
-    int newCursor = desiredCursorPosition
-        + ((userOptionSet != null) ? userOptionSet.getNoOfSpacesRemoved() : 0);
-
-    String subString = remainingBuffer;
-    if (newCursor != cursorStart) {
-      int sizedReducedAdj = sizeReduced ? -1 : 0;
-      int begin = newCursor + sizedReducedAdj - cursorStart;
-      subString = remainingBuffer.substring(begin).trim();
-    }
-
-    // Exception handling
-    if (coe != null // hasException
-        && newCursor < cursor // newCursorIsEarlierThanCursor
-        && completionCandidates.size() == 0 // zeroCompletionCandidates
-        && !(PreprocessorUtils.containsOnlyWhiteSpaces(subString) // onlyHasWhiteSpaces
-            || ((subString.endsWith(SyntaxConstants.LONG_OPTION_SPECIFIER)
-                && subString.startsWith(SyntaxConstants.LONG_OPTION_SPECIFIER)) // isHypenHyphen
-                || (subString.startsWith(SyntaxConstants.SHORT_OPTION_SPECIFIER)
-                    && subString.endsWith(SyntaxConstants.SHORT_OPTION_SPECIFIER))))) { // isHyphen
+  static String convertToSimpleParserInput(String userInput) {
+    List<String> inputTokens = splitUserInput(userInput);
+    return getSimpleParserInputFromTokens(inputTokens);
+  }
 
-      ExceptionHandler.handleException(coe);
-      return cursor;
+  static List<String> splitUserInput(String userInput) {
+    // make sure the userInput ends with a white space, because our regex expects the the quotes
+    // ends with at least one white space. We will trim the results after we found it.
+    userInput = userInput + " ";
+    // first split with whitespaces except in quotes
+    List<String> splitWithWhiteSpaces = new ArrayList<>();
+    Matcher m = PATTERN.matcher(userInput);
+    while (m.find()) {
+      splitWithWhiteSpaces.add(m.group().trim());
     }
 
-    // If nothing has been specified for auto-completion then we need to suggest options
-    if (completionCandidates.size() == 0) {
-      if (mandatoryOptions.size() > 0) {
-
-        for (Option option : mandatoryOptions) {
-          completionCandidates.add(
-              new Completion(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + option.getLongOption(),
-                  option.getLongOption(), "", 0));
-        }
+    List<String> furtherSplitWithEquals = new ArrayList<>();
+    for (String token : splitWithWhiteSpaces) {
+      // if this token has equal sign, split around the first occurrance of it
+      int indexOfFirstEqual = token.indexOf('=');
+      if (indexOfFirstEqual < 0) {
+        furtherSplitWithEquals.add(token);
       } else {
-        // As all the mandatory options have been specified we can prompt the
-        // user for optional options.
-        unspecifiedOptions =
-            getUnspecifiedOptionsWithMode(unspecifiedOptions, commandTarget, optionsPresentMap);
-        for (Option option : unspecifiedOptions) {
-          completionCandidates.add(
-              new Completion(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + option.getLongOption(),
-                  option.getLongOption(), "", 0));
+        String left = token.substring(0, indexOfFirstEqual);
+        String right = token.substring(indexOfFirstEqual + 1);
+        if (left.length() > 0) {
+          furtherSplitWithEquals.add(left);
+        }
+        if (right.length() > 0) {
+          furtherSplitWithEquals.add(right);
         }
       }
     }
-    return newCursor;
+    return furtherSplitWithEquals;
   }
 
-  private List<Option> getUnspecifiedOptionsWithMode(List<Option> unspecifiedOptions,
-      CommandTarget commandTarget, Map<String, Option> optionsPresentMap) {
+  static String getSimpleParserInputFromTokens(List<String> tokens) {
+    // make a copy of the input since we need to do add/remove
+    List<String> inputTokens = new ArrayList<>();
 
-    Collection<CommandMode> cmodes =
-        CommandModes.getInstance().getCommandModes(commandTarget.getCommandName());
-    if (cmodes != null) {
-      List<Option> filteredList = new ArrayList<Option>();
+    // get the --J arguments from the list of tokens
+    int firstJIndex = -1;
+    List<String> jArguments = new ArrayList<>();
 
-      // Populate with default options
-      CommandMode defaultMode = CommandModes.getInstance()
-          .getCommandMode(commandTarget.getCommandName(), CommandModes.DEFAULT_MODE);
-      for (String opt : defaultMode.options) {
-        for (Option option : unspecifiedOptions) {
-          if (option.getLongOption().equals(opt)) {
-            filteredList.add(option);
-          }
+    for (int i = 0; i < tokens.size(); i++) {
+      String token = tokens.get(i);
+      if ("--J".equals(token)) {
+        if (firstJIndex < 1) {
+          firstJIndex = i;
         }
-      }
+        i++;
 
-      // Now add options only for detected command mode
-      boolean leadOptionFound = false;
-      for (CommandMode cmd : cmodes) {
-        if (optionsPresentMap.containsKey(cmd.leadOption)) {
-          leadOptionFound = true;
-          for (String opt : cmd.options) {
-            if (!optionsPresentMap.containsKey(opt)) {
-              for (Option option : unspecifiedOptions) {
-                if (option.getLongOption().equals(opt)) {
-                  filteredList.add(option);
-                }
-              }
-            }
+        if (i < tokens.size()) {
+          String jArg = tokens.get(i);
+          if (jArg.charAt(0) == '"' || jArg.charAt(0) == '\'') {
+            jArg = jArg.substring(1, jArg.length() - 1);
           }
-          break;
-        }
-      }
-
-      if (leadOptionFound) {
-        return filteredList;
-      }
-
-      if (optionsPresentMap.isEmpty()) {
-        // Here return only lead-option of the command-modes
-        filteredList.clear();
-        for (CommandMode cmd2 : cmodes) {
-          for (Option option2 : unspecifiedOptions) {
-            if (option2.getLongOption().equals(cmd2.leadOption)) {
-              filteredList.add(option2);
-            }
+          if (jArg.length() > 0) {
+            jArguments.add(jArg);
           }
         }
-        return filteredList;
+      } else {
+        inputTokens.add(token);
       }
-      return unspecifiedOptions;
-    } else {
-      return unspecifiedOptions;
     }
-  }
-
-  private void checkOptionSetForValidCommandModes(OptionSet userOptionSet,
-      CommandTarget commandTarget) throws CliCommandMultiModeOptionException {
-    CommandModes modes = CommandModes.getInstance();
-    Collection<CommandMode> cmodes = modes.getCommandModes(commandTarget.getCommandName());
 
-    if (cmodes != null) {
-      CommandMode defaultMode =
-          modes.getCommandMode(commandTarget.getCommandName(), CommandModes.DEFAULT_MODE);
-      Map<String, Option> userOptions = new HashMap<String, Option>();
-      Map<String, CommandMode> loToModeMap = new HashMap<String, CommandMode>();
-      for (Option option : commandTarget.getOptionParser().getOptions()) {
-        if (userOptionSet.hasOption(option)) {
-          userOptions.put(option.getLongOption(), option);
+    // concatenate the remaining tokens with space
+    StringBuffer rawInput = new StringBuffer();
+    // firstJIndex must be less than or equal to the length of the inputToken
+    for (int i = 0; i <= inputTokens.size(); i++) {
+      // stick the --J arguments in the orginal first --J position
+      if (i == firstJIndex) {
+        rawInput.append("--J ");
+        if (jArguments.size() > 0) {
+          rawInput.append("\"").append(StringUtils.join(jArguments, ",")).append("\" ");
         }
       }
-
-      List<String> leadOptionList = new ArrayList<String>();
-      for (CommandMode cmd : cmodes) {
-        loToModeMap.put(cmd.leadOption, cmd);
-        if (userOptions.containsKey(cmd.leadOption)) {
-          leadOptionList.add(cmd.leadOption);
-        }
-
-        if (leadOptionList.size() > 1) {
-
-          StringBuilder sb = new StringBuilder();
-          for (String leadOption : leadOptionList) {
-            sb.append(loToModeMap.get(leadOption).name).append(",");
-          }
-          throw new CliCommandMultiModeOptionException(commandTarget,
-              userOptions.get(cmd.leadOption), sb.toString(),
-              CliCommandMultiModeOptionException.MULTIPLE_LEAD_OPTIONS);
-        }
-      }
-
-      if (leadOptionList.size() == 1) {
-        CommandMode modeDetected = loToModeMap.get(leadOptionList.get(0));
-        for (Option opt : userOptions.values()) {
-          // Check only for non-default options, default options are allowed with any other mode
-          if (!isDefaultOption(opt.getLongOption(), defaultMode)) {
-            boolean isOptionFromDetectedMode = false;
-            if (modeDetected.options.length > 0) {
-              for (String commandOpt : modeDetected.options) {
-                if (commandOpt.equals(opt.getLongOption())) {
-                  isOptionFromDetectedMode = true;
-                }
-              }
-              if (!isOptionFromDetectedMode) {
-                throw new CliCommandMultiModeOptionException(commandTarget, opt,
-                    opt.getLongOption(),
-                    CliCommandMultiModeOptionException.OPTIONS_FROM_MULTIPLE_MODES);
-              }
-            }
-          }
-        }
+      // then add the next inputToken
+      if (i < inputTokens.size()) {
+        rawInput.append(inputTokens.get(i)).append(" ");
       }
     }
-  }
 
-  private boolean isDefaultOption(String longOption, CommandMode commandMode) {
-    for (String str : commandMode.options) {
-      if (longOption.equals(str)) {
-        return true;
-      }
-    }
-    return false;
+    return rawInput.toString().trim();
   }
 
-  private boolean endsWithOptionSpecifiers(String userInput) {
-    userInput = userInput.trim();
-    if (userInput.endsWith(" " + SyntaxConstants.LONG_OPTION_SPECIFIER)
-        || userInput.endsWith(" " + SyntaxConstants.SHORT_OPTION_SPECIFIER)) {
-      return true;
-    } else {
-      return false;
-    }
-  }
+  @Override
+  public GfshParseResult parse(String userInput) {
+    String rawInput = convertToSimpleParserInput(userInput);
 
-  /*
-   * Verifies whether the userInput has any one of the following: --some-opt --s --some-opt=some-val
-   * --something-else
-   */
-  private boolean hasOptionSpecified(String userInput) {
-    userInput = userInput.trim();
-    return Pattern.matches("^(.*)(-+)(\\w+)(.*)$", userInput);
-  }
+    // User SimpleParser to parse the input
+    ParseResult result = super.parse(rawInput);
 
-  private String getSystemProvidedValue(Parameter parameter) {
-    if (parameter.isSystemProvided()) {
-      // TODO fetch from system properties
-      // Assume value is null for now.
-      return null;
-    } else {
-      return null;
+    if (result != null) {
+      return new GfshParseResult(result.getMethod(), result.getInstance(), result.getArguments(),
+          userInput);
     }
-  }
 
-  private boolean perfectMatch(List<Completion> completionCandidates, String... argumentValue) {
-    // Here only the last value should match one of the
-    // completionCandidates
-    if (argumentValue.length > 0) {
-      for (Completion completion : completionCandidates) {
-        if (completion.getValue().equals(argumentValue[argumentValue.length - 1])) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  private void modifyCompletionCandidates(List<Completion> completionCandidates, String prefix,
-      String... existingData) {
-    modifyCompletionCandidates(completionCandidates, prefix, false, existingData);
-  }
-
-  private void modifyCompletionCandidates(List<Completion> completionCandidates, String prefix,
-      boolean endsWithValueSeparator, String... existingData) {
-    List<Completion> temp = new ArrayList<Completion>();
-    while (completionCandidates.size() > 0) {
-      temp.add(completionCandidates.remove(0));
-    }
-    for (Completion completion : temp) {
-      boolean includeCompletion = true;
-      String value = completion.getValue();
-      if (existingData != null) {
-        for (String string : existingData) {
-          if (string != null) {
-            // Check whether that value matches any of the
-            // existingData
-            // If it matches any one of existing data then we do not
-            // need to include it in the list of completion
-            // candidates
-            if (value.equals(string)) {
-              includeCompletion = false;
-            }
-          }
-        }
-        if (includeCompletion) {
-          if (existingData[existingData.length - 1] != null
-              && (!value.startsWith(existingData[existingData.length - 1])
-                  && !endsWithValueSeparator)) {
-            includeCompletion = false;
-          }
-        }
-      }
-      if (includeCompletion) {
-        // Also we only need to check with the last string of
-        // existingData
-        // whether the completion value starts with it.
-        completionCandidates
-            .add(new Completion(prefix + completion.getValue(), completion.getValue(), "", 0));
-      }
-    }
-  }
-
-  @SuppressWarnings({"rawtypes", "unchecked"})
-  private boolean getAllPossibleValuesForParameter(List<Completion> completionCandidates,
-      Parameter parameter, String existingData, GfshMethodTarget gfshMethodTarget) {
-    Converter<?> converter = parameter.getConverter();
-    // Check if any new converter is available which
-    // satisfies the requirements for this argument
-    if (converter == null) {
-      parameter.setConverter(
-          commandManager.getConverter(parameter.getDataType(), parameter.getContext()));
-      converter = parameter.getConverter();
-    }
-    // If still we do not have any matching converters, we return
-    if (converter == null) {
-      return false;
-    } else {
-      // Now pass the getAllPossibleValues function of Converter interface
-      // all the required parameters
-
-      // Check whether it is a MultipleValueConverter
-      String valueSeparator = SyntaxConstants.VALUE_SEPARATOR;
-      if (parameter instanceof Option && ((Option) parameter).getValueSeparator() != null) {
-        valueSeparator = ((Option) parameter).getValueSeparator();
-      }
-      if (converter instanceof MultipleValueConverter) {
-        ((MultipleValueConverter) converter).getAllPossibleValues(completionCandidates,
-            parameter.getDataType(), ParserUtils.splitValues(existingData, valueSeparator),
-            parameter.getContext(),
-            new MethodTarget(gfshMethodTarget.getMethod(), gfshMethodTarget.getTarget(),
-                gfshMethodTarget.getRemainingBuffer(), gfshMethodTarget.getKey()));
-      } else {
-        converter.getAllPossibleValues(completionCandidates, parameter.getDataType(), existingData,
-            parameter.getContext(),
-            new MethodTarget(gfshMethodTarget.getMethod(), gfshMethodTarget.getTarget(),
-                gfshMethodTarget.getRemainingBuffer(), gfshMethodTarget.getKey()));
-      }
-    }
-    if (completionCandidates.size() > 0) {
-      return true;
-    } else {
-      return false;
-    }
+    return null;
   }
 
   /**
    *
+   * The super class's completeAdvanced has the following limitations: 1) for option name
+   * completion, you need to end your buffer with --. 2) For command name completion, you need to
+   * end your buffer with a space. 3) the above 2 completions, the returned value is always 0, and
+   * the completion is the entire command 4) for value completion, you also need to end your buffer
+   * with space, the returned value is the length of the original string, and the completion strings
+   * are the possible values.
+   *
+   * With these limitations, we will need to overwrite this command with some customization
+   *
+   * @param userInput
+   * @param cursor this input is ignored, we always move the cursor to the end of the userInput
+   * @param candidates
+   * @return the cursor point at which the candidate string will begin, this is important if you
+   *         have only one candidate, cause tabbing will use it to complete the string for you.
    */
-  public ParseResult parse(String userInput) {
-    GfshParseResult parseResult = null;
-    // First remove the trailing white spaces
-    userInput = StringUtils.stripEnd(userInput, null);
-    if ((ParserUtils.contains(userInput, SyntaxConstants.COMMAND_DELIMITER)
-        && StringUtils.endsWithIgnoreCase(userInput, SyntaxConstants.COMMAND_DELIMITER))) {
-      userInput = StringUtils.removeEnd(userInput, SyntaxConstants.COMMAND_DELIMITER);
-    }
 
-    try {
-      boolean error = false;
-      CliCommandOptionException coe = null;
-      List<CommandTarget> targets = locateTargets(ParserUtils.trimBeginning(userInput), false);
-      if (targets.size() > 1) {
-        if (userInput.length() > 0) {
-          handleCondition(
-              CliStrings.format(
-                  CliStrings.GFSHPARSER__MSG__AMBIGIOUS_COMMAND_0_FOR_ASSISTANCE_USE_1_OR_HINT_HELP,
-                  new Object[] {userInput, AbstractShell.completionKeys}),
-              CommandProcessingException.COMMAND_NAME_AMBIGUOUS, userInput);
-        }
-      } else {
-        if (targets.size() == 1) {
-          OptionSet parse = null;
-          List<MethodParameter> parameters = new ArrayList<MethodParameter>();
-          Map<String, String> paramValMap = new HashMap<String, String>();
-          CommandTarget commandTarget = targets.get(0);
-          GfshMethodTarget gfshMethodTarget = commandTarget.getGfshMethodTarget();
-          preConfigureConverters(commandTarget);
-
-          try {
-            parse = commandTarget.getOptionParser().parse(gfshMethodTarget.getRemainingBuffer());
-          } catch (CliException ce) {
-            if (ce instanceof CliCommandOptionException) {
-              coe = (CliCommandOptionException) ce;
-              coe.setCommandTarget(commandTarget);
-              parse = coe.getOptionSet();
-              error = true;
-            }
-          }
-
-          try {
-            checkOptionSetForValidCommandModes(parse, commandTarget);
-          } catch (CliCommandMultiModeOptionException ce) {
-            error = true;
-            coe = ce;
-          }
+  @Override
+  public int completeAdvanced(String userInput, int cursor, final List<Completion> candidates) {
+    // move the cursor to the end of the input
+    cursor = userInput.length();
+    List<String> inputTokens = splitUserInput(userInput);
 
-          error = processArguments(parse, commandTarget, paramValMap, parameters, error);
-          // TODO: next call throws when space before closing "
-          error = processOptions(parse, commandTarget, paramValMap, parameters, error);
-
-          if (!error) {
-            Object[] methodParameters = new Object[parameters.size()];
-            for (MethodParameter parameter : parameters) {
-              methodParameters[parameter.getParameterNo()] = parameter.getParameter();
-            }
-            parseResult =
-                new GfshParseResult(gfshMethodTarget.getMethod(), gfshMethodTarget.getTarget(),
-                    methodParameters, userInput, commandTarget.getCommandName(), paramValMap);
-          } else {
-            if (coe != null) {
-              logWrapper.fine("Handling exception: " + coe.getMessage());
-              ExceptionHandler.handleException(coe); // TODO: this eats exception that would make it
-                                                     // easier to debug GemfireDataCommandsDUnitTest
-              // ExceptionHandler.handleException() only logs it on console.
-              // When on member, we need to handle this.
-              if (!CliUtil.isGfshVM()) {
-                handleCondition(CliStrings
-                    .format(CliStrings.GFSHPARSER__MSG__INVALID_COMMAND_STRING_0, userInput), coe,
-                    CommandProcessingException.COMMAND_INVALID, userInput);
-              }
-            }
-          }
-
-        } else {
-          String message =
-              CliStrings.format(CliStrings.GFSHPARSER__MSG__COMMAND_0_IS_NOT_VALID, userInput);
-          CommandTarget commandTarget = locateExactMatchingTarget(userInput);
-          if (commandTarget != null) {
-            String commandName = commandTarget.getCommandName();
-            AvailabilityTarget availabilityIndicator = commandTarget.getAvailabilityIndicator();
-            message = CliStrings.format(CliStrings.GFSHPARSER__MSG__0_IS_NOT_AVAILABLE_REASON_1,
-                new Object[] {commandName, availabilityIndicator.getAvailabilityDescription()});
-          }
-          handleCondition(message, CommandProcessingException.COMMAND_INVALID_OR_UNAVAILABLE,
-              userInput);
-        }
+    // check if the input is before any option is specified, e.g. (start, describe)
+    boolean inputIsBeforeOption = true;
+    for (String token : inputTokens) {
+      if (token.startsWith("--")) {
+        inputIsBeforeOption = false;
+        break;
       }
-    } catch (IllegalArgumentException e1) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e1));
-    } catch (IllegalAccessException e1) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e1));
-    } catch (InvocationTargetException e1) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e1));
     }
-    return parseResult;
-  }
 
-  // Pre-configure the converters so that we can test against them when parsing the command line
-  private void preConfigureConverters(CommandTarget commandTarget) {
-    for (Option option : commandTarget.getOptionParser().getOptions()) {
-      Converter<?> converter = option.getConverter();
-      if (converter == null) {
-        option.setConverter(commandManager.getConverter(option.getDataType(), option.getContext()));
-        converter = option.getConverter();
+    // in the case of we are still trying to complete the command name
+    if (inputIsBeforeOption) {
+      List<Completion> potentials = getCandidates(userInput);
+      if (potentials.size() == 1 && potentials.get(0).getValue().equals(userInput)) {
+        potentials = getCandidates(userInput.trim() + " ");
       }
-    }
 
-    for (Argument argument : commandTarget.getOptionParser().getArguments()) {
-      Converter<?> converter = argument.getConverter();
-      if (converter == null) {
-        argument.setConverter(
-            commandManager.getConverter(argument.getDataType(), argument.getContext()));
-        converter = argument.getConverter();
+      if (potentials.size() > 0) {
+        candidates.addAll(potentials);
+        return 0;
       }
+      // otherwise, falling down to the potentials.size==0 case below
     }
-  }
 
-  private boolean processOptions(OptionSet parse, CommandTarget commandTarget,
-      Map<String, String> paramValMap, List<MethodParameter> parameters, boolean errorState) {
-    boolean error = errorState;
-    for (Option option : commandTarget.getOptionParser().getOptions()) {
-      String value = null;
-      if (parse.hasOption(option)) {
-        if (parse.hasValue(option)) {
-          value = parse.getValue(option);
-        }
-        if (value == null) {
-          handleCondition(
-              CliStrings.format(CliStrings.GFSHPARSER__MSG__VALUE_REQUIRED_FOR_OPTION_0,
-                  option.getLongOption()),
-              CommandProcessingException.OPTION_VALUE_REQUIRED, option.getLongOption());
-          logWrapper.fine("Value required for Parameter " + option.getLongOption());
-          error = true;
-        }
-      } else {
-        if (option.isRequired()) {
-          handleCondition(
-              CliStrings.format(CliStrings.GFSHPARSER__MSG__COMMAND_OPTION_0_IS_REQUIRED_USE_HELP,
-                  option.getLongOption()),
-              CommandProcessingException.REQUIRED_OPTION_MISSING, option.getLongOption());
-          logWrapper.fine("Required Parameter " + option.getLongOption());
-          error = true;
-        } else {
-          // Try to get the unspecifiedDefaultValue for the
-          // option
-          value = option.getUnspecifiedDefaultValue();
-          if (value == null) {
-            // Now try the system provide value
-            value = getSystemProvidedValue(option);
-          }
-        }
-      }
+    // now we are either trying to complete the option or a value
+    // trying to get candidates using the converted input
+    String buffer = getSimpleParserInputFromTokens(inputTokens);
+    String lastToken = inputTokens.get(inputTokens.size() - 1);
+    boolean lastTokenIsOption = lastToken.startsWith("--");
+    // In the original user input, where to begin the candidate string for completion
+    int candidateBeginAt;
 
-      String valueSeparator = SyntaxConstants.VALUE_SEPARATOR;
-      if (option.getValueSeparator() != null) {
-        valueSeparator = option.getValueSeparator();
-      }
+    // initially assume we are trying to complete the last token
+    List<Completion> potentials = getCandidates(buffer);
 
-      Object object = getConversionObject(option.getConverter(), value, option.getDataType(),
-          option.getContext(), valueSeparator);
-      // Check if conversion fails
-      if (value != null && object == null) {
-        handleCondition(
-            CliStrings.format(CliStrings.GFSHPARSER__MSG__VALUE_0_IS_NOT_APPLICABLE_FOR_1,
-                new Object[] {value.trim(), option.getLongOption()}),
-            CommandProcessingException.OPTION_VALUE_INVALID, option.getLongOption() + "=" + value);
-        logWrapper
-            .fine("Value \"" + value.trim() + "\" is not applicable for " + option.getLongOption());
-        error = true;
-      }
-      parameters.add(new MethodParameter(object, option.getParameterNo()));
-      paramValMap.put(option.getLongOption(), value);
-    }
-    return error;
-  }
-
-  private boolean processArguments(OptionSet parse, CommandTarget commandTarget,
-      Map<String, String> paramValMap, List<MethodParameter> parameters, boolean errorState) {
-    boolean error = errorState;
-    for (Argument argument : commandTarget.getOptionParser().getArguments()) {
-      String value = null;
-
-      if (parse.hasArgument(argument)) {
-        value = parse.getValue(argument);
+    if (potentials.size() > 0) {
+      if (lastTokenIsOption) {
+        candidateBeginAt = buffer.length() - lastToken.length();
       } else {
-        if (argument.isRequired()) {
-          handleCondition(
-              CliStrings.format(CliStrings.GFSHPARSER__MSG__COMMAND_ARGUMENT_0_IS_REQUIRED_USE_HELP,
-                  argument.getArgumentName()),
-              CommandProcessingException.REQUIRED_ARGUMENT_MISSING, argument.getArgumentName());
-          logWrapper.fine("Required Argument " + argument.getArgumentName());
-          error = true;
-        } else {
-          // try to get unspecifiedDefaultValue for
-          // the argument
-          value = argument.getUnspecifiedDefaultValue();
-          if (value == null) {
-            // Now try the system provided value
-            value = getSystemProvidedValue(argument);
-          }
-        }
-
-      }
-
-      Object conversionObject = getConversionObject(argument.getConverter(), value,
-          argument.getDataType(), argument.getContext(), SyntaxConstants.VALUE_SEPARATOR);
-      if (value != null && conversionObject == null) {
-        handleCondition(
-            CliStrings.format(CliStrings.GFSHPARSER__MSG__VALUE_0_IS_NOT_APPLICABLE_FOR_1,
-                new Object[] {value.trim(), argument.getArgumentName()}),
-            CommandProcessingException.ARGUMENT_INVALID, argument.getArgumentName() + "=" + value);
-        logWrapper.fine("Value '" + value.trim() + "' not applicable for argument: "
-            + argument.getArgumentName());
-        error = true;
-      } else {
-        parameters.add(new MethodParameter(conversionObject, argument.getParameterNo()));
-        paramValMap.put(argument.getArgumentName(), value);
+        // need to return the index before the "=" sign, since later on we are going to add the
+        // "=" sign to the completion candidates
+        candidateBeginAt = buffer.length() - lastToken.length() - 1;
       }
     }
-    return error;
-  }
-
-  @SuppressWarnings({"rawtypes", "unchecked"})
-  private Object getConversionObject(Converter<?> converter, String string, Class<?> dataType,
-      String context, String valueSeparator) {
-
-    try {
-      if (converter != null && converter instanceof MultipleValueConverter) {
-        return ((MultipleValueConverter) converter).convertFromText(
-            ParserUtils.splitValues(((string != null) ? string.trim() : null), valueSeparator),
-            dataType, context);
+    // if the last token is already complete, add either space or " --" and try again
+    else {
+      candidateBeginAt = buffer.length();
+      // last token is an option
+      if (lastTokenIsOption) {
+        // add a space to the buffer to get the option value candidates
+        potentials = getCandidates(buffer + " ");
+        lastTokenIsOption = false;
       }
-
-      // Remove outer single or double quotes if found
-      boolean hasDoubleQuotes = string.startsWith("\"") && string.endsWith("\"");
-      boolean hasSingleQuotes = string.startsWith("\'") && string.endsWith("\'");
-
-      while (string != null && (hasDoubleQuotes || hasSingleQuotes)) {
-        string = string.substring(1, string.length() - 1);
-        hasDoubleQuotes = string.startsWith("\"") && string.endsWith("\"");
-        hasSingleQuotes = string.startsWith("\'") && string.endsWith("\'");
-      }
-
-      if (converter != null) {
-        return converter.convertFromText((string != null) ? string.trim() : null, dataType,
-            context);
+      // last token is a value, we need to add " --" to it and retry to get the next list of options
+      else {
+        potentials = getCandidates(buffer + " --");
+        lastTokenIsOption = true;
       }
-
-      // TODO consider multiple value case for primitives
-      if (string != null) {
-        if (String.class.isAssignableFrom(dataType)) {
-          return string.trim();
-        } else if (Byte.class.isAssignableFrom(dataType) || byte.class.isAssignableFrom(dataType)) {
-          return Integer.parseInt(string);
-        } else if (Short.class.isAssignableFrom(dataType)
-            || short.class.isAssignableFrom(dataType)) {
-          return Integer.parseInt(string);
-        } else if (Boolean.class.isAssignableFrom(dataType)
-            || boolean.class.isAssignableFrom(dataType)) {
-          return Integer.parseInt(string);
-        } else if (Integer.class.isAssignableFrom(dataType)
-            || int.class.isAssignableFrom(dataType)) {
-          return Integer.parseInt(string);
-        } else if (Long.class.isAssignableFrom(dataType) || long.class.isAssignableFrom(dataType)) {
-          return Long.parseLong(string);
-        } else if (Float.class.isAssignableFrom(dataType)
-            || float.class.isAssignableFrom(dataType)) {
-          return Float.parseFloat(string);
-        } else if (Double.class.isAssignableFrom(dataType)
-            || double.class.isAssignableFrom(dataType)) {
-          return Double.parseDouble(string);
-        } else if (Character.class.isAssignableFrom(dataType)
-            || char.class.isAssignableFrom(dataType)) {
-          if (string.length() == 1) {
-            string.charAt(0);
-          } else {
-            // FIXME Use a constant here
-            return '0';
-          }
-        }
-      }
-    } catch (Exception e) {
-      // TODO add logging
-      // Do nothing, just return null
     }
-    return null;
-  }
-
-  private List<CommandTarget> locateTargets(String userInput)
-      throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
-    return locateTargets(userInput, true);
-  }
-
-
-  private List<CommandTarget> locateTargets(String userInput, boolean matchIncomplete)
-      throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
-    List<CommandTarget> commandTargets = new ArrayList<CommandTarget>();
-    Map<String, CommandTarget> commands = commandManager.getCommands();
-    // Now we need to locate the CommandTargets from the entries in the map
-    for (String commandName : commands.keySet()) {
-      if (userInput.startsWith(commandName)) {
-        // This means that the user has entered the command
-        CommandTarget commandTarget = commands.get(commandName);
-        if (isAvailable(commandTarget, commandName)) {
-          String remainingBuffer = StringUtils.removeStart(userInput, commandName);
-          if (remainingBuffer.length() == 0 || remainingBuffer.startsWith(" ")
-              || remainingBuffer.startsWith(GfshParser.LINE_SEPARATOR)) {
-            // We need to duplicate with a new MethodTarget as this
-            // parser will be used in a concurrent execution environment
-            if (!commandTargets.contains(commandTarget)) {
-              // This test is necessary as the command may have similar
-              // synonyms or which are prefix for the command
-              commandTargets.add(commandTarget.duplicate(commandName, remainingBuffer));
-            }
-          }
-        }
-      } else if (matchIncomplete && commandName.startsWith(userInput)) {
-        // This means that the user is yet to enter the command properly
-        CommandTarget commandTarget = commands.get(commandName);
-        if (isAvailable(commandTarget, commandName)) {
-          // We need to duplicate with a new MethodTarget as this
-          // parser will be used in a concurrent execution environment
-          if (!commandTargets.contains(commandTarget)) {
-            // This test is necessary as the command may have similar
-            // synonyms or which are prefix for the command
-            commandTargets.add(commandTarget.duplicate(commandName));
-          }
-        }
-      }
-    }
-    return commandTargets;
-  }
-
-  // TODO - Abhishek - create an inner CommandTargetLocater instead of multiple
-  // methods like these.
-  private CommandTarget locateExactMatchingTarget(final String userInput)// exact matching
-      throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
-    CommandTarget commandTarget = null;
-
-    Map<String, CommandTarget> commandTargetsMap = commandManager.getCommands();
-    // Reverse sort the command names because we should start from longer names
-    // E.g. Consider commands "A", "A B" & user input as "A B --opt1=val1"
-    // In this case, "A B" is the most probable match & should be matched first
-    // which can be achieved by reversing natural order of sorting.
-    Set<String> commandNamesReverseSorted = new TreeSet<String>(Collections.reverseOrder());
-    commandNamesReverseSorted.addAll(commandTargetsMap.keySet());
-
-    // Now we need to locate the CommandTargets from the entries in the map
-    for (final String commandName : commandNamesReverseSorted) {
-      if (userInput.startsWith(commandName) && commandWordsMatch(userInput, commandName)) {
-        // This means that the user has entered the command & name matches exactly
-        commandTarget = commandTargetsMap.get(commandName);
-        if (commandTarget != null) {
-          String remainingBuffer = StringUtils.removeStart(userInput, commandName);
-          commandTarget = commandTarget.duplicate(commandName, remainingBuffer);
-          break;
-        }
-      }
-    }
-    return commandTarget;
-  }
-
-  private static boolean commandWordsMatch(final String userInput, final String commandName) {
-    boolean commandWordsMatch = true;
 
-    String[] commandNameWords = commandName.split(" ");
-    String[] userInputWords = userInput.split(" ");
-
-    // commandName is fixed & hence should have less or same number of words as
-    // the user input. E.g. "create disk-store" should match with
-    // "create disk-store --name=xyz" but not with "create disk store"
-    if (commandNameWords.length <= userInputWords.length) {
-      // if both have length zero, words can be considered to be matching.
-      for (int i = 0; i < commandNameWords.length; i++) {
-        if (!commandNameWords[i].equals(userInputWords[i])) {
-          commandWordsMatch = false;
-          break;
-        }
-      }
+    // manipulate the candidate strings
+    if (lastTokenIsOption) {
+      // strip off the beginning part of the candidates from the cursor point
+      potentials.replaceAll(
+          completion -> new Completion(completion.getValue().substring(candidateBeginAt)));
     } else {
-      commandWordsMatch = false;
-    }
-
-    return commandWordsMatch;
-  }
-
-
-  private Map<String, CommandTarget> getRequiredCommandTargets(Set<String> requiredCommands) {
-    Map<String, CommandTarget> existingCommands = commandManager.getCommands();
-    Map<String, CommandTarget> requiredCommandsMap = existingCommands;
-
-    if (requiredCommands != null && !requiredCommands.isEmpty()) {
-      requiredCommandsMap = new TreeMap<String, CommandTarget>();
-      for (String commandName : requiredCommands) {
-        CommandTarget commandTarget = existingCommands.get(commandName);
-        if (commandTarget != null) {
-          requiredCommandsMap.put(commandName, commandTarget);
-        }
-      }
-    }
-
-    return requiredCommandsMap;
-  }
-
-  private Map<Short, List<CommandTarget>> findMatchingCommands(String userSpecifiedCommand,
-      Set<String> requiredCommands)
-      throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
-
-    Map<String, CommandTarget> existingCommands = getRequiredCommandTargets(requiredCommands);
-    CommandTarget exactCommandTarget = existingCommands.get(userSpecifiedCommand);
-
-    // 1. First find exactly matching commands.
-    List<CommandTarget> exactCommandTargets = Collections.emptyList();
-    if (exactCommandTarget != null) {
-      // This means that the user has entered the command
-      // NOTE: we are not skipping synonym here.
-      exactCommandTargets = Collections.singletonList(exactCommandTarget);
+      // these potentials do not have "=" in front of them, manually add them
+      potentials.replaceAll(completion -> new Completion("=" + completion.getValue()));
     }
 
-    // 2. Now find command names that start with 'userSpecifiedCommand'
-    List<CommandTarget> possibleCommandTargets = new ArrayList<CommandTarget>();
-    // Now we need to locate the CommandTargets from the entries in the map
-    for (Map.Entry<String, CommandTarget> entry : existingCommands.entrySet()) {
-      CommandTarget commandTarget = entry.getValue();
-      String commandName = commandTarget.getCommandName();
-      // This check is done to remove commands that are synonyms as
-      // CommandTarget.getCommandName() will return name & not a synonym
-      if (entry.getKey().equals(commandName)) {
-        if (commandName.startsWith(userSpecifiedCommand)
-            && !commandTarget.equals(exactCommandTarget)) {
-          // This means that the user is yet to enter the command properly
-          possibleCommandTargets.add(commandTarget);
-        }
-      }
-    }
+    candidates.addAll(potentials);
 
-    Map<Short, List<CommandTarget>> commandTargetsArr = new HashMap<Short, List<CommandTarget>>();
-    commandTargetsArr.put(EXACT_TARGET, exactCommandTargets);
-    commandTargetsArr.put(MATCHING_TARGETS, possibleCommandTargets);
-    return commandTargetsArr;
+    // usually we want to return the cursor at candidateBeginAt, but since we consolidated
+    // --J options into one, and added quotes around we need to consider the length difference
+    // between userInput and the converted input
+    cursor = candidateBeginAt + (userInput.trim().length() - buffer.length());
+    return cursor;
   }
 
-
-  private boolean isAvailable(CommandTarget commandTarget, String commandName)
-      throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
-    AvailabilityTarget availabilityIndicator = commandTarget.getAvailabilityIndicator();
-    if (availabilityIndicator == null) {
-      availabilityIndicator = commandManager.getAvailabilityIndicator(commandName);
-      commandTarget.setAvailabilityIndicator(availabilityIndicator);
-    }
-    return commandTarget.isAvailable();
+  // convenience method for testing
+  int completeAdvanced(String userInput, final List<Completion> candidates) {
+    return completeAdvanced(userInput, userInput.length(), candidates);
   }
 
-  public List<String> obtainHelpCommandNames(String userInput) {
-    List<String> commandNames = new ArrayList<String>();
-
-    try {
-      if (userInput == null) {
-        userInput = "";
-      }
-
-      List<CommandTarget> commandTargets = new ArrayList<CommandTarget>();
-      Map<Short, List<CommandTarget>> matchingCommandsMap = findMatchingCommands(userInput, null);
-      commandTargets.addAll(matchingCommandsMap.get(EXACT_TARGET));
-      commandTargets.addAll(matchingCommandsMap.get(MATCHING_TARGETS));
-
-      for (CommandTarget commandTarget : commandTargets) {
-        commandNames.add(commandTarget.getCommandName());
-      }
-    } catch (IllegalArgumentException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    } catch (IllegalAccessException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    } catch (InvocationTargetException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    }
-
-    return commandNames;
-  }
-
-  public String obtainHelp(String userInput, Set<String> commandNames) {
-    final boolean withinShell = commandNames == null || commandNames.isEmpty();
-    final String appName = withinShell ? "" : HelpUtils.EXE_PREFIX_FOR_EXTERNAL_HELP;
-
-    StringBuilder helpText = new StringBuilder();
-    try {
-      if (userInput == null) {
-        userInput = "";
-      }
-
-      Map<Short, List<CommandTarget>> matchingCommandsMap =
-          findMatchingCommands(userInput, commandNames);
-      List<CommandTarget> exactCommandTargets = matchingCommandsMap.get(EXACT_TARGET);
-      List<CommandTarget> matchingCommandTargets = matchingCommandsMap.get(MATCHING_TARGETS);
-      matchingCommandsMap.clear();
-
-      if (exactCommandTargets.isEmpty() && matchingCommandTargets.isEmpty()) {
-        // No matching commands
-        helpText.append(CliStrings.GFSHPARSER__MSG__NO_MATCHING_COMMAND)
-            .append(GfshParser.LINE_SEPARATOR);
-      } else {
-        if (exactCommandTargets.size() == 1) {
-          helpText.append(obtainCommandSpecificHelp(exactCommandTargets.get(0), withinShell));
-          if (!matchingCommandTargets.isEmpty()) {
-            helpText.append(GfshParser.LINE_SEPARATOR);
-            helpText.append(CliStrings
-                .format(CliStrings.GFSHPARSER__MSG__OTHER_COMMANDS_STARTING_WITH_0_ARE, userInput));
-            for (int i = 0; i < matchingCommandTargets.size(); i++) {
-              CommandTarget commandTarget = matchingCommandTargets.get(i);
-              helpText.append(commandTarget.getCommandName());
-              if (i < matchingCommandTargets.size() - 1) {
-                helpText.append(", ");
-              }
-            }
-            helpText.append(GfshParser.LINE_SEPARATOR);
-          }
-        } else {
-          List<CommandTarget> commandTargets = new ArrayList<CommandTarget>();
-          commandTargets.addAll(exactCommandTargets);
-          commandTargets.addAll(matchingCommandTargets);
-          for (CommandTarget commandTarget : commandTargets) {
-            String availability = commandTarget.isAvailable() ? HelpUtils.HELP__COMMAND_AVAILABLE
-                : HelpUtils.HELP__COMMAND_NOTAVAILABLE;
-            // Many matching commands, provide one line description
-            helpText.append(commandTarget.getCommandName());
-            if (withinShell) {
-              helpText.append(" (").append(availability).append(")");
-            }
-            helpText.append(GfshParser.LINE_SEPARATOR);
-            helpText.append(Gfsh.wrapText(commandTarget.getCommandHelp(), 1))
-                .append(GfshParser.LINE_SEPARATOR);
-          }
-          helpText.append(GfshParser.LINE_SEPARATOR);
-
-          if (withinShell) {
-            helpText
-                .append(Gfsh.wrapText(
-                    CliStrings.format(
-                        CliStrings.GFSHPARSER__MSG__USE_0_HELP_COMMAND_TODISPLAY_DETAILS, appName),
-                    0))
-                .append(GfshParser.LINE_SEPARATOR);
-            helpText.append(Gfsh.wrapText(
-                CliStrings.format(CliStrings.GFSHPARSER__MSG__HELP_CAN_ALSO_BE_OBTAINED_BY_0_KEY,
-                    AbstractShell.completionKeys),
-                0));
-          }
-        }
-      }
-    } catch (IllegalArgumentException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    } catch (IllegalAccessException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    } catch (InvocationTargetException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    }
-    return helpText.toString();
-  }
-
-  private String obtainCommandSpecificHelp(CommandTarget commandTarget, boolean withinShell) {
-    NewHelp newHelp = HelpUtils.getNewHelp(commandTarget, withinShell);
-    return newHelp.toString();
-  }
-
-  public List<String> getCommandNames(String string) {
-    List<String> commandNames = new ArrayList<String>();
-    try {
-      if (string == null) {
-        string = "";
-      }
-      List<CommandTarget> locateTargets = locateTargets(string);
-      for (CommandTarget commandTarget : locateTargets) {
-        String key = commandTarget.getGfshMethodTarget().getKey();
-        if (key.startsWith(string)) {
-          commandNames.add(key);
-        }
-      }
-    } catch (IllegalArgumentException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    } catch (IllegalAccessException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    } catch (InvocationTargetException e) {
-      logWrapper.warning(CliUtil.stackTraceAsString(e));
-    }
-    return commandNames;
-  }
-
-  // ///////////////// Parser interface Methods End //////////////////////////
-
-  private void handleCondition(String message, int errorType, Object errorData) {
-    this.handleCondition(message, null, errorType, errorData);
+  /**
+   * test only used to demonstrate what's the super class's completeAdvanced behavior
+   *
+   */
+  int completeSuperAdvanced(String userInput, final List<Completion> candidates) {
+    return super.completeAdvanced(userInput, userInput.length(), candidates);
   }
 
-  private void handleCondition(String message, Throwable th, int errorType, Object errorData) {
-    if (CliUtil.isGfshVM()) {
-      logWarning(message); // TODO - Abhishek add throwable if debug is ON
-    } else {
-      if (th != null) {
-        throw new CommandProcessingException(message + ": " + th.getMessage(), errorType,
-            errorData);
-      }
-      throw new CommandProcessingException(message, errorType, errorData);
-    }
+  /**
+   * @param buffer use the buffer to find the completion candidates
+   *
+   *        Note the cursor maynot be the size the buffer
+   */
+  private List<Completion> getCandidates(String buffer) {
+    List<Completion> candidates = new ArrayList<>();
+    // always pass the buffer length as the cursor position for simplicity purpose
+    super.completeAdvanced(buffer, buffer.length(), candidates);
+    // trimming the candidates
+    candidates.replaceAll(completion -> new Completion(completion.getValue().trim()));
+    return candidates;
   }
 
-  private void logWarning(String message) {
-    if (canLogToConsole()) {
-      consoleLogger
-          .warning(CLIConsoleBufferUtil.processMessegeForExtraCharactersFromConsoleBuffer(message));
-    } else {
-      Gfsh.println(message);
-    }
+  @Override
+  public void obtainHelp(String command) {
+    if (StringUtils.isBlank(command))
+      super.obtainHelp(command);
+    else
+      consoleLogger.info(commandManager.obtainHelp(command));
   }
 
-  private boolean canLogToConsole() {
-    Gfsh gfsh = Gfsh.getCurrentInstance();
-    return gfsh != null && !gfsh.isHeadlessMode() && consoleLogger != null;
+  public void obtainHint(String topic) {
+    consoleLogger.info(commandManager.obtainHint(topic));
   }
-
-  // private void logInfo(String message) {
-  // if (consoleLogger != null) {
-  // consoleLogger.info(message);
-  // } else {
-  // Gfsh.println(message);
-  // }
-  // }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java
index fc0427e..c5c73dc 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java
@@ -14,22 +14,19 @@
  */
 package org.apache.geode.management.internal.cli;
 
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.PureJavaMode;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
 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.GfshHistory;
-
 import org.springframework.shell.core.ExitShellRequest;
 
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import joptsimple.OptionException;
 import joptsimple.OptionParser;
 import joptsimple.OptionSet;
@@ -84,28 +81,14 @@ public final class Launcher {
   private static final String MSG_INVALID_COMMAND_OR_OPTION = "Invalid command or option : {0}."
       + GfshParser.LINE_SEPARATOR + "Use 'gfsh help' to display additional information.";
 
-  private final Set<String> allowedCommandLineCommands;
-  private final OptionParser commandLineParser;
-  private StartupTimeLogHelper startupTimeLogHelper;
-
   static {
     // See 47325
     System.setProperty(PureJavaMode.PURE_MODE_PROPERTY, "true");
   }
 
-  public static void main(final String[] args) {
-    // first check whether required dependencies exist in the classpath
-    // should we start without tomcat/servlet jars?
-    String nonExistingDependency = CliUtil.cliDependenciesExist(true);
-    if (nonExistingDependency != null) {
-      System.err.println("Required (" + nonExistingDependency
-          + ") libraries not found in the classpath. gfsh can't start.");
-      return;
-    }
-
-    Launcher launcher = new Launcher();
-    System.exit(launcher.parseCommandLine(args));
-  }
+  private final Set<String> allowedCommandLineCommands;
+  private final OptionParser commandLineParser;
+  private StartupTimeLogHelper startupTimeLogHelper;
 
   protected Launcher() {
     this.startupTimeLogHelper = new StartupTimeLogHelper();
@@ -138,6 +121,20 @@ public final class Launcher {
     this.commandLineParser.posixlyCorrect(false);
   }
 
+  public static void main(final String[] args) {
+    // first check whether required dependencies exist in the classpath
+    // should we start without tomcat/servlet jars?
+    String nonExistingDependency = CliUtil.cliDependenciesExist(true);
+    if (nonExistingDependency != null) {
+      System.err.println("Required (" + nonExistingDependency
+          + ") libraries not found in the classpath. gfsh can't start.");
+      return;
+    }
+
+    Launcher launcher = new Launcher();
+    System.exit(launcher.parseCommandLine(args));
+  }
+
   private int parseCommandLineCommand(final String... args) {
     Gfsh gfsh = null;
     try {
@@ -253,7 +250,7 @@ public final class Launcher {
   }
 
   private int parseCommandLine(final String... args) {
-    if (args.length > 0 && !args[0].startsWith(SyntaxConstants.SHORT_OPTION_SPECIFIER)) {
+    if (args.length > 0 && !args[0].startsWith(GfshParser.SHORT_OPTION_SPECIFIER)) {
       return parseCommandLineCommand(args);
     }
 
@@ -276,6 +273,7 @@ public final class Launcher {
   }
 
   private void printUsage(final Gfsh gfsh, final PrintStream stream) {
+    int terminalWidth = gfsh.getTerminalWidth();
     StringBuilder usageBuilder = new StringBuilder();
     stream.print("Pivotal GemFire(R) v");
     stream.print(GemFireVersion.getGemFireVersion());
@@ -289,23 +287,22 @@ public final class Launcher {
     stream.println(Gfsh.wrapText(
         "Commands may be any that are available from the interactive gfsh prompt.  "
             + "For commands that require a Manager to complete, the first command in the list must be \"connect\".",
-        1));
-    stream.println(GfshParser.LINE_SEPARATOR + "AVAILABLE COMMANDS");
-    stream.print(gfsh.obtainHelp("", this.allowedCommandLineCommands));
+        1, terminalWidth));
     stream.println("EXAMPLES");
     stream.println("gfsh");
-    stream.println(Gfsh.wrapText("Start GFSH in interactive mode.", 1));
+    stream.println(Gfsh.wrapText("Start GFSH in interactive mode.", 1, terminalWidth));
     stream.println("gfsh -h");
-    stream.println(
-        Gfsh.wrapText("Displays 'this' help. ('gfsh --help' or 'gfsh help' is equivalent)", 1));
+    stream.println(Gfsh.wrapText(
+        "Displays 'this' help. ('gfsh --help' or 'gfsh help' is equivalent)", 1, terminalWidth));
     stream.println("gfsh help start locator");
-    stream.println(Gfsh.wrapText("Display help for the \"start locator\" command.", 1));
+    stream.println(
+        Gfsh.wrapText("Display help for the \"start locator\" command.", 1, terminalWidth));
     stream.println("gfsh start locator --name=locator1");
-    stream.println(Gfsh.wrapText("Start a Locator with the name \"locator1\".", 1));
+    stream.println(Gfsh.wrapText("Start a Locator with the name \"locator1\".", 1, terminalWidth));
     stream.println("gfsh -e \"connect\" -e \"list members\"");
     stream.println(Gfsh.wrapText(
         "Connect to a running Locator using the default connection information and run the \"list members\" command.",
-        1));
+        1, terminalWidth));
     stream.println();
 
     printExecuteUsage(stream);

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/annotation/CliArgument.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/annotation/CliArgument.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/annotation/CliArgument.java
deleted file mode 100644
index e20e731..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/annotation/CliArgument.java
+++ /dev/null
@@ -1,81 +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.annotation;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-import org.springframework.shell.core.Converter;
-import org.springframework.shell.core.annotation.CliCommand;
-
-/**
- * Annotation for Argument of a Command
- * 
- * @since GemFire 7.0
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.PARAMETER)
-public @interface CliArgument {
-
-  /**
-   * @return name of the argument, useful during help and warning messages
-   */
-  String name();
-
-  /**
-   * @return a help message for this option (the default is a blank String, which means there is no
-   *         help)
-   */
-  String help() default "";
-
-  /**
-   * @return true if this argument must be specified one way or the other by the user (defaults to
-   *         false)
-   */
-  boolean mandatory() default false;
-
-  /**
-   * Returns a string providing context-specific information (e.g. a comma-delimited set of
-   * keywords) to the {@link Converter} that handles the annotated parameter's type.
-   * <p>
-   * For example, if a method parameter "thing" of type "Thing" is annotated as follows:
-   * 
-   * <pre>
-   * <code>@CliArgument(..., argumentContext = "foo,bar", ...) Thing thing</code>
-   * </pre>
-   * 
-   * ... then the {@link Converter} that converts the text entered by the user into an instance of
-   * Thing will be passed "foo,bar" as the value of the <code>optionContext</code> parameter in its
-   * public methods. This allows the behaviour of that Converter to be individually customised for
-   * each {@link CliArgument} of each {@link CliCommand}.
-   * 
-   * @return a non-<code>null</code> string (can be empty)
-   */
-  String argumentContext() default "";
-
-  /**
-   * @return if true, the user cannot specify this option and it is provided by the shell
-   *         infrastructure (defaults to false)
-   */
-  boolean systemProvided() default false;
-
-  /**
-   * @return the default value to use if this argument is unspecified by the user (defaults to
-   *         __NULL__, which causes null to be presented to any non-primitive parameter)
-   */
-  String unspecifiedDefaultValue() default "__NULL__";
-}


[5/9] geode git commit: GEODE-1597: use Spring shell's parser and delete our own parsing code

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/utils/FormatOutput.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/utils/FormatOutput.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/utils/FormatOutput.java
deleted file mode 100644
index 44998a0..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/utils/FormatOutput.java
+++ /dev/null
@@ -1,33 +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.help.utils;
-
-import java.util.*;
-
-public class FormatOutput {
-
-  public static String converListToString(List<String> outputStringList) {
-    Iterator<String> iters = outputStringList.iterator();
-
-    StringBuilder sb = new StringBuilder(200);
-
-    while (iters.hasNext()) {
-      sb.append("\n");
-      sb.append((String) iters.next());
-    }
-
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/utils/HelpUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/utils/HelpUtils.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/utils/HelpUtils.java
deleted file mode 100644
index 11765c5..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/utils/HelpUtils.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.cli.help.utils;
-
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.internal.cli.help.format.*;
-import org.apache.geode.management.internal.cli.modes.CommandModes;
-import org.apache.geode.management.internal.cli.modes.CommandModes.CommandMode;
-import org.apache.geode.management.internal.cli.parser.Argument;
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * @since GemFire 7.0
- */
-public class HelpUtils {
-  public static final String EXE_PREFIX_FOR_EXTERNAL_HELP =
-      org.apache.geode.management.internal.cli.shell.Gfsh.GFSH_APP_NAME + " ";
-  public static final String HELP__COMMAND_AVAILABLE = "Available";
-  public static final String HELP__COMMAND_NOTAVAILABLE = "Not Available";
-
-  private static final String NAME_NAME = "NAME";
-  private static final String SYNONYMS_NAME = "SYNONYMS";
-  private static final String SYNOPSIS_NAME = "SYNOPSIS";
-  private static final String SYNTAX_NAME = "SYNTAX";
-  private static final String ARGUMENTS_NAME = "ARGUMENTS";
-  private static final String OPTIONS_NAME = "PARAMETERS";
-  private static final String IS_AVAILABLE_NAME = "IS AVAILABLE";
-  private static final String MODES = "MODES";
-
-  private static final String REQUIRED_SUB_NAME = "Required: ";
-  private static final String DEFAULTVALUE_SUB_NAME = "Default value: ";
-  private static final String SYNONYMS_SUB_NAME = "Synonyms: ";
-  private static final String SPECIFIEDDEFAULTVALUE_SUB_NAME =
-      "Default (if the parameter is specified without value): ";
-  private static final String UNSPECIFIEDDEFAULTVALUE_VALUE_SUB_NAME =
-      "Default (if the parameter is not specified): ";
-
-  private static final String VALUE_FIELD = "value";
-  private static final String TRUE_TOKEN = "true";
-  private static final String FALSE_TOKEN = "false";
-
-
-  private static Help help(Block[] blocks) {
-    return new Help().setBlocks(blocks);
-  }
-
-  private static Block block(String heading, Row... rows) {
-    return new Block().setHeading(heading).setRows(rows);
-  }
-
-  private static Row row(String... info) {
-    return new Row().setInfo(info);
-  }
-
-  @Deprecated
-  public static Help getHelp(CommandTarget commandTarget) {
-    List<Block> blocks = new ArrayList<Block>();
-    // First we will have the block for NAME of the command
-    blocks.add(block(NAME_NAME, row(commandTarget.getCommandName())));
-    // Now add synonyms if any
-    if (commandTarget.getSynonyms() != null) {
-      blocks.add(block(SYNONYMS_NAME, row(commandTarget.getSynonyms())));
-    }
-
-
-
-    // Now comes the turn to display synopsis if any
-    if (commandTarget.getCommandHelp() != null && !commandTarget.getCommandHelp().equals("")) {
-      blocks.add(block(SYNOPSIS_NAME, row(commandTarget.getCommandHelp())));
-    }
-    // Now display the syntax for the command
-    StringBuffer buffer = new StringBuffer();
-    buffer.append(commandTarget.getCommandName());
-    // Create a list which will store optional arguments
-    List<Argument> optionalArguments = new ArrayList<Argument>();
-    for (Argument argument : commandTarget.getOptionParser().getArguments()) {
-      if (argument.isRequired()) {
-        buffer.append(" " + argument.getArgumentName());
-      } else {
-        optionalArguments.add(argument);
-      }
-    }
-    for (Argument argument : optionalArguments) {
-      buffer.append(" " + "[" + argument.getArgumentName() + "]");
-    }
-    // Create a list which will store optional options
-    List<Option> optionalOptions = new ArrayList<Option>();
-    for (Option option : commandTarget.getOptionParser().getOptions()) {
-      if (option.isRequired()) {
-        buffer.append(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + option.getLongOption());
-        // String temp = SyntaxConstants.OPTION_VALUE_SPECIFIER + VALUE_TOKEN + "("
-        // + SyntaxConstants.OPTION_VALUE_SEPARATOR + VALUE_TOKEN + ")*";
-        String temp = buildOptionHelpText(option);
-        //
-        if (option.getSpecifiedDefaultValue() != null
-            && !option.getSpecifiedDefaultValue().equals("")) {
-          buffer.append("(");
-          buffer.append(temp);
-          buffer.append(")?");
-        } else {
-          buffer.append(temp);
-        }
-      } else {
-        optionalOptions.add(option);
-      }
-    }
-    for (Option option : optionalOptions) {
-      buffer.append(" " + "[" + SyntaxConstants.LONG_OPTION_SPECIFIER + option.getLongOption());
-      // String temp = SyntaxConstants.OPTION_VALUE_SPECIFIER + VALUE_TOKEN + "("
-      // + SyntaxConstants.OPTION_VALUE_SEPARATOR + VALUE_TOKEN + ")*";
-      String temp = buildOptionHelpText(option);
-      //
-      if (option.getSpecifiedDefaultValue() != null
-          && !option.getSpecifiedDefaultValue().equals("")) {
-        buffer.append("(");
-        buffer.append(temp);
-        buffer.append(")?");
-      } else {
-        buffer.append(temp);
-      }
-      buffer.append("]");
-    }
-    blocks.add(block(SYNTAX_NAME, row(buffer.toString())));
-    // Detailed description of Arguments
-    if (commandTarget.getOptionParser().getArguments().size() > 0) {
-      List<Row> rows = new ArrayList<Row>();
-      for (Argument argument : commandTarget.getOptionParser().getArguments()) {
-        rows.add(row(argument.getArgumentName()
-            + ((argument.getHelp() != null && !argument.getHelp().equals(""))
-                ? ":" + argument.getHelp() : "")));
-      }
-      Row[] rowsArray = new Row[rows.size()];
-      blocks.add(block(ARGUMENTS_NAME, rows.toArray(rowsArray)));
-    }
-
-    // Detailed description of Options
-    if (commandTarget.getOptionParser().getOptions().size() > 0) {
-      List<Row> rows = new ArrayList<Row>();
-      for (Option option : commandTarget.getOptionParser().getOptions()) {
-        rows.add(
-            row(option.getLongOption() + ((option.getHelp() != null && !option.getHelp().equals(""))
-                ? ":" + option.getHelp() : "")));
-      }
-      Row[] rowsArray = new Row[rows.size()];
-      blocks.add(block(OPTIONS_NAME, rows.toArray(rowsArray)));
-    }
-    Block[] blocksArray = new Block[blocks.size()];
-    for (int i = 0; i < blocks.size(); i++) {
-      blocksArray[i] = blocks.get(i);
-    }
-    return help(blocksArray);
-  }
-
-  /**
-   * Builds help for the specified command.
-   * 
-   * @param commandTarget command specific target to use to generate help
-   * @param withinShell if <code>true</code> includes availabilty & doesn't include application name
-   * @return built NewHelp object for the given command target
-   */
-  public static NewHelp getNewHelp(CommandTarget commandTarget, boolean withinShell) {
-    DataNode root = new DataNode(null, new ArrayList<DataNode>());
-    // First we will have the block for NAME of the command
-    DataNode name = new DataNode(NAME_NAME, new ArrayList<DataNode>());
-    name.addChild(new DataNode(commandTarget.getCommandName(), null));
-    root.addChild(name);
-    if (withinShell) {// include availabilty info
-      DataNode availability = new DataNode(IS_AVAILABLE_NAME, new ArrayList<DataNode>());
-      boolean isAvailable = false;
-      try {
-        isAvailable = commandTarget.isAvailable();
-      } catch (Exception e) {
-        isAvailable = false;
-      }
-      availability.addChild(new DataNode(String.valueOf(isAvailable), null));
-      root.addChild(availability);
-    }
-    // Now add synonyms if any
-    if (commandTarget.getSynonyms() != null) {
-      DataNode synonyms = new DataNode(SYNONYMS_NAME, new ArrayList<DataNode>());
-      for (String string : commandTarget.getSynonyms()) {
-        synonyms.addChild(new DataNode(string, null));
-      }
-      root.addChild(synonyms);
-    }
-
-
-    // Now comes the turn to display synopsis if any
-    if (commandTarget.getCommandHelp() != null && !commandTarget.getCommandHelp().equals("")) {
-      DataNode synopsis = new DataNode(SYNOPSIS_NAME, new ArrayList<DataNode>());
-      synopsis.addChild(new DataNode(commandTarget.getCommandHelp(), null));
-      root.addChild(synopsis);
-    }
-
-
-    // Now display the syntax for the command
-    StringBuffer buffer = new StringBuffer();
-    if (withinShell) {
-      buffer.append(commandTarget.getCommandName());
-    } else { // add app name in the syntax
-      buffer.append(EXE_PREFIX_FOR_EXTERNAL_HELP).append(commandTarget.getCommandName());
-    }
-    appendArguments(buffer, commandTarget);
-    appendOptions(buffer, commandTarget);
-    DataNode syntax = new DataNode(SYNTAX_NAME, new ArrayList<DataNode>());
-    syntax.addChild(new DataNode(buffer.toString(), null));
-    root.addChild(syntax);
-
-
-    // Detailed description of Arguments
-    if (commandTarget.getOptionParser().getArguments().size() > 0) {
-      DataNode arguments = new DataNode(ARGUMENTS_NAME, new ArrayList<DataNode>());
-      for (Argument argument : commandTarget.getOptionParser().getArguments()) {
-        DataNode argumentNode = new DataNode(argument.getArgumentName(), new ArrayList<DataNode>());
-        argumentNode
-            .addChild(new DataNode(((argument.getHelp() != null && !argument.getHelp().equals(""))
-                ? argument.getHelp() : ""), null));
-        argumentNode.addChild(new DataNode(
-            REQUIRED_SUB_NAME + ((argument.isRequired()) ? TRUE_TOKEN : FALSE_TOKEN), null));
-        if (argument.getUnspecifiedDefaultValue() != null) {
-          argumentNode.addChild(
-              new DataNode(DEFAULTVALUE_SUB_NAME + argument.getUnspecifiedDefaultValue(), null));
-        }
-        arguments.addChild(argumentNode);
-      }
-      root.addChild(arguments);
-    }
-
-
-    try {
-      CommandModes modes = CommandModes.getInstance();
-      Collection<CommandMode> comModes = modes.getCommandModes(commandTarget.getCommandName());
-      DataNode modesDN = new DataNode(MODES, new ArrayList<DataNode>());
-      if (comModes != null) {
-        for (CommandMode cmd : comModes) {
-          StringBuffer sb = new StringBuffer();
-          List<Option> optionalOptions = new ArrayList<Option>();
-
-          sb.append(commandTarget.getCommandName()).append(" ");
-          if (!cmd.name.equals("default"))
-            appendRequiredOption(sb, getOption(commandTarget, cmd.leadOption));
-
-          for (String opt : cmd.options) {
-            if (!opt.equals(cmd.leadOption)) {
-              Option option = getOption(commandTarget, opt);
-              if (option.isRequired()) {
-                appendRequiredOption(sb, option);
-              } else
-                optionalOptions.add(option);
-            }
-          }
-
-          for (Option optOpt : optionalOptions)
-            appendOption(sb, optOpt);
-
-          DataNode modeDN = new DataNode(cmd.text, new ArrayList<DataNode>());
-          modeDN.addChild(new DataNode(sb.toString(), null));
-          modesDN.addChild(modeDN);
-        }
-        root.addChild(modesDN);
-      } else {
-        // modesDN.addChild(new DataNode("No command modes found", null));
-        // root.addChild(modesDN);
-      }
-
-    } catch (Exception e) {
-    } finally {
-
-    }
-
-    // Detailed description of Options
-    if (commandTarget.getOptionParser().getOptions().size() > 0) {
-      DataNode options = new DataNode(OPTIONS_NAME, new ArrayList<DataNode>());
-      for (Option option : commandTarget.getOptionParser().getOptions()) {
-        DataNode optionNode = new DataNode(option.getLongOption(), new ArrayList<DataNode>());
-        optionNode.addChild(new DataNode(
-            ((option.getHelp() != null && !option.getHelp().equals("")) ? option.getHelp() : ""),
-            null));
-        if (option.getSynonyms() != null && option.getSynonyms().size() > 0) {
-          StringBuilder builder = new StringBuilder();
-          for (String string : option.getSynonyms()) {
-            if (builder.length() > 0) {
-              builder.append(",");
-            }
-            builder.append(string);
-          }
-          optionNode.addChild(new DataNode(SYNONYMS_SUB_NAME + builder.toString(), null));
-        }
-        optionNode.addChild(new DataNode(
-            REQUIRED_SUB_NAME + ((option.isRequired()) ? TRUE_TOKEN : FALSE_TOKEN), null));
-        if (option.getSpecifiedDefaultValue() != null
-            && !option.getSpecifiedDefaultValue().equals("")) {
-          optionNode.addChild(new DataNode(
-              SPECIFIEDDEFAULTVALUE_SUB_NAME + option.getSpecifiedDefaultValue(), null));
-        }
-        if (option.getUnspecifiedDefaultValue() != null
-            && !option.getUnspecifiedDefaultValue().equals("")) {
-          optionNode.addChild(new DataNode(
-              UNSPECIFIEDDEFAULTVALUE_VALUE_SUB_NAME + option.getUnspecifiedDefaultValue(), null));
-        }
-        options.addChild(optionNode);
-      }
-      root.addChild(options);
-    }
-    return new NewHelp(root);
-  }
-
-  private static Option getOption(CommandTarget commandTarget, String opt) {
-    for (Option option : commandTarget.getOptionParser().getOptions()) {
-      if (option.getLongOption().equals(opt))
-        return option;
-    }
-    return null;
-  }
-
-  private static void appendOptions(StringBuffer buffer, CommandTarget commandTarget) {
-    List<Option> optionalOptions = new ArrayList<Option>();
-    for (Option option : commandTarget.getOptionParser().getOptions()) {
-      if (option.isRequired()) {
-        appendRequiredOption(buffer, option);
-      } else {
-        optionalOptions.add(option);
-      }
-    }
-    for (Option option : optionalOptions) {
-      appendOption(buffer, option);
-    }
-  }
-
-  private static void appendRequiredOption(StringBuffer buffer, Option option) {
-    buffer.append(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + option.getLongOption());
-    String temp = buildOptionHelpText(option);
-    if (option.getSpecifiedDefaultValue() != null
-        && !option.getSpecifiedDefaultValue().equals("")) {
-      buffer.append("(").append(temp).append(")?");
-    } else {
-      buffer.append(temp);
-    }
-  }
-
-  private static void appendOption(StringBuffer buffer, Option option) {
-    buffer.append(" " + "[" + SyntaxConstants.LONG_OPTION_SPECIFIER + option.getLongOption());
-    String temp = buildOptionHelpText(option);
-    if (option.getSpecifiedDefaultValue() != null
-        && !option.getSpecifiedDefaultValue().equals("")) {
-      buffer.append("(").append(temp).append(")?");
-    } else {
-      buffer.append(temp);
-    }
-    buffer.append("]");
-  }
-
-  private static void appendArguments(StringBuffer buffer, CommandTarget commandTarget) {
-    // Create a list which will store optional arguments
-    List<Argument> optionalArguments = new ArrayList<Argument>();
-    for (Argument argument : commandTarget.getOptionParser().getArguments()) {
-      if (argument.isRequired()) {
-        buffer.append(" " + argument.getArgumentName());
-      } else {
-        optionalArguments.add(argument);
-      }
-    }
-    for (Argument argument : optionalArguments) {
-      buffer.append(" " + "[" + argument.getArgumentName() + "]");
-    }
-  }
-
-  public static String buildOptionHelpText(Option option) {
-    String temp = SyntaxConstants.OPTION_VALUE_SPECIFIER + VALUE_FIELD;
-    if ((option.getValueSeparator() != null
-        && !CliMetaData.ANNOTATION_NULL_VALUE.equals(option.getValueSeparator())
-        && !option.getValueSeparator().equals(""))
-        || isCollectionOrArrayType(option.getDataType())) {
-      temp += "(" + option.getValueSeparator() + VALUE_FIELD + ")*";
-    }
-    return temp;
-  }
-
-  private static boolean isCollectionOrArrayType(Class<?> typeToCheck) {
-    return typeToCheck != null
-        && (typeToCheck.isArray() || Collection.class.isAssignableFrom(typeToCheck));
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index 126eb47..a793235 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -134,8 +134,10 @@ public class CliStrings {
   public static final String TOPIC_SHARED_CONFIGURATION = "Cluster Configuration";
   public static final String TOPIC_SHARED_CONFIGURATION_HELP =
       "Configuration for cluster and various groups. It consists of cache.xml, geode properties and deployed jars.\nChanges due to gfshs command are persisted to the locator hosting the cluster configuration service.";
-  public static final String TOPIC_CHANGELOGLEVEL =
-      "User can change the log-level for a  member run time and generate log contents as per the need";
+  public static final String TOPIC_LOGS = "Logs";
+  public static final String TOPIC_LOGS__DESC = "Generate log contents as per the need";
+  public static final String TOPIC_CLIENT = "Client";
+  public static final String TOPIC_CLIENT__DESC = "Client status";
 
   /*-*************************************************************************
    * ********* String Constants other than command name, options & help ******
@@ -1592,7 +1594,6 @@ public class CliStrings {
   /* 'list client' command */
   public static final String LIST_CLIENTS = "list clients";
   public static final String LIST_CLIENT__HELP = "Display list of connected clients";
-  public static final String TOPIC_LIST = "Display list of connected clients";
   public static final String LIST_CLIENT_COULD_NOT_RETRIEVE_CLIENT_LIST_0 =
       "Could not retrieve list of clients. Reason : {0}";
   public static final String LIST_CLIENT_COULD_NOT_RETRIEVE_SERVER_LIST =

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/multistep/CLIMultiStepHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/multistep/CLIMultiStepHelper.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/multistep/CLIMultiStepHelper.java
index 0d21543..d53261d 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/multistep/CLIMultiStepHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/multistep/CLIMultiStepHelper.java
@@ -14,10 +14,6 @@
  */
 package org.apache.geode.management.internal.cli.multistep;
 
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.geode.LogWriter;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.management.cli.Result;
@@ -36,10 +32,13 @@ import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.result.ResultData;
 import org.apache.geode.management.internal.cli.result.TabularResultData;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
-
 import org.springframework.shell.event.ParseResult;
 import org.springframework.util.ReflectionUtils;
 
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Utility class to abstract CompositeResultData for Multi-step commands Also contain execution
  * strategy for multi-step commands
@@ -57,7 +56,7 @@ public class CLIMultiStepHelper {
   public static final String STEP_ARGS = "stepArgs";
   public static final int DEFAULT_PAGE_SIZE = 20;
 
-  public static Object execCLISteps(LogWrapper logWrapper, Gfsh shell, ParseResult parseResult) {
+  public static Result execCLISteps(LogWrapper logWrapper, Gfsh shell, ParseResult parseResult) {
     CLIStep[] steps = (CLIStep[]) ReflectionUtils.invokeMethod(parseResult.getMethod(),
         parseResult.getInstance(), parseResult.getArguments());
     if (steps != null) {

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Argument.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Argument.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Argument.java
deleted file mode 100644
index 9acbc2a..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Argument.java
+++ /dev/null
@@ -1,71 +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.parser;
-
-/**
- * Argument of a Command
- * 
- * @since GemFire 7.0
- *
- */
-public class Argument extends Parameter {
-  private String argumentName;
-
-  public String getArgumentName() {
-    return argumentName;
-  }
-
-  public void setArgumentName(String argumentName) {
-    this.argumentName = argumentName;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 13;
-    int result = 1;
-    result = prime * result + ((argumentName == null) ? 0 : argumentName.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    Argument argument = (Argument) obj;
-    if (argumentName == null) {
-      if (argument.getArgumentName() != null) {
-        return false;
-      }
-    } else if (!argumentName.equals(argument.getArgumentName())) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append(Argument.class.getSimpleName()).append("[name=" + argumentName)
-        .append(",help=" + help).append(",required" + required + "]");
-    return builder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/AvailabilityTarget.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/AvailabilityTarget.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/AvailabilityTarget.java
deleted file mode 100644
index eff5fd2..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/AvailabilityTarget.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.cli.parser;
-
-import java.lang.reflect.Method;
-
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.CliMetaData.AvailabilityMetadata;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-
-/**
- * Used for checking availability of a command
- *
- * @since GemFire 7.0
- */
-public class AvailabilityTarget {
-
-  private final Object target;
-  private final Method method;
-  private final String availabilityDescription;
-
-  public AvailabilityTarget(Object target, Method method) {
-    this.target = target;
-    this.method = method;
-    AvailabilityMetadata availabilityMetadata =
-        this.method.getAnnotation(CliMetaData.AvailabilityMetadata.class);
-    String specifiedAvailabilityDesc =
-        CliStrings.AVAILABILITYTARGET_MSG_DEFAULT_UNAVAILABILITY_DESCRIPTION;
-    if (availabilityMetadata != null) {
-      specifiedAvailabilityDesc = availabilityMetadata.availabilityDescription();
-    }
-    this.availabilityDescription = specifiedAvailabilityDesc;
-  }
-
-  public Object getTarget() {
-    return target;
-  }
-
-  public Method getMethod() {
-    return method;
-  }
-
-  public String getAvailabilityDescription() {
-    return availabilityDescription;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 17;
-    int result = 8;
-    result = prime * result + ((target == null) ? 0 : target.hashCode());
-    result = prime * result + ((method == null) ? 0 : method.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    AvailabilityTarget availabilityTarget = (AvailabilityTarget) obj;
-    if (target == null) {
-      if (availabilityTarget.getTarget() != null) {
-        return false;
-      }
-    } else if (!target.equals(availabilityTarget.getTarget())) {
-      return false;
-    }
-    if (method == null) {
-      if (availabilityTarget.getMethod() != null) {
-        return false;
-      }
-    } else if (!method.equals(availabilityTarget.getMethod())) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append(AvailabilityTarget.class.getSimpleName());
-    builder.append("[" + "target=" + target);
-    builder.append(",method=" + method);
-    builder.append("]");
-    return builder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/CommandTarget.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/CommandTarget.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/CommandTarget.java
deleted file mode 100644
index 3dfc01a..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/CommandTarget.java
+++ /dev/null
@@ -1,176 +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.parser;
-
-import java.lang.reflect.InvocationTargetException;
-
-import org.apache.geode.management.internal.cli.GfshParser;
-
-/**
- * Used by {@link GfshParser} to store details of a command
- * 
- * @since GemFire 7.0
- * 
- */
-public class CommandTarget {
-  private final String commandName;
-  private final String[] synonyms;
-  private final String commandHelp;
-  private final GfshMethodTarget gfshMethodTarget;
-  private final GfshOptionParser optionParser;
-  private AvailabilityTarget availabilityIndicator;
-
-  public CommandTarget(String commandName, String[] synonyms, GfshMethodTarget methodTarget,
-      GfshOptionParser optionParser, AvailabilityTarget availabilityIndicator, String commandHelp) {
-    this.commandName = commandName;
-    this.synonyms = synonyms;
-    this.gfshMethodTarget = methodTarget;
-    this.optionParser = optionParser;
-    this.availabilityIndicator = availabilityIndicator;
-    this.commandHelp = commandHelp;
-  }
-
-  public GfshMethodTarget getGfshMethodTarget() {
-    return gfshMethodTarget;
-  }
-
-  public GfshOptionParser getOptionParser() {
-    return optionParser;
-  }
-
-  public boolean isAvailable()
-      throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
-    if (availabilityIndicator != null) {
-      return (Boolean) availabilityIndicator.getMethod().invoke(availabilityIndicator.getTarget());
-    } else {
-      return true;
-    }
-  }
-
-  public AvailabilityTarget getAvailabilityIndicator() {
-    return availabilityIndicator;
-  }
-
-  // TODO Change for concurrent access
-  public void setAvailabilityIndicator(AvailabilityTarget availabilityIndicator) {
-    this.availabilityIndicator = availabilityIndicator;
-  }
-
-  public String getCommandHelp() {
-    return commandHelp;
-  }
-
-  public CommandTarget duplicate(String key) {
-    return duplicate(key, null);
-  }
-
-  public CommandTarget duplicate(String key, String remainingBuffer) {
-    return new CommandTarget(
-        commandName, synonyms, new GfshMethodTarget(gfshMethodTarget.getMethod(),
-            gfshMethodTarget.getTarget(), remainingBuffer, key),
-        optionParser, availabilityIndicator, commandHelp);
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 47;
-    int result = 3;
-    result = prime * result + ((commandName == null) ? 0 : commandName.hashCode());
-    result = prime * result + ((commandHelp == null) ? 0 : commandHelp.hashCode());
-    result = prime * result + ((gfshMethodTarget == null) ? 0 : gfshMethodTarget.hashCode());
-    result = prime * result + ((optionParser == null) ? 0 : optionParser.hashCode());
-    result =
-        prime * result + ((availabilityIndicator == null) ? 0 : availabilityIndicator.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    // If two command targets have the same OptionParser
-    // then they are equal
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    CommandTarget commandTarget = (CommandTarget) obj;
-    if (commandName == null) {
-      if (commandTarget.getCommandName() != null) {
-        return false;
-      }
-    } else if (!commandName.equals(commandTarget.getCommandName())) {
-      return false;
-    }
-    if (commandHelp == null) {
-      if (commandTarget.getCommandHelp() != null) {
-        return false;
-      }
-    } else if (!commandHelp.equals(commandTarget.getCommandHelp())) {
-      return false;
-    }
-    if (gfshMethodTarget == null) {
-      if (commandTarget.getGfshMethodTarget() != null) {
-        return false;
-      }
-    } else if (!gfshMethodTarget.equals(commandTarget.getGfshMethodTarget())) {
-      return false;
-    }
-    if (optionParser == null) {
-      if (commandTarget.getOptionParser() != null) {
-        return false;
-      }
-    } else if (!optionParser.equals(commandTarget.getOptionParser())) {
-      return false;
-    }
-    if (availabilityIndicator == null) {
-      if (commandTarget.getAvailabilityIndicator() != null) {
-        return false;
-      }
-    } else if (!availabilityIndicator.equals(commandTarget.getAvailabilityIndicator())) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append(CommandTarget.class.getSimpleName()).append("[" + "commandName=" + commandName)
-        .append(",commandHelp=" + commandHelp);
-    builder.append(",synonyms=");
-    if (synonyms != null) {
-      for (String string : synonyms) {
-        builder.append(string + " ");
-      }
-    }
-    builder.append(",gfshMethodTarget=" + gfshMethodTarget);
-    builder.append(",optionParser=" + optionParser);
-    builder.append(",availabilityIndicator=" + availabilityIndicator);
-    builder.append("]");
-    return builder.toString();
-  }
-
-  public String getCommandName() {
-    return commandName;
-  }
-
-  public String[] getSynonyms() {
-    return synonyms;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/GfshMethodTarget.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/GfshMethodTarget.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/GfshMethodTarget.java
deleted file mode 100644
index 6f28830..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/GfshMethodTarget.java
+++ /dev/null
@@ -1,121 +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.parser;
-
-import java.lang.reflect.Method;
-
-import org.apache.commons.lang.StringUtils;
-import org.springframework.util.Assert;
-import org.springframework.util.ObjectUtils;
-
-/**
- * A method that can be executed via a shell command.
- * 
- * @since GemFire 7.0
- */
-public class GfshMethodTarget {
-
-  // Fields
-  private final Method method;
-  private final Object target;
-  private final String remainingBuffer;
-  private final String key;
-
-  /**
-   * Constructor for a <code>null remainingBuffer</code> and <code>key</code>
-   * 
-   * @param method the method to invoke (required)
-   * @param target the object on which the method is to be invoked (required)
-   */
-  public GfshMethodTarget(final Method method, final Object target) {
-    this(method, target, null, null);
-  }
-
-  /**
-   * Constructor that allows all fields to be set
-   * 
-   * @param method the method to invoke (required)
-   * @param target the object on which the method is to be invoked (required)
-   * @param remainingBuffer can be blank
-   * @param key can be blank
-   */
-  public GfshMethodTarget(final Method method, final Object target, final String remainingBuffer,
-      final String key) {
-    Assert.notNull(method, "Method is required");
-    Assert.notNull(target, "Target is required");
-    this.key = StringUtils.trimToEmpty(key);
-    this.method = method;
-    this.remainingBuffer = remainingBuffer;
-    this.target = target;
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other) {
-      return true;
-    }
-    if (other == null) {
-      return false;
-    }
-    if (getClass() != other.getClass()) {
-      return false;
-    }
-    GfshMethodTarget gfshMethodTarget = (GfshMethodTarget) other;
-    if (method == null) {
-      if (gfshMethodTarget.getMethod() != null) {
-        return false;
-      }
-    } else if (!method.equals(gfshMethodTarget.getMethod())) {
-      return false;
-    }
-    if (target == null) {
-      if (gfshMethodTarget.getTarget() != null) {
-        return false;
-      }
-    } else if (!target.equals(gfshMethodTarget.getTarget())) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    return ObjectUtils.nullSafeHashCode(new Object[] {method, target});
-  }
-
-  @Override
-  public final String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append(GfshMethodTarget.class.getSimpleName()).append("[key=" + key).append(
-        ",remainingBuffer=" + remainingBuffer + ",target=" + target + ",method=" + method + "]");
-    return builder.toString();
-  }
-
-  public String getKey() {
-    return this.key;
-  }
-
-  public Method getMethod() {
-    return this.method;
-  }
-
-  public String getRemainingBuffer() {
-    return this.remainingBuffer;
-  }
-
-  public Object getTarget() {
-    return this.target;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/GfshOptionParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/GfshOptionParser.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/GfshOptionParser.java
deleted file mode 100644
index a64933c..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/GfshOptionParser.java
+++ /dev/null
@@ -1,37 +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.parser;
-
-import java.util.LinkedList;
-
-import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.exceptions.CliException;
-
-/**
- * Delegate used for parsing by {@link GfshParser}
- * 
- * @since GemFire 7.0
- */
-public interface GfshOptionParser {
-  public void setArguments(LinkedList<Argument> arguments);
-
-  public LinkedList<Argument> getArguments();
-
-  public void setOptions(LinkedList<Option> options);
-
-  public LinkedList<Option> getOptions();
-
-  OptionSet parse(String userInput) throws CliException;
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/MethodParameter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/MethodParameter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/MethodParameter.java
deleted file mode 100644
index 599fb00..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/MethodParameter.java
+++ /dev/null
@@ -1,39 +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.parser;
-
-/**
- * Object used for ordering method parameters
- * 
- * @since GemFire 7.0
- * 
- */
-public class MethodParameter {
-  private final Object parameter;
-  private final int parameterNo;
-
-  public MethodParameter(Object parameter, int parameterNo) {
-    this.parameter = parameter;
-    this.parameterNo = parameterNo;
-  }
-
-  public Object getParameter() {
-    return parameter;
-  }
-
-  public int getParameterNo() {
-    return parameterNo;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Option.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Option.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Option.java
deleted file mode 100644
index 4eec112..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Option.java
+++ /dev/null
@@ -1,217 +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.parser;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.geode.management.internal.cli.parser.preprocessor.PreprocessorUtils;
-
-/**
- * Option of a Command
- * 
- * @since GemFire 7.0
- * 
- */
-public class Option extends Parameter {
-
-  private static final String NULL = "__NULL__";
-  private static final char SHORT_OPTION_DEFAULT = '\u0000';
-  // Used for Option Identification
-  private char shortOption;
-  private String longOption;
-  private List<String> synonyms;
-  private List<String> aggregate;
-
-  // Option Value related
-  private String specifiedDefaultValue;
-
-  // Constraints on Option
-  private boolean withRequiredArgs;
-  private String valueSeparator;
-
-  public Option() {
-    aggregate = new ArrayList<String>();
-  }
-
-  public Option(char shortOption) {
-    this(shortOption, null, null);
-  }
-
-  public Option(char shortOption, List<String> synonyms) {
-    this(shortOption, null, synonyms);
-  }
-
-  public Option(String longOption) {
-    this(SHORT_OPTION_DEFAULT, longOption, null);
-  }
-
-  public Option(String longOption, List<String> synonyms) {
-    this(SHORT_OPTION_DEFAULT, longOption, synonyms);
-  }
-
-  public Option(char shortOption, String longOption) {
-    this(shortOption, longOption, null);
-  }
-
-  public Option(char shortOption, String longOption, List<String> synonyms) {
-    aggregate = new ArrayList<String>();
-    this.shortOption = shortOption;
-    this.longOption = longOption;
-    this.synonyms = synonyms;
-    if (shortOption != SHORT_OPTION_DEFAULT) {
-      aggregate.add("" + shortOption);
-    }
-    if (longOption != null) {
-      aggregate.add(longOption);
-    }
-    if (synonyms != null) {
-      aggregate.addAll(synonyms);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append(Option.class.getSimpleName()).append("[longOption=" + longOption)
-        .append(",help=" + help).append(",required=" + required + "]");
-    return builder.toString();
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 41;
-    int result = 1;
-    result = prime * result + ((longOption == null) ? 0 : longOption.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    Option option = (Option) obj;
-    if (longOption == null) {
-      if (option.getLongOption() != null) {
-        return false;
-      }
-    } else if (!longOption.equals(option.getLongOption())) {
-      return false;
-    }
-    return true;
-  }
-
-  public List<String> getAggregate() {
-    return aggregate;
-  }
-
-  public char getShortOption() {
-    return shortOption;
-  }
-
-  public boolean setShortOption(char shortOption) {
-    if (shortOption != SHORT_OPTION_DEFAULT) {
-      int index = aggregate.indexOf("" + this.shortOption);
-      if (index != -1) {
-        return false;
-      } else {
-        this.shortOption = shortOption;
-        aggregate.add("" + shortOption);
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public String getLongOption() {
-    return longOption;
-  }
-
-  public boolean setLongOption(String longOption) {
-    longOption = longOption.trim();
-    if (!longOption.equals("")) {
-      if (this.longOption == null) {
-        int index = aggregate.indexOf(longOption);
-        if (index != -1) {
-          return false;
-        } else {
-          this.longOption = longOption;
-          aggregate.add(longOption);
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  public List<String> getSynonyms() {
-    return synonyms;
-  }
-
-  public void setSynonyms(List<String> synonyms) {
-    this.synonyms = new ArrayList<String>();
-    for (String string : synonyms) {
-      if (!string.equals("")) {
-        this.synonyms.add(string);
-      }
-    }
-    if (this.synonyms.size() > 0) {
-      this.aggregate.addAll(this.synonyms);
-    }
-  }
-
-  public boolean isWithRequiredArgs() {
-    return withRequiredArgs;
-  }
-
-  public void setWithRequiredArgs(boolean withRequiredArgs) {
-    this.withRequiredArgs = withRequiredArgs;
-  }
-
-  public String[] getStringArray() {
-    String[] stringArray = new String[aggregate.size()];
-    for (int i = 0; i < stringArray.length; i++) {
-      stringArray[i] = aggregate.get(i);
-    }
-    return stringArray;
-  }
-
-  public String getSpecifiedDefaultValue() {
-    if (specifiedDefaultValue.equals(NULL)) {
-      return null;
-    } else {
-      return specifiedDefaultValue;
-    }
-  }
-
-  public void setSpecifiedDefaultValue(String specifiedDefaultValue) {
-    this.specifiedDefaultValue = PreprocessorUtils.trim(specifiedDefaultValue).getString();
-  }
-
-  public String getValueSeparator() {
-    return valueSeparator;
-  }
-
-  public void setValueSeparator(String valueSeparator) {
-    this.valueSeparator = valueSeparator;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/OptionSet.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/OptionSet.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/OptionSet.java
deleted file mode 100644
index 42270e5..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/OptionSet.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.cli.parser;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Stores the result after parsing
- * 
- * @since GemFire 7.0
- *
- */
-public class OptionSet {
-  private Map<Option, String> optionsMap;
-  private Map<Argument, String> argumentsMap;
-  private int noOfSpacesRemoved;
-  private List<String> split;
-  private String userInput;
-
-  public OptionSet() {
-    optionsMap = new HashMap<Option, String>();
-    argumentsMap = new HashMap<Argument, String>();
-  }
-
-  public void put(Argument argument, String value) {
-    argumentsMap.put(argument, value);
-  }
-
-  public void put(Option option, String value) {
-    optionsMap.put(option, value);
-  }
-
-  public boolean hasOption(Option option) {
-    return optionsMap.containsKey(option);
-  }
-
-  public boolean hasArgument(Argument argument) {
-    String string = argumentsMap.get(argument);
-    if (string != null) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  public boolean hasValue(Option option) {
-    String string = optionsMap.get(option);
-    if (string != null && !string.equals("__NULL__")) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  public String getValue(Argument argument) {
-    return argumentsMap.get(argument);
-  }
-
-  public String getValue(Option option) {
-    return optionsMap.get(option);
-  }
-
-  public boolean areArgumentsPresent() {
-    if (!argumentsMap.isEmpty()) {
-      return true;
-    } else
-      return false;
-  }
-
-  public boolean areOptionsPresent() {
-    if (!optionsMap.isEmpty()) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  public int getNoOfSpacesRemoved() {
-    return noOfSpacesRemoved;
-  }
-
-  public void setNoOfSpacesRemoved(int noOfSpacesRemoved) {
-    this.noOfSpacesRemoved = noOfSpacesRemoved;
-  }
-
-  /**
-   * @return the split
-   */
-  public List<String> getSplit() {
-    return split;
-  }
-
-  /**
-   * @param split the split to set
-   */
-  public void setSplit(List<String> split) {
-    this.split = split;
-  }
-
-  public String getUserInput() {
-    return userInput;
-  }
-
-  public void setUserInput(String userInput) {
-    this.userInput = userInput;
-  }
-
-  @Override
-  public String toString() {
-    return "OptionSet [optionsMap=" + optionsMap + ", argumentsMap=" + argumentsMap
-        + ", noOfSpacesRemoved=" + noOfSpacesRemoved + ", split=" + split + ", userInput="
-        + userInput + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Parameter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Parameter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Parameter.java
deleted file mode 100644
index dc371b3..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/Parameter.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.cli.parser;
-
-import org.springframework.shell.core.Converter;
-
-import org.apache.geode.management.internal.cli.parser.preprocessor.PreprocessorUtils;
-
-/**
- * Parameter of a Command
- * 
- * @since GemFire 7.0
- */
-public abstract class Parameter {
-  // help for the Parameter
-  protected String help;
-
-  // Constraint on the Parameter
-  protected boolean required;
-
-  // Useful for Value conversion
-  protected String context;
-  protected Converter<?> converter;
-
-  // Data type of the option
-  protected Class<?> dataType;
-
-  // Necessary for preserving order in
-  // ParseResult object
-  protected int parameterNo;
-
-  // value related
-  protected boolean systemProvided;
-  protected String unspecifiedDefaultValue;
-
-  public String getHelp() {
-    return help;
-  }
-
-  public void setHelp(String help) {
-    this.help = help;
-  }
-
-  public boolean isRequired() {
-    return required;
-  }
-
-  public void setRequired(boolean required) {
-    this.required = required;
-  }
-
-  public String getContext() {
-    return context;
-  }
-
-  public void setContext(String context) {
-    this.context = context;
-  }
-
-  public Converter<?> getConverter() {
-    return converter;
-  }
-
-  // TODO Change for concurrent access.
-  public void setConverter(Converter<?> converter) {
-    this.converter = converter;
-  }
-
-  public Class<?> getDataType() {
-    return dataType;
-  }
-
-  public void setDataType(Class<?> dataType) {
-    this.dataType = dataType;
-  }
-
-  public int getParameterNo() {
-    return parameterNo;
-  }
-
-  public void setParameterNo(int parameterNo) {
-    this.parameterNo = parameterNo;
-  }
-
-  public boolean isSystemProvided() {
-    return systemProvided;
-  }
-
-  public void setSystemProvided(boolean systemProvided) {
-    this.systemProvided = systemProvided;
-  }
-
-  public String getUnspecifiedDefaultValue() {
-    if (unspecifiedDefaultValue.equals("__NULL__")) {
-      return null;
-    } else {
-      return unspecifiedDefaultValue;
-    }
-  }
-
-  public void setUnspecifiedDefaultValue(String unspecifiedDefaultValue) {
-    this.unspecifiedDefaultValue = PreprocessorUtils.trim(unspecifiedDefaultValue).getString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/ParserUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/ParserUtils.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/ParserUtils.java
deleted file mode 100644
index 80f1286..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/ParserUtils.java
+++ /dev/null
@@ -1,186 +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.parser;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.geode.management.internal.cli.parser.preprocessor.PreprocessorUtils;
-
-/**
- * 
- * Utility class for parsing and pre-processing
- * 
- * The methods herein always ensure that the syntax is proper before performing the desired
- * operation
- * 
- * @since GemFire 7.0
- */
-public class ParserUtils {
-  public static String[] split(String input, String splitAround) {
-    if (input != null && splitAround != null) {
-      List<String> parts = new ArrayList<String>();
-      StringBuffer part = new StringBuffer();
-      outer: for (int i = 0; i < input.length(); i++) {
-        char ch = input.charAt(i);
-        if (splitAround.startsWith("" + ch)) {
-          // First check whether syntax is valid
-          if (PreprocessorUtils.isSyntaxValid(part.toString())) {
-            // This means that we need to check further whether
-            // the splitAround is present
-            StringBuffer temp = new StringBuffer("");
-            for (int j = 0; j < splitAround.length() && (i + j) < input.length(); j++) {
-              temp.append(input.charAt((i + j)));
-              if (temp.toString().equals(splitAround)) {
-                parts.add(part.toString().trim());
-                part.delete(0, part.length());
-                i = i + j + 1;
-                if (i < input.length()) {
-                  ch = input.charAt(i);
-                } else {
-                  break outer;
-                }
-                break;
-              }
-            }
-          }
-        }
-        part.append(ch);
-      }
-      // Need to copy the last part in the parts list
-      if (part.length() > 0) {
-        if (!PreprocessorUtils.containsOnlyWhiteSpaces(part.toString())) {
-          if (!part.toString().equals(splitAround))
-            parts.add(part.toString().trim());
-        }
-      }
-      // Convert the list into an array
-      String[] split = new String[parts.size()];
-      for (int i = 0; i < split.length; i++) {
-        split[i] = parts.get(i);
-      }
-      return split;
-    } else {
-      return null;
-    }
-  }
-
-  public static String[] splitValues(String value, String valueSeparator) {
-    if (value != null && valueSeparator != null) {
-      String[] split = split(value, valueSeparator);
-      if (value.endsWith(valueSeparator)
-          && PreprocessorUtils.isSyntaxValid(split[split.length - 1])) {
-        String[] extendedSplit = new String[split.length + 1];
-        for (int i = 0; i < split.length; i++) {
-          extendedSplit[i] = split[i];
-        }
-        extendedSplit[split.length] = "";
-        return extendedSplit;
-      }
-
-      // Remove quotes from the beginning and end of split strings
-      for (int i = 0; i < split.length; i++) {
-        if ((split[i].startsWith("\"") && split[i].endsWith("\""))
-            || (split[i].startsWith("\'") && split[i].endsWith("\'"))) {
-          split[i] = split[i].substring(1, split[i].length() - 1);
-        }
-      }
-
-      return split;
-    } else {
-      return null;
-    }
-  }
-
-  public static boolean contains(String value, String subString) {
-    if (value != null && subString != null) {
-      // Here we need to keep in mind that once we get the substring, we
-      // should check whether the syntax remains valid
-      StringBuffer part = new StringBuffer();
-      for (int i = 0; i < value.length(); i++) {
-        char ch = value.charAt(i);
-        if (subString.startsWith("" + ch)) {
-          StringBuffer subPart = new StringBuffer(ch);
-          if (PreprocessorUtils.isSyntaxValid(part.toString())) {
-            for (int j = 0; j < subString.length() && (i + j) < value.length(); j++) {
-              subPart.append("" + value.charAt(i + j));
-              if (subPart.toString().equals(subString)) {
-                // The subString is present
-                // We can return from here
-                return true;
-              }
-            }
-          }
-        }
-        part.append(ch);
-      }
-    }
-    return false;
-  }
-
-  public static int lastIndexOf(String value, String subString) {
-    int index = -1;
-    if (value != null && subString != null) {
-      StringBuffer part = new StringBuffer();
-      outer: for (int i = 0; i < value.length(); i++) {
-        char ch = value.charAt(i);
-        if (subString.startsWith("" + ch)) {
-          StringBuffer subPart = new StringBuffer(ch);
-          if (PreprocessorUtils.isSyntaxValid(part.toString())) {
-            for (int j = 0; j < subString.length() && (i + j) < value.length(); j++) {
-              subPart.append(value.charAt(i + j));
-              if (subPart.toString().equals(subString)) {
-                // The subString is present
-                // We can return from here
-                index = i;
-                part.delete(0, part.length());
-                i += j + 1;
-                if (i < value.length()) {
-                  ch = value.charAt(i);
-                } else {
-                  break outer;
-                }
-              }
-            }
-          }
-        }
-        part.append(ch);
-      }
-    }
-    return index;
-  }
-
-  public static String getPadding(int numOfSpaces) {
-    char[] arr = new char[numOfSpaces];
-    Arrays.fill(arr, ' ');
-    return new String(arr);
-  }
-
-  public static String trimBeginning(String stringToTrim) {
-    if (stringToTrim.startsWith(" ")) {
-      int i = 0;
-      for (; i < stringToTrim.length(); i++) {
-        if (stringToTrim.charAt(i) != ' ') {
-          break;
-        }
-      }
-      stringToTrim = stringToTrim.substring(i);
-    }
-
-    return stringToTrim;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/SyntaxConstants.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/SyntaxConstants.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/SyntaxConstants.java
deleted file mode 100644
index 52d9212..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/SyntaxConstants.java
+++ /dev/null
@@ -1,34 +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.parser;
-
-/**
- * Syntax constants for the parser
- * 
- * @since GemFire 7.0
- *
- */
-// TODO merge with CliConstants
-public class SyntaxConstants {
-  public static final String OPTION_VALUE_SPECIFIER = "=";
-  public static final String VALUE_SEPARATOR = ",";
-  public static final String ARGUMENT_SEPARATOR = "?";
-  public static final String OPTION_SEPARATOR = " ";
-  public static final String SHORT_OPTION_SPECIFIER = "-";
-  public static final String LONG_OPTION_SPECIFIER = "--";
-  public static final String COMMAND_DELIMITER = ";";
-  public static final String CONTINUATION_CHARACTER = "\\";
-}
-

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/jopt/JoptOptionParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/jopt/JoptOptionParser.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/jopt/JoptOptionParser.java
deleted file mode 100644
index 52224d4..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/jopt/JoptOptionParser.java
+++ /dev/null
@@ -1,302 +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.parser.jopt;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-
-import joptsimple.ArgumentAcceptingOptionSpec;
-import joptsimple.OptionException;
-import joptsimple.OptionParser;
-import joptsimple.OptionSpecBuilder;
-
-import org.apache.commons.lang.StringUtils;
-
-import org.apache.geode.management.internal.cli.MultipleValueConverter;
-import org.apache.geode.management.internal.cli.exceptions.CliCommandOptionException;
-import org.apache.geode.management.internal.cli.exceptions.ExceptionGenerator;
-import org.apache.geode.management.internal.cli.parser.Argument;
-import org.apache.geode.management.internal.cli.parser.GfshOptionParser;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
-import org.apache.geode.management.internal.cli.parser.preprocessor.Preprocessor;
-import org.apache.geode.management.internal.cli.parser.preprocessor.PreprocessorUtils;
-import org.apache.geode.management.internal.cli.parser.preprocessor.TrimmedInput;
-import org.apache.geode.management.internal.cli.util.HyphenFormatter;
-
-/**
- * Implementation of {@link GfshOptionParser} which internally makes use of
- * {@link joptsimple.OptionParser}
- *
- * Newly constructed JoptOptionParser must be loaded with arguments and options before parsing
- * command strings.
- * 
- * @since GemFire 7.0
- */
-public class JoptOptionParser implements GfshOptionParser {
-
-  private OptionParser parser;
-  private LinkedList<Argument> arguments = new LinkedList<Argument>();
-  private LinkedList<Option> options;
-
-  /**
-   * Constructor
-   */
-  public JoptOptionParser() {
-    parser = new OptionParser(true);
-    parser.allowsUnrecognizedOptions();
-  }
-
-  public void setArguments(LinkedList<Argument> arguments) {
-    List<Argument> optional = new LinkedList<Argument>();
-    // Let us arrange arguments as mandatory arguments
-    // followed by optional arguments
-    for (Argument argument : arguments) {
-      if (argument.isRequired()) {
-        this.arguments.add(argument);
-      } else {
-        optional.add(argument);
-      }
-    }
-    for (Argument argument : optional) {
-      this.arguments.add(argument);
-    }
-  }
-
-  public void setOptions(LinkedList<Option> options) {
-    this.options = options;
-    for (Option option : options) {
-      addJoptOptionObject(option);
-    }
-  }
-
-  private void addJoptOptionObject(Option option) {
-    OptionSpecBuilder optionBuilder = null;
-
-    optionBuilder = parser.acceptsAll(option.getAggregate(), option.getHelp());
-
-    /* Now set the the attributes related to the option */
-
-    ArgumentAcceptingOptionSpec<String> argumentSpecs = null;
-
-    if (option.isWithRequiredArgs()) {
-      argumentSpecs = optionBuilder.withRequiredArg();
-    } else {
-      argumentSpecs = optionBuilder.withOptionalArg();
-    }
-
-    // TODO: temporarily commented out as workaround for GEODE-1598
-    // if (option.isRequired()) {
-    // argumentSpecs.required();
-    // }
-    if (option.getValueSeparator() != null) {
-      argumentSpecs.withValuesSeparatedBy(option.getValueSeparator());
-    }
-  }
-
-  public OptionSet parse(String userInput) throws CliCommandOptionException {
-    OptionSet optionSet = new OptionSet();
-    optionSet.setUserInput(userInput != null ? userInput.trim() : "");
-    if (userInput != null) {
-      TrimmedInput input = PreprocessorUtils.trim(userInput);
-      String[] preProcessedInput =
-          preProcess(new HyphenFormatter().formatCommand(input.getString()));
-      joptsimple.OptionSet joptOptionSet = null;
-      CliCommandOptionException ce = null;
-      // int factor = 0;
-      try {
-        joptOptionSet = parser.parse(preProcessedInput);
-      } catch (OptionException e) {
-        ce = processException(e);
-        // TODO: joptOptionSet = e.getDetected(); // removed when geode-joptsimple was removed
-      }
-      if (joptOptionSet != null) {
-
-        // Make sure there are no miscellaneous, unknown strings that cannot be identified as
-        // either options or arguments.
-        if (joptOptionSet.nonOptionArguments().size() > arguments.size()) {
-          String unknownString = (String) joptOptionSet.nonOptionArguments().get(arguments.size()); // added
-                                                                                                    // cast
-                                                                                                    // when
-                                                                                                    // geode-joptsimple
-                                                                                                    // was
-                                                                                                    // removed
-          // If the first option is un-parseable then it will be returned as "<option>=<value>"
-          // since it's
-          // been interpreted as an argument. However, all subsequent options will be returned as
-          // "<option>".
-          // This hack splits off the string before the "=" sign if it's the first case.
-          if (unknownString.matches("^-*\\w+=.*$")) {
-            unknownString = unknownString.substring(0, unknownString.indexOf('='));
-          }
-          // TODO: ce =
-          // processException(OptionException.createUnrecognizedOptionException(unknownString,
-          // joptOptionSet)); // removed when geode-joptsimple was removed
-        }
-
-        // First process the arguments
-        StringBuffer argument = new StringBuffer();
-        int j = 0;
-        for (int i = 0; i < joptOptionSet.nonOptionArguments().size()
-            && j < arguments.size(); i++) {
-          argument = argument.append(joptOptionSet.nonOptionArguments().get(i));
-          // Check for syntax of arguments before adding them to the
-          // option set as we want to support quoted arguments and those
-          // in brackets
-          if (PreprocessorUtils.isSyntaxValid(argument.toString())) {
-            optionSet.put(arguments.get(j), argument.toString());
-            j++;
-            argument.delete(0, argument.length());
-          }
-        }
-        if (argument.length() > 0) {
-          // Here we do not need to check for the syntax of the argument
-          // because the argument list is now over and this is the last
-          // argument which was not added due to improper syntax
-          optionSet.put(arguments.get(j), argument.toString());
-        }
-
-        // Now process the options
-        for (Option option : options) {
-          List<String> synonyms = option.getAggregate();
-          for (String string : synonyms) {
-            if (joptOptionSet.has(string)) {
-              // Check whether the user has actually entered the
-              // full option or just the start
-              boolean present = false;
-              outer: for (String inputSplit : preProcessedInput) {
-                if (inputSplit.startsWith(SyntaxConstants.LONG_OPTION_SPECIFIER)) {
-                  // Remove option prefix
-                  inputSplit =
-                      StringUtils.removeStart(inputSplit, SyntaxConstants.LONG_OPTION_SPECIFIER);
-                  // Remove value specifier
-                  inputSplit =
-                      StringUtils.removeEnd(inputSplit, SyntaxConstants.OPTION_VALUE_SPECIFIER);
-                  if (!inputSplit.equals("")) {
-                    if (option.getLongOption().equals(inputSplit)) {
-                      present = true;
-                      break outer;
-                    } else {
-                      for (String optionSynonym : option.getSynonyms()) {
-                        if (optionSynonym.equals(inputSplit)) {
-                          present = true;
-                          break outer;
-                        }
-                      }
-                    }
-                  }
-                }
-              }
-              if (present) {
-                if (joptOptionSet.hasArgument(string)) {
-                  List<?> arguments = joptOptionSet.valuesOf(string);
-                  if (arguments.size() > 1
-                      && !(option.getConverter() instanceof MultipleValueConverter)
-                      && option.getValueSeparator() == null) {
-                    List<String> optionList = new ArrayList<String>(1);
-                    optionList.add(string);
-                    // TODO: ce = processException(new
-                    // MultipleArgumentsForOptionException(optionList, joptOptionSet)); // removed
-                    // when geode-joptsimple was removed
-                  } else if ((arguments.size() == 1
-                      && !(option.getConverter() instanceof MultipleValueConverter))
-                      || option.getValueSeparator() == null) {
-                    optionSet.put(option, arguments.get(0).toString().trim());
-                  } else {
-                    StringBuffer value = new StringBuffer();
-                    String valueSeparator = option.getValueSeparator();
-                    for (Object object : joptOptionSet.valuesOf(string)) {
-                      if (value.length() == 0) {
-                        value.append((String) object);
-                      } else {
-                        if (valueSeparator != null) {
-                          value.append(valueSeparator + ((String) object).trim());
-                        } else {
-                          value.append(((String) object).trim());
-                        }
-                      }
-                    }
-                    optionSet.put(option, value.toString());
-                  }
-                } else {
-                  optionSet.put(option, option.getSpecifiedDefaultValue());
-                }
-                break;
-              }
-            }
-          }
-        }
-      }
-
-      // Convert the preProcessedInput into List<String>
-      List<String> split = new ArrayList<String>();
-      for (int i = 0; i < preProcessedInput.length; i++) {
-        split.add(preProcessedInput[i]);
-      }
-      optionSet.setNoOfSpacesRemoved(input.getNoOfSpacesRemoved() /* + factor */);
-      optionSet.setSplit(split);
-      if (ce != null) {
-        ce.setOptionSet(optionSet);
-        throw ce;
-      }
-    }
-    return optionSet;
-  }
-
-  private CliCommandOptionException processException(final OptionException exception) {
-    return ExceptionGenerator.generate(getOption(exception), exception);
-  }
-
-  private Option getOption(OptionException oe) {
-    Option exceptionOption = null;
-    Iterator<String> iterator = oe.options().iterator();
-    outermost: for (Option option : options) {
-      /* outer: */for (String string : option.getAggregate()) {
-        /* inner: */while (iterator.hasNext()) {
-          String joptOption = iterator.next();
-          if (string.equals(joptOption)) {
-            exceptionOption = option;
-            break outermost;
-          }
-        }
-      }
-    }
-
-    if (exceptionOption == null) {
-      if (oe.options() != null) {
-        if (oe.options().size() > 0) {
-          exceptionOption = new Option(oe.options().iterator().next());
-        }
-      }
-    }
-    return exceptionOption;
-  }
-
-  private String[] preProcess(String userInput) {
-    return Preprocessor.split(userInput);
-  }
-
-  public LinkedList<Argument> getArguments() {
-    return arguments;
-  }
-
-  public LinkedList<Option> getOptions() {
-    return options;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/EnclosingCharacters.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/EnclosingCharacters.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/EnclosingCharacters.java
deleted file mode 100644
index a35e626..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/EnclosingCharacters.java
+++ /dev/null
@@ -1,32 +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.parser.preprocessor;
-
-/**
- * Used for Supporting enclosed input
- * 
- * @since GemFire 7.0
- *
- */
-public final class EnclosingCharacters {
-  public static final Character DOUBLE_QUOTATION = '\"';
-  public static final Character SINGLE_QUOTATION = '\'';
-  public static final Character OPENING_CURLY_BRACE = '{';
-  public static final Character CLOSING_CURLY_BRACE = '}';
-  public static final Character OPENING_SQUARE_BRACKET = '[';
-  public static final Character CLOSING_SQUARE_BRACKET = ']';
-  public static final Character OPENING_CIRCULAR_BRACKET = '(';
-  public static final Character CLOSING_CIRCULAR_BRACKET = ')';
-}


[7/9] geode git commit: GEODE-1597: use Spring shell's parser and delete our own parsing code

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java
index 39d0442..3d155e5 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ClientCommands.java
@@ -15,16 +15,6 @@
 
 package org.apache.geode.management.internal.cli.commands;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import javax.management.ObjectName;
-
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.execute.FunctionService;
@@ -47,12 +37,21 @@ import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
-
 import org.springframework.shell.core.CommandMarker;
 import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import javax.management.ObjectName;
+
 /**
  * 
  * @since GemFire 8.0
@@ -65,7 +64,7 @@ public class ClientCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.LIST_CLIENTS, help = CliStrings.LIST_CLIENT__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_LIST})
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_CLIENT})
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
   public Result listClient() {
     Result result = null;
@@ -154,7 +153,7 @@ public class ClientCommands implements CommandMarker {
 
 
   @CliCommand(value = CliStrings.DESCRIBE_CLIENT, help = CliStrings.DESCRIBE_CLIENT__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_LIST})
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_CLIENT})
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
   public Result describeClient(@CliOption(key = CliStrings.DESCRIBE_CLIENT__ID, mandatory = true,
       help = CliStrings.DESCRIBE_CLIENT__ID__HELP) String clientId) {

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
index 50c9caa..5dfc1b8 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
@@ -202,11 +202,9 @@ public class ConfigCommands extends AbstractCommandsSupport {
   public Result exportConfig(
       @CliOption(key = {CliStrings.EXPORT_CONFIG__MEMBER},
           optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-          help = CliStrings.EXPORT_CONFIG__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String member,
+          help = CliStrings.EXPORT_CONFIG__MEMBER__HELP) String member,
       @CliOption(key = {CliStrings.EXPORT_CONFIG__GROUP}, optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.EXPORT_CONFIG__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String group,
+          help = CliStrings.EXPORT_CONFIG__GROUP__HELP) String group,
       @CliOption(key = {CliStrings.EXPORT_CONFIG__DIR},
           help = CliStrings.EXPORT_CONFIG__DIR__HELP) String dir) {
     InfoResultData infoData = ResultBuilder.createInfoResultData();

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
index 28ce092..c4455ad 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
@@ -124,8 +124,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           help = CliStrings.CREATE_REGION__USEATTRIBUTESFROM__HELP) String useAttributesFrom,
       @CliOption(key = CliStrings.CREATE_REGION__GROUP, optionContext = ConverterHint.MEMBERGROUP,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.CREATE_REGION__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String[] groups,
+          help = CliStrings.CREATE_REGION__GROUP__HELP) String[] groups,
       @CliOption(key = CliStrings.CREATE_REGION__SKIPIFEXISTS, unspecifiedDefaultValue = "true",
           specifiedDefaultValue = "true",
           help = CliStrings.CREATE_REGION__SKIPIFEXISTS__HELP) boolean skipIfExists,
@@ -133,11 +132,9 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
       // the following should all be in alphabetical order according to
       // their key string
       @CliOption(key = CliStrings.CREATE_REGION__ASYNCEVENTQUEUEID,
-          help = CliStrings.CREATE_REGION__ASYNCEVENTQUEUEID__HELP) @CliMetaData(
-              valueSeparator = ",") String[] asyncEventQueueIds,
+          help = CliStrings.CREATE_REGION__ASYNCEVENTQUEUEID__HELP) String[] asyncEventQueueIds,
       @CliOption(key = CliStrings.CREATE_REGION__CACHELISTENER,
-          help = CliStrings.CREATE_REGION__CACHELISTENER__HELP) @CliMetaData(
-              valueSeparator = ",") String[] cacheListener,
+          help = CliStrings.CREATE_REGION__CACHELISTENER__HELP) String[] cacheListener,
       @CliOption(key = CliStrings.CREATE_REGION__CACHELOADER,
           help = CliStrings.CREATE_REGION__CACHELOADER__HELP) String cacheLoader,
       @CliOption(key = CliStrings.CREATE_REGION__CACHEWRITER,
@@ -186,8 +183,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
       @CliOption(key = CliStrings.CREATE_REGION__ENTRYEXPIRATIONTTLACTION,
           help = CliStrings.CREATE_REGION__ENTRYEXPIRATIONTTLACTION__HELP) String entryExpirationTTLAction,
       @CliOption(key = CliStrings.CREATE_REGION__GATEWAYSENDERID,
-          help = CliStrings.CREATE_REGION__GATEWAYSENDERID__HELP) @CliMetaData(
-              valueSeparator = ",") String[] gatewaySenderIds,
+          help = CliStrings.CREATE_REGION__GATEWAYSENDERID__HELP) String[] gatewaySenderIds,
       @CliOption(key = CliStrings.CREATE_REGION__KEYCONSTRAINT,
           help = CliStrings.CREATE_REGION__KEYCONSTRAINT__HELP) String keyConstraint,
       @CliOption(key = CliStrings.CREATE_REGION__LOCALMAXMEMORY,
@@ -424,8 +420,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           help = CliStrings.ALTER_REGION__REGION__HELP) String regionPath,
       @CliOption(key = CliStrings.ALTER_REGION__GROUP, optionContext = ConverterHint.MEMBERGROUP,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.ALTER_REGION__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String[] groups,
+          help = CliStrings.ALTER_REGION__GROUP__HELP) String[] groups,
       @CliOption(key = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIME,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, specifiedDefaultValue = "-1",
           help = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIME__HELP) Integer entryExpirationIdleTime,
@@ -456,8 +451,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           help = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTLACTION__HELP) String regionExpirationTTLAction,
       @CliOption(key = CliStrings.ALTER_REGION__CACHELISTENER,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, specifiedDefaultValue = "",
-          help = CliStrings.ALTER_REGION__CACHELISTENER__HELP) @CliMetaData(
-              valueSeparator = ",") String[] cacheListeners,
+          help = CliStrings.ALTER_REGION__CACHELISTENER__HELP) String[] cacheListeners,
       @CliOption(key = CliStrings.ALTER_REGION__CACHELOADER,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           specifiedDefaultValue = "null",
@@ -468,12 +462,10 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           help = CliStrings.ALTER_REGION__CACHEWRITER__HELP) String cacheWriter,
       @CliOption(key = CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, specifiedDefaultValue = "",
-          help = CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID__HELP) @CliMetaData(
-              valueSeparator = ",") String[] asyncEventQueueIds,
+          help = CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID__HELP) String[] asyncEventQueueIds,
       @CliOption(key = CliStrings.ALTER_REGION__GATEWAYSENDERID,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, specifiedDefaultValue = "",
-          help = CliStrings.ALTER_REGION__GATEWAYSENDERID__HELP) @CliMetaData(
-              valueSeparator = ",") String[] gatewaySenderIds,
+          help = CliStrings.ALTER_REGION__GATEWAYSENDERID__HELP) String[] gatewaySenderIds,
       @CliOption(key = CliStrings.ALTER_REGION__CLONINGENABLED,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           specifiedDefaultValue = "false",

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DataCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DataCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DataCommands.java
index 6324b5c..29d68bd 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DataCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DataCommands.java
@@ -14,29 +14,6 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.StringTokenizer;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.shiro.subject.Subject;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
 import org.apache.geode.LogWriter;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheClosedException;
@@ -80,6 +57,28 @@ import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.result.TabularResultData;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 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.shiro.subject.Subject;
+import org.springframework.shell.core.CommandMarker;
+import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringTokenizer;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 /**
  * 
@@ -1080,7 +1079,8 @@ public class DataCommands implements CommandMarker {
       relatedTopic = {CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_REGION})
   @CliCommand(value = {CliStrings.REMOVE}, help = CliStrings.REMOVE__HELP)
   public Result remove(
-      @CliOption(key = {CliStrings.REMOVE__KEY}, help = CliStrings.REMOVE__KEY__HELP) String key,
+      @CliOption(key = {CliStrings.REMOVE__KEY}, help = CliStrings.REMOVE__KEY__HELP,
+          specifiedDefaultValue = "") String key,
       @CliOption(key = {CliStrings.REMOVE__REGION}, mandatory = true,
           help = CliStrings.REMOVE__REGION__HELP,
           optionContext = ConverterHint.REGIONPATH) String regionPath,

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java
index 1444088..0cf7078 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java
@@ -77,8 +77,7 @@ public final class DeployCommands extends AbstractCommandsSupport {
       relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
   public final Result deploy(
       @CliOption(key = {CliStrings.DEPLOY__GROUP}, help = CliStrings.DEPLOY__GROUP__HELP,
-          optionContext = ConverterHint.MEMBERGROUP) @CliMetaData(
-              valueSeparator = ",") String[] groups,
+          optionContext = ConverterHint.MEMBERGROUP) String[] groups,
       @CliOption(key = {CliStrings.DEPLOY__JAR}, help = CliStrings.DEPLOY__JAR__HELP) String jar,
       @CliOption(key = {CliStrings.DEPLOY__DIR}, help = CliStrings.DEPLOY__DIR__HELP) String dir) {
     try {
@@ -156,11 +155,9 @@ public final class DeployCommands extends AbstractCommandsSupport {
   @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public final Result undeploy(
       @CliOption(key = {CliStrings.UNDEPLOY__GROUP}, help = CliStrings.UNDEPLOY__GROUP__HELP,
-          optionContext = ConverterHint.MEMBERGROUP) @CliMetaData(
-              valueSeparator = ",") String[] groups,
+          optionContext = ConverterHint.MEMBERGROUP) String[] groups,
       @CliOption(key = {CliStrings.UNDEPLOY__JAR}, help = CliStrings.UNDEPLOY__JAR__HELP,
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE) @CliMetaData(
-              valueSeparator = ",") String jars) {
+          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE) String jars) {
 
     try {
       TabularResultData tabularData = ResultBuilder.createTabularResultData();
@@ -227,8 +224,7 @@ public final class DeployCommands extends AbstractCommandsSupport {
   @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
   public final Result listDeployed(@CliOption(key = {CliStrings.LIST_DEPLOYED__GROUP},
-      help = CliStrings.LIST_DEPLOYED__GROUP__HELP) @CliMetaData(
-          valueSeparator = ",") String group) {
+      help = CliStrings.LIST_DEPLOYED__GROUP__HELP) String group) {
 
     try {
       TabularResultData tabularData = ResultBuilder.createTabularResultData();

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java
index 9ad2060..a96d3c2 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java
@@ -317,13 +317,10 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
           unspecifiedDefaultValue = "32768",
           help = CliStrings.CREATE_DISK_STORE__WRITE_BUFFER_SIZE__HELP) int writeBufferSize,
       @CliOption(key = CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE, mandatory = true,
-          help = CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE__HELP,
-          optionContext = ConverterHint.STRING_DISABLER) @CliMetaData(
-              valueSeparator = ",") String[] directoriesAndSizes,
+          help = CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE__HELP) String[] directoriesAndSizes,
       @CliOption(key = CliStrings.CREATE_DISK_STORE__GROUP,
           help = CliStrings.CREATE_DISK_STORE__GROUP__HELP,
-          optionContext = ConverterHint.MEMBERGROUP) @CliMetaData(
-              valueSeparator = ",") String[] groups,
+          optionContext = ConverterHint.MEMBERGROUP) String[] groups,
       @CliOption(key = CliStrings.CREATE_DISK_STORE__DISK_USAGE_WARNING_PCT,
           unspecifiedDefaultValue = "90",
           help = CliStrings.CREATE_DISK_STORE__DISK_USAGE_WARNING_PCT__HELP) float diskUsageWarningPercentage,
@@ -425,9 +422,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
           help = CliStrings.COMPACT_DISK_STORE__NAME__HELP) String diskStoreName,
       @CliOption(key = CliStrings.COMPACT_DISK_STORE__GROUP,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.COMPACT_DISK_STORE__GROUP__HELP,
-          optionContext = ConverterHint.STRING_DISABLER) @CliMetaData(
-              valueSeparator = ",") String[] groups) {
+          help = CliStrings.COMPACT_DISK_STORE__GROUP__HELP) String[] groups) {
     Result result = null;
 
     try {
@@ -568,16 +563,13 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
           help = CliStrings.COMPACT_OFFLINE_DISK_STORE__NAME__HELP) String diskStoreName,
       @CliOption(key = CliStrings.COMPACT_OFFLINE_DISK_STORE__DISKDIRS, mandatory = true,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.COMPACT_OFFLINE_DISK_STORE__DISKDIRS__HELP,
-          optionContext = ConverterHint.DIRS + ":" + ConverterHint.STRING_DISABLER) @CliMetaData(
-              valueSeparator = ",") String[] diskDirs,
+          help = CliStrings.COMPACT_OFFLINE_DISK_STORE__DISKDIRS__HELP) String[] diskDirs,
       @CliOption(key = CliStrings.COMPACT_OFFLINE_DISK_STORE__MAXOPLOGSIZE,
           unspecifiedDefaultValue = "-1",
           help = CliStrings.COMPACT_OFFLINE_DISK_STORE__MAXOPLOGSIZE__HELP) long maxOplogSize,
       @CliOption(key = CliStrings.COMPACT_OFFLINE_DISK_STORE__J,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.COMPACT_OFFLINE_DISK_STORE__J__HELP) @CliMetaData(
-              valueSeparator = ",") String[] jvmProps) {
+          help = CliStrings.COMPACT_OFFLINE_DISK_STORE__J__HELP) String[] jvmProps) {
     Result result = null;
     LogWrapper logWrapper = LogWrapper.getInstance();
 
@@ -712,16 +704,13 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
           help = CliStrings.UPGRADE_OFFLINE_DISK_STORE__NAME__HELP) String diskStoreName,
       @CliOption(key = CliStrings.UPGRADE_OFFLINE_DISK_STORE__DISKDIRS, mandatory = true,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.UPGRADE_OFFLINE_DISK_STORE__DISKDIRS__HELP,
-          optionContext = ConverterHint.DIRS + ":" + ConverterHint.STRING_DISABLER) @CliMetaData(
-              valueSeparator = ",") String[] diskDirs,
+          help = CliStrings.UPGRADE_OFFLINE_DISK_STORE__DISKDIRS__HELP) String[] diskDirs,
       @CliOption(key = CliStrings.UPGRADE_OFFLINE_DISK_STORE__MAXOPLOGSIZE,
           unspecifiedDefaultValue = "-1",
           help = CliStrings.UPGRADE_OFFLINE_DISK_STORE__MAXOPLOGSIZE__HELP) long maxOplogSize,
       @CliOption(key = CliStrings.UPGRADE_OFFLINE_DISK_STORE__J,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.UPGRADE_OFFLINE_DISK_STORE__J__HELP) @CliMetaData(
-              valueSeparator = ",") String[] jvmProps)
+          help = CliStrings.UPGRADE_OFFLINE_DISK_STORE__J__HELP) String[] jvmProps)
       throws InterruptedException {
 
 
@@ -1155,8 +1144,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
       @CliOption(key = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKSTORENAME, mandatory = true,
           help = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKSTORENAME__HELP) String diskStoreName,
       @CliOption(key = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKDIRS, mandatory = true,
-          help = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKDIRS__HELP) @CliMetaData(
-              valueSeparator = ",") String[] diskDirs,
+          help = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKDIRS__HELP) String[] diskDirs,
       @CliOption(key = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__PDX_TYPES,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__PDX_TYPES__HELP) Boolean listPdxTypes,
@@ -1200,8 +1188,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
       @CliOption(key = CliStrings.EXPORT_OFFLINE_DISK_STORE__DISKSTORENAME, mandatory = true,
           help = CliStrings.EXPORT_OFFLINE_DISK_STORE__DISKSTORENAME__HELP) String diskStoreName,
       @CliOption(key = CliStrings.EXPORT_OFFLINE_DISK_STORE__DISKDIRS, mandatory = true,
-          help = CliStrings.EXPORT_OFFLINE_DISK_STORE__DISKDIRS__HELP) @CliMetaData(
-              valueSeparator = ",") String[] diskDirs,
+          help = CliStrings.EXPORT_OFFLINE_DISK_STORE__DISKDIRS__HELP) String[] diskDirs,
       @CliOption(key = CliStrings.EXPORT_OFFLINE_DISK_STORE__DIR, mandatory = true,
           help = CliStrings.EXPORT_OFFLINE_DISK_STORE__DIR__HELP) String dir) {
 
@@ -1245,12 +1232,10 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
           help = CliStrings.VALIDATE_DISK_STORE__NAME__HELP) String diskStoreName,
       @CliOption(key = CliStrings.VALIDATE_DISK_STORE__DISKDIRS, mandatory = true,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.VALIDATE_DISK_STORE__DISKDIRS__HELP) @CliMetaData(
-              valueSeparator = ",") String[] diskDirs,
+          help = CliStrings.VALIDATE_DISK_STORE__DISKDIRS__HELP) String[] diskDirs,
       @CliOption(key = CliStrings.VALIDATE_DISK_STORE__J,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.VALIDATE_DISK_STORE__J__HELP) @CliMetaData(
-              valueSeparator = ",") String[] jvmProps) {
+          help = CliStrings.VALIDATE_DISK_STORE__J__HELP) String[] jvmProps) {
     try {
       String resultString = new String();
 
@@ -1325,8 +1310,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
           help = CliStrings.ALTER_DISK_STORE__REGIONNAME__HELP) String regionName,
       @CliOption(key = CliStrings.ALTER_DISK_STORE__DISKDIRS,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.ALTER_DISK_STORE__DISKDIRS__HELP,
-          mandatory = true) @CliMetaData(valueSeparator = ",") String[] diskDirs,
+          help = CliStrings.ALTER_DISK_STORE__DISKDIRS__HELP, mandatory = true) String[] diskDirs,
       @CliOption(key = CliStrings.ALTER_DISK_STORE__COMPRESSOR,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           specifiedDefaultValue = "none",
@@ -1437,8 +1421,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
           help = CliStrings.DESTROY_DISK_STORE__NAME__HELP) String name,
       @CliOption(key = CliStrings.DESTROY_DISK_STORE__GROUP,
           help = CliStrings.DESTROY_DISK_STORE__GROUP__HELP,
-          optionContext = ConverterHint.MEMBERGROUP) @CliMetaData(
-              valueSeparator = ",") String[] groups) {
+          optionContext = ConverterHint.MEMBERGROUP) String[] groups) {
     try {
       TabularResultData tabularData = ResultBuilder.createTabularResultData();
       boolean accumulatedData = false;

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java
index 752ca2a..e21a334 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java
@@ -559,11 +559,9 @@ public class FunctionCommands implements CommandMarker {
       @CliOption(key = CliStrings.LIST_FUNCTION__MATCHES,
           help = CliStrings.LIST_FUNCTION__MATCHES__HELP) String matches,
       @CliOption(key = CliStrings.LIST_FUNCTION__GROUP, optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.LIST_FUNCTION__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String groups,
+          help = CliStrings.LIST_FUNCTION__GROUP__HELP) String groups,
       @CliOption(key = CliStrings.LIST_FUNCTION__MEMBER, optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.LIST_FUNCTION__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String members) {
+          help = CliStrings.LIST_FUNCTION__MEMBER__HELP) String members) {
     TabularResultData tabularData = ResultBuilder.createTabularResultData();
     boolean accumulatedData = false;
 

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshHelpCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshHelpCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshHelpCommands.java
index 1d1b28e..29e6bae 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshHelpCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/GfshHelpCommands.java
@@ -16,85 +16,29 @@ package org.apache.geode.management.internal.cli.commands;
 
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.ConverterHint;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.CommandManager;
-import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.annotation.CliArgument;
-import org.apache.geode.management.internal.cli.help.CliTopic;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
-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.Gfsh;
 import org.springframework.shell.core.CommandMarker;
 import org.springframework.shell.core.annotation.CliCommand;
-
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
+import org.springframework.shell.core.annotation.CliOption;
 
 /**
- * 
- * 
  * @since GemFire 7.0
  */
 public class GfshHelpCommands implements CommandMarker {
-
-  private Gfsh getGfsh() {
-    return Gfsh.getCurrentInstance();
-  }
-
   @CliCommand(value = CliStrings.HELP, help = CliStrings.HELP__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_HELP})
-  public Result obtainHelp(
-      @CliArgument(name = CliStrings.HELP__COMMAND, argumentContext = CliStrings.PARAM_CONTEXT_HELP,
-          help = CliStrings.HELP__COMMAND__HELP) String commandString) {
-    return ResultBuilder.createInfoResult(getGfsh().obtainHelp(commandString, null));
+  public void obtainHelp(@CliOption(key = {"", CliStrings.SH__COMMAND},
+      help = "Command name to provide help for") String buffer) {
+    Gfsh.getCurrentInstance().getGfshParser().obtainHelp(buffer);
   }
 
-
-
   @CliCommand(value = CliStrings.HINT, help = CliStrings.HINT__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_HELP})
-  public Result hint(
-      @CliArgument(name = CliStrings.HINT__TOPICNAME, argumentContext = ConverterHint.HINTTOPIC,
+  public void hint(
+      @CliOption(key = {"", CliStrings.HINT__TOPICNAME}, optionContext = ConverterHint.HINTTOPIC,
           help = CliStrings.HINT__TOPICNAME) String topicName) {
-    Result result = null;
-    CommandManager commandManager = CommandManager.getExisting();
-    if (commandManager == null) {
-      result =
-          ResultBuilder.createShellClientErrorResult(CliStrings.HINT__MSG__SHELL_NOT_INITIALIZED);
-    } else {
-      StringBuilder builder = new StringBuilder();
-      if (topicName == null) {
-        builder.append(CliStrings.HINT__MSG__TOPICS_AVAILABLE).append(GfshParser.LINE_SEPARATOR);
-        Set<String> topicNameSet = commandManager.getTopicNames();
-        for (String topic : topicNameSet) {
-          builder.append(topic).append(GfshParser.LINE_SEPARATOR);
-        }
-        result = ResultBuilder.createInfoResult(builder.toString());
-      } else {
-        CliTopic topic = commandManager.getTopic(topicName);
-        if (topic == null) {
-          result = ResultBuilder
-              .createInfoResult(CliStrings.format(CliStrings.HINT__MSG__UNKNOWN_TOPIC, topicName));
-        } else {
-          CompositeResultData compositeResultData = ResultBuilder.createCompositeResultData();
-          SectionResultData commandHelpSection =
-              compositeResultData.addSection("Commands And Help");
-          compositeResultData.setHeader(topic.getOneLinerDescription());
-          Map<String, String> commandsNameHelp = topic.getCommandsNameHelp();
-          Set<Entry<String, String>> entries = commandsNameHelp.entrySet();
-
-          for (Entry<String, String> entry : entries) {
-            commandHelpSection.addData(entry.getKey(), entry.getValue());
-          }
-
-          result = ResultBuilder.buildResult(compositeResultData);
-        }
-      }
-    }
-
-    return result;
+    Gfsh.getCurrentInstance().getGfshParser().obtainHint(topicName);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java
index 313d1bd..0e681d7 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/LauncherLifecycleCommands.java
@@ -241,10 +241,9 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
       @CliOption(key = CliStrings.START_LOCATOR__MAXHEAP,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.START_LOCATOR__MAXHEAP__HELP) final String maxHeap,
-      @CliOption(key = CliStrings.START_LOCATOR__J, optionContext = ConverterHint.STRING_LIST,
+      @CliOption(key = CliStrings.START_LOCATOR__J,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.START_LOCATOR__J__HELP) @CliMetaData(
-              valueSeparator = ",") final String[] jvmArgsOpts,
+          help = CliStrings.START_LOCATOR__J__HELP) final String[] jvmArgsOpts,
       @CliOption(key = CliStrings.START_LOCATOR__CONNECT, unspecifiedDefaultValue = "true",
           specifiedDefaultValue = "true",
           help = CliStrings.START_LOCATOR__CONNECT__HELP) final boolean connect,
@@ -1356,10 +1355,9 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
       @CliOption(key = CliStrings.START_SERVER__INITIAL_HEAP,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.START_SERVER__INITIAL_HEAP__HELP) final String initialHeap,
-      @CliOption(key = CliStrings.START_SERVER__J, optionContext = ConverterHint.STRING_LIST,
+      @CliOption(key = CliStrings.START_SERVER__J,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.START_SERVER__J__HELP) @CliMetaData(
-              valueSeparator = ",") final String[] jvmArgsOpts,
+          help = CliStrings.START_SERVER__J__HELP) final String[] jvmArgsOpts,
       @CliOption(key = CliStrings.START_SERVER__LOCATORS,
           optionContext = ConverterHint.LOCATOR_DISCOVERY_CONFIG,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
@@ -2056,10 +2054,9 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
       @CliOption(key = CliStrings.START_JCONSOLE__VERSION, specifiedDefaultValue = "true",
           unspecifiedDefaultValue = "false",
           help = CliStrings.START_JCONSOLE__VERSION__HELP) final boolean version,
-      @CliOption(key = CliStrings.START_JCONSOLE__J, optionContext = ConverterHint.STRING_LIST,
+      @CliOption(key = CliStrings.START_JCONSOLE__J,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.START_JCONSOLE__J__HELP) @CliMetaData(
-              valueSeparator = ",") final List<String> jvmArgs) {
+          help = CliStrings.START_JCONSOLE__J__HELP) final List<String> jvmArgs) {
     try {
       String[] jconsoleCommandLine =
           createJConsoleCommandLine(null, interval, notile, pluginpath, version, jvmArgs);
@@ -2226,11 +2223,9 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   @CliCommand(value = CliStrings.START_JVISUALVM, help = CliStrings.START_JVISUALVM__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_MANAGER,
       CliStrings.TOPIC_GEODE_JMX, CliStrings.TOPIC_GEODE_M_AND_M})
-  public Result startJVisualVM(
-      @CliOption(key = CliStrings.START_JCONSOLE__J, optionContext = ConverterHint.STRING_LIST,
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.START_JCONSOLE__J__HELP) @CliMetaData(
-              valueSeparator = ",") final List<String> jvmArgs) {
+  public Result startJVisualVM(@CliOption(key = CliStrings.START_JCONSOLE__J,
+      unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
+      help = CliStrings.START_JCONSOLE__J__HELP) final List<String> jvmArgs) {
     try {
       String[] jvisualvmCommandLine = createJVisualVMCommandLine(jvmArgs);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java
index 22981e7..b5fb9fe 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java
@@ -396,10 +396,11 @@ public class MiscellaneousCommands implements CommandMarker {
   @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DEBUG_UTIL})
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
   // TODO : Verify the auto-completion for multiple values.
-  public Result netstat(@CliOption(key = CliStrings.NETSTAT__MEMBER, mandatory = false,
-      unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-      optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-      help = CliStrings.NETSTAT__MEMBER__HELP) @CliMetaData(valueSeparator = ",") String[] members,
+  public Result netstat(
+      @CliOption(key = CliStrings.NETSTAT__MEMBER, mandatory = false,
+          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
+          optionContext = ConverterHint.ALL_MEMBER_IDNAME,
+          help = CliStrings.NETSTAT__MEMBER__HELP) String[] members,
       @CliOption(key = CliStrings.NETSTAT__GROUP, mandatory = false,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           optionContext = ConverterHint.MEMBERGROUP,
@@ -823,8 +824,7 @@ public class MiscellaneousCommands implements CommandMarker {
       @CliOption(key = {CliStrings.SHOW_METRICS__CACHESERVER__PORT},
           help = CliStrings.SHOW_METRICS__CACHESERVER__PORT__HELP) String cacheServerPortString,
       @CliOption(key = {CliStrings.SHOW_METRICS__CATEGORY},
-          help = CliStrings.SHOW_METRICS__CATEGORY__HELP) @CliMetaData(
-              valueSeparator = ",") String[] categories) {
+          help = CliStrings.SHOW_METRICS__CATEGORY__HELP) String[] categories) {
 
     Result result = null;
     try {
@@ -1894,11 +1894,11 @@ public class MiscellaneousCommands implements CommandMarker {
 
 
   @CliCommand(value = CliStrings.CHANGE_LOGLEVEL, help = CliStrings.CHANGE_LOGLEVEL__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_CHANGELOGLEVEL},
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_LOGS},
       interceptor = "org.apache.geode.management.internal.cli.commands.MiscellaneousCommands$ChangeLogLevelInterceptor")
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.WRITE)
   public Result changeLogLevel(
-      @CliOption(key = CliStrings.CHANGE_LOGLEVEL__MEMBER, unspecifiedDefaultValue = "",
+      @CliOption(key = CliStrings.CHANGE_LOGLEVEL__MEMBER,
           help = CliStrings.CHANGE_LOGLEVEL__MEMBER__HELP) String[] memberIds,
       @CliOption(key = CliStrings.CHANGE_LOGLEVEL__GROUPS, unspecifiedDefaultValue = "",
           help = CliStrings.CHANGE_LOGLEVEL__GROUPS__HELP) String[] grps,

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXCommands.java
index 4327dec..0ce8ec2 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXCommands.java
@@ -50,27 +50,21 @@ public class PDXCommands extends AbstractCommandsSupport {
   @CliCommand(value = CliStrings.CONFIGURE_PDX, help = CliStrings.CONFIGURE_PDX__HELP)
   @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION)
   @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
-  public Result configurePDX(@CliOption(key = CliStrings.CONFIGURE_PDX__READ__SERIALIZED,
-      unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-      help = CliStrings.CONFIGURE_PDX__READ__SERIALIZED__HELP) Boolean readSerialized,
-
+  public Result configurePDX(
+      @CliOption(key = CliStrings.CONFIGURE_PDX__READ__SERIALIZED,
+          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
+          help = CliStrings.CONFIGURE_PDX__READ__SERIALIZED__HELP) Boolean readSerialized,
       @CliOption(key = CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS__HELP) Boolean ignoreUnreadFields,
-
       @CliOption(key = CliStrings.CONFIGURE_PDX__DISKSTORE,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, specifiedDefaultValue = "",
           help = CliStrings.CONFIGURE_PDX__DISKSTORE__HELP) String diskStore,
-
-      @CliMetaData(valueSeparator = ",") @CliOption(
-          key = CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES,
+      @CliOption(key = CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           specifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES__HELP) String[] patterns,
-
-
-      @CliMetaData(valueSeparator = ",") @CliOption(
-          key = CliStrings.CONFIGURE_PDX__PORTABLE__AUTO__SERIALIZER__CLASSES,
+      @CliOption(key = CliStrings.CONFIGURE_PDX__PORTABLE__AUTO__SERIALIZER__CLASSES,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           specifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.CONFIGURE_PDX__PORTABLE__AUTO__SERIALIZER__CLASSES__HELP) String[] portablePatterns) {
@@ -179,8 +173,7 @@ public class PDXCommands extends AbstractCommandsSupport {
           help = CliStrings.PDX_DISKSTORE__HELP) String diskStore,
 
       @CliOption(key = CliStrings.PDX_DISKDIR, mandatory = true,
-          help = CliStrings.PDX_DISKDIR__HELP) @CliMetaData(
-              valueSeparator = ",") String[] diskDirs) {
+          help = CliStrings.PDX_DISKDIR__HELP) String[] diskDirs) {
 
     try {
       final File[] dirs = new File[diskDirs.length];

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueueCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueueCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueueCommands.java
index 8c56833..6208adb 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueueCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueueCommands.java
@@ -97,16 +97,13 @@ public class QueueCommands extends AbstractCommandsSupport {
           unspecifiedDefaultValue = "KEY",
           help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY__HELP) String orderPolicy,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER,
-          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER__HELP) @CliMetaData(
-              valueSeparator = ",") String[] gatewayEventFilters,
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER__HELP) String[] gatewayEventFilters,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER,
           help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER__HELP) String gatewaySubstitutionListener,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER, mandatory = true,
           help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER__HELP) String listener,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE,
-          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE__HELP,
-          optionContext = ConverterHint.STRING_DISABLER) @CliMetaData(
-              valueSeparator = ",") String[] listenerParamsAndValues) {
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE__HELP) String[] listenerParamsAndValues) {
 
     Properties listenerProperties = new Properties();
 

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
index b37feab..ff3b29f 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
@@ -37,7 +37,6 @@ import org.apache.geode.management.internal.SSLUtil;
 import org.apache.geode.management.internal.cli.CliUtil;
 import org.apache.geode.management.internal.cli.GfshParser;
 import org.apache.geode.management.internal.cli.LogWrapper;
-import org.apache.geode.management.internal.cli.annotation.CliArgument;
 import org.apache.geode.management.internal.cli.converters.ConnectionEndpointConverter;
 import org.apache.geode.management.internal.cli.domain.ConnectToLocatorResult;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
@@ -93,6 +92,155 @@ import javax.net.ssl.TrustManagerFactory;
  */
 public class ShellCommands implements CommandMarker {
 
+  // millis that connect --locator will wait for a response from the locator.
+  private final static int CONNECT_LOCATOR_TIMEOUT_MS = 60000; // see bug 45971
+
+  public static int getConnectLocatorTimeoutInMS() {
+    return ShellCommands.CONNECT_LOCATOR_TIMEOUT_MS;
+  }
+
+  /* package-private */
+  static Map<String, String> loadPropertiesFromURL(URL gfSecurityPropertiesUrl) {
+    Map<String, String> propsMap = Collections.emptyMap();
+
+    if (gfSecurityPropertiesUrl != null) {
+      InputStream inputStream = null;
+      try {
+        Properties props = new Properties();
+        inputStream = gfSecurityPropertiesUrl.openStream();
+        props.load(inputStream);
+        if (!props.isEmpty()) {
+          Set<String> jmxSpecificProps = new HashSet<String>();
+          propsMap = new LinkedHashMap<String, String>();
+          Set<Entry<Object, Object>> entrySet = props.entrySet();
+          for (Entry<Object, Object> entry : entrySet) {
+
+            String key = (String) entry.getKey();
+            if (key.endsWith(DistributionConfig.JMX_SSL_PROPS_SUFFIX)) {
+              key =
+                  key.substring(0, key.length() - DistributionConfig.JMX_SSL_PROPS_SUFFIX.length());
+              jmxSpecificProps.add(key);
+
+              propsMap.put(key, (String) entry.getValue());
+            } else if (!jmxSpecificProps.contains(key)) {// Prefer properties ending with "-jmx"
+              // over default SSL props.
+              propsMap.put(key, (String) entry.getValue());
+            }
+          }
+          props.clear();
+          jmxSpecificProps.clear();
+        }
+      } catch (IOException io) {
+        throw new RuntimeException(
+            CliStrings.format(CliStrings.CONNECT__MSG__COULD_NOT_READ_CONFIG_FROM_0,
+                CliUtil.decodeWithDefaultCharSet(gfSecurityPropertiesUrl.getPath())),
+            io);
+      } finally {
+        IOUtils.close(inputStream);
+      }
+    }
+    return propsMap;
+  }
+
+  // Copied from DistributedSystem.java
+  public static URL getFileUrl(String fileName) {
+    File file = new File(fileName);
+
+    if (file.exists()) {
+      try {
+        return IOUtils.tryGetCanonicalFileElseGetAbsoluteFile(file).toURI().toURL();
+      } catch (MalformedURLException ignore) {
+      }
+    }
+
+    file = new File(System.getProperty("user.home"), fileName);
+
+    if (file.exists()) {
+      try {
+        return IOUtils.tryGetCanonicalFileElseGetAbsoluteFile(file).toURI().toURL();
+      } catch (MalformedURLException ignore) {
+      }
+    }
+
+    return ClassPathLoader.getLatest().getResource(ShellCommands.class, fileName);
+  }
+
+  public static ConnectToLocatorResult connectToLocator(String host, int port, int timeout,
+      Map<String, String> props) throws IOException {
+    // register DSFID types first; invoked explicitly so that all message type
+    // initializations do not happen in first deserialization on a possibly
+    // "precious" thread
+    DSFIDFactory.registerTypes();
+
+    JmxManagerLocatorResponse locatorResponse =
+        JmxManagerLocatorRequest.send(host, port, timeout, props);
+
+    if (StringUtils.isBlank(locatorResponse.getHost()) || locatorResponse.getPort() == 0) {
+      Throwable locatorResponseException = locatorResponse.getException();
+      String exceptionMessage = CliStrings.CONNECT__MSG__LOCATOR_COULD_NOT_FIND_MANAGER;
+
+      if (locatorResponseException != null) {
+        String locatorResponseExceptionMessage = locatorResponseException.getMessage();
+        locatorResponseExceptionMessage = (!StringUtils.isBlank(locatorResponseExceptionMessage)
+            ? locatorResponseExceptionMessage : locatorResponseException.toString());
+        exceptionMessage = "Exception caused JMX Manager startup to fail because: '"
+            .concat(locatorResponseExceptionMessage).concat("'");
+      }
+
+      throw new IllegalStateException(exceptionMessage, locatorResponseException);
+    }
+
+    ConnectionEndpoint memberEndpoint =
+        new ConnectionEndpoint(locatorResponse.getHost(), locatorResponse.getPort());
+
+    String resultMessage = CliStrings.format(CliStrings.CONNECT__MSG__CONNECTING_TO_MANAGER_AT_0,
+        memberEndpoint.toString(false));
+
+    return new ConnectToLocatorResult(memberEndpoint, resultMessage,
+        locatorResponse.isJmxManagerSslEnabled());
+  }
+
+  private static InfoResultData executeCommand(Gfsh gfsh, String userCommand, boolean useConsole)
+      throws IOException {
+    InfoResultData infoResultData = ResultBuilder.createInfoResultData();
+
+    String cmdToExecute = userCommand;
+    String cmdExecutor = "/bin/sh";
+    String cmdExecutorOpt = "-c";
+    if (SystemUtils.isWindows()) {
+      cmdExecutor = "cmd";
+      cmdExecutorOpt = "/c";
+    } else if (useConsole) {
+      cmdToExecute = cmdToExecute + " </dev/tty >/dev/tty";
+    }
+    String[] commandArray = {cmdExecutor, cmdExecutorOpt, cmdToExecute};
+
+    ProcessBuilder builder = new ProcessBuilder();
+    builder.command(commandArray);
+    builder.directory();
+    builder.redirectErrorStream();
+    Process proc = builder.start();
+
+    BufferedReader input = new BufferedReader(new InputStreamReader(proc.getInputStream()));
+
+    String lineRead = "";
+    while ((lineRead = input.readLine()) != null) {
+      infoResultData.addLine(lineRead);
+    }
+
+    proc.getOutputStream().close();
+
+    try {
+      if (proc.waitFor() != 0) {
+        gfsh.logWarning("The command '" + userCommand + "' did not complete successfully", null);
+      }
+    } catch (final InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new IllegalStateException(e.getMessage(), e);
+    }
+    return infoResultData;
+  }
+
   private Gfsh getGfsh() {
     return Gfsh.getCurrentInstance();
   }
@@ -113,13 +261,6 @@ public class ShellCommands implements CommandMarker {
     return exitShellRequest;
   }
 
-  // millis that connect --locator will wait for a response from the locator.
-  private final static int CONNECT_LOCATOR_TIMEOUT_MS = 60000; // see bug 45971
-
-  public static int getConnectLocatorTimeoutInMS() {
-    return ShellCommands.CONNECT_LOCATOR_TIMEOUT_MS;
-  }
-
   @CliCommand(value = {CliStrings.CONNECT}, help = CliStrings.CONNECT__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_JMX,
       CliStrings.TOPIC_GEODE_MANAGER})
@@ -213,7 +354,6 @@ public class ShellCommands implements CommandMarker {
     return result;
   }
 
-
   private Result httpConnect(Map<String, String> sslConfigProps, boolean useSsl, String url,
       String userName, String passwordToUse) {
     Gfsh gfsh = getGfsh();
@@ -394,7 +534,6 @@ public class ShellCommands implements CommandMarker {
     return ResultBuilder.createConnectionErrorResult(errorMessage);
   }
 
-
   private String decrypt(String password) {
     if (password != null) {
       return PasswordUtil.decrypt(password);
@@ -584,112 +723,6 @@ public class ShellCommands implements CommandMarker {
     return sslConfigProps;
   }
 
-  private static String getGfshLogsCheckMessage(String logFilePath) {
-    return CliStrings.format(CliStrings.GFSH__PLEASE_CHECK_LOGS_AT_0, logFilePath);
-  }
-
-  /* package-private */
-  static Map<String, String> loadPropertiesFromURL(URL gfSecurityPropertiesUrl) {
-    Map<String, String> propsMap = Collections.emptyMap();
-
-    if (gfSecurityPropertiesUrl != null) {
-      InputStream inputStream = null;
-      try {
-        Properties props = new Properties();
-        inputStream = gfSecurityPropertiesUrl.openStream();
-        props.load(inputStream);
-        if (!props.isEmpty()) {
-          Set<String> jmxSpecificProps = new HashSet<String>();
-          propsMap = new LinkedHashMap<String, String>();
-          Set<Entry<Object, Object>> entrySet = props.entrySet();
-          for (Entry<Object, Object> entry : entrySet) {
-
-            String key = (String) entry.getKey();
-            if (key.endsWith(DistributionConfig.JMX_SSL_PROPS_SUFFIX)) {
-              key =
-                  key.substring(0, key.length() - DistributionConfig.JMX_SSL_PROPS_SUFFIX.length());
-              jmxSpecificProps.add(key);
-
-              propsMap.put(key, (String) entry.getValue());
-            } else if (!jmxSpecificProps.contains(key)) {// Prefer properties ending with "-jmx"
-                                                         // over default SSL props.
-              propsMap.put(key, (String) entry.getValue());
-            }
-          }
-          props.clear();
-          jmxSpecificProps.clear();
-        }
-      } catch (IOException io) {
-        throw new RuntimeException(
-            CliStrings.format(CliStrings.CONNECT__MSG__COULD_NOT_READ_CONFIG_FROM_0,
-                CliUtil.decodeWithDefaultCharSet(gfSecurityPropertiesUrl.getPath())),
-            io);
-      } finally {
-        IOUtils.close(inputStream);
-      }
-    }
-    return propsMap;
-  }
-
-  // Copied from DistributedSystem.java
-  public static URL getFileUrl(String fileName) {
-    File file = new File(fileName);
-
-    if (file.exists()) {
-      try {
-        return IOUtils.tryGetCanonicalFileElseGetAbsoluteFile(file).toURI().toURL();
-      } catch (MalformedURLException ignore) {
-      }
-    }
-
-    file = new File(System.getProperty("user.home"), fileName);
-
-    if (file.exists()) {
-      try {
-        return IOUtils.tryGetCanonicalFileElseGetAbsoluteFile(file).toURI().toURL();
-      } catch (MalformedURLException ignore) {
-      }
-    }
-
-    return ClassPathLoader.getLatest().getResource(ShellCommands.class, fileName);
-  }
-
-  public static ConnectToLocatorResult connectToLocator(String host, int port, int timeout,
-      Map<String, String> props) throws IOException {
-    // register DSFID types first; invoked explicitly so that all message type
-    // initializations do not happen in first deserialization on a possibly
-    // "precious" thread
-    DSFIDFactory.registerTypes();
-
-    JmxManagerLocatorResponse locatorResponse =
-        JmxManagerLocatorRequest.send(host, port, timeout, props);
-
-    if (StringUtils.isBlank(locatorResponse.getHost()) || locatorResponse.getPort() == 0) {
-      Throwable locatorResponseException = locatorResponse.getException();
-      String exceptionMessage = CliStrings.CONNECT__MSG__LOCATOR_COULD_NOT_FIND_MANAGER;
-
-      if (locatorResponseException != null) {
-        String locatorResponseExceptionMessage = locatorResponseException.getMessage();
-        locatorResponseExceptionMessage = (!StringUtils.isBlank(locatorResponseExceptionMessage)
-            ? locatorResponseExceptionMessage : locatorResponseException.toString());
-        exceptionMessage = "Exception caused JMX Manager startup to fail because: '"
-            .concat(locatorResponseExceptionMessage).concat("'");
-      }
-
-      throw new IllegalStateException(exceptionMessage, locatorResponseException);
-    }
-
-    ConnectionEndpoint memberEndpoint =
-        new ConnectionEndpoint(locatorResponse.getHost(), locatorResponse.getPort());
-
-    String resultMessage = CliStrings.format(CliStrings.CONNECT__MSG__CONNECTING_TO_MANAGER_AT_0,
-        memberEndpoint.toString(false));
-
-    return new ConnectToLocatorResult(memberEndpoint, resultMessage,
-        locatorResponse.isJmxManagerSslEnabled());
-  }
-
-
   @CliCommand(value = {CliStrings.DISCONNECT}, help = CliStrings.DISCONNECT__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_JMX,
       CliStrings.TOPIC_GEODE_MANAGER})
@@ -725,7 +758,6 @@ public class ShellCommands implements CommandMarker {
     return result;
   }
 
-
   @CliCommand(value = {CliStrings.DESCRIBE_CONNECTION}, help = CliStrings.DESCRIBE_CONNECTION__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_JMX})
   public Result describeConnection() {
@@ -750,7 +782,6 @@ public class ShellCommands implements CommandMarker {
     return result;
   }
 
-
   @CliCommand(value = {CliStrings.ECHO}, help = CliStrings.ECHO__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH})
   public Result echo(@CliOption(key = {CliStrings.ECHO__STR, ""},
@@ -784,7 +815,6 @@ public class ShellCommands implements CommandMarker {
     return resultData;
   }
 
-
   @CliCommand(value = {CliStrings.SET_VARIABLE}, help = CliStrings.SET_VARIABLE__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH})
   public Result setVariable(
@@ -806,22 +836,6 @@ public class ShellCommands implements CommandMarker {
     return result;
   }
 
-  // Enable when "use region" command is required. See #46110
-  // @CliCommand(value = { CliStrings.USE_REGION }, help = CliStrings.USE_REGION__HELP)
-  // @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH,
-  // CliStrings.TOPIC_GEODE_REGION})
-  // public Result useRegion(
-  // @CliArgument(name = CliStrings.USE_REGION__REGION,
-  // unspecifiedDefaultValue = "/",
-  // argumentContext = CliStrings.PARAM_CONTEXT_REGIONPATH,
-  // help = CliStrings.USE_REGION__REGION__HELP)
-  // String toRegion) {
-  // Gfsh gfsh = Gfsh.getCurrentInstance();
-  //
-  // gfsh.setPromptPath(toRegion);
-  // return ResultBuilder.createInfoResult("");
-  // }
-
   @CliCommand(value = {CliStrings.DEBUG}, help = CliStrings.DEBUG__HELP)
   @CliMetaData(shellOnly = true,
       relatedTopic = {CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_DEBUG_UTIL})
@@ -875,8 +889,7 @@ public class ShellCommands implements CommandMarker {
 
       GfshHistory gfshHistory = gfsh.getGfshHistory();
       Iterator<?> it = gfshHistory.entries();
-      boolean flagForLineNumbers =
-          (saveHistoryTo != null && saveHistoryTo.length() > 0) ? false : true;
+      boolean flagForLineNumbers = !(saveHistoryTo != null && saveHistoryTo.length() > 0);
       long lineNumber = 0;
 
       while (it.hasNext()) {
@@ -956,7 +969,6 @@ public class ShellCommands implements CommandMarker {
 
   }
 
-
   @CliCommand(value = {CliStrings.RUN}, help = CliStrings.RUN__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH})
   public Result executeScript(
@@ -979,7 +991,6 @@ public class ShellCommands implements CommandMarker {
     return result;
   }
 
-
   @CliCommand(value = CliStrings.ENCRYPT, help = CliStrings.ENCRYPT__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_DEBUG_UTIL})
   public Result encryptPassword(@CliOption(key = CliStrings.ENCRYPT_STRING,
@@ -1011,7 +1022,7 @@ public class ShellCommands implements CommandMarker {
   @CliCommand(value = {CliStrings.SH}, help = CliStrings.SH__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH})
   public Result sh(
-      @CliArgument(name = CliStrings.SH__COMMAND, mandatory = true,
+      @CliOption(key = {"", CliStrings.SH__COMMAND}, mandatory = true,
           help = CliStrings.SH__COMMAND__HELP) String command,
       @CliOption(key = CliStrings.SH__USE_CONSOLE, specifiedDefaultValue = "true",
           unspecifiedDefaultValue = "false",
@@ -1032,47 +1043,6 @@ public class ShellCommands implements CommandMarker {
     return result;
   }
 
-  private static InfoResultData executeCommand(Gfsh gfsh, String userCommand, boolean useConsole)
-      throws IOException {
-    InfoResultData infoResultData = ResultBuilder.createInfoResultData();
-
-    String cmdToExecute = userCommand;
-    String cmdExecutor = "/bin/sh";
-    String cmdExecutorOpt = "-c";
-    if (SystemUtils.isWindows()) {
-      cmdExecutor = "cmd";
-      cmdExecutorOpt = "/c";
-    } else if (useConsole) {
-      cmdToExecute = cmdToExecute + " </dev/tty >/dev/tty";
-    }
-    String[] commandArray = {cmdExecutor, cmdExecutorOpt, cmdToExecute};
-
-    ProcessBuilder builder = new ProcessBuilder();
-    builder.command(commandArray);
-    builder.directory();
-    builder.redirectErrorStream();
-    Process proc = builder.start();
-
-    BufferedReader input = new BufferedReader(new InputStreamReader(proc.getInputStream()));
-
-    String lineRead = "";
-    while ((lineRead = input.readLine()) != null) {
-      infoResultData.addLine(lineRead);
-    }
-
-    proc.getOutputStream().close();
-
-    try {
-      if (proc.waitFor() != 0) {
-        gfsh.logWarning("The command '" + userCommand + "' did not complete successfully", null);
-      }
-    } catch (final InterruptedException e) {
-      throw new IllegalStateException(e);
-    }
-    return infoResultData;
-  }
-
-
   @CliAvailabilityIndicator({CliStrings.CONNECT, CliStrings.DISCONNECT,
       CliStrings.DESCRIBE_CONNECTION})
   public boolean isAvailable() {

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java
index 239db48..cd8f6eb 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/WanCommands.java
@@ -71,14 +71,12 @@ public class WanCommands extends AbstractCommandsSupport {
   @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public Result createGatewaySender(@CliOption(key = CliStrings.CREATE_GATEWAYSENDER__GROUP,
       optionContext = ConverterHint.MEMBERGROUP,
-      help = CliStrings.CREATE_GATEWAYSENDER__GROUP__HELP) @CliMetaData(
-          valueSeparator = ",") String[] onGroups,
+      help = CliStrings.CREATE_GATEWAYSENDER__GROUP__HELP) String[] onGroups,
 
       @CliOption(key = CliStrings.CREATE_GATEWAYSENDER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.CREATE_GATEWAYSENDER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember,
+          help = CliStrings.CREATE_GATEWAYSENDER__MEMBER__HELP) String onMember,
 
       @CliOption(key = CliStrings.CREATE_GATEWAYSENDER__ID, mandatory = true,
           help = CliStrings.CREATE_GATEWAYSENDER__ID__HELP) String id,
@@ -129,12 +127,10 @@ public class WanCommands extends AbstractCommandsSupport {
           help = CliStrings.CREATE_GATEWAYSENDER__ORDERPOLICY__HELP) String orderPolicy,
 
       @CliOption(key = CliStrings.CREATE_GATEWAYSENDER__GATEWAYEVENTFILTER,
-          help = CliStrings.CREATE_GATEWAYSENDER__GATEWAYEVENTFILTER__HELP) @CliMetaData(
-              valueSeparator = ",") String[] gatewayEventFilters,
+          help = CliStrings.CREATE_GATEWAYSENDER__GATEWAYEVENTFILTER__HELP) String[] gatewayEventFilters,
 
       @CliOption(key = CliStrings.CREATE_GATEWAYSENDER__GATEWAYTRANSPORTFILTER,
-          help = CliStrings.CREATE_GATEWAYSENDER__GATEWAYTRANSPORTFILTER__HELP) @CliMetaData(
-              valueSeparator = ",") String[] gatewayTransportFilter) {
+          help = CliStrings.CREATE_GATEWAYSENDER__GATEWAYTRANSPORTFILTER__HELP) String[] gatewayTransportFilter) {
 
     Result result = null;
 
@@ -192,13 +188,11 @@ public class WanCommands extends AbstractCommandsSupport {
 
       @CliOption(key = CliStrings.START_GATEWAYSENDER__GROUP,
           optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.START_GATEWAYSENDER__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String onGroup,
+          help = CliStrings.START_GATEWAYSENDER__GROUP__HELP) String onGroup,
 
       @CliOption(key = CliStrings.START_GATEWAYSENDER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.START_GATEWAYSENDER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember) {
+          help = CliStrings.START_GATEWAYSENDER__MEMBER__HELP) String onMember) {
 
     Result result = null;
     final String id = senderId.trim();
@@ -316,13 +310,11 @@ public class WanCommands extends AbstractCommandsSupport {
 
       @CliOption(key = CliStrings.PAUSE_GATEWAYSENDER__GROUP,
           optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.PAUSE_GATEWAYSENDER__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String onGroup,
+          help = CliStrings.PAUSE_GATEWAYSENDER__GROUP__HELP) String onGroup,
 
       @CliOption(key = CliStrings.PAUSE_GATEWAYSENDER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.PAUSE_GATEWAYSENDER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember) {
+          help = CliStrings.PAUSE_GATEWAYSENDER__MEMBER__HELP) String onMember) {
 
     Result result = null;
     if (senderId != null)
@@ -396,13 +388,10 @@ public class WanCommands extends AbstractCommandsSupport {
 
       @CliOption(key = CliStrings.RESUME_GATEWAYSENDER__GROUP,
           optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.RESUME_GATEWAYSENDER__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String onGroup,
-
+          help = CliStrings.RESUME_GATEWAYSENDER__GROUP__HELP) String onGroup,
       @CliOption(key = CliStrings.RESUME_GATEWAYSENDER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.RESUME_GATEWAYSENDER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember) {
+          help = CliStrings.RESUME_GATEWAYSENDER__MEMBER__HELP) String onMember) {
 
     Result result = null;
     if (senderId != null)
@@ -527,13 +516,11 @@ public class WanCommands extends AbstractCommandsSupport {
 
       @CliOption(key = CliStrings.STOP_GATEWAYSENDER__GROUP,
           optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.STOP_GATEWAYSENDER__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String onGroup,
+          help = CliStrings.STOP_GATEWAYSENDER__GROUP__HELP) String onGroup,
 
       @CliOption(key = CliStrings.STOP_GATEWAYSENDER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.STOP_GATEWAYSENDER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember) {
+          help = CliStrings.STOP_GATEWAYSENDER__MEMBER__HELP) String onMember) {
 
     Result result = null;
     if (senderId != null)
@@ -592,14 +579,12 @@ public class WanCommands extends AbstractCommandsSupport {
   @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public Result createGatewayReceiver(@CliOption(key = CliStrings.CREATE_GATEWAYRECEIVER__GROUP,
       optionContext = ConverterHint.MEMBERGROUP,
-      help = CliStrings.CREATE_GATEWAYRECEIVER__GROUP__HELP) @CliMetaData(
-          valueSeparator = ",") String[] onGroups,
+      help = CliStrings.CREATE_GATEWAYRECEIVER__GROUP__HELP) String[] onGroups,
 
       @CliOption(key = CliStrings.CREATE_GATEWAYRECEIVER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.CREATE_GATEWAYRECEIVER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember,
+          help = CliStrings.CREATE_GATEWAYRECEIVER__MEMBER__HELP) String onMember,
 
       @CliOption(key = CliStrings.CREATE_GATEWAYRECEIVER__MANUALSTART,
           help = CliStrings.CREATE_GATEWAYRECEIVER__MANUALSTART__HELP) Boolean manualStart,
@@ -620,8 +605,7 @@ public class WanCommands extends AbstractCommandsSupport {
           help = CliStrings.CREATE_GATEWAYRECEIVER__SOCKETBUFFERSIZE__HELP) Integer socketBufferSize,
 
       @CliOption(key = CliStrings.CREATE_GATEWAYRECEIVER__GATEWAYTRANSPORTFILTER,
-          help = CliStrings.CREATE_GATEWAYRECEIVER__GATEWAYTRANSPORTFILTER__HELP) @CliMetaData(
-              valueSeparator = ",") String[] gatewayTransportFilters) {
+          help = CliStrings.CREATE_GATEWAYRECEIVER__GATEWAYTRANSPORTFILTER__HELP) String[] gatewayTransportFilters) {
 
     Result result = null;
 
@@ -735,13 +719,11 @@ public class WanCommands extends AbstractCommandsSupport {
   @ResourceOperation(resource = Resource.DATA, operation = Operation.MANAGE)
   public Result startGatewayReceiver(@CliOption(key = CliStrings.START_GATEWAYRECEIVER__GROUP,
       optionContext = ConverterHint.MEMBERGROUP,
-      help = CliStrings.START_GATEWAYRECEIVER__GROUP__HELP) @CliMetaData(
-          valueSeparator = ",") String onGroup,
+      help = CliStrings.START_GATEWAYRECEIVER__GROUP__HELP) String onGroup,
 
       @CliOption(key = CliStrings.START_GATEWAYRECEIVER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.START_GATEWAYRECEIVER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember) {
+          help = CliStrings.START_GATEWAYRECEIVER__MEMBER__HELP) String onMember) {
     Result result = null;
 
 
@@ -800,13 +782,11 @@ public class WanCommands extends AbstractCommandsSupport {
 
       @CliOption(key = CliStrings.STOP_GATEWAYRECEIVER__GROUP,
           optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.STOP_GATEWAYRECEIVER__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String onGroup,
+          help = CliStrings.STOP_GATEWAYRECEIVER__GROUP__HELP) String onGroup,
 
       @CliOption(key = CliStrings.STOP_GATEWAYRECEIVER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.STOP_GATEWAYRECEIVER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember) {
+          help = CliStrings.STOP_GATEWAYRECEIVER__MEMBER__HELP) String onMember) {
 
     Result result = null;
 
@@ -865,11 +845,9 @@ public class WanCommands extends AbstractCommandsSupport {
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
   public Result listGateway(
       @CliOption(key = CliStrings.LIST_GATEWAY__MEMBER, optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.LIST_GATEWAY__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember,
+          help = CliStrings.LIST_GATEWAY__MEMBER__HELP) String onMember,
       @CliOption(key = CliStrings.LIST_GATEWAY__GROUP, optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.LIST_GATEWAY__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String onGroup) {
+          help = CliStrings.LIST_GATEWAY__GROUP__HELP) String onGroup) {
 
     Result result = null;
     Cache cache = CacheFactory.getAnyInstance();
@@ -957,13 +935,11 @@ public class WanCommands extends AbstractCommandsSupport {
 
       @CliOption(key = CliStrings.STATUS_GATEWAYSENDER__GROUP,
           optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.STATUS_GATEWAYSENDER__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String onGroup,
+          help = CliStrings.STATUS_GATEWAYSENDER__GROUP__HELP) String onGroup,
 
       @CliOption(key = CliStrings.STATUS_GATEWAYSENDER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.STATUS_GATEWAYSENDER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember) {
+          help = CliStrings.STATUS_GATEWAYSENDER__MEMBER__HELP) String onMember) {
 
     Result result = null;
     if (senderId != null)
@@ -1016,13 +992,11 @@ public class WanCommands extends AbstractCommandsSupport {
   @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
   public Result statusGatewayReceiver(@CliOption(key = CliStrings.STATUS_GATEWAYRECEIVER__GROUP,
       optionContext = ConverterHint.MEMBERGROUP,
-      help = CliStrings.STATUS_GATEWAYRECEIVER__GROUP__HELP) @CliMetaData(
-          valueSeparator = ",") String onGroup,
+      help = CliStrings.STATUS_GATEWAYRECEIVER__GROUP__HELP) String onGroup,
 
       @CliOption(key = CliStrings.STATUS_GATEWAYRECEIVER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
-          help = CliStrings.STATUS_GATEWAYRECEIVER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember) {
+          help = CliStrings.STATUS_GATEWAYRECEIVER__MEMBER__HELP) String onMember) {
 
     Result result = null;
 
@@ -1073,13 +1047,11 @@ public class WanCommands extends AbstractCommandsSupport {
   public Result destroyGatewaySender(
       @CliOption(key = CliStrings.DESTROY_GATEWAYSENDER__GROUP,
           optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.DESTROY_GATEWAYSENDER__GROUP__HELP) @CliMetaData(
-              valueSeparator = ",") String[] onGroups,
+          help = CliStrings.DESTROY_GATEWAYSENDER__GROUP__HELP) String[] onGroups,
       @CliOption(key = CliStrings.DESTROY_GATEWAYSENDER__MEMBER,
           optionContext = ConverterHint.MEMBERIDNAME,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.DESTROY_GATEWAYSENDER__MEMBER__HELP) @CliMetaData(
-              valueSeparator = ",") String onMember,
+          help = CliStrings.DESTROY_GATEWAYSENDER__MEMBER__HELP) String onMember,
       @CliOption(key = CliStrings.DESTROY_GATEWAYSENDER__ID, mandatory = true,
           optionContext = ConverterHint.GATEWAY_SENDER_ID,
           help = CliStrings.DESTROY_GATEWAYSENDER__ID__HELP) String id) {

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/BooleanConverter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/BooleanConverter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/BooleanConverter.java
deleted file mode 100644
index 5e9cdb9..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/BooleanConverter.java
+++ /dev/null
@@ -1,54 +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.converters;
-
-import java.util.List;
-
-import org.springframework.shell.core.Completion;
-import org.springframework.shell.core.Converter;
-import org.springframework.shell.core.MethodTarget;
-
-/**
- * {@link Converter} for {@link Boolean}. Use this BooleanConverter instead of SHL's
- * BooleanConverter. Removed completion & conversion for values like 0, 1, yes, no.
- *
- * @since GemFire 7.0
- */
-public class BooleanConverter implements Converter<Boolean> {
-
-  public boolean supports(final Class<?> requiredType, final String optionContext) {
-    return Boolean.class.isAssignableFrom(requiredType)
-        || boolean.class.isAssignableFrom(requiredType);
-  }
-
-  public Boolean convertFromText(final String value, final Class<?> requiredType,
-      final String optionContext) {
-    if ("true".equalsIgnoreCase(value)) {
-      return true;
-    } else if ("false".equalsIgnoreCase(value)) {
-      return false;
-    } else {
-      throw new IllegalArgumentException("Cannot convert " + value + " to type Boolean.");
-    }
-  }
-
-  public boolean getAllPossibleValues(final List<Completion> completions,
-      final Class<?> requiredType, final String existingData, final String optionContext,
-      final MethodTarget target) {
-    completions.add(new Completion("true"));
-    completions.add(new Completion("false"));
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/DirConverter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/DirConverter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/DirConverter.java
deleted file mode 100644
index c97057b..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/DirConverter.java
+++ /dev/null
@@ -1,158 +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.converters;
-
-import java.io.File;
-import java.util.List;
-
-import org.springframework.shell.core.Completion;
-import org.springframework.shell.core.MethodTarget;
-import org.springframework.shell.support.util.FileUtils;
-import org.springframework.util.Assert;
-
-import org.apache.geode.management.cli.ConverterHint;
-import org.apache.geode.management.internal.cli.MultipleValueAdapter;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-
-/**
- *
- * @since GemFire 7.0
- * 
- * 
- */
-public class DirConverter extends MultipleValueAdapter<String[]> {
-
-  private static final String HOME_DIRECTORY_SYMBOL = "~";
-  // Constants
-  private static final String home = System.getProperty("user.home");
-
-  @Override
-  public String[] convertFromText(String[] values, Class<?> targetType, String context) {
-    return values;
-  }
-
-  @Override
-  public boolean getAllPossibleValues(List<Completion> completions, Class<?> targetType,
-      String[] existingData, String context, MethodTarget target) {
-    String adjustedUserInput = convertUserInputIntoAFullyQualifiedPath(
-        (existingData != null) ? existingData[existingData.length - 1] : "");
-
-    String directoryData =
-        adjustedUserInput.substring(0, adjustedUserInput.lastIndexOf(File.separator) + 1);
-    adjustedUserInput =
-        adjustedUserInput.substring(adjustedUserInput.lastIndexOf(File.separator) + 1);
-
-    populate(completions, adjustedUserInput,
-        ((existingData != null) ? existingData[existingData.length - 1] : ""), directoryData);
-
-    return true;
-  }
-
-  protected void populate(final List<Completion> completions, final String adjustedUserInput,
-      final String originalUserInput, final String directoryData) {
-    File directory = new File(directoryData);
-
-    if (!directory.isDirectory()) {
-      return;
-    }
-
-    for (File file : directory.listFiles()) {
-      if (adjustedUserInput == null || adjustedUserInput.length() == 0
-          || file.getName().toLowerCase().startsWith(adjustedUserInput.toLowerCase())) {
-
-        String completion = "";
-        if (directoryData.length() > 0)
-          completion += directoryData;
-        completion += file.getName();
-
-        completion = convertCompletionBackIntoUserInputStyle(originalUserInput, completion);
-
-        if (file.isDirectory()) {
-          completions.add(new Completion(completion + File.separator));
-        }
-      }
-    }
-  }
-
-  private String convertCompletionBackIntoUserInputStyle(final String originalUserInput,
-      final String completion) {
-    if (FileUtils.denotesAbsolutePath(originalUserInput)) {
-      // Input was originally as a fully-qualified path, so we just keep the
-      // completion in that form
-      return completion;
-    }
-    if (originalUserInput.startsWith(HOME_DIRECTORY_SYMBOL)) {
-      // Input originally started with this symbol, so replace the user's home
-      // directory with it again
-      Assert.notNull(home, "Home directory could not be determined from system properties");
-      return HOME_DIRECTORY_SYMBOL + completion.substring(home.length());
-    }
-    // The path was working directory specific, so strip the working directory
-    // given the user never typed it
-    return completion.substring(getWorkingDirectoryAsString().length());
-  }
-
-  /**
-   * If the user input starts with a tilde character (~), replace the tilde character with the
-   * user's home directory. If the user input does not start with a tilde, simply return the
-   * original user input without any changes if the input specifies an absolute path, or return an
-   * absolute path based on the working directory if the input specifies a relative path.
-   * 
-   * @param userInput the user input, which may commence with a tilde (required)
-   * @return a string that is guaranteed to no longer contain a tilde as the first character (never
-   *         null)
-   */
-  private String convertUserInputIntoAFullyQualifiedPath(final String userInput) {
-    if (FileUtils.denotesAbsolutePath(userInput)) {
-      // Input is already in a fully-qualified path form
-      return userInput;
-    }
-    if (userInput.startsWith(HOME_DIRECTORY_SYMBOL)) {
-      // Replace this symbol with the user's actual home directory
-      Assert.notNull(home, "Home directory could not be determined from system properties");
-      if (userInput.length() > 1) {
-        return home + userInput.substring(1);
-      }
-    }
-    // The path is working directory specific, so prepend the working directory
-    String fullPath = getWorkingDirectoryAsString() + userInput;
-    return fullPath;
-  }
-
-  private String getWorkingDirectoryAsString() {
-    try {
-      return getWorkingDirectory().getCanonicalPath() + File.separator;
-    } catch (Exception e) {
-      throw new IllegalStateException(e);
-    }
-  }
-
-  /**
-   * @return the "current working directory" this {@link DirConverter} should use if the user fails
-   *         to provide an explicit directory in their input (required)
-   */
-  private File getWorkingDirectory() {
-    return Gfsh.getCurrentInstance().getHome();
-  }
-
-  @Override
-  public boolean supports(Class<?> type, String optionContext) {
-    if (String[].class.isAssignableFrom(type) && optionContext.equals(ConverterHint.DIRS)) {
-      return true;
-    }
-    return false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/EnumConverter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/EnumConverter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/EnumConverter.java
deleted file mode 100644
index 354cef9..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/EnumConverter.java
+++ /dev/null
@@ -1,64 +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.converters;
-
-import java.util.List;
-
-import org.springframework.shell.core.Completion;
-import org.springframework.shell.core.Converter;
-import org.springframework.shell.core.MethodTarget;
-
-/**
- * {@link Converter} for {@link Enum}.
- *
- * @since GemFire 7.0
- */
-/*
- * Use this EnumConverter instead of SHL's EnumConverter. Added null check for existingData in
- * getAllPossibleValues
- * 
- * Original authors: Ben Alex & Alan Stewart
- */
-@SuppressWarnings("all") // Enum parameter warning
-public class EnumConverter implements Converter<Enum> {
-
-  public Enum convertFromText(final String value, final Class<?> requiredType,
-      final String optionContext) {
-    Class<Enum> enumClass = (Class<Enum>) requiredType;
-    return Enum.valueOf(enumClass, value);
-  }
-
-  public boolean getAllPossibleValues(final List<Completion> completions,
-      final Class<?> requiredType, final String existingData, final String optionContext,
-      final MethodTarget target) {
-    Class<Enum> enumClass = (Class<Enum>) requiredType;
-    for (Enum enumValue : enumClass.getEnumConstants()) {
-      String candidate = enumValue.name();
-      // GemFire/gfsh addition - check 'existingData == null'. GfshParser can
-      // pass existingData as null
-      if ("".equals(existingData) || existingData == null || candidate.startsWith(existingData)
-          || existingData.startsWith(candidate)
-          || candidate.toUpperCase().startsWith(existingData.toUpperCase())
-          || existingData.toUpperCase().startsWith(candidate.toUpperCase())) {
-        completions.add(new Completion(candidate));
-      }
-    }
-    return true;
-  }
-
-  public boolean supports(final Class<?> requiredType, final String optionContext) {
-    return Enum.class.isAssignableFrom(requiredType);
-  }
-}


[6/9] geode git commit: GEODE-1597: use Spring shell's parser and delete our own parsing code

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/HelpConverter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/HelpConverter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/HelpConverter.java
deleted file mode 100644
index e670274..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/HelpConverter.java
+++ /dev/null
@@ -1,68 +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.converters;
-
-import java.util.List;
-
-import org.springframework.shell.core.Completion;
-import org.springframework.shell.core.Converter;
-import org.springframework.shell.core.MethodTarget;
-
-import org.apache.geode.management.internal.cli.commands.GfshHelpCommands;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-
-/**
- * {@link Converter} for {@link GfshHelpCommands#obtainHelp(String)}
- * 
- *
- * @since GemFire 7.0
- */
-public class HelpConverter implements Converter<String> {
-
-  @Override
-  public String convertFromText(String existingData, Class<?> dataType, String optionContext) {
-
-    if (optionContext.equals(CliStrings.PARAM_CONTEXT_HELP)) {
-      return existingData.replaceAll("\"", "").replaceAll("'", "");
-    } else {
-      return null;
-    }
-  }
-
-  @Override
-  public boolean getAllPossibleValues(List<Completion> completionCandidates, Class<?> dataType,
-      String existingData, String optionContext, MethodTarget arg4) {
-
-    List<String> commandNames = Gfsh.getCurrentInstance().obtainHelpCommandNames(existingData);
-
-    for (String string : commandNames) {
-      completionCandidates.add(new Completion(string));
-    }
-    if (completionCandidates.size() > 0) {
-      return true;
-    }
-    return false;
-  }
-
-  @Override
-  public boolean supports(Class<?> arg0, String optionContext) {
-    if (String.class.isAssignableFrom(arg0)
-        && optionContext.equals(CliStrings.PARAM_CONTEXT_HELP)) {
-      return true;
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/HintTopicConverter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/HintTopicConverter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/HintTopicConverter.java
deleted file mode 100644
index b6f9f81..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/HintTopicConverter.java
+++ /dev/null
@@ -1,71 +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.converters;
-
-import java.util.List;
-import java.util.Set;
-
-import org.springframework.shell.core.Completion;
-import org.springframework.shell.core.Converter;
-import org.springframework.shell.core.MethodTarget;
-
-import org.apache.geode.management.cli.ConverterHint;
-import org.apache.geode.management.internal.cli.CommandManager;
-
-/**
- * 
- * @since GemFire 7.0
- */
-public class HintTopicConverter implements Converter<String> {
-
-  @Override
-  public boolean supports(Class<?> type, String optionContext) {
-    return String.class.equals(type) && ConverterHint.HINTTOPIC.equals(optionContext);
-  }
-
-  @Override
-  public String convertFromText(String value, Class<?> targetType, String optionContext) {
-    return value;
-  }
-
-  @Override
-  public boolean getAllPossibleValues(List<Completion> completions, Class<?> targetType,
-      String existingData, String optionContext, MethodTarget target) {
-    if (String.class.equals(targetType) && ConverterHint.HINTTOPIC.equals(optionContext)) {
-      CommandManager commandManager = CommandManager.getExisting();
-      if (commandManager != null) {
-        Set<String> topicNames = commandManager.getTopicNames();
-
-        for (String topicName : topicNames) {
-          if (existingData != null && !existingData.isEmpty()) {
-            if (topicName.startsWith(existingData)) { // match exact case
-              completions.add(new Completion(topicName));
-            } else if (topicName.toLowerCase().startsWith(existingData.toLowerCase())) { // match
-                                                                                         // case
-                                                                                         // insensitive
-              String completionStr = existingData + topicName.substring(existingData.length());
-
-              completions.add(new Completion(completionStr));
-            }
-          } else {
-            completions.add(new Completion(topicName));
-          }
-        }
-      }
-    }
-
-    return !completions.isEmpty();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/LogLevelConverter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/LogLevelConverter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/LogLevelConverter.java
index 3a26240..b303e77 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/LogLevelConverter.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/LogLevelConverter.java
@@ -41,7 +41,7 @@ public class LogLevelConverter implements Converter<String> {
 
   @Override
   public boolean supports(Class<?> type, String optionContext) {
-    return String.class.equals(type) && ConverterHint.LOG_LEVEL.equals(optionContext);
+    return String.class.equals(type) && optionContext.contains(ConverterHint.LOG_LEVEL);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/StringArrayConverter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/StringArrayConverter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/StringArrayConverter.java
deleted file mode 100644
index eacf181..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/StringArrayConverter.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.cli.converters;
-
-import java.util.List;
-
-import org.springframework.shell.core.Completion;
-import org.springframework.shell.core.MethodTarget;
-
-import org.apache.geode.management.cli.ConverterHint;
-import org.apache.geode.management.internal.cli.MultipleValueAdapter;
-
-/**
- *
- * @since GemFire 7.0
- * 
- * 
- */
-public class StringArrayConverter extends MultipleValueAdapter<String[]> {
-
-  @Override
-  public String[] convertFromText(String[] value, Class<?> targetType, String context) {
-    return value;
-  }
-
-  @Override
-  public boolean getAllPossibleValues(List<Completion> completions, Class<?> targetType,
-      String[] existingData, String context, MethodTarget target) {
-    return false;
-  }
-
-  @Override
-  public boolean supports(Class<?> type, String optionContext) {
-    if (String[].class.isAssignableFrom(type) && !optionContext.equals(ConverterHint.DIRS)) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/StringListConverter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/StringListConverter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/StringListConverter.java
deleted file mode 100644
index eab096b..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/converters/StringListConverter.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.cli.converters;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.springframework.shell.core.Completion;
-import org.springframework.shell.core.MethodTarget;
-
-import org.apache.geode.management.cli.ConverterHint;
-import org.apache.geode.management.internal.cli.MultipleValueAdapter;
-
-/**
- * 
- * 
- * @since GemFire 7.0
- */
-public class StringListConverter extends MultipleValueAdapter<List<String>> {
-
-  @Override
-  public boolean supports(Class<?> type, String optionContext) {
-    return List.class.isAssignableFrom(type) && ConverterHint.STRING_LIST.equals(optionContext);
-  }
-
-  @Override
-  public List<String> convertFromText(String[] value, Class<?> targetType, String context) {
-    List<String> list = null;
-
-    if (List.class.isAssignableFrom(targetType) && ConverterHint.STRING_LIST.equals(context)
-        && value != null && value.length > 0) {
-      list = new ArrayList<String>(Arrays.asList(value));
-    }
-    return list;
-  }
-
-  @Override
-  public boolean getAllPossibleValues(List<Completion> completions, Class<?> targetType,
-      String[] existingData, String context, MethodTarget target) {
-    return List.class.isAssignableFrom(targetType) && ConverterHint.STRING_LIST.equals(context);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandException.java
deleted file mode 100644
index 7e5cba0..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandException.java
+++ /dev/null
@@ -1,66 +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.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandException extends CliException {
-  private static final long serialVersionUID = 968411094429216130L;
-
-  private CommandTarget commandTarget;
-  private OptionSet optionSet;
-
-  public CliCommandException(final CommandTarget commandTarget) {
-    this(commandTarget, null, null);
-  }
-
-  public CliCommandException(final CommandTarget commandTarget, final OptionSet optionSet) {
-    this(commandTarget, optionSet, null);
-  }
-
-  public CliCommandException(final CommandTarget commandTarget, final OptionSet optionSet,
-      final Throwable cause) {
-    super(cause);
-    this.setCommandTarget(commandTarget);
-    this.setOptionSet(optionSet);
-  }
-
-  public CommandTarget getCommandTarget() {
-    return commandTarget;
-  }
-
-  /**
-   * TODO: make this immutable
-   *
-   * @param commandTarget the commandTarget to set
-   */
-  public void setCommandTarget(CommandTarget commandTarget) {
-    this.commandTarget = commandTarget;
-  }
-
-  public OptionSet getOptionSet() {
-    return optionSet;
-  }
-
-  /**
-   * TODO: make this immutable
-   *
-   * @param optionSet the optionSet to set
-   */
-  public void setOptionSet(OptionSet optionSet) {
-    this.optionSet = optionSet;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandInvalidException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandInvalidException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandInvalidException.java
deleted file mode 100644
index a140059..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandInvalidException.java
+++ /dev/null
@@ -1,39 +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.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandInvalidException extends CliCommandException {
-  private static final long serialVersionUID = -2195809850441234116L;
-
-  public CliCommandInvalidException(final CommandTarget commandTarget) {
-    this(commandTarget, null, null);
-  }
-
-  public CliCommandInvalidException(final CommandTarget commandTarget, OptionSet optionSet) {
-    this(commandTarget, optionSet, null);
-  }
-
-  public CliCommandInvalidException(final Throwable cause) {
-    this(null, null, cause);
-  }
-
-  public CliCommandInvalidException(final CommandTarget commandTarget, OptionSet optionSet,
-      Throwable cause) {
-    super(commandTarget, optionSet, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandMultiModeOptionException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandMultiModeOptionException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandMultiModeOptionException.java
deleted file mode 100644
index acbc496..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandMultiModeOptionException.java
+++ /dev/null
@@ -1,49 +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.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandMultiModeOptionException extends CliCommandOptionException {
-  private static final long serialVersionUID = -5658813370141696448L;
-
-  public static final int MULTIPLE_LEAD_OPTIONS = 1; // TODO: move or delete
-  public static final int OPTIONS_FROM_MULTIPLE_MODES = 2; // TODO: move or delete
-
-  private String leadOptionString;
-  private int code;
-
-  public CliCommandMultiModeOptionException(final CommandTarget commandTarget, final Option option,
-      final String string, final int code) {
-    this(commandTarget, option, string, code, null);
-  }
-
-  public CliCommandMultiModeOptionException(final CommandTarget commandTarget, final Option option,
-      final String string, final int code, final Throwable cause) {
-    super(commandTarget, option, cause);
-    this.leadOptionString = string;
-    this.code = code;
-  }
-
-  public String getLeadOptionString() {
-    return leadOptionString;
-  }
-
-  public int getCode() {
-    return code;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandNotAvailableException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandNotAvailableException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandNotAvailableException.java
deleted file mode 100644
index c471df2..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandNotAvailableException.java
+++ /dev/null
@@ -1,36 +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.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandNotAvailableException extends CliCommandException {
-  private static final long serialVersionUID = -631339463163773007L;
-
-  public CliCommandNotAvailableException(final CommandTarget commandTarget) {
-    this(commandTarget, null, null);
-  }
-
-  public CliCommandNotAvailableException(final CommandTarget commandTarget,
-      final OptionSet optionSet) {
-    this(commandTarget, optionSet, null);
-  }
-
-  public CliCommandNotAvailableException(final CommandTarget commandTarget,
-      final OptionSet optionSet, final Throwable cause) {
-    super(commandTarget, optionSet, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionException.java
deleted file mode 100644
index a7e56be..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionException.java
+++ /dev/null
@@ -1,65 +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.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandOptionException extends CliCommandException {
-  private static final long serialVersionUID = -5443638512704442487L;
-
-  private Option option;
-
-  public CliCommandOptionException(final CommandTarget commandTarget, final Option option) {
-    this(commandTarget, option, null, null);
-  }
-
-  public CliCommandOptionException(final CommandTarget commandTarget, final Option option,
-      final OptionSet optionSet) {
-    this(commandTarget, option, optionSet, null);
-  }
-
-  public CliCommandOptionException(final CommandTarget commandTarget, final Option option,
-      final Throwable cause) {
-    this(commandTarget, option, null, cause);
-  }
-
-  public CliCommandOptionException(final Throwable cause) {
-    this(null, null, null, cause);
-  }
-
-  public CliCommandOptionException(final CommandTarget commandTarget, final Option option,
-      final OptionSet optionSet, final Throwable cause) {
-    super(commandTarget, optionSet, cause);
-    this.setOption(option);
-  }
-
-  /**
-   * @return option for which the exception occurred
-   */
-  public Option getOption() {
-    return option;
-  }
-
-  /**
-   * TODO: make this immutable
-   *
-   * @param option the option to set
-   */
-  public void setOption(Option option) {
-    this.option = option;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionHasMultipleValuesException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionHasMultipleValuesException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionHasMultipleValuesException.java
deleted file mode 100644
index 4b365e3..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionHasMultipleValuesException.java
+++ /dev/null
@@ -1,47 +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.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandOptionHasMultipleValuesException extends CliCommandOptionValueException {
-
-  private static final long serialVersionUID = -5277268341319591711L;
-
-  public CliCommandOptionHasMultipleValuesException(final CommandTarget commandTarget,
-      final Option option, final String value) {
-    this(commandTarget, option, null, value, null);
-  }
-
-  public CliCommandOptionHasMultipleValuesException(final CommandTarget commandTarget,
-      final Option option, final OptionSet optionSet, final String value) {
-    this(commandTarget, option, optionSet, value, null);
-  }
-
-  public CliCommandOptionHasMultipleValuesException(final Throwable cause) {
-    this(null, null, null, null, cause);
-  }
-
-  public CliCommandOptionHasMultipleValuesException(final Option option, final Throwable cause) {
-    this(null, option, null, null, cause);
-  }
-
-  public CliCommandOptionHasMultipleValuesException(final CommandTarget commandTarget,
-      final Option option, final OptionSet optionSet, final String value, final Throwable cause) {
-    super(commandTarget, option, optionSet, value, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionInvalidException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionInvalidException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionInvalidException.java
deleted file mode 100644
index 1db8906..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionInvalidException.java
+++ /dev/null
@@ -1,37 +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.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandOptionInvalidException extends CliCommandOptionException {
-  private static final long serialVersionUID = 8773148664471110429L;
-
-  public CliCommandOptionInvalidException(final CommandTarget commandTarget, final Option option) {
-    this(commandTarget, option, null, null);
-  }
-
-  public CliCommandOptionInvalidException(final CommandTarget commandTarget, final Option option,
-      final OptionSet optionSet) {
-    this(commandTarget, option, optionSet, null);
-  }
-
-  public CliCommandOptionInvalidException(final CommandTarget commandTarget, final Option option,
-      final OptionSet optionSet, final Throwable cause) {
-    super(commandTarget, option, optionSet, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionMissingException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionMissingException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionMissingException.java
deleted file mode 100644
index f263dce..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionMissingException.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.management.internal.cli.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandOptionMissingException extends CliCommandOptionException {
-  private static final long serialVersionUID = 7152881150151676813L;
-
-  public CliCommandOptionMissingException(final CommandTarget commandTarget, final Option option) {
-    this(commandTarget, option, null, null);
-  }
-
-  public CliCommandOptionMissingException(final CommandTarget commandTarget, final Option option,
-      final OptionSet optionSet) {
-    this(commandTarget, option, optionSet, null);
-  }
-
-  public CliCommandOptionMissingException(final Throwable cause) {
-    this(null, null, null, cause);
-  }
-
-  public CliCommandOptionMissingException(final Option option, final Throwable cause) {
-    this(null, option, null, cause);
-  }
-
-  public CliCommandOptionMissingException(final CommandTarget commandTarget, final Option option,
-      final OptionSet optionSet, final Throwable cause) {
-    super(commandTarget, option, optionSet, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionNotApplicableException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionNotApplicableException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionNotApplicableException.java
deleted file mode 100644
index 9814778..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionNotApplicableException.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.cli.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandOptionNotApplicableException extends CliCommandOptionException {
-  private static final long serialVersionUID = 4190478428338602501L;
-
-  public CliCommandOptionNotApplicableException(final CommandTarget commandTarget,
-      final Option option) {
-    this(commandTarget, option, null, null);
-  }
-
-  public CliCommandOptionNotApplicableException(final CommandTarget commandTarget,
-      final Option option, final OptionSet optionSet) {
-    this(commandTarget, option, optionSet, null);
-  }
-
-  public CliCommandOptionNotApplicableException(final Throwable cause) {
-    this(null, null, null, cause);
-  }
-
-  public CliCommandOptionNotApplicableException(final Option option, final Throwable cause) {
-    this(null, option, null, cause);
-  }
-
-  public CliCommandOptionNotApplicableException(final CommandTarget commandTarget,
-      final Option option, final OptionSet optionSet, Throwable cause) {
-    super(commandTarget, option, optionSet, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueConversionException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueConversionException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueConversionException.java
deleted file mode 100644
index 7dbf869..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueConversionException.java
+++ /dev/null
@@ -1,38 +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.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandOptionValueConversionException extends CliCommandOptionValueException {
-  private static final long serialVersionUID = 5144720637801591L;
-
-  public CliCommandOptionValueConversionException(final CommandTarget commandTarget,
-      final Option option, final String value) {
-    this(commandTarget, option, null, value, null);
-  }
-
-  public CliCommandOptionValueConversionException(final CommandTarget commandTarget,
-      final Option option, final OptionSet optionSet, final String value) {
-    this(commandTarget, option, optionSet, value, null);
-  }
-
-  public CliCommandOptionValueConversionException(final CommandTarget commandTarget,
-      final Option option, final OptionSet optionSet, final String value, final Throwable cause) {
-    super(commandTarget, option, optionSet, value, null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueException.java
deleted file mode 100644
index ee02df8..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueException.java
+++ /dev/null
@@ -1,49 +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.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandOptionValueException extends CliCommandOptionException {
-  private static final long serialVersionUID = -7339487978861146474L;
-
-  private final String value;
-
-  public CliCommandOptionValueException(final CommandTarget commandTarget, final Option option,
-      final String value) {
-    this(commandTarget, option, null, value, null);
-  }
-
-  public CliCommandOptionValueException(final CommandTarget commandTarget, final Option option,
-      final OptionSet optionSet, final String value) {
-    this(commandTarget, option, null, value, null);
-  }
-
-  public CliCommandOptionValueException(final Throwable cause) {
-    this(null, null, null, null, cause);
-  }
-
-  public CliCommandOptionValueException(final CommandTarget commandTarget, final Option option,
-      final OptionSet optionSet, final String value, final Throwable cause) {
-    super(commandTarget, option, optionSet, cause);
-    this.value = value;
-  }
-
-  public String getValue() {
-    return value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueMissingException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueMissingException.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueMissingException.java
deleted file mode 100644
index 023a878..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/CliCommandOptionValueMissingException.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.cli.exceptions;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-public class CliCommandOptionValueMissingException extends CliCommandOptionValueException {
-  private static final long serialVersionUID = 7842061609469545533L;
-
-  public CliCommandOptionValueMissingException(final CommandTarget commandTarget,
-      final Option option, final String value) {
-    this(commandTarget, option, null, value, null);
-  }
-
-  public CliCommandOptionValueMissingException(final CommandTarget commandTarget,
-      final Option option, final OptionSet optionSet, final String value) {
-    this(commandTarget, option, optionSet, value, null);
-  }
-
-  public CliCommandOptionValueMissingException(final Throwable cause) {
-    this(null, null, null, null, cause);
-  }
-
-  public CliCommandOptionValueMissingException(final Option option, final Throwable cause) {
-    this(null, option, null, null, cause);
-  }
-
-  public CliCommandOptionValueMissingException(final CommandTarget commandTarget,
-      final Option option, final OptionSet optionSet, final String value, final Throwable cause) {
-    super(commandTarget, option, optionSet, value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/ExceptionGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/ExceptionGenerator.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/ExceptionGenerator.java
deleted file mode 100644
index fda3135..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/ExceptionGenerator.java
+++ /dev/null
@@ -1,48 +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.exceptions;
-
-import joptsimple.OptionException;
-
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-
-/**
- * Converts joptsimple exceptions into corresponding exceptions for cli
- *
- * TODO: delete this class
- */
-public class ExceptionGenerator {
-
-  public static CliCommandOptionException generate(Option option, OptionException cause) {
-    if (cause.getClass().getSimpleName().contains("MissingRequiredOptionException")) {
-      return new CliCommandOptionMissingException(option, cause);
-
-    } else if (cause.getClass().getSimpleName()
-        .contains("OptionMissingRequiredArgumentException")) {
-      return new CliCommandOptionValueMissingException(option, cause);
-
-    } else if (cause.getClass().getSimpleName().contains("UnrecognizedOptionException")) {
-      return new CliCommandOptionNotApplicableException(option, cause);
-
-    } else if (cause.getClass().getSimpleName().contains("MultipleArgumentsForOptionException")) {
-      return new CliCommandOptionHasMultipleValuesException(option, cause);
-
-    } else {
-      return new CliCommandOptionException(cause);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/ExceptionHandler.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/ExceptionHandler.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/ExceptionHandler.java
deleted file mode 100644
index 95afbaf..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/exceptions/ExceptionHandler.java
+++ /dev/null
@@ -1,92 +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.exceptions;
-
-import java.util.logging.Logger;
-
-import org.apache.geode.management.internal.cli.util.CLIConsoleBufferUtil;
-
-/**
- * Prints the warning according the CliException
- */
-public class ExceptionHandler {
-
-  private static Logger LOGGER = Logger.getLogger(ExceptionHandler.class.getCanonicalName());
-
-  // FIXME define handling when no match is present
-  public static void handleException(CliException ce) {
-    if (ce instanceof CliCommandNotAvailableException) {
-      handleCommandNotAvailableException((CliCommandNotAvailableException) ce);
-    } else if (ce instanceof CliCommandInvalidException) {
-      handleCommandInvalidException((CliCommandInvalidException) ce);
-    } else if (ce instanceof CliCommandOptionException) {
-      handleOptionException((CliCommandOptionException) ce);
-    }
-  }
-
-  private static void handleMultiModeOptionException(CliCommandMultiModeOptionException ce) {
-    switch (ce.getCode()) {
-      case CliCommandMultiModeOptionException.MULTIPLE_LEAD_OPTIONS:
-        LOGGER.warning(CLIConsoleBufferUtil.processMessegeForExtraCharactersFromConsoleBuffer(
-            "Input command contains multiple lead-options from modes : "
-                + ce.getLeadOptionString()));
-        break;
-      case CliCommandMultiModeOptionException.OPTIONS_FROM_MULTIPLE_MODES:
-        LOGGER.warning(CLIConsoleBufferUtil.processMessegeForExtraCharactersFromConsoleBuffer(
-            "Input command contains options from multilpe modes : " + ce.getLeadOptionString()));
-        break;
-    }
-  }
-
-  private static void handleCommandInvalidException(CliCommandInvalidException ccie) {
-    LOGGER.warning(CLIConsoleBufferUtil.processMessegeForExtraCharactersFromConsoleBuffer(
-        ccie.getCommandTarget().getGfshMethodTarget().getKey() + " is not a valid Command"));
-  }
-
-  private static void handleCommandNotAvailableException(CliCommandNotAvailableException ccnae) {
-    LOGGER.warning(CLIConsoleBufferUtil.processMessegeForExtraCharactersFromConsoleBuffer(
-        ccnae.getCommandTarget().getGfshMethodTarget().getKey()
-            + " is not available at the moment"));
-  }
-
-  private static void handleOptionException(CliCommandOptionException ccoe) {
-    if (ccoe instanceof CliCommandOptionNotApplicableException) {
-      handleOptionInvalidExcpetion((CliCommandOptionNotApplicableException) ccoe);
-    } else if (ccoe instanceof CliCommandOptionValueException) {
-      handleOptionValueException((CliCommandOptionValueException) ccoe);
-    } else if (ccoe instanceof CliCommandMultiModeOptionException) {
-      handleMultiModeOptionException((CliCommandMultiModeOptionException) ccoe);
-    }
-  }
-
-  private static void handleOptionInvalidExcpetion(CliCommandOptionNotApplicableException cconae) {
-    String messege = "Parameter " + cconae.getOption().getLongOption() + " is not applicable for "
-        + cconae.getCommandTarget().getGfshMethodTarget().getKey();
-    LOGGER.warning(CLIConsoleBufferUtil.processMessegeForExtraCharactersFromConsoleBuffer(messege));
-  }
-
-  private static void handleOptionValueException(CliCommandOptionValueException ccove) {
-    if (ccove instanceof CliCommandOptionHasMultipleValuesException) {
-      // unfortunately by changing from geode-joptsimple to jopt-simple we will lose ALL such
-      // debugging info from exceptions
-      // String parameter = ccove != null && ccove.getOption() != null ?
-      // ccove.getOption().getLongOption() : "<null>";
-      String parameter = ccove.getOption().getLongOption();
-      String message = "Parameter " + parameter + " can only be specified once";
-      LOGGER
-          .warning(CLIConsoleBufferUtil.processMessegeForExtraCharactersFromConsoleBuffer(message));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/CliTopic.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/CliTopic.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/CliTopic.java
deleted file mode 100644
index 791cdca..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/CliTopic.java
+++ /dev/null
@@ -1,132 +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.help;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-
-/**
- * 
- * 
- * @since GemFire 7.0
- */
-public class CliTopic implements Comparable<CliTopic> {
-  private static final Map<String, String> nameDescriptionMap = new HashMap<String, String>();
-
-  static {
-    nameDescriptionMap.put(CliStrings.DEFAULT_TOPIC_GEODE, CliStrings.DEFAULT_TOPIC_GEODE__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_REGION__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_WAN, CliStrings.TOPIC_GEODE_WAN__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_JMX, CliStrings.TOPIC_GEODE_JMX__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_DISKSTORE,
-        CliStrings.TOPIC_GEODE_DISKSTORE__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_LOCATOR, CliStrings.TOPIC_GEODE_LOCATOR__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_SERVER, CliStrings.TOPIC_GEODE_SERVER__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_MANAGER, CliStrings.TOPIC_GEODE_MANAGER__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_STATISTICS,
-        CliStrings.TOPIC_GEODE_STATISTICS__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_LIFECYCLE,
-        CliStrings.TOPIC_GEODE_LIFECYCLE__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_M_AND_M, CliStrings.TOPIC_GEODE_M_AND_M__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_DATA__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_CONFIG, CliStrings.TOPIC_GEODE_CONFIG__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_FUNCTION, CliStrings.TOPIC_GEODE_FUNCTION__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_HELP, CliStrings.TOPIC_GEODE_HELP__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_DEBUG_UTIL,
-        CliStrings.TOPIC_GEODE_DEBUG_UTIL__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GFSH__DESC);
-  }
-
-
-  private final String name;
-  private final String oneLinerDescription;
-  private Set<CommandTarget> commandTargets;
-
-  public CliTopic(String name) {
-    this.name = name;
-    this.oneLinerDescription = nameDescriptionMap.get(this.name);
-    this.commandTargets = new HashSet<CommandTarget>();
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public String getOneLinerDescription() {
-    return oneLinerDescription;
-  }
-
-  public void addCommandTarget(CommandTarget commandTarget) {
-    commandTargets.add(commandTarget);
-  }
-
-  public Map<String, String> getCommandsNameHelp() {
-    Map<String, String> commandsNameHelp = new TreeMap<String, String>();
-
-    for (CommandTarget commandTarget : commandTargets) {
-      commandsNameHelp.put(commandTarget.getCommandName(), commandTarget.getCommandHelp());
-    }
-
-    return commandsNameHelp;
-  }
-
-  @Override
-  public int compareTo(CliTopic o) {
-    if (o != null) {
-      return this.name.compareTo(o.name);
-    } else {
-      return -1;
-    }
-  }
-
-  // hashCode & equals created using Eclipse
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((name == null) ? 0 : name.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (!getClass().isInstance(obj)) {
-      return false;
-    }
-    CliTopic other = (CliTopic) obj;
-    if (name == null) {
-      if (other.name != null) {
-        return false;
-      }
-    } else if (!name.equals(other.name)) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    return CliTopic.class.getSimpleName() + "[" + name + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/HelpBlock.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/HelpBlock.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/HelpBlock.java
new file mode 100644
index 0000000..4383044
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/HelpBlock.java
@@ -0,0 +1,86 @@
+/*
+ * 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.help;
+
+import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.management.internal.cli.GfshParser;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ *
+ */
+public class HelpBlock {
+  private String data = null;
+  private List<HelpBlock> children = new ArrayList<>();
+  // indent level
+  private int level = -1;
+
+  public HelpBlock() {}
+
+  public HelpBlock(String data) {
+    if (!StringUtils.isBlank(data)) {
+      this.data = data;
+      this.level = 0;
+    }
+  }
+
+  public String getData() {
+    return data;
+  }
+
+  public List<HelpBlock> getChildren() {
+    return children;
+  }
+
+  public int getLevel() {
+    return this.level;
+  }
+
+  public void addChild(HelpBlock helpBlock) {
+    // before adding another block as the child, increment the indent level
+    helpBlock.setLevel(level + 1);
+    children.add(helpBlock);
+  }
+
+  // recursively set the indent level of the decendents
+  public void setLevel(int level) {
+    this.level = level;
+    for (HelpBlock child : children) {
+      child.setLevel(level + 1);
+    }
+  }
+
+  @Override
+  public String toString() {
+    // no indentation, no wrapping
+    return toString(-1);
+  }
+
+  public String toString(int terminalWidth) {
+    StringBuffer builder = new StringBuffer();
+
+    if (data != null) {
+      builder.append(Gfsh.wrapText(data, level, terminalWidth));
+      builder.append(GfshParser.LINE_SEPARATOR);
+    }
+    for (HelpBlock child : children) {
+      builder.append(child.toString(terminalWidth));
+    }
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/Helper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/Helper.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/Helper.java
new file mode 100644
index 0000000..fe2dc7d
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/Helper.java
@@ -0,0 +1,345 @@
+/*
+ * 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.help;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.internal.cli.GfshParser;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.springframework.shell.core.MethodTarget;
+import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+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 java.util.Set;
+import java.util.TreeMap;
+
+/**
+ * 
+ * 
+ * @since GemFire 7.0
+ */
+public class Helper {
+
+  static final String NAME_NAME = "NAME";
+  static final String SYNONYMS_NAME = "SYNONYMS";
+  static final String SYNOPSIS_NAME = "SYNOPSIS";
+  static final String SYNTAX_NAME = "SYNTAX";
+  static final String OPTIONS_NAME = "PARAMETERS";
+  static final String IS_AVAILABLE_NAME = "IS AVAILABLE";
+
+  static final String REQUIRED_SUB_NAME = "Required: ";
+  static final String SYNONYMS_SUB_NAME = "Synonyms: ";
+  static final String SPECIFIEDDEFAULTVALUE_SUB_NAME =
+      "Default (if the parameter is specified without value): ";
+  static final String UNSPECIFIEDDEFAULTVALUE_VALUE_SUB_NAME =
+      "Default (if the parameter is not specified): ";
+
+  static final String VALUE_FIELD = "value";
+  static final String TRUE_TOKEN = "true";
+  static final String FALSE_TOKEN = "false";
+
+  private final Map<String, Topic> topics = new HashMap<>();
+  private final Map<String, Method> commands = new TreeMap<String, Method>();
+  private final Map<String, MethodTarget> availabilityIndicators =
+      new HashMap<String, MethodTarget>();
+
+  public Helper() {
+    initTopic(CliStrings.DEFAULT_TOPIC_GEODE, CliStrings.DEFAULT_TOPIC_GEODE__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_REGION__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_WAN, CliStrings.TOPIC_GEODE_WAN__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_JMX, CliStrings.TOPIC_GEODE_JMX__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_DISKSTORE, CliStrings.TOPIC_GEODE_DISKSTORE__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_LOCATOR, CliStrings.TOPIC_GEODE_LOCATOR__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_SERVER, CliStrings.TOPIC_GEODE_SERVER__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_MANAGER, CliStrings.TOPIC_GEODE_MANAGER__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_STATISTICS, CliStrings.TOPIC_GEODE_STATISTICS__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_LIFECYCLE, CliStrings.TOPIC_GEODE_LIFECYCLE__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_M_AND_M, CliStrings.TOPIC_GEODE_M_AND_M__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_DATA__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_CONFIG, CliStrings.TOPIC_GEODE_CONFIG__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_FUNCTION, CliStrings.TOPIC_GEODE_FUNCTION__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_HELP, CliStrings.TOPIC_GEODE_HELP__DESC);
+    initTopic(CliStrings.TOPIC_GEODE_DEBUG_UTIL, CliStrings.TOPIC_GEODE_DEBUG_UTIL__DESC);
+    initTopic(CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GFSH__DESC);
+    initTopic(CliStrings.TOPIC_LOGS, CliStrings.TOPIC_LOGS__DESC);
+    initTopic(CliStrings.TOPIC_CLIENT, CliStrings.TOPIC_CLIENT__DESC);
+  }
+
+  private void initTopic(String topic, String desc) {
+    topics.put(topic, new Topic(topic, desc));
+  }
+
+  public void addCommand(CliCommand command, Method commandMethod) {
+    // put all the command synonyms in the command map
+    Arrays.stream(command.value()).forEach(cmd -> {
+      commands.put(cmd, commandMethod);
+    });
+
+    // resolve the hint message for each method
+    CliMetaData cliMetaData = commandMethod.getDeclaredAnnotation(CliMetaData.class);
+    if (cliMetaData == null)
+      return;
+    String[] related = cliMetaData.relatedTopic();
+
+    // for hint message, we only need to show the first synonym
+    String commandString = command.value()[0];
+    if (related == null) {
+      return;
+    }
+    Arrays.stream(related).forEach(topic -> {
+      Topic foundTopic = topics.get(topic);
+      if (foundTopic == null) {
+        throw new IllegalArgumentException("No such topic found in the initial map: " + topic);
+      }
+      foundTopic.addRelatedCommand(commandString, command.help());
+    });
+  }
+
+  public void addAvailabilityIndicator(CliAvailabilityIndicator availability, MethodTarget target) {
+    Arrays.stream(availability.value()).forEach(command -> {
+      availabilityIndicators.put(command, target);
+    });
+  }
+
+  public String getHelp(String buffer, int terminalWidth) {
+    Method method = commands.get(buffer);
+    if (method == null) {
+      return "no help exists for this command.";
+    }
+    return getHelp(method).toString(terminalWidth);
+  }
+
+  public String getHint(String buffer) {
+    StringBuilder builder = new StringBuilder();
+    // if no topic is provided, return a list of topics
+    if (StringUtils.isBlank(buffer)) {
+      builder.append(CliStrings.HINT__MSG__TOPICS_AVAILABLE).append(GfshParser.LINE_SEPARATOR)
+          .append(GfshParser.LINE_SEPARATOR);
+
+      List<String> sortedTopics = new ArrayList<>(topics.keySet());
+      Collections.sort(sortedTopics);
+      sortedTopics.stream()
+          .forEachOrdered(topic -> builder.append(topic).append(GfshParser.LINE_SEPARATOR));
+      return builder.toString();
+    }
+
+    Topic topic = topics.get(buffer);
+    if (topic == null) {
+      return CliStrings.format(CliStrings.HINT__MSG__UNKNOWN_TOPIC, buffer);
+    }
+
+    builder.append(topic.desc).append(GfshParser.LINE_SEPARATOR).append(GfshParser.LINE_SEPARATOR);
+    Collections.sort(topic.relatedCommands);
+    topic.relatedCommands.stream().forEachOrdered(command -> builder.append(command.command)
+        .append(": ").append(command.desc).append(GfshParser.LINE_SEPARATOR));
+    return builder.toString();
+  }
+
+  private HelpBlock getHelp(Method method) {
+    return getHelp(method.getDeclaredAnnotation(CliCommand.class), method.getParameterAnnotations(),
+        method.getParameterTypes());
+  }
+
+  HelpBlock getHelp(CliCommand cliCommand, Annotation[][] annotations, Class<?>[] parameterTypes) {
+    String commandName = cliCommand.value()[0];
+    HelpBlock root = new HelpBlock();
+    // First we will have the block for NAME of the command
+    HelpBlock name = new HelpBlock(NAME_NAME);
+    name.addChild(new HelpBlock(commandName));
+    root.addChild(name);
+
+    // add the availability flag
+    HelpBlock availability = new HelpBlock(IS_AVAILABLE_NAME);
+    boolean available = true;
+    MethodTarget target = availabilityIndicators.get(commandName);
+    if (target != null) {
+      try {
+        available = (Boolean) target.getMethod().invoke(target.getTarget());
+      } catch (Exception e) {
+      }
+    }
+    availability.addChild(new HelpBlock(available + ""));
+    root.addChild(availability);
+
+    // Now add synonyms if any
+    String[] allNames = cliCommand.value();
+    if (allNames.length > 1) {
+      HelpBlock synonyms = new HelpBlock(SYNONYMS_NAME);
+      for (int i = 1; i < allNames.length; i++) {
+        synonyms.addChild(new HelpBlock(allNames[i]));
+      }
+      root.addChild(synonyms);
+    }
+
+    // Now comes the turn to display synopsis if any
+    if (!StringUtils.isBlank(cliCommand.help())) {
+      HelpBlock synopsis = new HelpBlock(SYNOPSIS_NAME);
+      synopsis.addChild(new HelpBlock(cliCommand.help()));
+      root.addChild(synopsis);
+    }
+
+    // Now display the syntax for the command
+    HelpBlock syntaxBlock = new HelpBlock(SYNTAX_NAME);
+    String syntax = getSyntaxString(commandName, annotations, parameterTypes);
+    syntaxBlock.addChild(new HelpBlock(syntax));
+    root.addChild(syntaxBlock);
+
+    // Detailed description of all the Options
+    if (annotations.length > 0) {
+      HelpBlock options = new HelpBlock(OPTIONS_NAME);
+      for (int i = 0; i < annotations.length; i++) {
+        CliOption cliOption = getAnnotation(annotations[i], CliOption.class);
+        HelpBlock optionNode = getOptionDetail(cliOption);
+        options.addChild(optionNode);
+      }
+      root.addChild(options);
+    }
+    return root;
+  }
+
+  HelpBlock getOptionDetail(CliOption cliOption) {
+    HelpBlock optionNode = new HelpBlock(getPrimaryKey(cliOption));
+    String help = cliOption.help();
+    optionNode.addChild(new HelpBlock((!StringUtils.isBlank(help) ? help : "")));
+    if (getSynonyms(cliOption).size() > 0) {
+      StringBuilder builder = new StringBuilder();
+      for (String string : getSynonyms(cliOption)) {
+        if (builder.length() > 0) {
+          builder.append(",");
+        }
+        builder.append(string);
+      }
+      optionNode.addChild(new HelpBlock(SYNONYMS_SUB_NAME + builder.toString()));
+    }
+    optionNode.addChild(
+        new HelpBlock(REQUIRED_SUB_NAME + ((cliOption.mandatory()) ? TRUE_TOKEN : FALSE_TOKEN)));
+    if (!isNullOrBlank(cliOption.specifiedDefaultValue())) {
+      optionNode.addChild(
+          new HelpBlock(SPECIFIEDDEFAULTVALUE_SUB_NAME + cliOption.specifiedDefaultValue()));
+    }
+    if (!isNullOrBlank(cliOption.unspecifiedDefaultValue())) {
+      optionNode.addChild(new HelpBlock(
+          UNSPECIFIEDDEFAULTVALUE_VALUE_SUB_NAME + cliOption.unspecifiedDefaultValue()));
+    }
+    return optionNode;
+  }
+
+  private <T> T getAnnotation(Annotation[] annotations, Class<?> klass) {
+    for (Annotation annotation : annotations) {
+      if (klass.isAssignableFrom(annotation.getClass())) {
+        return (T) annotation;
+      }
+    }
+    return null;
+  }
+
+  String getSyntaxString(String commandName, Annotation[][] annotations, Class[] parameterTypes) {
+    StringBuffer buffer = new StringBuffer();
+    buffer.append(commandName);
+    for (int i = 0; i < annotations.length; i++) {
+      CliOption cliOption = getAnnotation(annotations[i], CliOption.class);
+      String optionString = getOptionString(cliOption, parameterTypes[i]);
+      if (cliOption.mandatory()) {
+        buffer.append(" ").append(optionString);
+      } else {
+        buffer.append(" [").append(optionString).append("]");
+      }
+    }
+    return buffer.toString();
+  }
+
+  /**
+   * this builds the following format of strings: key (as in sh and help) --key=value --key(=value)?
+   * (if has specifiedDefaultValue) --key=value(,value)* (if the value is a list)
+   *
+   * @return option string
+   */
+  private static String getOptionString(CliOption cliOption, Class<?> optionType) {
+    String key0 = cliOption.key()[0];
+    if ("".equals(key0)) {
+      return (cliOption.key()[1]);
+    }
+
+    StringBuffer buffer = new StringBuffer();
+    buffer.append(GfshParser.LONG_OPTION_SPECIFIER).append(key0);
+
+    boolean hasSpecifiedDefault = !isNullOrBlank(cliOption.specifiedDefaultValue());
+
+    if (hasSpecifiedDefault) {
+      buffer.append("(");
+    }
+
+    buffer.append(GfshParser.OPTION_VALUE_SPECIFIER).append(VALUE_FIELD);
+
+    if (hasSpecifiedDefault) {
+      buffer.append(")?");
+    }
+
+    if (isCollectionOrArrayType(optionType)) {
+      buffer.append("(").append(",").append(VALUE_FIELD).append(")*");
+    }
+
+    return buffer.toString();
+  }
+
+  private static boolean isCollectionOrArrayType(Class<?> typeToCheck) {
+    return typeToCheck != null
+        && (typeToCheck.isArray() || Collection.class.isAssignableFrom(typeToCheck));
+  }
+
+  private static String getPrimaryKey(CliOption option) {
+    String[] keys = option.key();
+    if (keys.length == 0) {
+      throw new RuntimeException("Invalid option keys");
+    } else if ("".equals(keys[0])) {
+      return keys[1];
+    } else {
+      return keys[0];
+    }
+  }
+
+  private static List<String> getSynonyms(CliOption option) {
+    List<String> synonyms = new ArrayList<>();
+    String[] keys = option.key();
+    if (keys.length < 2)
+      return synonyms;
+    // if the primary key is empty (like sh and help command), then there should be no synonyms.
+    if ("".equals(keys[0]))
+      return synonyms;
+
+    for (int i = 1; i < keys.length; i++) {
+      synonyms.add(keys[i]);
+    }
+    return synonyms;
+  }
+
+  private static boolean isNullOrBlank(String value) {
+    return StringUtils.isBlank(value) || CliMetaData.ANNOTATION_NULL_VALUE.equals(value);
+  }
+
+  public Set<String> getCommands() {
+    return commands.keySet();
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/Topic.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/Topic.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/Topic.java
new file mode 100644
index 0000000..3ea896c
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/Topic.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.help;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class Topic implements Comparable<Topic> {
+  String topic;
+  String desc;
+  List<Command> relatedCommands = new ArrayList<>();
+
+  public Topic(String topic, String desc) {
+    this.topic = topic;
+    this.desc = desc;
+  }
+
+  public void addRelatedCommand(String command, String desc) {
+    relatedCommands.add(new Command(command, desc));
+  }
+
+  @Override
+  public int compareTo(Topic o) {
+    return topic.compareTo(o.topic);
+  }
+
+  public class Command implements Comparable<Command> {
+    String command;
+    String desc;
+
+    public Command(String command, String desc) {
+      this.command = command;
+      this.desc = desc;
+    }
+
+    @Override
+    public int compareTo(Command o) {
+      return this.command.compareTo(o.command);
+    }
+  }
+
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Block.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Block.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Block.java
deleted file mode 100644
index dde1a20..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Block.java
+++ /dev/null
@@ -1,42 +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.help.format;
-
-/**
- *
- */
-public class Block {
-  private String heading;
-  private Row[] rows;
-
-  public String getHeading() {
-    return heading;
-  }
-
-  public Block setHeading(String heading) {
-    this.heading = heading;
-    return this;
-  }
-
-  public Row[] getRows() {
-    return rows;
-  }
-
-  public Block setRows(Row[] rows) {
-    this.rows = rows;
-    return this;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/DataNode.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/DataNode.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/DataNode.java
deleted file mode 100644
index 8f5e570..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/DataNode.java
+++ /dev/null
@@ -1,48 +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.help.format;
-
-import java.util.List;
-
-/**
- *
- */
-public class DataNode {
-  String data;
-  List<DataNode> children;
-
-  public DataNode(String data, List<DataNode> dataNode) {
-    this.data = data;
-    this.children = dataNode;
-  }
-
-  public String getData() {
-    return data;
-  }
-
-  public List<DataNode> getChildren() {
-    return children;
-  }
-
-  public boolean addChild(DataNode dataNode) {
-    if (this.children != null) {
-      this.children.add(dataNode);
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Help.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Help.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Help.java
deleted file mode 100644
index 68c10bb..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Help.java
+++ /dev/null
@@ -1,44 +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.help.format;
-
-/**
- *
- */
-public class Help {
-  private Block[] blocks;
-
-  public Block[] getBlocks() {
-    return blocks;
-  }
-
-  public Help setBlocks(Block[] block) {
-    this.blocks = block;
-    return this;
-  }
-
-  @Override
-  public String toString() {
-    StringBuffer buffer = new StringBuffer();
-    for (Block block : blocks) {
-      buffer.append(block.getHeading() + "\n");
-      for (Row row : block.getRows()) {
-        buffer.append("\t" + row.getInfo()[0] + "\n");
-      }
-      buffer.append("\n");
-    }
-    return buffer.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/NewHelp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/NewHelp.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/NewHelp.java
deleted file mode 100644
index 90f9eda..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/NewHelp.java
+++ /dev/null
@@ -1,52 +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.help.format;
-
-import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-
-/**
- * 
- * @since GemFire 7.0
- */
-public class NewHelp {
-  DataNode root;
-
-  public NewHelp(DataNode root) {
-    this.root = root;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    append(builder, root, 0);
-    return builder.toString();
-  }
-
-  private void append(StringBuilder builder, DataNode dataNode, int level) {
-    if (dataNode != null && builder != null) {
-      String data = dataNode.getData();
-      if (data != null && !data.equals("")) {
-        builder.append(Gfsh.wrapText(data, level - 1));
-        builder.append(GfshParser.LINE_SEPARATOR);
-      }
-      if (dataNode.getChildren() != null && dataNode.getChildren().size() > 0) {
-        for (DataNode child : dataNode.getChildren()) {
-          append(builder, child, level + 1);
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Row.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Row.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Row.java
deleted file mode 100644
index ac1ca21..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/help/format/Row.java
+++ /dev/null
@@ -1,28 +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.help.format;
-
-public class Row {
-  private String[] info;
-
-  public String[] getInfo() {
-    return info;
-  }
-
-  public Row setInfo(String[] info) {
-    this.info = info;
-    return this;
-  }
-}


[2/9] geode git commit: GEODE-1597: use Spring shell's parser and delete our own parsing code

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/JoptOptionParserTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/JoptOptionParserTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/JoptOptionParserTest.java
deleted file mode 100644
index 9815a9c..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/JoptOptionParserTest.java
+++ /dev/null
@@ -1,527 +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;
-
-import static org.assertj.core.api.Assertions.*;
-import static org.mockito.Mockito.*;
-
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-
-import org.apache.geode.management.internal.cli.exceptions.CliCommandOptionException;
-import org.apache.geode.management.internal.cli.exceptions.CliCommandOptionMissingException;
-import org.apache.geode.management.internal.cli.exceptions.CliCommandOptionNotApplicableException;
-import org.apache.geode.management.internal.cli.parser.Argument;
-import org.apache.geode.management.internal.cli.parser.Option;
-import org.apache.geode.management.internal.cli.parser.OptionSet;
-import org.apache.geode.management.internal.cli.parser.jopt.JoptOptionParser;
-import org.apache.geode.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-@RunWith(JUnitParamsRunner.class)
-public class JoptOptionParserTest {
-
-  private JoptOptionParser emptyOptionParser;
-  private OptionSet emptyOptionSet;
-
-  private Argument requiredArgument;
-  private Argument optionalArgument;
-
-  private Option requiredOption;
-  private Option optionalOption;
-
-  private JoptOptionParser simpleOptionParser;
-  private JoptOptionParser exampleOptionParser;
-
-  @Before
-  public void setUp() throws Exception {
-    this.emptyOptionParser = new JoptOptionParser();
-    this.emptyOptionSet = new OptionSet();
-    defineSimpleOptionParser();
-    defineExampleOptionParser();
-  }
-
-  @Test
-  public void getArgumentsIsEmptyByDefault() throws Exception {
-    assertThat(this.emptyOptionParser.getArguments()).isEmpty();
-  }
-
-  @Test
-  public void getOptionsIsNullByDefault() throws Exception {
-    assertThat(this.emptyOptionParser.getOptions()).isNull();
-  }
-
-  @Test
-  public void parseNullReturnsDefaultOptionSet() throws Exception {
-    OptionSet optionSet = this.emptyOptionParser.parse(null);
-    assertThat(optionSet.areArgumentsPresent()).isEqualTo(emptyOptionSet.areArgumentsPresent());
-    assertThat(optionSet.areOptionsPresent()).isEqualTo(emptyOptionSet.areOptionsPresent());
-    assertThat(optionSet.getNoOfSpacesRemoved()).isEqualTo(emptyOptionSet.getNoOfSpacesRemoved());
-    assertThat(optionSet.getSplit()).isEqualTo(emptyOptionSet.getSplit());
-    assertThat(optionSet.getNoOfSpacesRemoved()).isEqualTo(emptyOptionSet.getNoOfSpacesRemoved());
-    assertThat(optionSet.getUserInput()).isEqualTo(""); // emptyOptionSet.getUserInput());
-    assertThat(optionSet.getValue((Argument) null))
-        .isEqualTo(emptyOptionSet.getValue((Argument) null));
-    assertThat(optionSet.getValue((Option) null)).isEqualTo(emptyOptionSet.getValue((Option) null));
-  }
-
-  @Test
-  public void parseEmptyThrowsNullPointerException() throws Exception {
-    assertThatThrownBy(() -> this.emptyOptionParser.parse(""))
-        .isInstanceOf(NullPointerException.class);
-  }
-
-  @Test
-  public void setArgumentsShouldCreateCopy() throws Exception {
-    Argument argument = mock(Argument.class);
-    when(argument.isRequired()).thenReturn(true);
-
-    LinkedList<Argument> arguments = new LinkedList<>();
-    arguments.add(argument);
-
-    this.emptyOptionParser.setArguments(arguments);
-
-    assertThat(this.emptyOptionParser.getArguments()).isNotSameAs(arguments);
-    assertThat(this.emptyOptionParser.getArguments()).hasSize(1);
-
-    arguments.clear();
-
-    assertThat(arguments).hasSize(0);
-    assertThat(this.emptyOptionParser.getArguments()).hasSize(1);
-  }
-
-  @Test
-  public void setArgumentsShouldKeepRequiredBeforeOptional() throws Exception {
-    Argument requiredArgument1 = mock(Argument.class);
-    when(requiredArgument1.isRequired()).thenReturn(true);
-    Argument optionalArgument1 = mock(Argument.class);
-    when(optionalArgument1.isRequired()).thenReturn(false);
-
-    LinkedList<Argument> arguments = new LinkedList<>();
-    arguments.add(requiredArgument1);
-    arguments.add(optionalArgument1);
-
-    this.emptyOptionParser.setArguments(arguments);
-
-    LinkedList<Argument> argumentsReturned = this.emptyOptionParser.getArguments();
-
-    assertThat(argumentsReturned).hasSize(2);
-    assertThat(argumentsReturned.getFirst()).isSameAs(requiredArgument1);
-    assertThat(argumentsReturned.getLast()).isSameAs(optionalArgument1);
-  }
-
-  @Test
-  public void setArgumentsShouldMoveRequiredBeforeOptional() throws Exception {
-    Argument requiredArgument1 = mock(Argument.class);
-    when(requiredArgument1.isRequired()).thenReturn(true);
-    Argument optionalArgument1 = mock(Argument.class);
-    when(optionalArgument1.isRequired()).thenReturn(false);
-
-    LinkedList<Argument> arguments = new LinkedList<>();
-    arguments.add(optionalArgument1);
-    arguments.add(requiredArgument1);
-
-    this.emptyOptionParser.setArguments(arguments);
-
-    LinkedList<Argument> argumentsReturned = this.emptyOptionParser.getArguments();
-
-    assertThat(argumentsReturned).hasSize(2);
-    assertThat(argumentsReturned.getFirst()).isSameAs(requiredArgument1);
-    assertThat(argumentsReturned.getLast()).isSameAs(optionalArgument1);
-  }
-
-  @Test
-  public void setOptionsShouldKeepSameInstance() throws Exception {
-    Option option = mock(Option.class);
-    ArrayList aggregate = new ArrayList<String>();
-    aggregate.add("option");
-    when(option.getAggregate()).thenReturn(aggregate);
-    when(option.getHelp()).thenReturn("help text");
-
-    LinkedList<Option> options = new LinkedList<>();
-    options.add(option);
-
-    this.emptyOptionParser.setOptions(options);
-
-    assertThat(this.emptyOptionParser.getOptions()).isSameAs(options);
-    assertThat(this.emptyOptionParser.getOptions()).hasSize(1);
-
-    options.clear();
-
-    assertThat(options).hasSize(0);
-    assertThat(this.emptyOptionParser.getOptions()).hasSize(0);
-  }
-
-  @Test
-  public void parseInputWithDefinedArgumentShouldWork() throws Exception {
-    LinkedList<Argument> arguments = new LinkedList<>();
-    LinkedList<Option> options = new LinkedList<>();
-
-    arguments.add(this.requiredArgument);
-
-    JoptOptionParser optionParser = new JoptOptionParser();
-    optionParser.setArguments(arguments);
-    optionParser.setOptions(options);
-
-    OptionSet optionSet = optionParser.parse("command1 argument1_value");
-    assertThat(optionSet.areArgumentsPresent()).isTrue();
-    assertThat(optionSet.hasArgument(this.requiredArgument)).isTrue();
-  }
-
-  @Test
-  public void parseInputWithOneArgumentShouldFindJustOneArgument() throws Exception {
-    LinkedList<Argument> arguments = new LinkedList<>();
-    LinkedList<Option> options = new LinkedList<>();
-
-    arguments.add(this.requiredArgument);
-
-    JoptOptionParser optionParser = new JoptOptionParser();
-    optionParser.setArguments(arguments);
-    optionParser.setOptions(options);
-
-    OptionSet optionSet = optionParser.parse("command1 argument1_value");
-    assertThat(optionSet.areArgumentsPresent()).isTrue();
-    assertThat(optionSet.hasArgument(this.requiredArgument)).isTrue();
-    assertThat(optionSet.hasArgument(this.optionalArgument)).isFalse();
-  }
-
-  @Test
-  public void parseInputWithTwoArgumentsShouldFindTwoArguments() throws Exception {
-    LinkedList<Argument> arguments = new LinkedList<>();
-    LinkedList<Option> options = new LinkedList<>();
-
-    arguments.add(this.requiredArgument);
-    arguments.add(this.optionalArgument);
-
-    JoptOptionParser optionParser = new JoptOptionParser();
-    optionParser.setArguments(arguments);
-    optionParser.setOptions(options);
-
-    OptionSet optionSet = optionParser.parse("command1 argument1_value? argument2_value");
-    assertThat(optionSet.areArgumentsPresent()).isTrue();
-    assertThat(optionSet.hasArgument(this.requiredArgument)).isTrue();
-    assertThat(optionSet.hasArgument(this.optionalArgument)).isTrue();
-  }
-
-  @Test
-  public void parseInputWithUndefinedArgumentShouldNotThrow() throws Exception {
-    LinkedList<Argument> arguments = new LinkedList<>();
-    LinkedList<Option> options = new LinkedList<>();
-
-    arguments.add(this.requiredArgument);
-
-    JoptOptionParser optionParser = new JoptOptionParser();
-    optionParser.setArguments(arguments);
-    optionParser.setOptions(options);
-
-    OptionSet optionSet = optionParser.parse("command1 argument1_value? argument2_value");
-    assertThat(optionSet.getUserInput()).isEqualTo("command1 argument1_value? argument2_value");
-  }
-
-  @Test
-  public void parseInputShouldIgnoreUndefinedOption() throws Exception {
-    // one fix for GEODE-1598 has a side effect of preventing our detection of undefined options
-    OptionSet optionSet =
-        this.simpleOptionParser.parse("command1 argument1_value argument2_value --undefinedOption");
-    assertThat(optionSet.areOptionsPresent()).isFalse();
-    assertThat(optionSet.hasOption(this.requiredOption)).isFalse();
-    assertThat(optionSet.hasOption(this.optionalOption)).isFalse();
-  }
-
-  @Test
-  public void parseInputWithOneOptionShouldFindOneOption() throws Exception {
-    OptionSet optionSet = this.simpleOptionParser.parse("command1 argument1_value --option1");
-    assertThat(optionSet.areOptionsPresent()).isTrue();
-    assertThat(optionSet.hasOption(this.requiredOption)).isTrue();
-    assertThat(optionSet.hasOption(this.optionalOption)).isFalse();
-  }
-
-  @Test
-  public void parseInputWithTwoOptionsShouldFindTwoOptions() throws Exception {
-    OptionSet optionSet =
-        this.simpleOptionParser.parse("command1 argument1_value --option1 --option2");
-    assertThat(optionSet.areOptionsPresent()).isTrue();
-    assertThat(optionSet.hasOption(this.requiredOption)).isTrue();
-    assertThat(optionSet.hasOption(this.optionalOption)).isTrue();
-  }
-
-  @Test
-  public void parseInputWithOptionWithValueShouldFindOption() throws Exception {
-    OptionSet optionSet = this.simpleOptionParser.parse("command1 argument1_value --option1=value");
-    assertThat(optionSet.areOptionsPresent()).isTrue();
-    assertThat(optionSet.hasOption(this.requiredOption)).isTrue();
-  }
-
-  @Test
-  public void parseInputWithOptionWithoutValueShouldFindOption() throws Exception {
-    OptionSet optionSet = this.simpleOptionParser.parse("command1 argument1_value --option1");
-    assertThat(optionSet.areOptionsPresent()).isTrue();
-    assertThat(optionSet.hasOption(this.requiredOption)).isTrue();
-  }
-
-  @Test
-  public void parseInputWithoutOptionShouldNotFindOptions() throws Exception {
-    LinkedList<Argument> arguments = new LinkedList<>();
-    LinkedList<Option> options = new LinkedList<>();
-
-    arguments.add(this.requiredArgument);
-
-    JoptOptionParser optionParser = new JoptOptionParser();
-    optionParser.setArguments(arguments);
-    optionParser.setOptions(options);
-
-    OptionSet optionSet = optionParser.parse("command1 argument1_value");
-    assertThat(optionSet.areOptionsPresent()).isFalse();
-    assertThat(optionSet.hasOption(this.requiredOption)).isFalse();
-  }
-
-  @Test
-  @Parameters(method = "exampleInputParameters")
-  public void parseInputWithExampleInputParametesr(String command, boolean expectException,
-      boolean hasArguments, boolean hasOptions) throws Exception {
-    if (expectException) {
-      assertThatThrownBy(() -> this.exampleOptionParser.parse(command))
-          .isExactlyInstanceOf(CliCommandOptionMissingException.class);
-      return;
-    }
-
-    OptionSet options = this.exampleOptionParser.parse(command);
-    assertThat(options).isNotNull();
-    assertThat(options.areArgumentsPresent()).isEqualTo(hasArguments);
-    assertThat(options.areOptionsPresent()).isEqualTo(hasOptions);
-  }
-
-  private static Object[] exampleInputParameters() {
-    return new Object[] {
-        // 0
-        new Object[] {" ARGUMENT1_VALUE \u2014option1=somevalue", false, true, false},
-        // 1
-        new Object[] {" ARGUMENT1_VALUE?      ARGUMENT2_VALUE -- ----------", false, true, false},
-        // 2
-        new Object[] {" --option1=value", false, false, true},
-        // 3
-        new Object[] {
-            "         ARGUMENT1_VALUE?       ARGUMENT2_VALUE --option1=option1value --option2",
-            false, true, true},
-        // 4
-        new Object[] {
-            "         ARGUMENT1_VALUE?       ARGUMENT2_VALUE --option1=option1value --option2=option2value --option3=option3value",
-            false, true, true},
-        // 5
-        new Object[] {
-            " --string=string1 --stringArray=1,2 --stringArray=3,4 --stringList=11,12,13 --integer=10 --stringArray=5 --stringList=14,15",
-            false, false, true},
-        // 6
-        new Object[] {" --stringArray=1,2 --stringArray='3,4'", false, false, true},
-        // 7
-        new Object[] {
-            " --string=\"1\" --colonArray=2:3:4 --stringArray=5,\"6,7\",8 --stringList=\"9,10,11,12\"",
-            false, false, true},
-        // 8
-        new Object[] {" --string=string1 --stringArray=1,2 --string=string2", false, false, true},
-        // 9
-        new Object[] {" this is just one argument?this is a second argument", false, true, false}};
-  }
-
-  private void defineSimpleOptionParser() {
-    LinkedList<Argument> arguments = new LinkedList<Argument>();
-    LinkedList<Option> options = new LinkedList<Option>();
-
-    this.requiredArgument = mock(Argument.class);
-    when(this.requiredArgument.getArgumentName()).thenReturn("argument1");
-    when(this.requiredArgument.getContext()).thenReturn("context for argument1");
-    when(this.requiredArgument.getHelp()).thenReturn("help for argument1");
-    when(this.requiredArgument.isRequired()).thenReturn(true);
-    arguments.add(this.requiredArgument);
-
-    this.optionalArgument = mock(Argument.class);
-    when(this.optionalArgument.getArgumentName()).thenReturn("argument2");
-    when(this.optionalArgument.getContext()).thenReturn("context for argument2");
-    when(this.optionalArgument.getHelp()).thenReturn("help for argument2");
-    when(this.optionalArgument.isRequired()).thenReturn(false);
-    when(this.optionalArgument.getUnspecifiedDefaultValue())
-        .thenReturn("{unspecified default value for argument2}");
-    when(this.optionalArgument.isSystemProvided()).thenReturn(false);
-    arguments.add(this.optionalArgument);
-
-    this.requiredOption = mock(Option.class);
-    when(this.requiredOption.getLongOption()).thenReturn("--option1");
-    List<String> aggregate = new ArrayList<>();
-    aggregate.add("option1");
-    when(this.requiredOption.getAggregate()).thenReturn(aggregate);
-    when(this.requiredOption.getLongOption()).thenReturn("option1");
-    when(this.requiredOption.getHelp()).thenReturn("help for option1");
-    when(this.requiredOption.getValueSeparator()).thenReturn("=");
-    when(this.requiredOption.isRequired()).thenReturn(true);
-    assertThat(this.requiredOption.getAggregate()).isNotEmpty();
-    options.add(this.requiredOption);
-
-    this.optionalOption = mock(Option.class);
-    when(this.optionalOption.getLongOption()).thenReturn("--option2");
-    aggregate = new ArrayList<>();
-    aggregate.add("option2");
-    when(this.optionalOption.getAggregate()).thenReturn(aggregate);
-    when(this.optionalOption.getLongOption()).thenReturn("option2");
-    when(this.optionalOption.getHelp()).thenReturn("help for option2");
-    when(this.optionalOption.getValueSeparator()).thenReturn("=");
-    when(this.optionalOption.isRequired()).thenReturn(false);
-    assertThat(this.optionalOption.getAggregate()).isNotEmpty();
-    options.add(this.optionalOption);
-
-    this.simpleOptionParser = new JoptOptionParser();
-    this.simpleOptionParser.setArguments(arguments);
-    this.simpleOptionParser.setOptions(options);
-  }
-
-  private void defineExampleOptionParser() {
-    LinkedList<Argument> arguments = new LinkedList<Argument>();
-    LinkedList<Option> options = new LinkedList<Option>();
-
-    Argument argument1 = mock(Argument.class);
-    when(argument1.getArgumentName()).thenReturn("argument1");
-    when(argument1.getContext()).thenReturn("context for argument1");
-    when(argument1.getHelp()).thenReturn("help for argument1");
-    when(argument1.isRequired()).thenReturn(true);
-    arguments.add(argument1);
-
-    Argument argument2 = mock(Argument.class);
-    when(argument2.getArgumentName()).thenReturn("argument2");
-    when(argument2.getContext()).thenReturn("context for argument2");
-    when(argument2.getHelp()).thenReturn("help for argument2");
-    when(argument2.isRequired()).thenReturn(false);
-    when(argument2.getUnspecifiedDefaultValue())
-        .thenReturn("{unspecified default value for argument2}");
-    when(argument2.isSystemProvided()).thenReturn(false);
-    arguments.add(argument2);
-
-    Argument argument3 = mock(Argument.class);
-    when(argument3.getArgumentName()).thenReturn("argument3");
-    when(argument3.getContext()).thenReturn("context for argument3");
-    when(argument3.getHelp()).thenReturn("help for argument3");
-    when(argument3.isRequired()).thenReturn(false);
-    when(argument3.getUnspecifiedDefaultValue())
-        .thenReturn("{unspecified default value for argument3}");
-    when(argument2.isSystemProvided()).thenReturn(false);
-    arguments.add(argument3);
-
-    Option option1 = mock(Option.class);
-    when(option1.getLongOption()).thenReturn("--option1");
-    List<String> aggregate1 = new ArrayList<>();
-    aggregate1.add("option1");
-    when(option1.getAggregate()).thenReturn(aggregate1);
-    when(option1.getLongOption()).thenReturn("option1");
-    when(option1.getHelp()).thenReturn("help for option1");
-    when(option1.getValueSeparator()).thenReturn("=");
-    when(option1.isRequired()).thenReturn(false);
-    assertThat(option1.getAggregate()).isNotEmpty();
-    options.add(option1);
-
-    Option option2 = mock(Option.class);
-    when(option2.getLongOption()).thenReturn("--option2");
-    List<String> aggregate2 = new ArrayList<>();
-    aggregate2.add("option2");
-    when(option2.getAggregate()).thenReturn(aggregate2);
-    when(option2.getLongOption()).thenReturn("option2");
-    when(option2.getHelp()).thenReturn("help for option2");
-    when(option2.getValueSeparator()).thenReturn("=");
-    when(option2.isRequired()).thenReturn(false);
-    assertThat(option2.getAggregate()).isNotEmpty();
-    options.add(option2);
-
-    Option option3 = mock(Option.class);
-    when(option3.getLongOption()).thenReturn("--option3");
-    List<String> aggregate3 = new ArrayList<>();
-    aggregate3.add("option3");
-    when(option3.getAggregate()).thenReturn(aggregate3);
-    when(option3.getLongOption()).thenReturn("option3");
-    when(option3.getHelp()).thenReturn("help for option3");
-    when(option3.getValueSeparator()).thenReturn("=");
-    when(option3.isRequired()).thenReturn(false);
-    assertThat(option3.getAggregate()).isNotEmpty();
-    options.add(option3);
-
-    Option stringOption = mock(Option.class);
-    when(stringOption.getLongOption()).thenReturn("--string");
-    List<String> aggregateStringOption = new ArrayList<>();
-    aggregateStringOption.add("string");
-    when(stringOption.getAggregate()).thenReturn(aggregateStringOption);
-    when(stringOption.getLongOption()).thenReturn("string");
-    when(stringOption.getHelp()).thenReturn("help for string");
-    when(stringOption.getValueSeparator()).thenReturn("=");
-    when(stringOption.isRequired()).thenReturn(false);
-    assertThat(stringOption.getAggregate()).isNotEmpty();
-    options.add(stringOption);
-
-    Option stringArrayOption = mock(Option.class);
-    when(stringArrayOption.getLongOption()).thenReturn("--stringArray");
-    List<String> aggregateStringArrayOption = new ArrayList<>();
-    aggregateStringArrayOption.add("stringArray");
-    when(stringArrayOption.getAggregate()).thenReturn(aggregateStringArrayOption);
-    when(stringArrayOption.getLongOption()).thenReturn("stringArray");
-    when(stringArrayOption.getHelp()).thenReturn("help for stringArray");
-    when(stringArrayOption.getValueSeparator()).thenReturn("=");
-    when(stringArrayOption.isRequired()).thenReturn(false);
-    assertThat(stringArrayOption.getAggregate()).isNotEmpty();
-    options.add(stringArrayOption);
-
-    Option stringListOption = mock(Option.class);
-    when(stringListOption.getLongOption()).thenReturn("--stringList");
-    List<String> aggregateStringListOption = new ArrayList<>();
-    aggregateStringListOption.add("stringList");
-    when(stringListOption.getAggregate()).thenReturn(aggregateStringListOption);
-    when(stringListOption.getLongOption()).thenReturn("stringList");
-    when(stringListOption.getHelp()).thenReturn("help for stringList");
-    when(stringListOption.getValueSeparator()).thenReturn("=");
-    when(stringListOption.isRequired()).thenReturn(false);
-    assertThat(stringListOption.getAggregate()).isNotEmpty();
-    options.add(stringListOption);
-
-    Option integerOption = mock(Option.class);
-    when(integerOption.getLongOption()).thenReturn("--integer");
-    List<String> aggregateIntegerOption = new ArrayList<>();
-    aggregateIntegerOption.add("integer");
-    when(integerOption.getAggregate()).thenReturn(aggregateIntegerOption);
-    when(integerOption.getLongOption()).thenReturn("integer");
-    when(integerOption.getHelp()).thenReturn("help for integer");
-    when(integerOption.getValueSeparator()).thenReturn("=");
-    when(integerOption.isRequired()).thenReturn(false);
-    assertThat(integerOption.getAggregate()).isNotEmpty();
-    options.add(integerOption);
-
-    Option colonArrayOption = mock(Option.class);
-    when(colonArrayOption.getLongOption()).thenReturn("--colonArray");
-    List<String> aggregateColonArrayOption = new ArrayList<>();
-    aggregateColonArrayOption.add("colonArray");
-    when(colonArrayOption.getAggregate()).thenReturn(aggregateColonArrayOption);
-    when(colonArrayOption.getLongOption()).thenReturn("colonArray");
-    when(colonArrayOption.getHelp()).thenReturn("help for colonArray");
-    when(colonArrayOption.getValueSeparator()).thenReturn("=");
-    when(colonArrayOption.isRequired()).thenReturn(false);
-    assertThat(colonArrayOption.getAggregate()).isNotEmpty();
-    options.add(colonArrayOption);
-
-    this.exampleOptionParser = new JoptOptionParser();
-    this.exampleOptionParser.setArguments(arguments);
-    this.exampleOptionParser.setOptions(options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/annotations/CliArgumentJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/annotations/CliArgumentJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/annotations/CliArgumentJUnitTest.java
deleted file mode 100644
index 161f7c6..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/annotations/CliArgumentJUnitTest.java
+++ /dev/null
@@ -1,154 +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.annotations;
-
-import static org.junit.Assert.*;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.management.internal.cli.annotation.CliArgument;
-import org.apache.geode.test.junit.categories.UnitTest;
-
-/**
- * Includes tests for checking assignment of {@link CliArgument}
- */
-@Category(UnitTest.class)
-public class CliArgumentJUnitTest {
-
-  private static final String ARGUMENT_NAME = "ARGUMENT_NAME";
-  private static final String ARGUMENT_HELP = "ARGUMENT_HELP";
-  private static final boolean ARGUMENT_MANDATORY = true;
-  private static final String ARGUMENT_CONTEXT = "ARGUMENT_CONTEXT";
-  private static final boolean SYSTEM_PROVIDED = true;
-  private static final String ARGUMENT_UNSPECIFIED_DEFAULT_VALUE =
-      "ARGUMENT_UNSPECIFIED_DEFAULT_VALUE";
-  private static final String MESSAGE_FOR_DEFAULT_ARGUMENT = "Testing for argument with defaults";
-  private static final String MESSAGE_FOR_ARGUMENT = "Testing for argument without defaults";
-
-  /**
-   * Test for {@link CliArgument#name()}
-   */
-  @Test
-  public void testName() throws Exception {
-    String name = ((CliArgument) (ArgumentTestingClass.class
-        .getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-            .name();
-    assertNotNull(name);
-    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, name, ARGUMENT_NAME);
-    name = ((CliArgument) (ArgumentTestingClass.class
-        .getMethod("argumentTestingMethod", String.class).getParameterAnnotations()[0][0])).name();
-    assertNotNull(name);
-    assertEquals(MESSAGE_FOR_ARGUMENT, name, ARGUMENT_NAME);
-  }
-
-  /**
-   * Test for {@link CliArgument#help()}
-   */
-  @Test
-  public void testHelp() throws Exception {
-    String help = ((CliArgument) (ArgumentTestingClass.class
-        .getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-            .help();
-    assertNotNull(help);
-    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, help, "");
-    help = ((CliArgument) (ArgumentTestingClass.class
-        .getMethod("argumentTestingMethod", String.class).getParameterAnnotations()[0][0])).help();
-    assertNotNull(help);
-    assertEquals(MESSAGE_FOR_ARGUMENT, help, ARGUMENT_HELP);
-  }
-
-  /**
-   * Test for {@link CliArgument#mandatory()}
-   */
-  @Test
-  public void testMandatory() throws Exception {
-    boolean mandatory = ((CliArgument) (ArgumentTestingClass.class
-        .getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-            .mandatory();
-    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, mandatory, false);
-    mandatory =
-        ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class)
-            .getParameterAnnotations()[0][0])).mandatory();
-    assertEquals(MESSAGE_FOR_ARGUMENT, mandatory, ARGUMENT_MANDATORY);
-  }
-
-  /**
-   * Test for {@link CliArgument#argumentContext()}
-   */
-  @Test
-  public void testArgumentContext() throws Exception {
-    String argumentContext = ((CliArgument) (ArgumentTestingClass.class
-        .getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-            .argumentContext();
-    assertNotNull(argumentContext);
-    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, argumentContext, "");
-    argumentContext =
-        ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class)
-            .getParameterAnnotations()[0][0])).argumentContext();
-    assertNotNull(argumentContext);
-    assertEquals(MESSAGE_FOR_ARGUMENT, argumentContext, ARGUMENT_CONTEXT);
-  }
-
-  /**
-   * Test for {@link CliArgument#systemProvided()}
-   */
-  @Test
-  public void testSystemProvided() throws Exception {
-    boolean systemProvided = ((CliArgument) (ArgumentTestingClass.class
-        .getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-            .systemProvided();
-    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, systemProvided, false);
-    systemProvided =
-        ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class)
-            .getParameterAnnotations()[0][0])).systemProvided();
-    assertEquals(MESSAGE_FOR_ARGUMENT, systemProvided, SYSTEM_PROVIDED);
-  }
-
-  /**
-   * Test for {@link CliArgument#unspecifiedDefaultValue()}
-   */
-  @Test
-  public void testUnspecifiedDefaultValue() throws Exception {
-    String unspecifiedDefaultValue = ((CliArgument) (ArgumentTestingClass.class
-        .getMethod("defaultArgumentTestingMethod", String.class).getParameterAnnotations()[0][0]))
-            .unspecifiedDefaultValue();
-    assertEquals(MESSAGE_FOR_DEFAULT_ARGUMENT, unspecifiedDefaultValue, "__NULL__");
-    unspecifiedDefaultValue =
-        ((CliArgument) (ArgumentTestingClass.class.getMethod("argumentTestingMethod", String.class)
-            .getParameterAnnotations()[0][0])).unspecifiedDefaultValue();
-    assertEquals(MESSAGE_FOR_ARGUMENT, unspecifiedDefaultValue, ARGUMENT_UNSPECIFIED_DEFAULT_VALUE);
-  }
-
-  /**
-   * Class used by the tests
-   */
-  private static class ArgumentTestingClass {
-
-    @SuppressWarnings("unused")
-    public static Object defaultArgumentTestingMethod(
-        @CliArgument(name = ARGUMENT_NAME) String defaultArgument) {
-      return null;
-    }
-
-    @SuppressWarnings("unused")
-    public static Object argumentTestingMethod(
-        @CliArgument(name = ARGUMENT_NAME, help = ARGUMENT_HELP, mandatory = ARGUMENT_MANDATORY,
-            argumentContext = ARGUMENT_CONTEXT, systemProvided = SYSTEM_PROVIDED,
-            unspecifiedDefaultValue = ARGUMENT_UNSPECIFIED_DEFAULT_VALUE) String argument) {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CliCommandTestBase.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CliCommandTestBase.java
index 165f664..afdb706 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CliCommandTestBase.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CliCommandTestBase.java
@@ -32,7 +32,6 @@ import org.apache.geode.management.ManagementService;
 import org.apache.geode.management.internal.cli.CommandManager;
 import org.apache.geode.management.internal.cli.HeadlessGfsh;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
 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;
@@ -43,42 +42,61 @@ import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
 import org.apache.geode.test.dunit.rules.DistributedRestoreSystemProperties;
 import org.junit.Rule;
 import org.junit.rules.TemporaryFolder;
+import org.springframework.shell.core.CommandMarker;
 
 import java.io.IOException;
 import java.io.PrintStream;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Map;
+import java.util.List;
 import java.util.Properties;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 /**
  * Base class for all the CLI/gfsh command dunit tests.
- * 
+ *
  * @deprecated use LocatorServerStartupRule and GfshShellConnectorRule instead.
  */
 public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
 
   public static final String USE_HTTP_SYSTEM_PROPERTY = "useHTTP";
-
-  private boolean useHttpOnConnect = Boolean.getBoolean(USE_HTTP_SYSTEM_PROPERTY);
-
-  private ManagementService managementService;
-
-  private transient HeadlessGfsh shell;
-
+  @Rule
+  public transient DistributedRestoreSystemProperties restoreSystemProperties =
+      new DistributedRestoreSystemProperties();
+  @Rule
+  public transient TemporaryFolder temporaryFolder = new TemporaryFolder();
   protected transient int httpPort;
   protected transient int jmxPort;
   protected transient String jmxHost;
   protected transient String gfshDir;
+  private boolean useHttpOnConnect = Boolean.getBoolean(USE_HTTP_SYSTEM_PROPERTY);
+  private ManagementService managementService;
+  private transient HeadlessGfsh shell;
 
-  @Rule
-  public transient DistributedRestoreSystemProperties restoreSystemProperties =
-      new DistributedRestoreSystemProperties();
+  public static boolean checkIfCommandsAreLoadedOrNot() {
+    CommandManager manager = new CommandManager();
+    List<CommandMarker> commands = manager.getCommandMarkers();
+    return commands.size() >= 1;
 
-  @Rule
-  public transient TemporaryFolder temporaryFolder = new TemporaryFolder();
+  }
+
+  protected static String commandResultToString(final CommandResult commandResult) {
+    assertNotNull(commandResult);
+
+    commandResult.resetToFirstLine();
+
+    StringBuilder buffer = new StringBuilder(commandResult.getHeader());
+
+    while (commandResult.hasNextLine()) {
+      buffer.append(commandResult.nextLine());
+    }
+
+    buffer.append(commandResult.getFooter());
+
+    return buffer.toString();
+  }
 
   @Override
   public final void postSetUp() throws Exception {
@@ -204,20 +222,6 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
     assertTrue(checkIfCommandsAreLoadedOrNot());
   }
 
-  public static boolean checkIfCommandsAreLoadedOrNot() {
-    CommandManager manager;
-    try {
-      manager = CommandManager.getInstance();
-      Map<String, CommandTarget> commands = manager.getCommands();
-      if (commands.size() < 1) {
-        return false;
-      }
-      return true;
-    } catch (ClassNotFoundException | IOException e) {
-      throw new RuntimeException("Could not load commands", e);
-    }
-  }
-
   /**
    * Stop the default management service.
    */
@@ -353,15 +357,6 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
   protected CommandResult executeCommandWithoutClear(HeadlessGfsh shell, String command) {
     assert (shell != null);
     assert (command != null);
-
-    try {
-      info("Executing command " + command + " with command Mgr " + CommandManager.getInstance());
-    } catch (ClassNotFoundException cnfex) {
-      throw new AssertionError(cnfex);
-    } catch (IOException ioex) {
-      throw new AssertionError(ioex);
-    }
-
     shell.executeCommand(command);
     if (shell.hasError()) {
       error("executeCommand completed with error : " + shell.getError());
@@ -396,22 +391,6 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
     printStream.print(commandResultToString(commandResult));
   }
 
-  protected static String commandResultToString(final CommandResult commandResult) {
-    assertNotNull(commandResult);
-
-    commandResult.resetToFirstLine();
-
-    StringBuilder buffer = new StringBuilder(commandResult.getHeader());
-
-    while (commandResult.hasNextLine()) {
-      buffer.append(commandResult.nextLine());
-    }
-
-    buffer.append(commandResult.getFooter());
-
-    return buffer.toString();
-  }
-
   /**
    * Utility method for finding the CommandResult object in the Map of CommandOutput objects.
    *

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
index 5f885e1..791e02b 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
@@ -906,11 +906,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     CommandStringBuilder commandStringBuilder = new CommandStringBuilder(CliStrings.ALTER_REGION);
     commandStringBuilder.addOption(CliStrings.ALTER_REGION__REGION, "/" + this.alterRegionName);
     commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER,
-        "com.cadrdunit.RegionAlterCacheListenerA");
-    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER,
-        "com.cadrdunit.RegionAlterCacheListenerB");
-    commandStringBuilder.addOption(CliStrings.ALTER_REGION__CACHELISTENER,
-        "com.cadrdunit.RegionAlterCacheListenerC");
+        "com.cadrdunit.RegionAlterCacheListenerA,com.cadrdunit.RegionAlterCacheListenerB,com.cadrdunit.RegionAlterCacheListenerC");
 
     cmdResult = executeCommand(commandStringBuilder.toString());
     assertEquals(Result.Status.OK, cmdResult.getStatus());

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java
index 9ed5bed..3a88faa 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DeployCommandsDUnitTest.java
@@ -18,16 +18,13 @@ import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
 import static org.apache.geode.test.dunit.Host.getHost;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.junit.Assert.assertNotNull;
 
 import org.apache.geode.internal.ClassBuilder;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
-import org.apache.geode.test.dunit.rules.Locator;
 import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
-import org.apache.geode.test.dunit.rules.Server;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.junit.Before;
 import org.junit.Rule;
@@ -213,7 +210,7 @@ public class DeployCommandsDUnitTest implements Serializable {
     });
 
     gfshConnector
-        .executeAndVerifyCommand("undeploy --jars=" + jar3.getName() + "," + jar4.getName());
+        .executeAndVerifyCommand("undeploy --jar=" + jar3.getName() + "," + jar4.getName());
     server1.invoke(() -> {
       assertThatCannotLoad(jarName3, class3);
       assertThatCannotLoad(jarName4, class4);
@@ -239,10 +236,10 @@ public class DeployCommandsDUnitTest implements Serializable {
   @Test
   public void testListDeployed() throws Exception {
     // Deploy a couple of JAR files which can be listed
-    gfshConnector.executeAndVerifyCommand(
-        "deploy jar --group=" + GROUP1 + " --jar=" + jar1.getCanonicalPath());
-    gfshConnector.executeAndVerifyCommand(
-        "deploy jar --group=" + GROUP2 + " --jar=" + jar2.getCanonicalPath());
+    gfshConnector
+        .executeAndVerifyCommand("deploy --group=" + GROUP1 + " --jar=" + jar1.getCanonicalPath());
+    gfshConnector
+        .executeAndVerifyCommand("deploy --group=" + GROUP2 + " --jar=" + jar2.getCanonicalPath());
 
     // List for all members
     gfshConnector.executeAndVerifyCommand("list deployed");

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
index 5b07f28..e9c61c5 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
@@ -1190,9 +1190,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
     commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__TIME_INTERVAL, "2023");
     commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__WRITE_BUFFER_SIZE, "3110");
     commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE,
-        diskStore1Dir1.getAbsolutePath() + "#1452637463");
-    commandStringBuilder.addOption(CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE,
-        diskStore1Dir2.getAbsolutePath());
+        diskStore1Dir1.getAbsolutePath() + "#1452637463," + diskStore1Dir2.getAbsolutePath());
     cmdResult = executeCommand(commandStringBuilder.toString());
     assertEquals(Result.Status.OK, cmdResult.getStatus());
     String stringResult = commandResultToString(cmdResult);

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
index e99a7fb..0e2a41e 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
@@ -14,34 +14,23 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.apache.geode.test.dunit.Assert.*;
-import static org.apache.geode.test.dunit.IgnoredException.*;
-import static org.apache.geode.test.dunit.LogWriterUtils.*;
-import static org.apache.geode.test.dunit.Wait.*;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.geode.cache.CacheListener;
-import org.apache.geode.cache.EntryEvent;
-import org.apache.geode.cache.util.CacheListenerAdapter;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.apache.geode.test.dunit.Assert.assertEquals;
+import static org.apache.geode.test.dunit.Assert.assertFalse;
+import static org.apache.geode.test.dunit.Assert.assertNotEquals;
+import static org.apache.geode.test.dunit.Assert.assertNotNull;
+import static org.apache.geode.test.dunit.Assert.assertNotSame;
+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.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.LogWriterUtils.getLogWriter;
+import static org.apache.geode.test.dunit.Wait.waitForCriterion;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.CacheListener;
 import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.EntryEvent;
 import org.apache.geode.cache.PartitionAttributes;
 import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.Region;
@@ -51,6 +40,7 @@ import org.apache.geode.cache.query.QueryInvalidException;
 import org.apache.geode.cache.query.data.Portfolio;
 import org.apache.geode.cache.query.internal.CompiledValue;
 import org.apache.geode.cache.query.internal.QCompiler;
+import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.lang.StringUtils;
@@ -84,6 +74,21 @@ import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.WaitCriterion;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.FlakyTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Dunit class for testing gemfire data commands : get, put, remove, select, rebalance
@@ -790,7 +795,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     // Remove empty key entry using gfsh remove command
     String command = "remove ";
 
-    command = command + " " + "--key=\"'" + key + "'\" --region=" + DATA_REGION_NAME_PATH;
+    command = command + " " + "--key=\"" + key + "\" --region=" + DATA_REGION_NAME_PATH;
     CommandResult cmdResult = executeCommand(command);
     printCommandOutput(cmdResult);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/HelpCommandsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/HelpCommandsIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/HelpCommandsIntegrationTest.java
deleted file mode 100644
index b91a1f3..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/HelpCommandsIntegrationTest.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.cli.commands;
-
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.internal.AvailablePortHelper;
-import org.apache.geode.management.internal.cli.CommandManager;
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.result.CommandResult;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.cli.shell.GfshConfig;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.contrib.java.lang.system.ProvideSystemProperty;
-import org.junit.experimental.categories.Category;
-
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.apache.geode.management.internal.cli.commands.CliCommandTestBase.commandResultToString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-@Category(IntegrationTest.class)
-public class HelpCommandsIntegrationTest {
-
-  private int jmxPort;
-
-  private Gfsh gfsh;
-
-  @ClassRule
-  public static final ProvideSystemProperty isGfsh = new ProvideSystemProperty("gfsh", "true");
-
-  @Before
-  public void setup() throws Exception {
-    jmxPort = AvailablePortHelper.getRandomAvailableTCPPort();
-
-    Properties localProps = new Properties();
-    localProps.setProperty(LOCATORS, "");
-    localProps.setProperty(MCAST_PORT, "0");
-    localProps.setProperty(JMX_MANAGER, "true");
-    localProps.setProperty(JMX_MANAGER_START, "true");
-    localProps.setProperty(JMX_MANAGER_PORT, String.valueOf(jmxPort));
-
-    new CacheFactory(localProps).create();
-
-    gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
-  }
-
-  @After
-  public void teardown() {
-    InternalDistributedSystem ids = InternalDistributedSystem.getConnectedInstance();
-    if (ids != null) {
-      ids.disconnect();
-    }
-  }
-
-  /**
-   * TODO:GEODE-1466: update golden file to geode.properties
-   */
-  @Test
-  public void testOfflineHelp() throws Exception {
-    Properties helpProps = new Properties();
-    helpProps.load(
-        HelpCommandsIntegrationTest.class.getResourceAsStream("golden-help-offline.properties"));
-
-    CommandManager cm = CommandManager.getInstance();
-    for (Map.Entry<String, CommandTarget> e : cm.getCommands().entrySet()) {
-      // Mock commands may have been produced in the VM by other tests
-      // 'quit' is an alias for 'exit' and doesn't produce help
-      if (e.getKey().contains("mock") || e.getKey().contains("quit")) {
-        continue;
-      }
-
-      CommandResult cr = (CommandResult) gfsh.executeCommand("help " + e.getKey()).getResult();
-      String gfshResult = commandResultToString(cr);
-
-      String goldParam = e.getKey().replace(" ", "-") + ".help";
-      String goldResult = helpProps.getProperty(goldParam);
-      assertNotNull("No golden text for: " + goldParam, goldResult);
-      assertEquals(goldResult.trim(), gfshResult.trim());
-
-      helpProps.remove(goldParam);
-    }
-
-    // No help should remain unchecked
-    assertEquals(0, helpProps.size());
-  }
-
-  @Test
-  public void testOnlineHelp() throws Exception {
-    Properties helpProps = new Properties();
-    helpProps.load(
-        HelpCommandsIntegrationTest.class.getResourceAsStream("golden-help-online.properties"));
-
-    gfsh.executeCommand("connect --jmx-manager=localhost[" + jmxPort + "]");
-
-    CommandManager cm = CommandManager.getInstance();
-    for (Map.Entry<String, CommandTarget> e : cm.getCommands().entrySet()) {
-      // Mock commands may have been produced in the VM by other tests
-      // 'quit' is an alias for 'exit' and doesn't produce help
-      if (e.getKey().contains("mock") || e.getKey().contains("quit")) {
-        continue;
-      }
-
-      CommandResult cr = (CommandResult) gfsh.executeCommand("help " + e.getKey()).getResult();
-      String gfshResult = commandResultToString(cr);
-
-      String goldParam = e.getKey().replace(" ", "-") + ".help";
-      String goldResult = helpProps.getProperty(goldParam);
-      assertNotNull("No golden text for: " + goldParam, goldResult);
-
-      String[] lines = gfshResult.split("\n");
-      gfshResult = String.join("\n", lines[0], lines[1], lines[2], lines[3]);
-
-      assertEquals(goldResult.trim(), gfshResult.trim());
-
-      helpProps.remove(goldParam);
-    }
-
-    // No help should remain unchecked
-    assertEquals(0, helpProps.size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/QueueCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/QueueCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/QueueCommandsDUnitTest.java
index 42a0624..04a8c13 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/QueueCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/QueueCommandsDUnitTest.java
@@ -193,9 +193,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER,
         "com.qcdunit.QueueCommandsDUnitTestHelper");
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE,
-        "param1");
-    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE,
-        "param2#value2");
+        "param1,param2#value2");
     cmdResult = executeCommand(commandStringBuilder.toString());
     assertEquals(Result.Status.OK, cmdResult.getStatus());
     stringResult = commandResultToString(cmdResult);

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/help/HelpBlockUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/help/HelpBlockUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/help/HelpBlockUnitTest.java
new file mode 100644
index 0000000..4eb4abf
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/help/HelpBlockUnitTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.help;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class HelpBlockUnitTest {
+  private HelpBlock firstBlock, secondBlock, thirdBlock;
+
+  @Before
+  public void before() {
+    firstBlock = new HelpBlock("First Line");
+    secondBlock = new HelpBlock("Second Line");
+    thirdBlock = new HelpBlock("Third Line");
+    assertThat(firstBlock.getLevel()).isEqualTo(0);
+    assertThat(secondBlock.getLevel()).isEqualTo(0);
+    assertThat(thirdBlock.getLevel()).isEqualTo(0);
+  }
+
+  @Test
+  public void testChildLevel() {
+    HelpBlock block = new HelpBlock();
+    assertThat(block.getLevel()).isEqualTo(-1);
+
+    firstBlock.addChild(secondBlock);
+    assertThat(firstBlock.getLevel()).isEqualTo(0);
+    assertThat(secondBlock.getLevel()).isEqualTo(1);
+
+    secondBlock.addChild(thirdBlock);
+    assertThat(firstBlock.getLevel()).isEqualTo(0);
+    assertThat(secondBlock.getLevel()).isEqualTo(1);
+    assertThat(thirdBlock.getLevel()).isEqualTo(2);
+
+    assertThat(firstBlock.getChildren()).contains(secondBlock);
+    assertThat(firstBlock.getChildren().size()).isEqualTo(1);
+    assertThat(secondBlock.getChildren()).contains(thirdBlock);
+    assertThat(secondBlock.getChildren().size()).isEqualTo(1);
+
+    // after manually set the level of the first block
+    firstBlock.setLevel(10);
+    assertThat(firstBlock.getLevel()).isEqualTo(10);
+    assertThat(firstBlock.getData()).isEqualTo("First Line");
+    assertThat(secondBlock.getLevel()).isEqualTo(11);
+    assertThat(secondBlock.getData()).isEqualTo("Second Line");
+    assertThat(thirdBlock.getLevel()).isEqualTo(12);
+    assertThat(thirdBlock.getData()).isEqualTo("Third Line");
+  }
+
+  @Test
+  public void testToString() {
+    firstBlock.addChild(secondBlock);
+    secondBlock.addChild(thirdBlock);
+
+    String result = firstBlock.toString(-1);
+    assertThat(result).isEqualTo("First Line\nSecond Line\nThird Line\n");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/help/HelperUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/help/HelperUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/help/HelperUnitTest.java
new file mode 100644
index 0000000..82d8d71
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/help/HelperUnitTest.java
@@ -0,0 +1,171 @@
+/*
+ * 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.help;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+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;
+import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+
+@Category(UnitTest.class)
+public class HelperUnitTest {
+  private Helper helper;
+  private CliCommand cliCommand;
+  private Method method;
+  private CliAvailabilityIndicator availabilityIndicator;
+  private CommandMarker commandMarker;
+
+  private Annotation[][] annotations;
+  private CliOption cliOption;
+
+  private Class<?>[] parameterType;
+  private HelpBlock optionBlock;
+
+  @Before
+  public void before() throws Exception {
+    helper = new Helper();
+    cliCommand = mock(CliCommand.class);
+    when(cliCommand.value()).thenReturn("test,test-synonym".split(","));
+    when(cliCommand.help()).thenReturn("This is a test description");
+
+    // the tests will test with one parameter and one annotation at a time.
+    cliOption = mock(CliOption.class);
+    when(cliOption.key()).thenReturn("option".split(","));
+    when(cliOption.help()).thenReturn("help of option");
+    when(cliOption.mandatory()).thenReturn(true);
+
+    annotations = new Annotation[1][1];
+    annotations[0][0] = cliOption;
+
+    parameterType = new Class[1];
+    parameterType[0] = String.class;
+
+    availabilityIndicator = mock(CliAvailabilityIndicator.class);
+
+  }
+
+  @Test
+  public void testGetHelp() {
+    HelpBlock helpBlock = helper.getHelp(cliCommand, annotations, parameterType);
+    String[] helpLines = helpBlock.toString().split("\n");
+    assertThat(helpLines.length).isEqualTo(14);
+    assertThat(helpLines[0]).isEqualTo(Helper.NAME_NAME);
+    assertThat(helpLines[2]).isEqualTo(Helper.IS_AVAILABLE_NAME);
+    assertThat(helpLines[4]).isEqualTo(Helper.SYNONYMS_NAME);
+    assertThat(helpLines[6]).isEqualTo(Helper.SYNOPSIS_NAME);
+    assertThat(helpLines[8]).isEqualTo(Helper.SYNTAX_NAME);
+    assertThat(helpLines[10]).isEqualTo(Helper.OPTIONS_NAME);
+  }
+
+  @Test
+  public void testGetSyntaxStringWithMandatory() {
+    String syntax = helper.getSyntaxString("test", annotations, parameterType);
+    assertThat(syntax).isEqualTo("test --option=value");
+    optionBlock = helper.getOptionDetail(cliOption);
+    assertThat(optionBlock.toString())
+        .isEqualTo("option\n" + "help of option\n" + "Required: true\n");
+  }
+
+  @Test
+  public void testGetSyntaxStringWithOutMandatory() {
+    when(cliOption.mandatory()).thenReturn(false);
+    String syntax = helper.getSyntaxString("test", annotations, parameterType);
+    assertThat(syntax).isEqualTo("test [--option=value]");
+    optionBlock = helper.getOptionDetail(cliOption);
+    assertThat(optionBlock.toString())
+        .isEqualTo("option\n" + "help of option\n" + "Required: false\n");
+  }
+
+  @Test
+  public void testGetSyntaxStringWithSecondaryOptionNameIgnored() {
+    when(cliOption.key()).thenReturn("option,option2".split(","));
+    String syntax = helper.getSyntaxString("test", annotations, parameterType);
+    assertThat(syntax).isEqualTo("test --option=value");
+    optionBlock = helper.getOptionDetail(cliOption);
+    assertThat(optionBlock.toString())
+        .isEqualTo("option\n" + "help of option\n" + "Synonyms: option2\n" + "Required: true\n");
+  }
+
+  @Test
+  public void testGetSyntaxStringWithSecondaryOptionName() {
+    when(cliOption.key()).thenReturn(",option2".split(","));
+    when(cliOption.mandatory()).thenReturn(true);
+    String syntax = helper.getSyntaxString("test", annotations, parameterType);
+    assertThat(syntax).isEqualTo("test option2");
+    optionBlock = helper.getOptionDetail(cliOption);
+    assertThat(optionBlock.toString())
+        .isEqualTo("option2\n" + "help of option\n" + "Required: true\n");
+  }
+
+  @Test
+  public void testGetSyntaxStringWithOptionalSecondaryOptionName() {
+    when(cliOption.key()).thenReturn(",option2".split(","));
+    when(cliOption.mandatory()).thenReturn(false);
+    String syntax = helper.getSyntaxString("test", annotations, parameterType);
+    assertThat(syntax).isEqualTo("test [option2]");
+    optionBlock = helper.getOptionDetail(cliOption);
+    assertThat(optionBlock.toString())
+        .isEqualTo("option2\n" + "help of option\n" + "Required: false\n");
+  }
+
+  @Test
+  public void testGetSyntaxStringWithStringArray() {
+    parameterType[0] = String[].class;
+    String syntax = helper.getSyntaxString("test", annotations, parameterType);
+    assertThat(syntax).isEqualTo("test --option=value(,value)*");
+    optionBlock = helper.getOptionDetail(cliOption);
+    assertThat(optionBlock.toString())
+        .isEqualTo("option\n" + "help of option\n" + "Required: true\n");
+  }
+
+  @Test
+  public void testGetSyntaxStringWithSpecifiedDefault() {
+    when(cliOption.specifiedDefaultValue()).thenReturn("true");
+    String syntax = helper.getSyntaxString("test", annotations, parameterType);
+    assertThat(syntax).isEqualTo("test --option(=value)?");
+
+    optionBlock = helper.getOptionDetail(cliOption);
+    assertThat(optionBlock.toString()).isEqualTo("option\n" + "help of option\n"
+        + "Required: true\n" + "Default (if the parameter is specified without value): true\n");
+
+  }
+
+  @Test
+  public void testGetSyntaxStringWithDefaultAndStringArray() {
+    parameterType[0] = String[].class;
+    when(cliOption.specifiedDefaultValue()).thenReturn("value1,value2");
+    String syntax = helper.getSyntaxString("test", annotations, parameterType);
+    assertThat(syntax).isEqualTo("test --option(=value)?(,value)*");
+
+    optionBlock = helper.getOptionDetail(cliOption);
+    assertThat(optionBlock.toString())
+        .isEqualTo("option\n" + "help of option\n" + "Required: true\n"
+            + "Default (if the parameter is specified without value): value1,value2\n");
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/ParserUtilsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/ParserUtilsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/ParserUtilsJUnitTest.java
deleted file mode 100644
index 9b22e64..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/ParserUtilsJUnitTest.java
+++ /dev/null
@@ -1,81 +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.parser;
-
-import static org.junit.Assert.*;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.test.junit.categories.UnitTest;
-
-/**
- * Includes tests for all utility methods in {@link ParserUtils}
- */
-@Category(UnitTest.class)
-public class ParserUtilsJUnitTest {
-
-  /**
-   * Test for {@link ParserUtils#split(String, String)}
-   */
-  @Test
-  public void testSplit() {
-    String input = "something::{::}::nothing";
-    String[] split = ParserUtils.split(input, "::");
-    assertEquals("Size of the split", 3, split.length);
-    assertEquals("First string", "something", split[0]);
-    assertEquals("Second string", "{::}", split[1]);
-    assertEquals("Third string", "nothing", split[2]);
-  }
-
-  /**
-   * Test for {@link ParserUtils#splitValues(String, String)}
-   */
-  @Test
-  public void testSplitValues() {
-    String input = "something::{::}::nothing::";
-    String[] split = ParserUtils.splitValues(input, "::");
-    assertEquals("Size of the split", 4, split.length);
-    assertEquals("First string", "something", split[0]);
-    assertEquals("Second string", "{::}", split[1]);
-    assertEquals("Third string", "nothing", split[2]);
-    assertEquals("Fourth string", "", split[3]);
-  }
-
-  /**
-   * Test for {@link ParserUtils#contains(String, String)}
-   */
-  @Test
-  public void testContains() {
-    String input = "something::{::}::nothing::";
-    assertTrue("Check Boolean", ParserUtils.contains(input, "::"));
-    input = "{something::{::}::nothing::}";
-    assertFalse("Check Boolean", ParserUtils.contains(input, "::"));
-  }
-
-  /**
-   * Test for {@link ParserUtils#lastIndexOf(String, String)}
-   */
-  @Test
-  public void testLastIndexOf() {
-    String input = "something::{::}::nothing::";
-    assertEquals("lastIndex", 24, ParserUtils.lastIndexOf(input, "::"));
-    input = "something::{::}::\"nothing::\"";
-    assertEquals("lastIndex", 15, ParserUtils.lastIndexOf(input, "::"));
-    input = "{something::{::}::\"nothing::\"}";
-    assertEquals("lastIndex", -1, ParserUtils.lastIndexOf(input, "::"));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorJUnitTest.java
deleted file mode 100644
index 97325cb..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorJUnitTest.java
+++ /dev/null
@@ -1,296 +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.parser.preprocessor;
-
-import static org.junit.Assert.*;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.test.junit.categories.UnitTest;
-
-/**
- * Test for Preprocessor
- */
-@Category(UnitTest.class)
-public class PreprocessorJUnitTest {
-
-  @Test
-  public void test1Arg() {
-    String input = "arg1";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 1, split.length);
-    assertEquals("First string", "arg1", split[0]);
-  }
-
-  @Test
-  public void test2Args() {
-    String input = "arg1?arg2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "arg1", split[0]);
-    assertEquals("Second string", "arg2", split[1]);
-  }
-
-  @Test
-  public void test1SpacedArg() {
-    String input = "arg1-1 arg1-2  ";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 1, split.length);
-    assertEquals("First string", "arg1-1 arg1-2", split[0]);
-  }
-
-  @Test
-  public void test1SpacedArg1Option() {
-    String input = "arg1-1 arg1-2 --option1=value1";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 3, split.length);
-    assertEquals("First string", "arg1-1 arg1-2", split[0]);
-    assertEquals("Second string", "--option1", split[1]);
-    assertEquals("Third string", "value1", split[2]);
-  }
-
-  @Test
-  public void test1OptionNoValue() {
-    String input = "--option1";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 1, split.length);
-    assertEquals("First string", "--option1", split[0]);
-  }
-
-  @Test
-  public void test2OptionsNoValue() {
-    String input = "--option1 --option2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "--option2", split[1]);
-  }
-
-  @Test
-  public void test2Options1Value() {
-    String input = "--option1=value1 --option2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 3, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "value1", split[1]);
-    assertEquals("Third string", "--option2", split[2]);
-  }
-
-  @Test
-  public void test1OptionHasValue() {
-    String input = "--option1=value1";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "value1", split[1]);
-  }
-
-  @Test
-  public void test1Arg1OptionHasValue() {
-    String input = "arg1 --option1=value1";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 3, split.length);
-    assertEquals("First string", "arg1", split[0]);
-    assertEquals("Second string", "--option1", split[1]);
-    assertEquals("Third string", "value1", split[2]);
-  }
-
-  @Test
-  public void test1OptionMissingValue() {
-    String input = "--option1=";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "__NULL__", split[1]);
-  }
-
-  @Test
-  public void test2OptionsMissingFirstValue() {
-    String input = "--option1= --option2=value2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 4, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "__NULL__", split[1]);
-    assertEquals("Third string", "--option2", split[2]);
-    assertEquals("Fourth string", "value2", split[3]);
-  }
-
-  @Test
-  public void testSingleQuotedArg() {
-    String input = "\'arg1-1= arg1-2\'?arg2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "\'arg1-1= arg1-2\'", split[0]);
-    assertEquals("Second string", "arg2", split[1]);
-  }
-
-  @Test
-  public void testDoubleQuotedArg() {
-    String input = "\"   \'arg1-1 =arg1-2   \"?arg2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "\"   \'arg1-1 =arg1-2   \"", split[0]);
-    assertEquals("Second string", "arg2", split[1]);
-  }
-
-  @Test
-  public void testSingleQuotedOption() {
-    String input = "--option1=\'value1-1 =value1-2\"\' --option2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 3, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "\'value1-1 =value1-2\"\'", split[1]);
-    assertEquals("Third string", "--option2", split[2]);
-  }
-
-  @Test
-  public void testDoubleQuotedOption() {
-    String input = "--option1= --option2=\"value2\"";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 4, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "__NULL__", split[1]);
-    assertEquals("Third string", "--option2", split[2]);
-    assertEquals("Fourth string", "\"value2\"", split[3]);
-  }
-
-  @Test
-  public void testSingleQuoteInsideDoubleQuote() {
-    String input = "--option1=\"   \'  value1  \'   \"";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "\"   \'  value1  \'   \"", split[1]);
-  }
-
-  @Test
-  public void testQuotedStringWithAdditonalData() {
-    String input = "--option1=\"   \'  value1  \'   \",moreData,\"  even more data\"";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "\"   \'  value1  \'   \",moreData,\"  even more data\"",
-        split[1]);
-  }
-
-  @Test
-  public void testBadOption() {
-    String input = "--option1=value1 -option2=value2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 4, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "value1", split[1]);
-    assertEquals("Third string", "-option2", split[2]);
-    assertEquals("Third string", "value2", split[3]);
-  }
-
-  @Test
-  public void testBadOptions() {
-    String input = "--option1=value1 -option3 -option4";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 4, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "value1", split[1]);
-    assertEquals("Third string", "-option3", split[2]);
-    assertEquals("Third string", "-option4", split[3]);
-  }
-
-  @Test
-  public void testExtraArgSpaces() {
-    String input = "   arg1?  arg2   ";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "arg1", split[0]);
-    assertEquals("Second string", "arg2", split[1]);
-  }
-
-  @Test
-  public void testExtraOptionSpaces() {
-    String input = "   --option1=value1    --option2=value2   ";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 4, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "value1", split[1]);
-    assertEquals("Third string", "--option2", split[2]);
-    assertEquals("Fourth string", "value2", split[3]);
-  }
-
-  @Test
-  public void testExtraArgAndOptionSpaces() {
-    String input = "   arg1   --option1=value1   ";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 3, split.length);
-    assertEquals("First string", "arg1", split[0]);
-    assertEquals("Second string", "--option1", split[1]);
-    assertEquals("Third string", "value1", split[2]);
-  }
-
-  @Test
-  public void testValueSpecifierAsPartOfValue() {
-    String input = "--option1=-Dprop1=value1 --option2=-Dprop2=value2 --option3";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 5, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "-Dprop1=value1", split[1]);
-    assertEquals("Third string", "--option2", split[2]);
-    assertEquals("Fourth string", "-Dprop2=value2", split[3]);
-    assertEquals("Fifth string", "--option3", split[4]);
-  }
-
-  @Test
-  public void testMissingOption() {
-    String input = "--option1=value1 value2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 3, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "value1", split[1]);
-    assertEquals("Third string", "value2", split[2]);
-  }
-
-  @Test
-  public void testUnclosedQuoteArg() {
-    String input = "\"arg1-1 arg1-2 --option1=value1 --option2=value2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 1, split.length);
-    assertEquals("First string", "\"arg1-1 arg1-2 --option1=value1 --option2=value2", split[0]);
-  }
-
-  @Test
-  public void testUnclosedQuoteOption() {
-    String input = "--option1=\"value1 --option2=value2";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second string", "\"value1 --option2=value2", split[1]);
-  }
-
-  @Test
-  public void testArgWithQuotedLongOptionSpec() {
-    String input = "\"--arg=value\"";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 1, split.length);
-    assertEquals("First string", "\"--arg=value\"", split[0]);
-  }
-
-  @Test
-  public void testOptionWithQuotedLongOptionSpec() {
-    String input = "--option1=\"--arg=value\"";
-    String[] split = Preprocessor.split(input);
-    assertEquals("Size of the split", 2, split.length);
-    assertEquals("First string", "--option1", split[0]);
-    assertEquals("Second", "\"--arg=value\"", split[1]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorUtilsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorUtilsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorUtilsJUnitTest.java
deleted file mode 100644
index b56cff2..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorUtilsJUnitTest.java
+++ /dev/null
@@ -1,121 +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.parser.preprocessor;
-
-import static org.junit.Assert.*;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.internal.lang.SystemUtils;
-import org.apache.geode.test.junit.categories.UnitTest;
-
-/**
- * Includes tests for all utility methods in {@link PreprocessorUtils}
- */
-@Category(UnitTest.class)
-public class PreprocessorUtilsJUnitTest {
-
-  /**
-   * Test for {@link PreprocessorUtils#simpleTrim(String)}
-   */
-  @Test
-  public void testSimpleTrim() {
-    String input = " 1 2 3 ";
-    TrimmedInput simpleTrim = PreprocessorUtils.simpleTrim(input);
-    assertEquals("No of spaces removed", 1, simpleTrim.getNoOfSpacesRemoved());
-    assertEquals("input after trimming", "1 2 3", simpleTrim.getString());
-
-    input = " 1 2 3      ";
-    simpleTrim = PreprocessorUtils.simpleTrim(input);
-    assertEquals("No of spaces removed", 1, simpleTrim.getNoOfSpacesRemoved());
-    assertEquals("input after trimming", "1 2 3", simpleTrim.getString());
-  }
-
-  /**
-   * Test for {@link PreprocessorUtils#trim(String)}
-   */
-  @Test
-  public void testTrim() {
-    String input = " command argument1 argument2 ";
-    TrimmedInput trim = PreprocessorUtils.trim(input);
-    assertEquals("No of spaces removed", 1, trim.getNoOfSpacesRemoved());
-    assertEquals("input after trimming", "command argument1 argument2", trim.getString());
-
-    input = "   command   argument1   argument2 ";
-    trim = PreprocessorUtils.trim(input);
-    assertEquals("No of spaces removed", 7, trim.getNoOfSpacesRemoved());
-    assertEquals("input after trimming", "command argument1 argument2", trim.getString());
-
-    input = "command argument1 argument2 -- -- - - - -- -- -- -- -- --- --------- - - - --- --";
-    trim = PreprocessorUtils.trim(input);
-    assertEquals("No of spaces removed", 0, trim.getNoOfSpacesRemoved());
-    assertEquals("input after trimming", "command argument1 argument2", trim.getString());
-
-    input = "command argument1 argument2 --";
-    trim = PreprocessorUtils.trim(input);
-    assertEquals("No of spaces removed", 0, trim.getNoOfSpacesRemoved());
-    assertEquals("input after trimming", "command argument1 argument2", trim.getString());
-
-    input = "command argument1 argument2 -";
-    trim = PreprocessorUtils.trim(input);
-    assertEquals("No of spaces removed", 0, trim.getNoOfSpacesRemoved());
-    assertEquals("input after trimming", "command argument1 argument2", trim.getString());
-  }
-
-  /**
-   * Test for {@link PreprocessorUtils#removeWhiteSpaces(String)}
-   */
-  @Test
-  public void testRemoveWhiteSpaces() {
-    String input = "1 2 3   ";
-    String output = PreprocessorUtils.removeWhiteSpaces(input);
-    assertEquals("Output after removing white spaces", "123", output);
-  }
-
-  /**
-   * Test for {@link PreprocessorUtils#isSyntaxValid(String)}
-   */
-  @Test
-  public void testIsSyntaxValid() {
-    assertTrue(PreprocessorUtils.isSyntaxValid("{}"));
-    assertFalse(PreprocessorUtils.isSyntaxValid("{{]}"));
-    assertTrue(PreprocessorUtils.isSyntaxValid("\"\""));
-    assertTrue(PreprocessorUtils.isSyntaxValid("\"{\'[]\'}\""));
-    assertFalse(PreprocessorUtils.isSyntaxValid("{\"}\""));
-  }
-
-  /**
-   * Test for {@link PreprocessorUtils#containsOnlyWhiteSpaces(String)}
-   */
-  @Test
-  public void testContainsOnlyWhiteSpaces() {
-    assertTrue(PreprocessorUtils
-        .containsOnlyWhiteSpaces("                                                  "));
-    assertFalse(PreprocessorUtils.containsOnlyWhiteSpaces("              d       "));
-  }
-
-  /**
-   * Test for {@link PreprocessorUtils#isWhitespace(char)}
-   */
-  @Test
-  public void testIsWhitespace() {
-    assertTrue(PreprocessorUtils.isWhitespace(' '));
-    assertTrue(PreprocessorUtils.isWhitespace('\t'));
-    assertTrue(PreprocessorUtils.isWhitespace('\n'));
-    assertEquals(SystemUtils.isWindows(), PreprocessorUtils.isWhitespace('\r'));
-  }
-
-}


[4/9] geode git commit: GEODE-1597: use Spring shell's parser and delete our own parsing code

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/Preprocessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/Preprocessor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/Preprocessor.java
deleted file mode 100644
index 0dd875a..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/Preprocessor.java
+++ /dev/null
@@ -1,151 +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.parser.preprocessor;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
-
-/**
- * 
- * <code><br></code> This Class will serve the same purpose as pre-processors do during compilation
- * of a program.
- * 
- * It will act as pre-processor for jopt.
- * 
- * It will split the user input into an array of strings as per the specifications of the command
- * for e.g; Different command might require different value separators, different value specifiers
- * and many more customizations
- * 
- * @since GemFire 7.0
- * 
- */
-public class Preprocessor {
-  private static final String VALUE_SPECIFIER = SyntaxConstants.OPTION_VALUE_SPECIFIER;
-  private static final String ARGUMENT_SEPARATOR = SyntaxConstants.ARGUMENT_SEPARATOR;
-  private static final String OPTION_SEPARATOR = SyntaxConstants.OPTION_SEPARATOR;
-  private static final String LONG_OPTION_SPECIFIER = SyntaxConstants.LONG_OPTION_SPECIFIER;
-  private static final String OPTION_DELIMITER = OPTION_SEPARATOR + LONG_OPTION_SPECIFIER;
-
-  public static String[] split(final String input) {
-    if (input == null) {
-      return null;
-    }
-
-    final String trimInput = PreprocessorUtils.trim(input).getString();
-    final int length = trimInput.length();
-    final List<String> returnStrings = new ArrayList<String>();
-
-    int index = 0; // Current location of the character(s) in the string being examined
-    int startOfString = 0; // Starting index of the string we're currently parsing and preparing to
-                           // save
-
-    // If this first string doesn't start with the long option specifier, then there are arguments.
-    // Process the arguments first.
-    if (!trimInput.regionMatches(index, LONG_OPTION_SPECIFIER, 0, LONG_OPTION_SPECIFIER.length())) {
-      // Until we find the first occurrence of Option Delimiter (" --")
-      while (index < length
-          && !trimInput.regionMatches(index, OPTION_DELIMITER, 0, OPTION_DELIMITER.length())) {
-        // Anything inside single or double quotes gets ignored
-        if (trimInput.charAt(index) == '\'' || trimInput.charAt(index) == '\"') {
-          char charToLookFor = trimInput.charAt(index++);
-          // Look for the next single or double quote. Those preceded by a '\' character are
-          // ignored.
-          while (index < length && (trimInput.charAt(index) != charToLookFor
-              || trimInput.charAt(index - 1) == '\\')) {
-            index++;
-          }
-        }
-
-        index++;
-        // 1. There are only arguments & we've reached the end OR
-        // 2. We are at index where option (" --") has started OR
-        // 3. One argument has finished & we are now at the next argument - check for Argument
-        // Separator (" ")
-        if (index >= length
-            || trimInput.regionMatches(index, OPTION_DELIMITER, 0, OPTION_DELIMITER.length())
-            || trimInput.regionMatches(index, ARGUMENT_SEPARATOR, 0, ARGUMENT_SEPARATOR.length())) {
-          String stringToAdd =
-              trimInput.substring(startOfString, (index > length ? length : index)).trim();
-          returnStrings.add(stringToAdd);
-
-          if (trimInput.regionMatches(index, ARGUMENT_SEPARATOR, 0, ARGUMENT_SEPARATOR.length())) {
-            index += ARGUMENT_SEPARATOR.length();
-          }
-          startOfString = index;
-        }
-      }
-      index += OPTION_SEPARATOR.length();
-    }
-
-    // Process the options
-    startOfString = index;
-    while (index < length) {
-      // Until we find the first occurrence of Option Separator (" ") or Value Specifier ("=")
-      while (index < length
-          && !trimInput.regionMatches(index, OPTION_SEPARATOR, 0, OPTION_SEPARATOR.length())
-          && !trimInput.regionMatches(index, VALUE_SPECIFIER, 0, VALUE_SPECIFIER.length())) {
-        index++;
-      }
-
-      if (startOfString != index) {
-        returnStrings.add(trimInput.substring(startOfString, index));
-        startOfString = index + 1;
-      }
-
-      // If value part (starting with "=") has started
-      if (trimInput.regionMatches(index++, VALUE_SPECIFIER, 0, VALUE_SPECIFIER.length())) {
-        startOfString = index;
-
-        // Keep going over chars until we find the option separator ("--")
-        while (index < length
-            && !trimInput.regionMatches(index, OPTION_SEPARATOR, 0, OPTION_SEPARATOR.length())) {
-          // Anything inside single or double quotes gets ignored
-          if (index < length
-              && (trimInput.charAt(index) == '\'' || trimInput.charAt(index) == '\"')) {
-            char charToLookFor = trimInput.charAt(index++);
-            // Look for the next single or double quote. Those preceded by a '\' character are
-            // ignored.
-            while (index < length && (trimInput.charAt(index) != charToLookFor
-                || trimInput.charAt(index - 1) == '\\')) {
-              index++;
-            }
-          }
-          index++;
-        }
-
-        // 1. We are done & at the end OR
-        // 2. There is another word which matches ("--")
-        if (index >= length
-            || trimInput.regionMatches(index, OPTION_SEPARATOR, 0, OPTION_SEPARATOR.length())) {
-          if (startOfString == index) {
-            // This place-holder value indicates to OptionParser that an option
-            // was specified without a value.
-            returnStrings.add("__NULL__");
-          } else {
-            String stringToAdd =
-                trimInput.substring(startOfString, (index > length ? length : index));
-            returnStrings.add(stringToAdd);
-          }
-          startOfString = index + 1;
-        }
-        index++;
-      }
-    }
-
-    return returnStrings.toArray(new String[0]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorUtils.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorUtils.java
deleted file mode 100644
index a1872c9..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/PreprocessorUtils.java
+++ /dev/null
@@ -1,327 +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.parser.preprocessor;
-
-
-import java.util.regex.Pattern;
-
-import org.apache.commons.lang.StringUtils;
-
-import org.apache.geode.internal.lang.SystemUtils;
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
-
-/**
- * The methods in this class will be used by the {@link Preprocessor} class to perform various
- * trivial operations
- * 
- * @since GemFire 7.0
- */
-public class PreprocessorUtils {
-
-  public static TrimmedInput simpleTrim(String input) {
-    if (input != null) {
-      // First remove the trailing white spaces, we do not need those
-      if (!containsOnlyWhiteSpaces(input)) {
-        input = StringUtils.stripEnd(input, null);
-      }
-      String output = input.trim();
-      return new TrimmedInput(output, input.length() - output.length());
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * 
-   * This function will trim the given input string. It will not only remove the spaces and tabs at
-   * the end but also compress multiple spaces and tabs to a single space
-   * 
-   * @param input The input string on which the trim operation needs to be performed
-   * @return String
-   */
-  public static TrimmedInput trim(final String input) {
-    return trim(input, true);
-  }
-
-  /**
-   * 
-   * This function will trim the given input string. It will not only remove the spaces and tabs at
-   * the end but also compress multiple spaces and tabs to a single space
-   * 
-   * @param input The input string on which the trim operation needs to be performed
-   * @param retainLineSeparator whether to retain the line separator.
-   * 
-   * @return String
-   */
-  public static TrimmedInput trim(final String input, final boolean retainLineSeparator) {
-    if (input != null) {
-      String inputCopy = input;
-      StringBuffer output = new StringBuffer();
-      // First remove the trailing white spaces, we do not need those
-      inputCopy = StringUtils.stripEnd(inputCopy, null);
-      // As this parser is for optionParsing, we also need to remove
-      // the trailing optionSpecifiers provided it has previous
-      // options. Remove the trailing LONG_OPTION_SPECIFIERs
-      // in a loop. It is necessary to check for previous options for
-      // the case of non-mandatory arguments.
-      // "^(.*)(\\s-+)$" - something that ends with a space followed by a series of hyphens.
-      while (Pattern.matches("^(.*)(\\s-+)$", inputCopy)) {
-        inputCopy = StringUtils.removeEnd(inputCopy, SyntaxConstants.SHORT_OPTION_SPECIFIER);
-
-        // Again we need to trim the trailing white spaces
-        // As we are in a loop
-        inputCopy = StringUtils.stripEnd(inputCopy, null);
-      }
-      // Here we made use of the String class function trim to remove the
-      // space and tabs if any at the
-      // beginning and the end of the string
-      int noOfSpacesRemoved = 0;
-      {
-        int length = inputCopy.length();
-        inputCopy = inputCopy.trim();
-        noOfSpacesRemoved += length - inputCopy.length();
-      }
-      // Now we need to compress the multiple spaces and tabs to single space
-      // and tabs but we also need to ignore the white spaces inside the
-      // quotes and parentheses
-
-      StringBuffer buffer = new StringBuffer();
-
-      boolean startWhiteSpace = false;
-      for (int i = 0; i < inputCopy.length(); i++) {
-        char ch = inputCopy.charAt(i);
-        buffer.append(ch);
-        if (PreprocessorUtils.isWhitespace(ch)) {
-          if (PreprocessorUtils.isSyntaxValid(buffer.toString())) {
-            if (startWhiteSpace) {
-              noOfSpacesRemoved++;
-            } else {
-              startWhiteSpace = true;
-              if (ch == '\n') {
-                if (retainLineSeparator) {
-                  output.append("\n");
-                }
-              } else {
-                output.append(" ");
-              }
-            }
-            buffer.delete(0, buffer.length());
-          } else {
-            output.append(ch);
-          }
-        } else {
-          startWhiteSpace = false;
-          output.append(ch);
-        }
-      }
-      return new TrimmedInput(output.toString(), noOfSpacesRemoved);
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * This function just does the simple job of removing white spaces from the given input string
-   * 
-   * @param input The input String from which the spaces need to be removed
-   * @return String
-   */
-  public static String removeWhiteSpaces(String input) {
-    if (input != null) {
-      input = trim(input).getString();
-      StringBuffer output = new StringBuffer();
-      for (int i = 0; i < input.length(); i++) {
-        char ch = input.charAt(i);
-        if (PreprocessorUtils.isWhitespace(ch)) {
-          continue;
-        } else {
-          output.append(ch);
-        }
-      }
-      return output.toString();
-    } else {
-      return null;
-    }
-  }
-
-  /**
-   * 
-   * This function will check for the validity of the input provided.
-   * 
-   * For e.g; '" input"' is valid but '" input' is not a valid input same is the case with input
-   * like a JSON string
-   * 
-   * @param input The input string which needs to be checked for proper syntax
-   * @return Boolean
-   */
-  public static Boolean isSyntaxValid(String input) {
-    if (input != null) {
-      // We will need two different stacks one for double quotation
-      // and the other one for checking brackets
-      Stack<Character> stack = new Stack<Character>();
-      if (input.length() > 0) {
-        for (int i = 0; i < input.length(); i++) {
-          char ch = input.charAt(i);
-          if ('\\' == ch) {
-            // It means that this is an escape sequence
-            // So skip the next character as well
-            i++;
-            continue;
-          }
-          if (isValueEnclosingChar(ch)) {
-            // First check whether the enclosing character
-            // is a double quotation.
-            if (EnclosingCharacters.DOUBLE_QUOTATION == ch) {
-              Character popped = stack.pop();
-              if (popped == EnclosingCharacters.DOUBLE_QUOTATION) {
-                // Everything is normal
-              } else {
-                // We just push both the characters onto the stack
-                if (popped != null) {
-                  stack.push(popped);
-                }
-                stack.push(ch);
-              }
-            } else if (EnclosingCharacters.SINGLE_QUOTATION == ch) {
-              Character popped = stack.pop();
-              if (popped == EnclosingCharacters.SINGLE_QUOTATION) {
-                // Everything is normal
-              } else {
-                // We just push both the characters onto the stack
-                if (popped != null) {
-                  stack.push(popped);
-                }
-                stack.push(ch);
-              }
-            } else {
-              if (isOpeningBracket(ch)) {
-                // If this a opening bracket then just push it onto
-                // the stack
-                stack.push(ch);
-              } else {
-                // This means that it is a closing bracket.
-                // Now pop a character form the stack and check
-                // whether both
-                // the brackets match each other
-                Character popped = stack.pop();
-                if (matches(popped, ch)) {
-                  // Everything is normal
-                } else {
-                  return false;
-                }
-              }
-            }
-          }
-        }
-      }
-      if (stack.isEmpty()) {
-        return true;
-      } else {
-        return false;
-      }
-    } else {
-      return false;
-    }
-  }
-
-  private static boolean matches(Character popped, char ch) {
-    if (popped != null) {
-      outer: {
-        if (isOpeningBracket(popped)) {
-          if (EnclosingCharacters.OPENING_SQUARE_BRACKET == popped) {
-            if (EnclosingCharacters.CLOSING_SQUARE_BRACKET == ch) {
-              return true;
-            } else {
-              break outer;
-            }
-          }
-          if (EnclosingCharacters.OPENING_CIRCULAR_BRACKET == popped) {
-            if (EnclosingCharacters.CLOSING_CIRCULAR_BRACKET == ch) {
-              return true;
-            } else {
-              break outer;
-            }
-          }
-          if (EnclosingCharacters.OPENING_CURLY_BRACE == popped) {
-            if (EnclosingCharacters.CLOSING_CURLY_BRACE == ch) {
-              return true;
-            } else {
-              break outer;
-            }
-          }
-        }
-      }
-      return false;
-    } else {
-      return false;
-    }
-  }
-
-  // Not used at present
-  @SuppressWarnings("unused")
-  private static boolean isClosingBracket(char ch) {
-    if (EnclosingCharacters.CLOSING_SQUARE_BRACKET == ch
-        || EnclosingCharacters.CLOSING_CIRCULAR_BRACKET == ch
-        || EnclosingCharacters.CLOSING_CURLY_BRACE == ch) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  private static boolean isOpeningBracket(char ch) {
-    if (EnclosingCharacters.OPENING_SQUARE_BRACKET == ch
-        || EnclosingCharacters.OPENING_CIRCULAR_BRACKET == ch
-        || EnclosingCharacters.OPENING_CURLY_BRACE == ch) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  private static boolean isValueEnclosingChar(char ch) {
-    if (EnclosingCharacters.OPENING_SQUARE_BRACKET == ch
-        || EnclosingCharacters.CLOSING_SQUARE_BRACKET == ch
-        || EnclosingCharacters.OPENING_CIRCULAR_BRACKET == ch
-        || EnclosingCharacters.CLOSING_CIRCULAR_BRACKET == ch
-        || EnclosingCharacters.OPENING_CURLY_BRACE == ch
-        || EnclosingCharacters.CLOSING_CURLY_BRACE == ch
-        || EnclosingCharacters.DOUBLE_QUOTATION == ch
-        || EnclosingCharacters.SINGLE_QUOTATION == ch) {
-      return true;
-    }
-    return false;
-  }
-
-  public static boolean containsOnlyWhiteSpaces(String input) {
-    if (input != null) {
-      for (int i = 0; i < input.length(); i++) {
-        if (!PreprocessorUtils.isWhitespace(input.charAt(i))) {
-          return false;
-        }
-      }
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  public static boolean isWhitespace(char ch) {
-    if (ch == ' ' || ch == '\t' || ch == '\n' || (ch == '\r' && SystemUtils.isWindows())) {
-      return true;
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/Stack.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/Stack.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/Stack.java
deleted file mode 100644
index ae47723..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/Stack.java
+++ /dev/null
@@ -1,52 +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.parser.preprocessor;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Basic Stack implementation, used by {@link PreprocessorUtils#isSyntaxValid(String)} for detecting
- * valid syntax
- * 
- * 
- * @param <T>
- */
-public class Stack<T> {
-  private List<T> list = new ArrayList<T>();
-
-  public void push(T object) {
-    list.add(object);
-  }
-
-  public T pop() {
-    if (list.size() > 0) {
-      int length = list.size();
-      T object = list.get(length - 1);
-      list.remove(length - 1);
-      return object;
-    } else {
-      return null;
-    }
-  }
-
-  public Boolean isEmpty() {
-    if (list.size() == 0) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/TrimmedInput.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/TrimmedInput.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/TrimmedInput.java
deleted file mode 100644
index 8740f00..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/parser/preprocessor/TrimmedInput.java
+++ /dev/null
@@ -1,44 +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.parser.preprocessor;
-
-/**
- * Used for trimming input before Pre-processing
- * 
- * @since GemFire 7.0
- * 
- */
-public class TrimmedInput {
-  private final int noOfSpacesRemoved;
-  private final String string;
-
-  public TrimmedInput(String string, int noOfSpacesRemoved) {
-    this.string = string;
-    this.noOfSpacesRemoved = noOfSpacesRemoved;
-  }
-
-  public String getString() {
-    return string;
-  }
-
-  public int getNoOfSpacesRemoved() {
-    return noOfSpacesRemoved;
-  }
-
-  @Override
-  public String toString() {
-    return string;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/CommandProcessor.java
----------------------------------------------------------------------
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
index c346eaf..c2c6e14 100755
--- 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
@@ -14,27 +14,25 @@
  */
 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.apache.geode.internal.security.IntegratedSecurityService;
 import org.apache.geode.internal.security.SecurityService;
 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.springframework.shell.core.Parser;
 import org.springframework.shell.event.ParseResult;
 
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.util.Map;
+import java.util.Properties;
+
 /**
  * 
  * 
@@ -42,8 +40,7 @@ import org.springframework.shell.event.ParseResult;
  */
 public class CommandProcessor {
   protected RemoteExecutionStrategy executionStrategy;
-  protected Parser parser;
-  private CommandManager commandManager;
+  private GfshParser gfshParser;
   private int lastExecutionStatus;
   private LogWrapper logWrapper;
 
@@ -59,9 +56,8 @@ public class CommandProcessor {
   }
 
   public CommandProcessor(Properties cacheProperties) throws ClassNotFoundException, IOException {
-    this.commandManager = CommandManager.getInstance(cacheProperties);
+    this.gfshParser = new GfshParser(cacheProperties);
     this.executionStrategy = new RemoteExecutionStrategy();
-    this.parser = new GfshParser(commandManager);
     this.logWrapper = LogWrapper.getInstance();
   }
 
@@ -73,7 +69,7 @@ public class CommandProcessor {
 
   protected Parser getParser() {
     synchronized (LOCK) {
-      return parser;
+      return gfshParser;
     }
   }
 
@@ -178,9 +174,8 @@ public class CommandProcessor {
 
   public void stop() {
     synchronized (LOCK) {
-      this.commandManager = null;
+      this.gfshParser = null;
       this.executionStrategy = null;
-      this.parser = null;
       this.isStopped = true;
     }
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
index bcf1b41..78921c5 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
@@ -26,11 +26,9 @@ import org.apache.geode.internal.util.SunAPINotFoundException;
 import org.apache.geode.management.cli.CommandProcessingException;
 import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.CliUtil;
-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.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
 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;
@@ -42,8 +40,6 @@ import org.apache.geode.management.internal.cli.shell.jline.GfshUnsupportedTermi
 import org.apache.geode.management.internal.cli.shell.unsafe.GfshSignalHandler;
 import org.apache.geode.management.internal.cli.util.CommentSkipHelper;
 import org.springframework.shell.core.AbstractShell;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.Converter;
 import org.springframework.shell.core.ExecutionStrategy;
 import org.springframework.shell.core.ExitShellRequest;
 import org.springframework.shell.core.JLineLogHandler;
@@ -65,7 +61,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Scanner;
-import java.util.Set;
 import java.util.TreeMap;
 import java.util.logging.Level;
 import java.util.logging.LogManager;
@@ -84,7 +79,7 @@ import java.util.logging.Logger;
  * <p />
  * Additionally, this class is used to maintain GemFire SHell (gfsh) specific information like:
  * environment TODO
- * 
+ *
  *
  * @since GemFire 7.0
  */
@@ -101,13 +96,8 @@ public class Gfsh extends JLineShell {
 
   public static final String LINE_INDENT = "    ";
   public static final String LINE_SEPARATOR = System.getProperty("line.separator");
-
-  private static final String DEFAULT_SECONDARY_PROMPT = ">";
-
   // Default Window dimensions
   public static final int DEFAULT_WIDTH = 100;
-  private static final int DEFAULT_HEIGHT = 100;
-
   public static final String ENV_APP_NAME = "APP_NAME";
   public static final String ENV_APP_CONTEXT_PATH = "APP_CONTEXT_PATH";
   public static final String ENV_APP_FETCH_SIZE = "APP_FETCH_SIZE";
@@ -119,7 +109,6 @@ public class Gfsh extends JLineShell {
   public static final String ENV_APP_LOG_FILE = "APP_LOG_FILE";
   public static final String ENV_APP_PWD = "APP_PWD";
   public static final String ENV_APP_RESULT_VIEWER = "APP_RESULT_VIEWER";
-
   // Environment Properties taken from the OS
   public static final String ENV_SYS_USER = "SYS_USER";
   public static final String ENV_SYS_USER_HOME = "SYS_USER_HOME";
@@ -138,41 +127,39 @@ public class Gfsh extends JLineShell {
   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();
+  public static boolean SUPPORT_MUTLIPLESHELL = false;
 
+  // private static final String ANIMATION_SLOT = "A"; //see 46072
   protected static PrintStream gfshout = System.out;
   protected static PrintStream gfsherr = System.err;
-
-  // private static final String ANIMATION_SLOT = "A"; //see 46072
-
+  protected static ThreadLocal<Gfsh> gfshThreadLocal = new ThreadLocal<Gfsh>();
   private static Gfsh instance;
-  private static final Object INSTANCE_LOCK = new Object();
-
+  // This flag is used to restrict column trimming to table only types
+  private static ThreadLocal<Boolean> resultTypeTL = new ThreadLocal<Boolean>();
+  private static String OS = System.getProperty("os.name").toLowerCase();
   private final Map<String, String> env = new TreeMap<String, String>();
   private final List<String> readonlyAppEnv = new ArrayList<String>();
-
   // Map to keep reference to actual user specified Command String
   // Should always have one value at the max
   private final Map<String, String> expandedPropCommandsMap = new HashMap<String, String>();
-
-  private final CommandManager commandManager;
   private final ExecutionStrategy executionStrategy;
   private final GfshParser parser;
-  private OperationInvoker operationInvoker;
-  private int lastExecutionStatus;
-  private Thread runner;
-  private boolean debugON;
   private final LogWrapper gfshFileLogger;
   private final GfshConfig gfshConfig;
   private final GfshHistory gfshHistory;
   private final ANSIHandler ansiHandler;
-  private Terminal terminal;
   private final boolean isHeadlessMode;
+  private OperationInvoker operationInvoker;
+  private int lastExecutionStatus;
+  private Thread runner;
+  private boolean debugON;
+  private Terminal terminal;
   private boolean supressScriptCmdOutput;
   private boolean isScriptRunning;
-
   private AbstractSignalNotificationHandler signalHandler;
-  // This flag is used to restrict column trimming to table only types
-  private static ThreadLocal<Boolean> resultTypeTL = new ThreadLocal<Boolean>();
 
   protected Gfsh() throws ClassNotFoundException, IOException {
     this(null);
@@ -180,7 +167,7 @@ public class Gfsh extends JLineShell {
 
   /**
    * Create a GemFire shell with console using the specified arguments.
-   * 
+   *
    * @param args arguments to be used to create a GemFire shell instance
    * @throws IOException
    * @throws ClassNotFoundException
@@ -192,7 +179,7 @@ public class Gfsh extends JLineShell {
   /**
    * Create a GemFire shell using the specified arguments. Console for user inputs is made available
    * if <code>launchShell</code> is set to <code>true</code>.
-   * 
+   *
    * @param launchShell whether to make Console available
    * @param args arguments to be used to create a GemFire shell instance or execute command
    * @throws IOException
@@ -219,14 +206,11 @@ public class Gfsh extends JLineShell {
     // 4. Customized History implementation
     this.gfshHistory = new GfshHistory();
 
-    // 5. Create CommandManager & load Commands & Converters
-    this.commandManager = CommandManager.getInstance();
-
     // 6. Set System Environment here
     initializeEnvironment();
     // 7. Create Roo/SpringShell framework objects
     this.executionStrategy = new GfshExecutionStrategy(this);
-    this.parser = new GfshParser(commandManager);
+    this.parser = new GfshParser();
     // 8. Set max History file size
     setHistorySize(gfshConfig.getHistorySize());
 
@@ -271,6 +255,195 @@ public class Gfsh extends JLineShell {
     }
   }
 
+  public static Gfsh getInstance(boolean launchShell, String[] args, GfshConfig gfshConfig)
+      throws ClassNotFoundException, IOException {
+    if (instance == null) {
+      synchronized (INSTANCE_LOCK) {
+        if (instance == null) {
+          instance = new Gfsh(launchShell, args, gfshConfig);
+          instance.executeInitFileIfPresent();
+        }
+      }
+    }
+
+    return instance;
+  }
+
+  public static boolean isInfoResult() {
+    if (resultTypeTL.get() == null) {
+      return false;
+    }
+    return resultTypeTL.get();
+  }
+
+  public static void println() {
+    gfshout.println();
+  }
+
+  public static <T> void println(T toPrint) {
+    gfshout.println(toPrint);
+  }
+
+  public static <T> void print(T toPrint) {
+    gfshout.print(toPrint);
+  }
+
+  public static <T> void printlnErr(T toPrint) {
+    gfsherr.println(toPrint);
+  }
+
+  // See 46369
+  private static String readLine(ConsoleReader reader, String prompt) throws IOException {
+    String earlierLine = reader.getCursorBuffer().toString();
+    String readLine = null;
+    try {
+      readLine = reader.readLine(prompt);
+    } catch (IndexOutOfBoundsException e) {
+      if (earlierLine.length() == 0) {
+        reader.println();
+        readLine = LINE_SEPARATOR;
+        reader.getCursorBuffer().cursor = 0;
+      } else {
+        readLine = readLine(reader, prompt);
+      }
+    }
+    return readLine;
+  }
+
+  private static String removeBackslash(String result) {
+    if (result.endsWith(GfshParser.CONTINUATION_CHARACTER)) {
+      result = result.substring(0, result.length() - 1);
+    }
+    return result;
+  }
+
+  public static void redirectInternalJavaLoggers() {
+    // Do we need to this on re-connect?
+    LogManager logManager = LogManager.getLogManager();
+
+    try {
+      Enumeration<String> loggerNames = logManager.getLoggerNames();
+
+      while (loggerNames.hasMoreElements()) {
+        String loggerName = loggerNames.nextElement();
+        if (loggerName.startsWith("java.") || loggerName.startsWith("javax.")) {
+          // System.out.println(loggerName);
+          Logger javaLogger = logManager.getLogger(loggerName);
+          /*
+           * From Java Docs: It is also important to note that the Logger associated with the String
+           * name may be garbage collected at any time if there is no strong reference to the
+           * Logger. The caller of this method must check the return value for null in order to
+           * properly handle the case where the Logger has been garbage collected.
+           */
+          if (javaLogger != null) {
+            LogWrapper.getInstance().setParentFor(javaLogger);
+          }
+        }
+      }
+    } catch (SecurityException e) {
+      LogWrapper.getInstance().warning(e.getMessage(), e);
+    }
+  }
+
+  public static Gfsh getCurrentInstance() {
+    if (!SUPPORT_MUTLIPLESHELL) {
+      return instance;
+    } else {
+      return gfshThreadLocal.get();
+    }
+  }
+
+  private static String extractKey(String input) {
+    return input.substring("${".length(), input.length() - "}".length());
+  }
+
+  public static ConsoleReader getConsoleReader() {
+    Gfsh gfsh = Gfsh.getCurrentInstance();
+    return (gfsh == null ? null : gfsh.reader);
+  }
+
+  /**
+   * Take a string and wrap it into multiple lines separated by CliConstants.LINE_SEPARATOR. Lines
+   * are separated based upon the terminal width, separated on word boundaries and may have extra
+   * spaces added to provide indentation.
+   *
+   * For example: if the terminal width were 5 and the string "123 456789 01234" were passed in with
+   * an indentation level of 2, then the returned string would be:
+   *
+   * <pre>
+   *         123
+   *         45678
+   *         9
+   *         01234
+   * </pre>
+   *
+   * @param string String to wrap (add breakpoints and indent)
+   * @param indentationLevel The number of indentation levels to use.
+   * @return The wrapped string.
+   */
+  public static String wrapText(final String string, final int indentationLevel,
+      final int terminalWidth) {
+    if (terminalWidth <= 1) {
+      return string;
+    }
+
+    final int maxLineLength = terminalWidth - 1;
+    final StringBuffer stringBuf = new StringBuffer();
+    int index = 0;
+    int startOfCurrentLine = 0;
+    while (index < string.length()) {
+      // Add the indentation
+      for (int i = 0; i < indentationLevel; i++) {
+        stringBuf.append(LINE_INDENT);
+      }
+      int currentLineLength = LINE_INDENT.length() * indentationLevel;
+
+      // Find the end of a line:
+      // 1. If the end of string is reached
+      // 2. If the width of the terminal has been reached
+      // 3. If a newline character was found in the string
+      while (index < string.length() && currentLineLength < maxLineLength
+          && string.charAt(index) != '\n') {
+        index++;
+        currentLineLength++;
+      }
+
+      // If the line was terminated with a newline character
+      if (index != string.length() && string.charAt(index) == '\n') {
+        stringBuf.append(string.substring(startOfCurrentLine, index));
+        stringBuf.append(LINE_SEPARATOR);
+        index++;
+        startOfCurrentLine = index;
+
+        // If the end of the string was reached or the last character just happened to be a space
+        // character
+      } else if (index == string.length() || string.charAt(index) == ' ') {
+        stringBuf.append(string.substring(startOfCurrentLine, index));
+        if (index != string.length()) {
+          stringBuf.append(LINE_SEPARATOR);
+          index++;
+        }
+
+      } else {
+        final int spaceCharIndex = string.lastIndexOf(" ", index);
+
+        // If no spaces were found then there's no logical wayto split the string
+        if (spaceCharIndex == -1) {
+          stringBuf.append(string.substring(startOfCurrentLine, index)).append(LINE_SEPARATOR);
+
+          // Else split the string cleanly between words
+        } else {
+          stringBuf.append(string.substring(startOfCurrentLine, spaceCharIndex))
+              .append(LINE_SEPARATOR);
+          index = spaceCharIndex + 1;
+        }
+      }
+
+      startOfCurrentLine = index;
+    }
+    return stringBuf.toString();
+  }
+
   /**
    * Initializes default environment variables to default values
    */
@@ -302,20 +475,6 @@ public class Gfsh extends JLineShell {
     env.put(ENV_APP_RESULT_VIEWER, String.valueOf(DEFAULT_APP_RESULT_VIEWER));
   }
 
-  public static Gfsh getInstance(boolean launchShell, String[] args, GfshConfig gfshConfig)
-      throws ClassNotFoundException, IOException {
-    if (instance == null) {
-      synchronized (INSTANCE_LOCK) {
-        if (instance == null) {
-          instance = new Gfsh(launchShell, args, gfshConfig);
-          instance.executeInitFileIfPresent();
-        }
-      }
-    }
-
-    return instance;
-  }
-
   public AbstractSignalNotificationHandler getSignalHandler() {
     return signalHandler;
   }
@@ -384,8 +543,6 @@ public class Gfsh extends JLineShell {
 
   }
 
-  //////////////////// JLineShell Class Methods Start //////////////////////////
-  //////////////////////// Implemented Methods ////////////////////////////////
   /**
    * See findResources in {@link AbstractShell}
    */
@@ -397,7 +554,7 @@ public class Gfsh extends JLineShell {
   /**
    * Returns the {@link ExecutionStrategy} implementation used by this implementation of
    * {@link AbstractShell}. {@link Gfsh} uses {@link GfshExecutionStrategy}.
-   * 
+   *
    * @return ExecutionStrategy used by Gfsh
    */
   @Override
@@ -408,19 +565,22 @@ public class Gfsh extends JLineShell {
   /**
    * Returns the {@link Parser} implementation used by this implementation of
    * {@link AbstractShell}.{@link Gfsh} uses {@link GfshParser}.
-   * 
+   *
    * @return Parser used by Gfsh
    */
   @Override
-  protected Parser getParser() {
+  public Parser getParser() {
+    return parser;
+  }
+
+  public GfshParser getGfshParser() {
     return parser;
   }
 
-  //////////////////////// Overridden Behavior /////////////////////////////////
   /**
    * Executes the given command string. We have over-ridden the behavior to extend the original
    * implementation to store the 'last command execution status'.
-   * 
+   *
    * @param line command string to be executed
    * @return true if execution is successful; false otherwise
    */
@@ -469,14 +629,6 @@ public class Gfsh extends JLineShell {
     return reader.readLine(textToPrompt, mask);
   }
 
-  public void add(final CommandMarker command) {
-    commandManager.add(command);
-  }
-
-  public void add(final Converter<?> converter) {
-    commandManager.add(converter);
-  }
-
   @Override
   public void printBannerAndWelcome() {
     printAsInfo(getBanner());
@@ -584,14 +736,6 @@ public class Gfsh extends JLineShell {
       return false;
   }
 
-  public static boolean isInfoResult() {
-    if (resultTypeTL.get() == null)
-      return false;
-    return resultTypeTL.get();
-  }
-
-  private static String OS = System.getProperty("os.name").toLowerCase();
-
   private boolean isUnix() {
     return !(OS.indexOf("win") >= 0);
   }
@@ -604,7 +748,6 @@ public class Gfsh extends JLineShell {
     }
   }
 
-  /////// Save multiple-line commands as single line in history - starts ///////
   @Override
   protected ConsoleReader createConsoleReader() {
     ConsoleReader consoleReader = super.createConsoleReader();
@@ -631,15 +774,6 @@ public class Gfsh extends JLineShell {
     return getVersion();
   }
 
-  // causes instability on MacOS See #46072
-  // public void flashMessage(String message) {
-  // if (reader != null) {
-  // flash(Level.FINE, message, ANIMATION_SLOT);
-  // }
-  // }
-  /////// Save multiple-line commands as single line in history - ends /////////
-  ///////////////////// JLineShell Class Methods End //////////////////////////
-
   public int getTerminalHeight() {
     return terminal != null ? terminal.getHeight() : DEFAULT_HEIGHT;
   }
@@ -658,22 +792,6 @@ public class Gfsh extends JLineShell {
     return DEFAULT_WIDTH;
   }
 
-  public static void println() {
-    gfshout.println();
-  }
-
-  public static <T> void println(T toPrint) {
-    gfshout.println(toPrint);
-  }
-
-  public static <T> void print(T toPrint) {
-    gfshout.print(toPrint);
-  }
-
-  public static <T> void printlnErr(T toPrint) {
-    gfsherr.println(toPrint);
-  }
-
   /**
    * @return the lastExecutionStatus
    */
@@ -684,7 +802,7 @@ public class Gfsh extends JLineShell {
 
   /**
    * Set the last command execution status
-   * 
+   *
    * @param lastExecutionStatus last command execution status
    */
   public void setLastExecutionStatus(int lastExecutionStatus) {
@@ -801,7 +919,7 @@ public class Gfsh extends JLineShell {
           linesBuffer.append(lineWithoutComments);
           linesBufferString = linesBuffer.toString();
           // NOTE: Similar code is in promptLoop()
-          if (!linesBufferString.endsWith(SyntaxConstants.CONTINUATION_CHARACTER)) { // see 45893
+          if (!linesBufferString.endsWith(GfshParser.CONTINUATION_CHARACTER)) { // see 45893
             // String command = null;
 
             List<String> commandList = MultiCommandHelper.getMultipleCommands(linesBufferString);
@@ -855,7 +973,7 @@ public class Gfsh extends JLineShell {
     return result;
   }
 
-  /////////////// For setting shell environment properties START ///////////////
+
   public String setEnvProperty(String propertyName, String propertyValue) {
     if (propertyName == null || propertyValue == null) {
       throw new IllegalArgumentException(
@@ -885,11 +1003,9 @@ public class Gfsh extends JLineShell {
   }
 
   public boolean isQuietMode() {
-    // System.out.println(env.get(CliConstants.ENV_APP_QUIET_EXECUTION));
     return Boolean.parseBoolean(env.get(ENV_APP_QUIET_EXECUTION));
   }
 
-  ////////////////// Providing Multiple-line support starts ///////////////////
   @Override
   public void promptLoop() {
     String line = null;
@@ -898,7 +1014,7 @@ public class Gfsh extends JLineShell {
       gfshHistory.setAutoFlush(false);
       // NOTE: Similar code is in executeScript()
       while (exitShellRequest == null && (line = readLine(reader, prompt)) != null) {
-        if (!line.endsWith(SyntaxConstants.CONTINUATION_CHARACTER)) { // see 45893
+        if (!line.endsWith(GfshParser.CONTINUATION_CHARACTER)) { // see 45893
           List<String> commandList = MultiCommandHelper.getMultipleCommands(line);
           for (String cmdLet : commandList) {
             String trimmedCommand = cmdLet.trim();
@@ -926,39 +1042,10 @@ public class Gfsh extends JLineShell {
     setShellStatus(Status.SHUTTING_DOWN);
   }
 
-  // See 46369
-  private static String readLine(ConsoleReader reader, String prompt) throws IOException {
-    String earlierLine = reader.getCursorBuffer().toString();
-    String readLine = null;
-    try {
-      readLine = reader.readLine(prompt);
-    } catch (IndexOutOfBoundsException e) {
-      if (earlierLine.length() == 0) {
-        reader.println();
-        readLine = LINE_SEPARATOR;
-        reader.getCursorBuffer().cursor = 0;
-      } else {
-        readLine = readLine(reader, prompt);
-      }
-    }
-    return readLine;
-  }
-
-  private static String removeBackslash(String result) {
-    if (result.endsWith(SyntaxConstants.CONTINUATION_CHARACTER)) {
-      result = result.substring(0, result.length() - 1);
-    }
-    return result;
-  }
-
   String getDefaultSecondaryPrompt() {
     return ansiHandler.decorateString(DEFAULT_SECONDARY_PROMPT, ANSIStyle.YELLOW);
   }
-  ///////////////// Providing Multiple-line support ends //////////////////////
 
-  /////////////// For setting shell environment properties END /////////////////
-
-  /////////////////////// OperationInvoker code START //////////////////////////
   public boolean isConnectedAndReady() {
     return operationInvoker != null && operationInvoker.isConnected() && operationInvoker.isReady();
   }
@@ -976,66 +1063,11 @@ public class Gfsh extends JLineShell {
   public void setOperationInvoker(final OperationInvoker operationInvoker) {
     this.operationInvoker = operationInvoker;
   }
-  //////////////////////// OperationInvoker code END //////////////////////////
-
-  //////////////////////// Fields for TestableShell Start //////////////////////
-  public static boolean SUPPORT_MUTLIPLESHELL = false;
-  protected static ThreadLocal<Gfsh> gfshThreadLocal = new ThreadLocal<Gfsh>();
-  //////////////////////// Fields for TestableShell End ////////////////////////
-
-  public static void redirectInternalJavaLoggers() {
-    // Do we need to this on re-connect?
-    LogManager logManager = LogManager.getLogManager();
-
-    try {
-      Enumeration<String> loggerNames = logManager.getLoggerNames();
-
-      while (loggerNames.hasMoreElements()) {
-        String loggerName = loggerNames.nextElement();
-        if (loggerName.startsWith("java.") || loggerName.startsWith("javax.")) {
-          // System.out.println(loggerName);
-          Logger javaLogger = logManager.getLogger(loggerName);
-          /*
-           * From Java Docs: It is also important to note that the Logger associated with the String
-           * name may be garbage collected at any time if there is no strong reference to the
-           * Logger. The caller of this method must check the return value for null in order to
-           * properly handle the case where the Logger has been garbage collected.
-           */
-          if (javaLogger != null) {
-            LogWrapper.getInstance().setParentFor(javaLogger);
-          }
-        }
-      }
-    } catch (SecurityException e) {
-      // e.printStackTrace();
-      LogWrapper.getInstance().warning(e.getMessage(), e);
-    }
-  }
-
-  public static Gfsh getCurrentInstance() {
-    if (!SUPPORT_MUTLIPLESHELL) {
-      return instance;
-    } else {
-      return gfshThreadLocal.get();
-    }
-  }
-
-  public String obtainHelp(String userInput, Set<String> requiredCommandNames) {
-    return parser.obtainHelp(userInput, requiredCommandNames);
-  }
-
-  public List<String> obtainHelpCommandNames(String userInput) {
-    return parser.obtainHelpCommandNames(userInput);
-  }
 
   public GfshConfig getGfshConfig() {
     return this.gfshConfig;
   }
 
-  public List<String> getCommandNames(String matchingWith) {
-    return parser.getCommandNames(matchingWith);
-  }
-
   @Override
   protected String getHistoryFileName() {
     return gfshConfig.getHistoryFileName();
@@ -1077,7 +1109,7 @@ public class Gfsh extends JLineShell {
       // contextPath = "." + CliConstants.DEFAULT_APP_CONTEXT_PATH;
     }
 
-    defaultPrompt = MessageFormat.format(defaultPrompt, new Object[] {clusterString, contextPath});
+    defaultPrompt = MessageFormat.format(defaultPrompt, clusterString, contextPath);
 
     return ansiHandler.decorateString(defaultPrompt, ANSIStyle.YELLOW);
   }
@@ -1120,114 +1152,6 @@ public class Gfsh extends JLineShell {
     }
     return output;
   }
-
-  private static String extractKey(String input) {
-    return input.substring("${".length(), input.length() - "}".length());
-  }
-
-  public static ConsoleReader getConsoleReader() {
-    Gfsh gfsh = Gfsh.getCurrentInstance();
-    return (gfsh == null ? null : gfsh.reader);
-  }
-
-  /**
-   * Take a string and wrap it into multiple lines separated by CliConstants.LINE_SEPARATOR. Lines
-   * are separated based upon the terminal width, separated on word boundaries and may have extra
-   * spaces added to provide indentation.
-   *
-   * For example: if the terminal width were 5 and the string "123 456789 01234" were passed in with
-   * an indentation level of 2, then the returned string would be:
-   *
-   * <pre>
-   *         123
-   *         45678
-   *         9
-   *         01234
-   * </pre>
-   *
-   * @param string String to wrap (add breakpoints and indent)
-   * @param indentationLevel The number of indentation levels to use.
-   * @return The wrapped string.
-   */
-  public static String wrapText(final String string, final int indentationLevel) {
-    Gfsh gfsh = getCurrentInstance();
-    if (gfsh == null) {
-      return string;
-    }
-
-    final int maxLineLength = gfsh.getTerminalWidth() - 1;
-    final StringBuffer stringBuf = new StringBuffer();
-    int index = 0;
-    int startOfCurrentLine = 0;
-    while (index < string.length()) {
-      // Add the indentation
-      for (int i = 0; i < indentationLevel; i++) {
-        stringBuf.append(LINE_INDENT);
-      }
-      int currentLineLength = LINE_INDENT.length() * indentationLevel;
-
-      // Find the end of a line:
-      // 1. If the end of string is reached
-      // 2. If the width of the terminal has been reached
-      // 3. If a newline character was found in the string
-      while (index < string.length() && currentLineLength < maxLineLength
-          && string.charAt(index) != '\n') {
-        index++;
-        currentLineLength++;
-      }
-
-      // If the line was terminated with a newline character
-      if (index != string.length() && string.charAt(index) == '\n') {
-        stringBuf.append(string.substring(startOfCurrentLine, index));
-        stringBuf.append(LINE_SEPARATOR);
-        index++;
-        startOfCurrentLine = index;
-
-        // If the end of the string was reached or the last character just happened to be a space
-        // character
-      } else if (index == string.length() || string.charAt(index) == ' ') {
-        stringBuf.append(string.substring(startOfCurrentLine, index));
-        if (index != string.length()) {
-          stringBuf.append(LINE_SEPARATOR);
-          index++;
-        }
-
-      } else {
-        final int spaceCharIndex = string.lastIndexOf(" ", index);
-
-        // If no spaces were found then there's no logical wayto split the string
-        if (spaceCharIndex == -1) {
-          stringBuf.append(string.substring(startOfCurrentLine, index)).append(LINE_SEPARATOR);
-
-          // Else split the string cleanly between words
-        } else {
-          stringBuf.append(string.substring(startOfCurrentLine, spaceCharIndex))
-              .append(LINE_SEPARATOR);
-          index = spaceCharIndex + 1;
-        }
-      }
-
-      startOfCurrentLine = index;
-    }
-    return stringBuf.toString();
-  }
-
-  // // for testing only
-  // public static void main(String[] args) {
-  // try {
-  // Gfsh gfsh = new Gfsh();
-  // String expandProperties = gfsh.expandProperties("execute function
-  // --id=group-with-arguments-with-result-collector
-  // --result-collector=management.operations.ops.FunctionOperations$CustomResultCollector
-  // --arguments=group-with-arguments-with-result-collector --group=managed1");
-  //// String expandProperties = gfsh.expandProperties("My name is ${NAME}");
-  // System.out.println(expandProperties);
-  // } catch (ClassNotFoundException e) {
-  // e.printStackTrace();
-  // } catch (IOException e) {
-  // e.printStackTrace();
-  // }
-  // }
 }
 
 

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
----------------------------------------------------------------------
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 d74f5d6..7c80e0d 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
@@ -72,12 +72,11 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
    */
   @Override
   public Object execute(ParseResult parseResult) {
-    Object result = null;
+    Result result = null;
     Method method = parseResult.getMethod();
     try {
       // Check if it's a multi-step command
-      Method reflectmethod = parseResult.getMethod();
-      MultiStepCommand cmd = reflectmethod.getAnnotation(MultiStepCommand.class);
+      MultiStepCommand cmd = method.getAnnotation(MultiStepCommand.class);
       if (cmd != null) {
         return execCLISteps(logWrapper, shell, parseResult);
       }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/MultiCommandHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/MultiCommandHelper.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/MultiCommandHelper.java
index 89f93d5..9eafff5 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/MultiCommandHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/MultiCommandHelper.java
@@ -14,22 +14,18 @@
  */
 package org.apache.geode.management.internal.cli.shell;
 
+import org.apache.geode.management.internal.cli.GfshParser;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
-
-/**
- * 
- *
- */
 public class MultiCommandHelper {
 
   public static List<String> getMultipleCommands(String input) {
     Map<Integer, List<String>> listMap = new HashMap<Integer, List<String>>();
-    String as[] = input.split(SyntaxConstants.COMMAND_DELIMITER);
+    String as[] = input.split(GfshParser.COMMAND_DELIMITER);
     int splitCount = 0;
     for (String a : as) {
       if (a.endsWith("\\")) {

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/jline/GfshHistory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/jline/GfshHistory.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/jline/GfshHistory.java
index dc4a42f..d0113af 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/jline/GfshHistory.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/jline/GfshHistory.java
@@ -14,13 +14,11 @@
  */
 package org.apache.geode.management.internal.cli.shell.jline;
 
+import jline.console.history.MemoryHistory;
+
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.geode.management.internal.cli.parser.preprocessor.PreprocessorUtils;
-
-import jline.console.history.MemoryHistory;
-
 /**
  * Overrides jline.History to add History without newline characters.
  * 
@@ -35,6 +33,13 @@ public class GfshHistory extends MemoryHistory {
   // let the history from history file get added initially
   private boolean autoFlush = true;
 
+  public static String redact(String buffer) {
+    String trimmed = buffer.trim();
+    Matcher matcher = passwordRe.matcher(trimmed);
+    String sanitized = matcher.replaceAll("$1*****");
+    return sanitized;
+  }
+
   public void addToHistory(String buffer) {
     if (isAutoFlush()) {
       super.add(redact(buffer));
@@ -48,12 +53,4 @@ public class GfshHistory extends MemoryHistory {
   public void setAutoFlush(boolean autoFlush) {
     this.autoFlush = autoFlush;
   }
-
-  public static String redact(String buffer) {
-    String trimmed = PreprocessorUtils.trim(buffer, false).getString();
-
-    Matcher matcher = passwordRe.matcher(trimmed);
-    String sanitized = matcher.replaceAll("$1*****");
-    return sanitized;
-  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/CommandStringBuilder.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/CommandStringBuilder.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/CommandStringBuilder.java
index 16efda5..4410fea 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/CommandStringBuilder.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/CommandStringBuilder.java
@@ -17,7 +17,6 @@ package org.apache.geode.management.internal.cli.util;
 import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.internal.lang.SystemUtils;
 import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
 
 
 /**
@@ -27,10 +26,10 @@ import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
  * @since GemFire 7.0
  */
 public class CommandStringBuilder {
-  private final String OPTION_MARKER = SyntaxConstants.LONG_OPTION_SPECIFIER;
-  private final String EQUAL_TO = SyntaxConstants.OPTION_VALUE_SPECIFIER;
-  private final String ARG_SEPARATOR = SyntaxConstants.OPTION_SEPARATOR;
-  private final String OPTION_SEPARATOR = SyntaxConstants.OPTION_SEPARATOR;
+  private final String OPTION_MARKER = GfshParser.LONG_OPTION_SPECIFIER;
+  private final String EQUAL_TO = GfshParser.OPTION_VALUE_SPECIFIER;
+  private final String ARG_SEPARATOR = GfshParser.OPTION_SEPARATOR;
+  private final String OPTION_SEPARATOR = GfshParser.OPTION_SEPARATOR;
   private final String SINGLE_SPACE = " ";
 
   private final StringBuffer buffer;
@@ -40,14 +39,13 @@ public class CommandStringBuilder {
     buffer = new StringBuffer(command);
   }
 
-  public CommandStringBuilder addArgument(String argument) {
-    if (hasOptions) {
-      throw new IllegalStateException(
-          "Arguments can't be specified after options. Built String is: " + buffer.toString());
+  private static String getLineSeparator() {
+    // Until TestableGfsh issue #46388 is resolved
+    if (SystemUtils.isWindows()) {
+      return "\r";
+    } else {
+      return GfshParser.LINE_SEPARATOR;
     }
-    buffer.append(ARG_SEPARATOR);
-    buffer.append(argument);
-    return this;
   }
 
   public CommandStringBuilder addOption(String option, String value) {
@@ -77,20 +75,10 @@ public class CommandStringBuilder {
 
   public CommandStringBuilder addNewLine() {
     buffer.append(SINGLE_SPACE); // add a space before continuation char
-    buffer.append(SyntaxConstants.CONTINUATION_CHARACTER);
     buffer.append(getLineSeparator());
     return this;
   }
 
-  private static String getLineSeparator() {
-    // Until TestableGfsh issue #46388 is resolved
-    if (SystemUtils.isWindows()) {
-      return "\r";
-    } else {
-      return GfshParser.LINE_SEPARATOR;
-    }
-  }
-
   public String getCommandString() {
     return buffer.toString();
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DataCommandsController.java
----------------------------------------------------------------------
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
index cfe6089..3a8ed82 100644
--- 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
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.management.internal.web.controllers;
 
-import java.util.concurrent.Callable;
-
 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;
@@ -28,6 +26,8 @@ import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 import org.springframework.web.context.request.WebRequest;
 
+import java.util.concurrent.Callable;
+
 /**
  * The DataCommandsController class implements GemFire Management REST API web service endpoints for
  * the Gfsh Data Commands.
@@ -115,7 +115,7 @@ public class DataCommandsController extends AbstractCommandsController {
     command.addOption(CliStrings.REMOVE__REGION, decode(regionNamePath));
     command.addOption(CliStrings.REMOVE__ALL, String.valueOf(allKeys));
 
-    if (hasValue(key)) {
+    if (key != null) {
       command.addOption(CliStrings.REMOVE__KEY, key);
     }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java
----------------------------------------------------------------------
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
index 527e059..604bdee 100644
--- 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
@@ -94,7 +94,7 @@ public class ExportLogController extends AbstractCommandsController {
   }
 
   ResponseEntity<InputStreamResource> getResponse(String result) {
-    // the result is json string from CommandResul
+    // the result is json string from CommandResult
     Result commandResult = ResultBuilder.fromJson(result);
     if (commandResult.getStatus().equals(Result.Status.OK)) {
       return getOKResponse(commandResult);

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/CommandManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/CommandManagerJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/CommandManagerJUnitTest.java
index 503ffb2..3189f9f 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/CommandManagerJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/CommandManagerJUnitTest.java
@@ -14,28 +14,17 @@
  */
 package org.apache.geode.management.internal.cli;
 
-import static org.junit.Assert.*;
-
-import java.lang.annotation.Annotation;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.ConverterHint;
 import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.annotation.CliArgument;
-import org.apache.geode.management.internal.cli.parser.Argument;
-import org.apache.geode.management.internal.cli.parser.AvailabilityTarget;
-import org.apache.geode.management.internal.cli.parser.CommandTarget;
-import org.apache.geode.management.internal.cli.parser.Option;
 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.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.shell.core.CommandMarker;
@@ -46,6 +35,8 @@ import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
+import java.util.List;
+
 /**
  * CommandManagerTest - Includes tests to check the CommandManager functions
  */
@@ -94,9 +85,11 @@ public class CommandManagerJUnitTest {
   private static final String OPTION3_UNSPECIFIED_DEFAULT_VALUE =
       "{unspecified default value for option3}";
 
-  @After
-  public void tearDown() {
-    CommandManager.clearInstance();
+  private CommandManager commandManager;
+
+  @Before
+  public void before() {
+    commandManager = new CommandManager();
   }
 
   /**
@@ -104,9 +97,9 @@ public class CommandManagerJUnitTest {
    */
   @Test
   public void testCommandManagerLoadCommands() throws Exception {
-    CommandManager commandManager = CommandManager.getInstance(true);
     assertNotNull(commandManager);
-    assertNotSame(0, commandManager.getCommands().size());
+    assertThat(commandManager.getCommandMarkers().size()).isGreaterThan(0);
+    assertThat(commandManager.getConverters().size()).isGreaterThan(0);
   }
 
   /**
@@ -114,125 +107,23 @@ public class CommandManagerJUnitTest {
    */
   @Test
   public void testCommandManagerInstance() throws Exception {
-    CommandManager commandManager = CommandManager.getInstance(true);
     assertNotNull(commandManager);
   }
 
   /**
-   * tests createOption method for creating option
-   */
-  @Test
-  public void testCommandManagerCreateOption() throws Exception {
-    CommandManager commandManager = CommandManager.getInstance(true);
-    assertNotNull(commandManager);
-
-    Method method = Commands.class.getMethod(COMMAND1_NAME, String.class, String.class,
-        String.class, String.class, String.class);
-
-    Annotation[][] annotations = method.getParameterAnnotations();
-    Class<?>[] parameterTypes = method.getParameterTypes();
-    List<String> optionNames = new ArrayList<String>();
-    optionNames.add(OPTION1_NAME);
-    optionNames.add(OPTION2_NAME);
-    optionNames.add(OPTION3_NAME);
-
-    int parameterNo = 0;
-    for (int i = 0; i < annotations.length; i++) {
-      Annotation[] annotation = annotations[i];
-      for (Annotation ann : annotation) {
-        if (ann instanceof CliOption) {
-          Option createdOption =
-              commandManager.createOption((CliOption) ann, parameterTypes[i], parameterNo);
-          assertTrue(optionNames.contains(createdOption.getLongOption()));
-          assertEquals(((CliOption) ann).help(), createdOption.getHelp());
-          if (((CliOption) ann).specifiedDefaultValue() != null
-              && ((CliOption) ann).specifiedDefaultValue().length() > 0) {
-            assertEquals(((CliOption) ann).specifiedDefaultValue().trim(),
-                createdOption.getSpecifiedDefaultValue().trim());
-          }
-          if (((CliOption) ann).unspecifiedDefaultValue() != null
-              && ((CliOption) ann).unspecifiedDefaultValue().length() > 0) {
-            assertEquals(((CliOption) ann).specifiedDefaultValue().trim(),
-                createdOption.getSpecifiedDefaultValue().trim());
-          }
-
-        }
-      }
-    }
-  }
-
-  /**
-   * tests createArgument method for creating argument
-   */
-  @Test
-  public void testCommandManagerCreateArgument() throws Exception {
-    CommandManager commandManager = CommandManager.getInstance(true);
-    assertNotNull(commandManager);
-
-    Method method = Commands.class.getMethod(COMMAND1_NAME, String.class, String.class,
-        String.class, String.class, String.class);
-
-    Annotation[][] annotations = method.getParameterAnnotations();
-    Class<?>[] parameterTypes = method.getParameterTypes();
-    List<String> argumentList = new ArrayList<String>();
-    argumentList.add(ARGUMENT1_NAME);
-    argumentList.add(ARGUMENT2_NAME);
-
-    int parameterNo = 0;
-    for (int i = 0; i < annotations.length; i++) {
-      Annotation[] annotation = annotations[i];
-      for (Annotation ann : annotation) {
-        if (ann instanceof CliArgument) {
-          Argument arg =
-              commandManager.createArgument((CliArgument) ann, parameterTypes[i], parameterNo);
-          assertEquals(true, argumentList.contains(arg.getArgumentName()));
-          assertEquals(((CliArgument) ann).mandatory(), arg.isRequired());
-          assertEquals(((CliArgument) ann).name().trim(), arg.getArgumentName().trim());
-          assertEquals(((CliArgument) ann).argumentContext().trim(), arg.getContext().trim());
-          assertEquals(((CliArgument) ann).help().trim(), arg.getHelp().trim());
-        }
-      }
-    }
-  }
-
-  /**
-   * tests availabilityIndicator for a method
-   */
-  @Test
-  public void testCommandManagerAvailabilityIndicator() throws Exception {
-    CommandManager commandManager = CommandManager.getInstance(true);
-    assertNotNull(commandManager);
-    commandManager.add(Commands.class.newInstance());
-    Map<String, CommandTarget> commands = commandManager.getCommands();
-    for (String commandName : commands.keySet()) {
-      if (commandName.equals(COMMAND1_NAME)) {
-        CommandTarget commandTarget = commands.get(commandName);
-        AvailabilityTarget availabilityIndicator = commandTarget.getAvailabilityIndicator();
-        if (availabilityIndicator == null) {
-          availabilityIndicator = commandManager.getAvailabilityIndicator(COMMAND1_NAME);
-          commandTarget.setAvailabilityIndicator(availabilityIndicator);
-        }
-        assertEquals(true, commandTarget.isAvailable());
-        break;
-      }
-    }
-  }
-
-  /**
    * Tests {@link CommandManager#loadPluginCommands()}.
    * 
    * @since GemFire 8.1
    */
   @Test
   public void testCommandManagerLoadPluginCommands() throws Exception {
-    CommandManager commandManager = CommandManager.getInstance(true);
     assertNotNull(commandManager);
 
     // see META-INF/services/org.springframework.shell.core.CommandMarker service loader file.
     assertTrue("Should find listed plugin.",
-        commandManager.getCommands().containsKey("mock plugin command"));
+        commandManager.getHelper().getCommands().contains("mock plugin command"));
     assertTrue("Should not find unlisted plugin.",
-        !commandManager.getCommands().containsKey("mock plugin command unlisted"));
+        !commandManager.getHelper().getCommands().contains("mock plugin command unlisted"));
   }
 
   /**
@@ -244,11 +135,10 @@ public class CommandManagerJUnitTest {
     @CliMetaData(shellOnly = true, relatedTopic = {"relatedTopicOfCommand1"})
     @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
     public static String command1(
-        @CliArgument(name = ARGUMENT1_NAME, argumentContext = ARGUMENT1_CONTEXT,
-            help = ARGUMENT1_HELP, mandatory = true) String argument1,
-        @CliArgument(name = ARGUMENT2_NAME, argumentContext = ARGUMENT2_CONTEXT,
-            help = ARGUMENT2_HELP, mandatory = false,
-            unspecifiedDefaultValue = ARGUMENT2_UNSPECIFIED_DEFAULT_VALUE,
+        @CliOption(key = ARGUMENT1_NAME, optionContext = ARGUMENT1_CONTEXT, help = ARGUMENT1_HELP,
+            mandatory = true) String argument1,
+        @CliOption(key = ARGUMENT2_NAME, optionContext = ARGUMENT2_CONTEXT, help = ARGUMENT2_HELP,
+            mandatory = false, unspecifiedDefaultValue = ARGUMENT2_UNSPECIFIED_DEFAULT_VALUE,
             systemProvided = false) String argument2,
         @CliOption(key = {OPTION1_NAME, OPTION1_SYNONYM}, help = OPTION1_HELP, mandatory = true,
             optionContext = OPTION1_CONTEXT,
@@ -272,18 +162,16 @@ public class CommandManagerJUnitTest {
     @CliCommand(value = {"testParamConcat"})
     @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
     public static Result testParamConcat(@CliOption(key = {"string"}) String string,
-        @CliOption(key = {"stringArray"}) @CliMetaData(valueSeparator = ",") String[] stringArray,
-        @CliOption(key = {"stringList"}, optionContext = ConverterHint.STRING_LIST) @CliMetaData(
-            valueSeparator = ",") List<String> stringList,
+        @CliOption(key = {"stringArray"}) String[] stringArray,
         @CliOption(key = {"integer"}) Integer integer,
-        @CliOption(key = {"colonArray"}) @CliMetaData(valueSeparator = ":") String[] colonArray) {
+        @CliOption(key = {"colonArray"}) String[] colonArray) {
       return null;
     }
 
     @CliCommand(value = {"testMultiWordArg"})
     @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public static Result testMultiWordArg(@CliArgument(name = "arg1") String arg1,
-        @CliArgument(name = "arg2") String arg2) {
+    public static Result testMultiWordArg(@CliOption(key = "arg1") String arg1,
+        @CliOption(key = "arg2") String arg2) {
       return null;
     }
 
@@ -300,10 +188,7 @@ public class CommandManagerJUnitTest {
 
     @Override
     public boolean supports(Class<?> type, String optionContext) {
-      if (type.isAssignableFrom(String.class)) {
-        return true;
-      }
-      return false;
+      return type.isAssignableFrom(String.class);
     }
 
     @Override


[3/9] geode git commit: GEODE-1597: use Spring shell's parser and delete our own parsing code

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserIntegrationTest.java
index f3e3bd8..a467b34 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserIntegrationTest.java
@@ -14,39 +14,43 @@
  */
 package org.apache.geode.management.internal.cli;
 
-import static org.assertj.core.api.Assertions.*;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
-import java.util.Arrays;
-import java.util.Map;
-
-import org.junit.After;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.test.junit.categories.IntegrationTest;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.springframework.shell.core.Completion;
 import org.springframework.shell.event.ParseResult;
 
-import org.apache.geode.test.junit.categories.IntegrationTest;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 
 @Category(IntegrationTest.class)
 public class GfshParserIntegrationTest {
-
-  private CommandManager commandManager;
-  private GfshParser parser;
+  private static GfshParser parser;
+  private List<Completion> candidates;
+  private String buffer;
+  private int cursor;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    parser = new GfshParser();
+  }
 
   @Before
   public void setUp() throws Exception {
-    CommandManager.clearInstance();
-    this.commandManager = CommandManager.getInstance(true);
-    this.parser = new GfshParser(commandManager);
+    this.candidates = new ArrayList<>();
   }
 
-  @After
-  public void tearDown() {
-    CommandManager.clearInstance();
-  }
-
-  private Map<String, String> params(String input, String commandName, String commandMethod) {
+  private Map<String, String> parseParams(String input, String commandMethod) {
     ParseResult parseResult = parser.parse(input);
+
     GfshParseResult gfshParseResult = (GfshParseResult) parseResult;
     Map<String, String> params = gfshParseResult.getParamValueStrings();
     for (String param : params.keySet()) {
@@ -55,108 +59,425 @@ public class GfshParserIntegrationTest {
 
     assertThat(gfshParseResult.getMethod().getName()).isEqualTo(commandMethod);
     assertThat(gfshParseResult.getUserInput()).isEqualTo(input.trim());
-    assertThat(gfshParseResult.getCommandName()).isEqualTo(commandName);
 
     return params;
   }
 
   @Test
-  public void optionStartsWithHyphenWithoutQuotes() throws Exception {
+  public void getSimpleParserInputTest() {
+    buffer = "start locator  --J=\"-Dgemfire.http-service-port=8080\" --name=loc1";
+    assertEquals("start locator --J \"-Dgemfire.http-service-port=8080\" --name loc1",
+        GfshParser.convertToSimpleParserInput(buffer));
+
+    buffer = "start locator --J=-Dgemfire.http-service-port=8080 --name=loc1 --J=-Ddummythinghere";
+    assertEquals(
+        "start locator --J \"-Dgemfire.http-service-port=8080,-Ddummythinghere\" --name loc1",
+        GfshParser.convertToSimpleParserInput(buffer));
+
+    buffer = "start locator --";
+    assertThat(GfshParser.convertToSimpleParserInput(buffer)).isEqualTo("start locator --");
+
+    buffer =
+        "start locator --J=-Dgemfire.http-service-port=8080 --name=loc1 --J=-Ddummythinghere --";
+    assertEquals(
+        "start locator --J \"-Dgemfire.http-service-port=8080,-Ddummythinghere\" --name loc1 --",
+        GfshParser.convertToSimpleParserInput(buffer));
+
+    buffer = "start server --name=name1 --locators=localhost --J=-Dfoo=bar";
+    assertEquals("start server --name name1 --locators localhost --J \"-Dfoo=bar\"",
+        GfshParser.convertToSimpleParserInput(buffer));
+  }
+
+  @Test
+  public void testParseOptionStartsWithHyphenWithoutQuotes() throws Exception {
     String input =
         "rebalance --exclude-region=/GemfireDataCommandsDUnitTestRegion2 --simulate=true --time-out=-1";
-    Map<String, String> params = params(input, "rebalance", "rebalance");
-
+    Map<String, String> params = parseParams(input, "rebalance");
     assertThat(params.get("exclude-region")).isEqualTo("/GemfireDataCommandsDUnitTestRegion2");
     assertThat(params.get("simulate")).isEqualTo("true");
-    assertThat(params.get("time-out")).isEqualTo("\"-1\"");
+    assertThat(params.get("time-out")).isEqualTo("-1");
   }
 
   @Test
-  public void optionStartsWithHyphenWithQuotes() throws Exception {
+  public void testParseOptionStartsWithHyphenWithQuotes() throws Exception {
     String input =
         "rebalance --exclude-region=/GemfireDataCommandsDUnitTestRegion2 --simulate=true --time-out=\"-1\"";
-    Map<String, String> params = params(input, "rebalance", "rebalance");
+    Map<String, String> params = parseParams(input, "rebalance");
 
     assertThat(params.get("exclude-region")).isEqualTo("/GemfireDataCommandsDUnitTestRegion2");
     assertThat(params.get("simulate")).isEqualTo("true");
-    assertThat(params.get("time-out")).isEqualTo("\"-1\"");
+    assertThat(params.get("time-out")).isEqualTo("-1");
   }
 
   @Test
-  public void optionContainingHyphen() throws Exception {
+  public void testParseOptionContainingHyphen() throws Exception {
     String input = "rebalance --exclude-region=/The-Region --simulate=true";
-    Map<String, String> params = params(input, "rebalance", "rebalance");
+    Map<String, String> params = parseParams(input, "rebalance");
 
     assertThat(params.get("exclude-region")).isEqualTo("/The-Region");
     assertThat(params.get("simulate")).isEqualTo("true");
   }
 
   @Test
-  public void optionContainingUnderscore() throws Exception {
+  public void testParseOptionContainingUnderscore() throws Exception {
     String input = "rebalance --exclude-region=/The_region --simulate=true";
-    Map<String, String> params = params(input, "rebalance", "rebalance");
+    Map<String, String> params = parseParams(input, "rebalance");
 
     assertThat(params.get("exclude-region")).isEqualTo("/The_region");
     assertThat(params.get("simulate")).isEqualTo("true");
   }
 
   @Test
-  public void oneJOptionWithQuotes() throws Exception {
+  public void testParseOneJOptionWithQuotes() throws Exception {
     String input = "start locator  --J=\"-Dgemfire.http-service-port=8080\" --name=loc1";
-    Map<String, String> params = params(input, "start locator", "startLocator");
+    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
-  public void oneJOptionWithSpaceInQuotes() throws Exception {
+  public void testParseOneJOptionWithSpaceInQuotes() throws Exception {
     String input = "start locator  --J=\"-Dgemfire.http-service-port= 8080\" --name=loc1";
-    Map<String, String> params = params(input, "start locator", "startLocator");
+    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
-  public void oneJOption() throws Exception {
+  public void testParseOneJOption() throws Exception {
     String input = "start locator --J=-Dgemfire.http-service-port=8080 --name=loc1";
-    Map<String, String> params = params(input, "start locator", "startLocator");
+    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
-  public void twoJOptions() throws Exception {
+  public void testParseTwoJOptions() throws Exception {
     String input =
         "start locator --J=-Dgemfire.http-service-port=8080 --name=loc1 --J=-Ddummythinghere";
-    Map<String, String> params = params(input, "start locator", "startLocator");
+    Map<String, String> params = parseParams(input, "startLocator");
 
     assertThat(params.get("name")).isEqualTo("loc1");
-    assertThat(params.get("J"))
-        .isEqualTo("\"-Dgemfire.http-service-port=8080\",\"-Ddummythinghere\"");
+    assertThat(params.get("J")).isEqualTo("-Dgemfire.http-service-port=8080,-Ddummythinghere");
   }
 
   @Test
-  public void twoJOptionsOneWithQuotesOneWithout() throws Exception {
+  public void testParseTwoJOptionsOneWithQuotesOneWithout() throws Exception {
     String input =
         "start locator --J=\"-Dgemfire.http-service-port=8080\" --name=loc1 --J=-Ddummythinghere";
-    Map<String, String> params = params(input, "start locator", "startLocator");
+    Map<String, String> params = parseParams(input, "startLocator");
 
     assertThat(params.get("name")).isEqualTo("loc1");
-    assertThat(params.get("J"))
-        .isEqualTo("\"-Dgemfire.http-service-port=8080\",\"-Ddummythinghere\"");
+    assertThat(params.get("J")).isEqualTo("-Dgemfire.http-service-port=8080,-Ddummythinghere");
   }
 
   @Test
-  public void oneJOptionWithQuotesAndLotsOfSpaces() throws Exception {
+  public void testParseOneJOptionWithQuotesAndLotsOfSpaces() throws Exception {
     String input =
         "start locator       --J=\"-Dgemfire.http-service-port=8080\"      --name=loc1         ";
-    Map<String, String> params = params(input, "start locator", "startLocator");
+    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
+  public void testCompletionDescibe() throws Exception {
+    buffer = "describe";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(7);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo("describe client");
+  }
+
+  @Test
+  public void testCompletionDescibeWithSpace() throws Exception {
+    buffer = "describe ";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(7);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo("describe client");
+  }
+
+  @Test
+  public void testCompletionDeploy() throws Exception {
+    buffer = "deploy";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(3);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + " --dir");
+  }
+
+  @Test
+  public void testCompletionDeployWithSpace() throws Exception {
+    buffer = "deploy ";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(3);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + "--dir");
+  }
+
+  @Test
+  public void testCompleteWithRequiredOption() throws Exception {
+    buffer = "describe config";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(1);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + " --member");
+  }
+
+  @Test
+  public void testCompleteWithRequiredOptionWithSpace() throws Exception {
+    buffer = "describe config ";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(1);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + "--member");
+  }
+
+  @Test
+  public void testCompleteCommand() throws Exception {
+    buffer = "start ser";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(1);
+    assertThat("start server").isEqualTo(getCompleted(buffer, cursor, candidates.get(0)));
+  }
+
+  @Test
+  public void testCompleteOptionWithOnlyOneCandidate() throws Exception {
+    buffer = "start server --nam";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(1);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + "e");
+  }
+
+  @Test
+  public void testCompleteOptionWithMultipleCandidates() throws Exception {
+    buffer = "start server --name=jinmei --loc";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(3);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0)))
+        .isEqualTo(buffer + "ator-wait-time");
+    assertThat(getCompleted(buffer, cursor, candidates.get(1))).isEqualTo(buffer + "ators");
+    assertThat(getCompleted(buffer, cursor, candidates.get(2))).isEqualTo(buffer + "k-memory");
+  }
+
+  @Test
+  public void testCompleteWithExtraSpace() throws Exception {
+    buffer = "start server --name=name1  --se";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo("start server --name=name1  ".length());
+    assertThat(candidates.size()).isEqualTo(3);
+    assertTrue(candidates.contains(new Completion("--server-port")));
+    assertThat(getCompleted(buffer, cursor, candidates.get(0)))
+        .isEqualTo(buffer + "curity-properties-file");
+  }
+
+  @Test
+  public void testCompleteWithDashInTheEnd() throws Exception {
+    buffer = "start server --name=name1 --";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo(buffer.length() - 2);
+    assertThat(candidates.size()).isEqualTo(50);
+    assertTrue(candidates.contains(new Completion("--properties-file")));
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + "J");
+  }
+
+  @Test
+  public void testCompleteWithSpace() throws Exception {
+    buffer = "start server --name=name1 ";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo(buffer.length() - 1);
+    assertThat(candidates.size()).isEqualTo(50);
+    assertTrue(candidates.contains(new Completion(" --properties-file")));
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + "--J");
+  }
+
+  @Test
+  public void testCompleteWithOutSpace() throws Exception {
+    buffer = "start server --name=name1";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo(buffer.length());
+    assertThat(candidates.size()).isEqualTo(50);
+    assertTrue(candidates.contains(new Completion(" --properties-file")));
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + " --J");
+  }
+
+  @Test
+  public void testCompleteJ() throws Exception {
+    buffer = "start server --name=name1 --J=";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo(buffer.length());
+    assertThat(candidates.size()).isEqualTo(0);
+  }
+
+  @Test
+  public void testCompleteWithValue() throws Exception {
+    buffer = "start server --name=name1 --J";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo(buffer.length());
+    assertThat(candidates.size()).isEqualTo(0);
+  }
+
+  @Test
+  public void testCompleteWithDash() throws Exception {
+    buffer = "start server --name=name1 --J=-Dfoo.bar --";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(49);
+  }
+
+  @Test
+  public void testCompleteWithMultipleJ() throws Exception {
+    buffer = "start server --name=name1 --J=-Dme=her --J=-Dfoo=bar --l";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo("start server --name=name1 --J=-Dme=her --J=-Dfoo=bar ".length());
+    assertThat(candidates.size()).isEqualTo(4);
+    assertTrue(candidates.contains(new Completion("--locators")));
+  }
+
+  @Test
+  public void testMultiJComplete() throws Exception {
+    buffer = "start server --name=name1 --J=-Dtest=test1 --J=-Dfoo=bar";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo(buffer.length());
+    assertThat(candidates.size()).isEqualTo(49);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0)))
+        .isEqualTo(buffer + " --assign-buckets");
+  }
+
+  @Test
+  public void testMultiJCompleteWithDifferentOrder() throws Exception {
+    buffer = "start server --J=-Dtest=test1 --J=-Dfoo=bar --name=name1";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo(buffer.length());
+    assertThat(candidates.size()).isEqualTo(49);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0)))
+        .isEqualTo(buffer + " --assign-buckets");
+  }
+
+  @Test
+  public void testJComplete3() throws Exception {
+    buffer = "start server --name=name1 --locators=localhost --J=-Dfoo=bar";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo(buffer.length());
+    assertThat(candidates.size()).isEqualTo(48);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0)))
+        .isEqualTo(buffer + " --assign-buckets");
+  }
+
+  @Test
+  public void testJComplete4() throws Exception {
+    buffer = "start server --name=name1 --locators=localhost  --J=-Dfoo=bar --";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(cursor).isEqualTo(buffer.length() - 2);
+    assertThat(candidates.size()).isEqualTo(48);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0)))
+        .isEqualTo(buffer + "assign-buckets");
+  }
+
+  @Test
+  public void testCompletRegionType() throws Exception {
+    buffer = "create region --name=test --type";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(23);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + "=LOCAL");
+  }
+
+  @Test
+  public void testCompletPartialRegionType() throws Exception {
+    buffer = "create region --name=test --type=LO";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(5);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + "CAL");
+  }
+
+  @Test
+  public void testCompletHelp() throws Exception {
+    buffer = "create region --name=test --type LO";
+    cursor = parser.completeSuperAdvanced(buffer, candidates);
+    System.out.println("");
+  }
+
+  @Test
+  public void testCompletLogLevel() throws Exception {
+    buffer = "change loglevel --loglevel";
+    cursor = parser.completeAdvanced(buffer, candidates);
+    assertThat(candidates.size()).isEqualTo(8);
+    assertThat(getCompleted(buffer, cursor, candidates.get(0))).isEqualTo(buffer + "=ALL");
+  }
+
+  @Test
+  public void testObtainHelp() {
+    String command = CliStrings.START_PULSE;
+    String helpString = "NAME\n" + "start pulse\n" + "IS AVAILABLE\n" + "true\n" + "SYNOPSIS\n"
+        + "Open a new window in the default Web browser with the URL for the Pulse application.\n"
+        + "SYNTAX\n" + "start pulse [--url=value]\n" + "PARAMETERS\n" + "url\n"
+        + "URL of the Pulse Web application.\n" + "Required: false\n"
+        + "Default (if the parameter is not specified): http://localhost:7070/pulse\n";
+    assertThat(parser.getCommandManager().obtainHelp(command)).isEqualTo(helpString);
+  }
+
+
+  @Test
+  public void testStringArrayConverter() {
+    String command = "create disk-store --name=foo --dir=bar";
+    GfshParseResult result = parser.parse(command);
+    assertThat(result).isNotNull();
+    assertThat(result.getParamValue("dir")).isEqualTo("bar");
+  }
+
+  @Test
+  public void testDirConverter() {
+    String command = "compact offline-disk-store --name=foo --disk-dirs=bar";
+    GfshParseResult result = parser.parse(command);
+    assertThat(result).isNotNull();
+    assertThat(result.getParamValue("disk-dirs")).isEqualTo("bar");
+  }
+
+  @Test
+  public void testMultiDirInvalid() throws Exception {
+    String command = "create disk-store --name=testCreateDiskStore1 --group=Group1 "
+        + "--allow-force-compaction=true --auto-compact=false --compaction-threshold=67 "
+        + "--max-oplog-size=355 --queue-size=5321 --time-interval=2023 --write-buffer-size=3110 "
+        + "--dir=/testCreateDiskStore1.1#1452637463 " + "--dir=/testCreateDiskStore1.2";
+    GfshParseResult result = parser.parse(command);
+    assertThat(result).isNull();
+  }
+
+  @Test
+  public void testMultiDirValid() throws Exception {
+    String command = "create disk-store --name=testCreateDiskStore1 --group=Group1 "
+        + "--allow-force-compaction=true --auto-compact=false --compaction-threshold=67 "
+        + "--max-oplog-size=355 --queue-size=5321 --time-interval=2023 --write-buffer-size=3110 "
+        + "--dir=/testCreateDiskStore1.1#1452637463,/testCreateDiskStore1.2";
+    GfshParseResult result = parser.parse(command);
+    assertThat(result).isNotNull();
+    assertThat(result.getParamValue("dir"))
+        .isEqualTo("/testCreateDiskStore1.1#1452637463,/testCreateDiskStore1.2");
+  }
+
+  @Test
+  public void testEmptyKey() throws Exception {
+    String command = "remove  --key=\"\" --region=/GemfireDataCommandsTestRegion";
+    GfshParseResult result = parser.parse(command);
+    assertThat(result).isNotNull();
+    assertThat(result.getParamValue("key")).isEqualTo("");
+  }
+
+  @Test
+  public void testJsonKey() throws Exception {
+    String command = "get --key=('id':'testKey0') --region=regionA";
+    GfshParseResult result = parser.parse(command);
+    assertThat(result).isNotNull();
+  }
+
+  @Test
+  public void testUnspecifiedValueToStringArray() {
+    String command = "change loglevel --loglevel=finer --groups=group1,group2";
+    ParseResult result = parser.parse(command);
+    String[] memberIdValue = (String[]) result.getArguments()[0];
+    assertThat(memberIdValue).isNull();
+  }
+
+  private String getCompleted(String buffer, int cursor, Completion completed) {
+    return buffer.substring(0, cursor) + completed.getValue();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/1fc0f0ca/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserJUnitTest.java
index 44e99f4..2fd8c2f 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserJUnitTest.java
@@ -14,41 +14,16 @@
  */
 package org.apache.geode.management.internal.cli;
 
-import static org.junit.Assert.*;
+import static org.assertj.core.api.Assertions.assertThat;
 
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.After;
+import org.apache.geode.test.junit.categories.UnitTest;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.springframework.shell.core.CommandMarker;
-import org.springframework.shell.core.Completion;
-import org.springframework.shell.core.Converter;
-import org.springframework.shell.core.MethodTarget;
-import org.springframework.shell.core.Parser;
-import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-import org.springframework.shell.event.ParseResult;
 
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.CommandProcessingException;
-import org.apache.geode.management.cli.ConverterHint;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.annotation.CliArgument;
-import org.apache.geode.management.internal.cli.converters.StringArrayConverter;
-import org.apache.geode.management.internal.cli.converters.StringListConverter;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.parser.SyntaxConstants;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-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.geode.test.junit.categories.UnitTest;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 
 /**
  * GfshParserJUnitTest - Includes tests to check the parsing and auto-completion capabilities of
@@ -57,807 +32,103 @@ import org.apache.geode.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class GfshParserJUnitTest {
 
-  private static final String COMMAND1_NAME = "command1";
-  private static final String COMMAND1_NAME_ALIAS = "command1_alias";
-  private static final String COMMAND2_NAME = "c2";
-
-  private static final String COMMAND1_HELP = "help for " + COMMAND1_NAME;
-
-  // ARGUMENTS
-  private static final String ARGUMENT1_NAME = "argument1";
-  private static final String ARGUMENT1_HELP = "help for argument1";
-  private static final String ARGUMENT1_CONTEXT = "context for argument 1";
-  private static final Completion[] ARGUMENT1_COMPLETIONS =
-      {new Completion("arg1"), new Completion("arg1alt")};
-  private static final String ARGUMENT2_NAME = "argument2";
-  private static final String ARGUMENT2_CONTEXT = "context for argument 2";
-  private static final String ARGUMENT2_HELP = "help for argument2";
-  private static final String ARGUMENT2_UNSPECIFIED_DEFAULT_VALUE =
-      "{unspecified default value for argument2}";
-  private static final Completion[] ARGUMENT2_COMPLETIONS =
-      {new Completion("arg2"), new Completion("arg2alt")};
-
-  // OPTIONS
-  private static final String OPTION1_NAME = "option1";
-  private static final String OPTION1_SYNONYM = "opt1";
-  private static final String OPTION1_HELP = "help for option1";
-  private static final String OPTION1_CONTEXT = "context for option1";
-  private static final Completion[] OPTION1_COMPLETIONS =
-      {new Completion("option1"), new Completion("option1Alternate")};
-  private static final String OPTION2_NAME = "option2";
-  private static final String OPTION2_HELP = "help for option2";
-  private static final String OPTION2_CONTEXT = "context for option2";
-  private static final String OPTION2_SPECIFIED_DEFAULT_VALUE =
-      "{specified default value for option2}";
-  private static final Completion[] OPTION2_COMPLETIONS =
-      {new Completion("option2"), new Completion("option2Alternate")};
-  private static final String OPTION3_NAME = "option3";
-  private static final String OPTION3_SYNONYM = "opt3";
-  private static final String OPTION3_HELP = "help for option3";
-  private static final String OPTION3_CONTEXT = "context for option3";
-  private static final String OPTION3_UNSPECIFIED_DEFAULT_VALUE =
-      "{unspecified default value for option3}";
-  private static final Completion[] OPTION3_COMPLETIONS =
-      {new Completion("option3"), new Completion("option3Alternate")};
-
-  private Method methodCommand1;
-  private Method methodTestParamConcat;
-  private Method methodTestMultiWordArg;
-
-  private CommandManager commandManager;
-  private GfshParser parser;
+  private String input;
+  private List<String> tokens;
 
   @Before
-  public void setUp() throws Exception {
-    methodCommand1 = Commands.class.getMethod("command1", String.class, String.class, String.class,
-        String.class, String.class);
-    methodTestParamConcat = Commands.class.getMethod("testParamConcat", String.class,
-        String[].class, List.class, Integer.class, String[].class);
-    methodTestMultiWordArg =
-        Commands.class.getMethod("testMultiWordArg", String.class, String.class);
-
-    // Make sure no prior tests leave the CommandManager in a funky state
-    CommandManager.clearInstance();
-
-    commandManager = CommandManager.getInstance(false);
-    commandManager.add(Commands.class.newInstance());
-    commandManager.add(SimpleConverter.class.newInstance());
-    commandManager.add(StringArrayConverter.class.newInstance());
-    commandManager.add(StringListConverter.class.newInstance());
-
-    // Set up the parser
-    parser = new GfshParser(commandManager);
-
-    CliUtil.isGfshVM = false;
+  public void before() {
+    tokens = new ArrayList<>();
   }
 
-  @After
-  public void tearDown() {
-    CommandManager.clearInstance();
-  }
-
-  /**
-   * Tests the auto-completion capability of {@link GfshParser} with the method
-   * {@link GfshParser#complete(String, int, List)}
-   */
   @Test
-  public void testComplete() throws Exception {
-    // Get the names of the command
-    String[] command1Names = ((CliCommand) methodCommand1.getAnnotation(CliCommand.class)).value();
-
-    // Input contains an entirely different string
-    String input = "moc";
-    List<String> completionCandidates = new ArrayList<String>();
-    List<String> completionValues = new ArrayList<String>();
-    parser.complete(input, input.length(), completionCandidates);
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains a string which is prefix
-    // of more than 1 command
-    input = "c";
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    // completions will come alphabetically sorted
-    completionValues.add(COMMAND2_NAME);
-    completionValues.add(COMMAND1_NAME);
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains only prefix of the command
-    // name which is not a prefix of other command.
-    // It may be the prefix for the synonym of command
-    input = command1Names[0].substring(0, 3);
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    completionValues.add(COMMAND1_NAME);
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains only the command name
-    input = command1Names[0];
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the completions for argument1
-    // For arguments, the formatted value will equal the actual arguments
-    // But the actual value will contain the ARGUMENT_SEPARATOR
-    for (Completion completion : ARGUMENT1_COMPLETIONS) {
-      completionValues.add(" " + completion.getValue());
-    }
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name and prefix of first
-    // argument
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue().substring(0, 3);
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the completions for argument2
-    // which have the provided first argument as the prefix
-    for (Completion completion : ARGUMENT1_COMPLETIONS) {
-      if (completion.getValue().startsWith(ARGUMENT1_COMPLETIONS[0].getValue().substring(0, 3))) {
-        completionValues.add(" " + completion.getValue());
-      }
-    }
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument
-    // and first option
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION1_NAME;
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the values for the first option
-    for (Completion completion : OPTION1_COMPLETIONS) {
-      completionValues.add(SyntaxConstants.OPTION_VALUE_SPECIFIER + completion.getValue());
-    }
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option and prefix of one of the values provided
-    // by the auto-completor.
-    input = command1Names[1] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER
-        + OPTION1_COMPLETIONS[0].getValue().substring(0, 2);
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the values for the first option
-    for (Completion completion : OPTION1_COMPLETIONS) {
-      if (completion.getValue().startsWith(OPTION1_COMPLETIONS[0].getValue().substring(0, 2))) {
-        completionValues.add(SyntaxConstants.OPTION_VALUE_SPECIFIER + completion.getValue());
-      }
-    }
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option and one of the values provided
-    // by the auto-completor.
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER + OPTION1_COMPLETIONS[0].getValue();
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the remaining options
-    // As only first option is mandatory, we expect the
-    // the other non-mandatory options.
-    completionValues.add(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION2_NAME);
-    completionValues.add(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION3_NAME);
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option, one value for the option and value separator at
-    // the end
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER + OPTION1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.VALUE_SEPARATOR;
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the other values for completion
-    completionValues.add(SyntaxConstants.VALUE_SEPARATOR + OPTION1_COMPLETIONS[1].getValue());
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option and both the values for the option
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER + OPTION1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.VALUE_SEPARATOR + OPTION1_COMPLETIONS[1].getValue();
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the remaining options
-    // As only first option is mandatory, we expect the
-    // the other non-mandatory options.
-    completionValues.add(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION2_NAME);
-    completionValues.add(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION3_NAME);
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option, both the values for the option and valueSeparator
-    // at the end
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER + OPTION1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.VALUE_SEPARATOR + OPTION1_COMPLETIONS[1].getValue()
-        + SyntaxConstants.VALUE_SEPARATOR;
-    clearAndSimpleComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect nothing for completion
-    assertSimpleCompletionValues(completionValues, completionCandidates);
-  }
-
-  private void clearAndSimpleComplete(List<String> completionCandidates,
-      List<String> completionValues, String input, Parser parser) {
-    completionCandidates.clear();
-    completionValues.clear();
-    parser.complete(input, input.length(), completionCandidates);
+  public void testSplitUserInputDoubleQuotes() {
+    input = "query --query=\"select * from /region\"";
+    tokens = GfshParser.splitUserInput(input);
+    assertThat(tokens.size()).isEqualTo(3);
+    assertThat(tokens.get(0)).isEqualTo("query");
+    assertThat(tokens.get(1)).isEqualTo("--query");
+    assertThat(tokens.get(2)).isEqualTo("\"select * from /region\"");
   }
 
-  private void assertSimpleCompletionValues(List<String> expected, List<String> actual) {
-    assertEquals("Check size", expected.size(), actual.size());
-    assertEquals(expected, actual);
-  }
-
-  /**
-   * Tests the auto-completion capability of {@link GfshParser} with the method
-   * {@link GfshParser#completeAdvanced(String, int, List)}
-   */
   @Test
-  public void testCompleteAdvanced() throws Exception {
-    // Get the names of the command
-    String[] command1Names = ((CliCommand) methodCommand1.getAnnotation(CliCommand.class)).value();
-
-    // Input contains an entirely different string
-    String input = "moc";
-    List<Completion> completionCandidates = new ArrayList<Completion>();
-    List<Completion> completionValues = new ArrayList<Completion>();
-    parser.completeAdvanced(input, input.length(), completionCandidates);
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains a string which is prefix
-    // of more than 1 command
-    input = "c";
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    // completions will come alphabetically sorted
-    completionValues.add(new Completion(COMMAND2_NAME));
-    completionValues.add(new Completion(COMMAND1_NAME));
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains only prefix of the command
-    // name which is not a prefix of other command.
-    // It may be the prefix for the synonym of command
-    input = command1Names[0].substring(0, 3);
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    completionValues.add(new Completion(COMMAND1_NAME));
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains only the command name
-    input = command1Names[0];
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the completions for argument1
-    // For arguments, the formatted value will equal the actual arguments
-    // But the actual value will contain the ARGUMENT_SEPARATOR
-    for (Completion completion : ARGUMENT1_COMPLETIONS) {
-      completionValues.add(
-          new Completion(" " + completion.getValue(), completion.getFormattedValue(), null, 0));
-    }
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name and prefix of first
-    // argument
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue().substring(0, 3);
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the completions for argument2
-    // which have the provided first argument as the prefix
-    for (Completion completion : ARGUMENT1_COMPLETIONS) {
-      if (completion.getValue().startsWith(ARGUMENT1_COMPLETIONS[0].getValue().substring(0, 3))) {
-        completionValues.add(
-            new Completion(" " + completion.getValue(), completion.getFormattedValue(), null, 0));
-      }
-    }
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument
-    // and first option
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION1_NAME;
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the values for the first option
-    for (Completion completion : OPTION1_COMPLETIONS) {
-      completionValues
-          .add(new Completion(SyntaxConstants.OPTION_VALUE_SPECIFIER + completion.getValue(),
-              completion.getValue(), null, 0));
-    }
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option and prefix of one of the values provided
-    // by the auto-completor.
-    input = command1Names[1] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER
-        + OPTION1_COMPLETIONS[0].getValue().substring(0, 2);
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the values for the first option
-    for (Completion completion : OPTION1_COMPLETIONS) {
-      if (completion.getValue().startsWith(OPTION1_COMPLETIONS[0].getValue().substring(0, 2))) {
-        completionValues
-            .add(new Completion(SyntaxConstants.OPTION_VALUE_SPECIFIER + completion.getValue(),
-                completion.getValue(), null, 0));
-      }
-    }
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option and one of the values provided
-    // by the auto-completor.
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER + OPTION1_COMPLETIONS[0].getValue();
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the remaining options
-    // As only first option is mandatory, we expect the
-    // the other non-mandatory options.
-    completionValues.add(new Completion(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION2_NAME,
-        OPTION2_NAME, null, 0));
-    completionValues.add(new Completion(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION3_NAME,
-        OPTION3_NAME, null, 0));
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option, one value for the option and value separator at
-    // the end
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER + OPTION1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.VALUE_SEPARATOR;
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the other values for completion
-    completionValues
-        .add(new Completion(SyntaxConstants.VALUE_SEPARATOR + OPTION1_COMPLETIONS[1].getValue(),
-            OPTION1_COMPLETIONS[1].getValue(), null, 0));
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option and both the values for the option
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER + OPTION1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.VALUE_SEPARATOR + OPTION1_COMPLETIONS[1].getValue();
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect the remaining options
-    // As only first option is mandatory, we expect the
-    // the other non-mandatory options.
-    completionValues.add(new Completion(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION2_NAME,
-        OPTION2_NAME, null, 0));
-    completionValues.add(new Completion(" " + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION3_NAME,
-        OPTION3_NAME, null, 0));
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-
-    // Input contains command name, first argument, second argument,
-    // first option, both the values for the option and valueSeparator
-    // at the end
-    input = command1Names[0] + " " + ARGUMENT1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + ARGUMENT2_COMPLETIONS[0].getValue()
-        + SyntaxConstants.ARGUMENT_SEPARATOR + " " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER + OPTION1_COMPLETIONS[0].getValue()
-        + SyntaxConstants.VALUE_SEPARATOR + OPTION1_COMPLETIONS[1].getValue()
-        + SyntaxConstants.VALUE_SEPARATOR;
-    clearAndAdvancedComplete(completionCandidates, completionValues, input, parser);
-    // Here we expect nothing for completion
-    assertAdvancedCompletionValues(completionValues, completionCandidates);
-  }
-
-  private void clearAndAdvancedComplete(List<Completion> completionCandidates,
-      List<Completion> completionValues, String input, Parser parser) {
-    completionCandidates.clear();
-    completionValues.clear();
-    parser.completeAdvanced(input, input.length(), completionCandidates);
+  public void testSplitUserInputSingleQuotes() {
+    input = "query --query='select * from /region'";
+    tokens = GfshParser.splitUserInput(input);
+    assertThat(tokens.size()).isEqualTo(3);
+    assertThat(tokens.get(0)).isEqualTo("query");
+    assertThat(tokens.get(1)).isEqualTo("--query");
+    assertThat(tokens.get(2)).isEqualTo("'select * from /region'");
   }
 
-  private void assertAdvancedCompletionValues(List<Completion> expected, List<Completion> actual) {
-    assertEquals("Check size", expected.size(), actual.size());
-    for (int i = 0; i < expected.size(); i++) {
-      assertEquals("Check completion value no." + i + ". Expected(" + expected.get(i)
-          + ") & Actual(" + actual.get(i) + ").", expected.get(i).getValue(),
-          actual.get(i).getValue());
-      if (expected.get(i).getFormattedValue() != null) {
-        assertEquals(
-            "Check completion formatted value no." + i + ". Expected("
-                + expected.get(i).getFormattedValue() + ") & Actual("
-                + actual.get(i).getFormattedValue() + ").",
-            expected.get(i).getFormattedValue(), actual.get(i).getFormattedValue());
-      }
-    }
+  @Test
+  public void testSplitUserInputWithJ() {
+    input =
+        "start server --name=server1  --J=\"-Dgemfire.start-dev-rest-api=true\" --J='-Dgemfire.http-service-port=8080' --J='-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=30000'";
+    tokens = GfshParser.splitUserInput(input);
+    assertThat(tokens.size()).isEqualTo(10);
+    assertThat(tokens.get(5)).isEqualTo("\"-Dgemfire.start-dev-rest-api=true\"");
+    assertThat(tokens.get(7)).isEqualTo("'-Dgemfire.http-service-port=8080'");
+    assertThat(tokens.get(9))
+        .isEqualTo("'-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=30000'");
   }
 
-  /**
-   * Test for checking parsing of {@link GfshParser} with method {@link GfshParser#parse(String)}
-   * <p>
-   * Does not include testing for multiple values as this change is still pending in spring-shell
-   */
   @Test
-  public void testParse() throws Exception {
-    // Get the names of the command
-    String[] command1Names = ((CliCommand) methodCommand1.getAnnotation(CliCommand.class)).value();
-
-    // Input contains an entirely different string
-    String input = "moc";
-    ParseResult parse = null;
-    CommandProcessingException expectedException = null;
-    try {
-      parse = parser.parse(input);
-    } catch (CommandProcessingException expected) {
-      expectedException = expected;
-    } finally {
-      assertNotNull("Expecting a " + CommandProcessingException.class
-          + " for an invalid command name: " + input, expectedException);
-      assertEquals(
-          "CommandProcessingException type doesn't match. " + "Actual("
-              + expectedException.getErrorType() + ") & Expected("
-              + CommandProcessingException.COMMAND_INVALID_OR_UNAVAILABLE + ") ",
-          expectedException.getErrorType(),
-          CommandProcessingException.COMMAND_INVALID_OR_UNAVAILABLE);
-    }
-
-    // Input contains a string which is prefix
-    // of more than 1 command
-    input = "c";
-    expectedException = null;
-    try {
-      parse = parser.parse(input);
-    } catch (CommandProcessingException e) {
-      expectedException = e;
-    } finally {
-      assertNotNull("Expecting a " + CommandProcessingException.class
-          + " for an invalid/incomplete command name: " + input, expectedException);
-      assertEquals(
-          "CommandProcessingException type doesn't match. Actual("
-              + expectedException.getErrorType() + ") & Expected("
-              + CommandProcessingException.COMMAND_INVALID_OR_UNAVAILABLE + ") ",
-          expectedException.getErrorType(),
-          CommandProcessingException.COMMAND_INVALID_OR_UNAVAILABLE);
-    }
-
-    // Input contains only prefix of the command
-    // name which is not a prefix of other command.
-    // It may be the prefix for the synonym of command
-    input = "com";
-    expectedException = null;
-    try {
-      parse = parser.parse(input);
-    } catch (CommandProcessingException expected) {
-      expectedException = expected;
-    } finally {
-      // FIXME - Nikhil/Abhishek prefix shouldn't work
-      assertNotNull("Expecting a " + CommandProcessingException.class
-          + " for an invalid/incomplete command name: " + input, expectedException);
-      assertEquals(
-          "CommandProcessingException type doesn't match. Actual("
-              + expectedException.getErrorType() + ") & Expected("
-              + CommandProcessingException.COMMAND_INVALID_OR_UNAVAILABLE + ") ",
-          expectedException.getErrorType(),
-          CommandProcessingException.COMMAND_INVALID_OR_UNAVAILABLE);
-    }
-
-    // Input contains only command name
-    input = command1Names[0];
-    expectedException = null;
-    try {
-      parse = parser.parse(input);
-    } catch (CommandProcessingException expected) {
-      expectedException = expected;
-    } finally {
-      assertNotNull("Expecting a " + CommandProcessingException.class
-          + " for an invalid/incomplete command name: " + input, expectedException);
-      assertEquals(
-          "CommandProcessingException type doesn't match. Actual("
-              + expectedException.getErrorType() + ") & Expected("
-              + CommandProcessingException.REQUIRED_ARGUMENT_MISSING + ") ",
-          CommandProcessingException.REQUIRED_ARGUMENT_MISSING, expectedException.getErrorType());
-    }
-
-    // Input contains first argument and first option with value
-    input = command1Names[0] + " ARGUMENT1_VALUE " + SyntaxConstants.LONG_OPTION_SPECIFIER
-        + OPTION1_NAME + SyntaxConstants.OPTION_VALUE_SPECIFIER + "somevalue";
-    parse = parser.parse(input);
-    assertNotNull(parse);
-    assertEquals("Check ParseResult method", parse.getMethod(), methodCommand1);
-    assertEquals("Check no. of method arguments", 5, parse.getArguments().length);
-    assertEquals("Check argument1", "ARGUMENT1_VALUE", parse.getArguments()[0]);
-    assertEquals("Check argument2", ARGUMENT2_UNSPECIFIED_DEFAULT_VALUE, parse.getArguments()[1]);
-    assertEquals("Check option1 value", "somevalue", parse.getArguments()[2]);
-    assertEquals("Check option2 value", null, parse.getArguments()[3]);
-    assertEquals("Check option3 value", OPTION3_UNSPECIFIED_DEFAULT_VALUE, parse.getArguments()[4]);
-
-    // Input contains only both arguments but is terminated by long option
-    // specifiers. These hyphens at the end are ignored by the parser
-    input = command1Names[1] + " ARGUMENT1_VALUE?      ARGUMENT2_VALUE -- ----------";
-    try {
-      parse = parser.parse(input);
-    } catch (CommandProcessingException expected) {
-      expectedException = expected;
-    } finally {
-      assertNotNull("Expecting a " + CommandProcessingException.class
-          + " for an invalid/incomplete command name: " + input, expectedException);
-      // assertEquals("CommandProcessingException type doesn't match. Actual("
-      // + expectedException.getErrorType() + ") & Expected("
-      // + CommandProcessingException.REQUIRED_OPTION_MISSING + ") ",
-      // expectedException.getErrorType(),
-      // CommandProcessingException.REQUIRED_OPTION_MISSING);
-    }
-
-    // Input contains both arguments. The first option is specified with value
-    // The second is specified without value and the third option is not
-    // specified
-    input = command1Names[1] + "         ARGUMENT1_VALUE?       ARGUMENT2_VALUE "
-        + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION1_NAME
-        + SyntaxConstants.OPTION_VALUE_SPECIFIER + "option1value" + " "
-        + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION2_NAME;
-    parse = parser.parse(input);
-    assertNotNull(parse);
-    assertEquals("Check ParseResult method", parse.getMethod(), methodCommand1);
-    assertEquals("Check no. of method arguments", 5, parse.getArguments().length);
-    assertEquals("Check argument1", "ARGUMENT1_VALUE", parse.getArguments()[0]);
-    assertEquals("Check argument2", "ARGUMENT2_VALUE", parse.getArguments()[1]);
-    assertEquals("Check option1 value", "option1value", parse.getArguments()[2]);
-    assertEquals("Check option2 value", OPTION2_SPECIFIED_DEFAULT_VALUE, parse.getArguments()[3]);
-    assertEquals("Check option3 value", OPTION3_UNSPECIFIED_DEFAULT_VALUE, parse.getArguments()[4]);
-
-    // Input contains both arguments. All the three options
-    // are specified with values
-    input = command1Names[1] + "         ARGUMENT1_VALUE?       ARGUMENT2_VALUE "
-        + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION1_SYNONYM
-        + SyntaxConstants.OPTION_VALUE_SPECIFIER + "option1value" + " "
-        + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION2_NAME
-        + SyntaxConstants.OPTION_VALUE_SPECIFIER + "option2value" + " "
-        + SyntaxConstants.LONG_OPTION_SPECIFIER + OPTION3_NAME
-        + SyntaxConstants.OPTION_VALUE_SPECIFIER + "option3value";
-    parse = parser.parse(input);
-    assertNotNull(parse);
-    assertEquals("Check ParseResult method", parse.getMethod(), methodCommand1);
-    assertEquals("Check no. of method arguments", 5, parse.getArguments().length);
-    assertEquals("Check argument1", "ARGUMENT1_VALUE", parse.getArguments()[0]);
-    assertEquals("Check argument2", "ARGUMENT2_VALUE", parse.getArguments()[1]);
-    assertEquals("Check option1 value", "option1value", parse.getArguments()[2]);
-    assertEquals("Check option2 value", "option2value", parse.getArguments()[3]);
-    assertEquals("Check option3 value", "option3value", parse.getArguments()[4]);
-
-    // Test concatenation of options when they appear more than once in the command
-    String command =
-        "testParamConcat --string=string1 --stringArray=1,2 --stringArray=3,4 --stringList=11,12,13 --integer=10 --stringArray=5 --stringList=14,15";
-    ParseResult parseResult = parser.parse(command);
-    assertNotNull(parseResult);
-    assertEquals("Check ParseResult method", parseResult.getMethod(), methodTestParamConcat);
-    assertEquals("Check no. of method arguments", 5, parseResult.getArguments().length);
-    Object[] arguments = parseResult.getArguments();
-    assertEquals(arguments[0], "string1");
-    assertEquals(((String[]) arguments[1])[0], "1");
-    assertEquals(((String[]) arguments[1])[1], "2");
-    assertEquals(((String[]) arguments[1])[2], "3");
-    assertEquals(((String[]) arguments[1])[3], "4");
-    assertEquals(((String[]) arguments[1])[4], "5");
-    assertEquals(((List) arguments[2]).get(0), "11");
-    assertEquals(((List) arguments[2]).get(1), "12");
-    assertEquals(((List) arguments[2]).get(2), "13");
-    assertEquals(((List) arguments[2]).get(3), "14");
-    assertEquals(((List) arguments[2]).get(4), "15");
-    assertEquals(arguments[3], 10);
-
-    // Test concatenation of options when they appear more than once in the command
-    command = "testParamConcat --stringArray=1,2 --stringArray=\'3,4\'";
-    parseResult = parser.parse(command);
-    assertNotNull(parseResult);
-    assertEquals("Check ParseResult method", parseResult.getMethod(), methodTestParamConcat);
-    assertEquals("Check no. of method arguments", 5, parseResult.getArguments().length);
-    arguments = parseResult.getArguments();
-    assertEquals(((String[]) arguments[1])[0], "1");
-    assertEquals(((String[]) arguments[1])[1], "2");
-    assertEquals(((String[]) arguments[1])[2], "3,4");
-
-    command =
-        "testParamConcat --string=\"1\" --colonArray=2:3:4 --stringArray=5,\"6,7\",8 --stringList=\"9,10,11,12\"";
-    parseResult = parser.parse(command);
-    assertNotNull(parseResult);
-    assertEquals("Check ParseResult method", parseResult.getMethod(), methodTestParamConcat);
-    assertEquals("Check no. of method arguments", 5, parseResult.getArguments().length);
-    arguments = parseResult.getArguments();
-    assertEquals(arguments[0], "1");
-    assertEquals(((String[]) arguments[1])[0], "5");
-    assertEquals(((String[]) arguments[1])[1], "6,7");
-    assertEquals(((String[]) arguments[1])[2], "8");
-    assertEquals(((List) arguments[2]).get(0), "9,10,11,12");
-    assertEquals(((String[]) arguments[4])[0], "2");
-    assertEquals(((String[]) arguments[4])[1], "3");
-    assertEquals(((String[]) arguments[4])[2], "4");
-
-    // try {
-    // command = "testParamConcat --string=string1 --stringArray=1,2 --string=string2";
-    // parseResult = parser.parse(command);
-    // fail("Should have received a CommandProcessingException due to 'string' being specified
-    // twice");
-    // } catch (CommandProcessingException expected) {
-    // // Expected
-    // }
-
-    command = "testMultiWordArg this is just one argument?this is a second argument";
-    parseResult = parser.parse(command);
-    assertNotNull(parseResult);
-    assertEquals("Check ParseResult method", parseResult.getMethod(), methodTestMultiWordArg);
-    assertEquals("Check no. of method arguments", 2, parseResult.getArguments().length);
-    arguments = parseResult.getArguments();
-    assertEquals(arguments[0], "this is just one argument");
-    assertEquals(arguments[1], "this is a second argument");
+  public void testSplitUserInputWithJNoQuotes() {
+    input =
+        "start server --name=server1  --J=-Dgemfire.start-dev-rest-api=true --J=-Dgemfire.http-service-port=8080";
+    tokens = GfshParser.splitUserInput(input);
+    assertThat(tokens.size()).isEqualTo(8);
+    assertThat(tokens.get(5)).isEqualTo("-Dgemfire.start-dev-rest-api=true");
+    assertThat(tokens.get(7)).isEqualTo("-Dgemfire.http-service-port=8080");
   }
 
   @Test
-  public void testDefaultAvailabilityMessage() throws Exception {
-    checkAvailabilityMessage(new AvailabilityCommands(), AvailabilityCommands.C2_NAME,
-        AvailabilityCommands.C2_MSG_UNAVAILABLE, AvailabilityCommands.C2_PROP);
+  public void testSplitJsonValue() throws Exception {
+    input = "get --key=('id':'testKey0') --region=regionA";
+    tokens = GfshParser.splitUserInput(input);
+    assertThat(tokens.size()).isEqualTo(5);
+    assertThat(tokens.get(2)).isEqualTo("('id':'testKey0')");
   }
 
-  @Ignore("This test was not previously enabled and it fails. Is it valid?")
   @Test
-  public void testCustomAvailabilityMessage() throws Exception {
-    checkAvailabilityMessage(new AvailabilityCommands(), AvailabilityCommands.C1_NAME,
-        AvailabilityCommands.C1_MSG_UNAVAILABLE, AvailabilityCommands.C1_PROP);
+  public void testGetSimpleParserInput() throws Exception {
+    String[] strings = {"command", "--option1", "value1", "--option2", "'test value'"};
+    Arrays.stream(strings).forEach(tokens::add);
+    assertThat(GfshParser.getSimpleParserInputFromTokens(tokens))
+        .isEqualTo("command --option1 value1 --option2 'test value'");
   }
 
-  public void checkAvailabilityMessage(CommandMarker availabilityCommands, String commandString,
-      String unavailableMessage, String availabiltyBooleanProp) throws Exception {
-    CommandManager cmdManager = CommandManager.getInstance(false);
-    cmdManager.add(availabilityCommands);
-
-    GfshParser parser = new GfshParser(cmdManager);
-    ParseResult parseResult = null;
-
-    // Case 1: Command is not available
-    try {
-      parseResult = parser.parse(commandString);
-    } catch (CommandProcessingException e) {
-      String actualMessage = e.getMessage();
-      String expectedMessage =
-          CliStrings.format(CliStrings.GFSHPARSER__MSG__0_IS_NOT_AVAILABLE_REASON_1,
-              new Object[] {commandString, unavailableMessage});
-      assertEquals("1. Unavailability message [" + actualMessage + "] is not as expected["
-          + expectedMessage + "].", actualMessage, expectedMessage);
-    }
-
-    // Case 2: Command is 'made' available
-    try {
-      System.setProperty(availabiltyBooleanProp, "true");
-      parseResult = parser.parse(commandString);
-      assertNotNull("ParseResult should not be null for available command.", parseResult);
-    } finally {
-      System.clearProperty(availabiltyBooleanProp);
-    }
-
-    // Case 3: Command is not available again
-    try {
-      parseResult = parser.parse(commandString);
-    } catch (CommandProcessingException e) {
-      String actualMessage = e.getMessage();
-      String expectedMessage =
-          CliStrings.format(CliStrings.GFSHPARSER__MSG__0_IS_NOT_AVAILABLE_REASON_1,
-              new Object[] {commandString, unavailableMessage});
-      assertEquals("2. Unavailabilty message [" + actualMessage + "] is not as expected["
-          + expectedMessage + "].", actualMessage, expectedMessage);
-    }
+  @Test
+  public void testGetSimpleParserInputWithJ() throws Exception {
+    String[] strings =
+        {"command", "--J", "-Dkey=value", "--option", "'test value'", "--J", "-Dkey2=value2"};
+    Arrays.stream(strings).forEach(tokens::add);
+    assertThat(GfshParser.getSimpleParserInputFromTokens(tokens))
+        .isEqualTo("command --J \"-Dkey=value,-Dkey2=value2\" --option 'test value'");
   }
 
-  static class Commands implements CommandMarker {
-
-    @CliCommand(value = {COMMAND1_NAME, COMMAND1_NAME_ALIAS}, help = COMMAND1_HELP)
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public static String command1(
-        @CliArgument(name = ARGUMENT1_NAME, argumentContext = ARGUMENT1_CONTEXT,
-            help = ARGUMENT1_HELP, mandatory = true) String argument1,
-        @CliArgument(name = ARGUMENT2_NAME, argumentContext = ARGUMENT2_CONTEXT,
-            help = ARGUMENT2_HELP, mandatory = false,
-            unspecifiedDefaultValue = ARGUMENT2_UNSPECIFIED_DEFAULT_VALUE,
-            systemProvided = false) String argument2,
-        @CliOption(key = {OPTION1_NAME, OPTION1_SYNONYM}, help = OPTION1_HELP, mandatory = true,
-            optionContext = OPTION1_CONTEXT) String option1,
-        @CliOption(key = {OPTION2_NAME}, help = OPTION2_HELP, mandatory = false,
-            optionContext = OPTION2_CONTEXT,
-            specifiedDefaultValue = OPTION2_SPECIFIED_DEFAULT_VALUE) String option2,
-        @CliOption(key = {OPTION3_NAME, OPTION3_SYNONYM}, help = OPTION3_HELP, mandatory = false,
-            optionContext = OPTION3_CONTEXT,
-            unspecifiedDefaultValue = OPTION3_UNSPECIFIED_DEFAULT_VALUE) String option3) {
-      return null;
-    }
-
-    @CliCommand(value = {COMMAND2_NAME})
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public static String command2() {
-      return null;
-    }
-
-    @CliCommand(value = {"testParamConcat"})
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public static Result testParamConcat(@CliOption(key = {"string"}) String string,
-        @CliOption(key = {"stringArray"}) @CliMetaData(valueSeparator = ",") String[] stringArray,
-        @CliOption(key = {"stringList"}, optionContext = ConverterHint.STRING_LIST) @CliMetaData(
-            valueSeparator = ",") List<String> stringList,
-        @CliOption(key = {"integer"}) Integer integer,
-        @CliOption(key = {"colonArray"}) @CliMetaData(valueSeparator = ":") String[] colonArray) {
-      return null;
-    }
-
-    @CliCommand(value = {"testMultiWordArg"})
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public static Result testMultiWordArg(@CliArgument(name = "arg1") String arg1,
-        @CliArgument(name = "arg2") String arg2) {
-      return null;
-    }
+  @Test
+  public void testGetSimpleParserInputWithJWithSingleQuotes() throws Exception {
+    String[] strings = {"command", "--J", "'-Dkey=value value'"};
+    Arrays.stream(strings).forEach(tokens::add);
+    assertThat(GfshParser.getSimpleParserInputFromTokens(tokens))
+        .isEqualTo("command --J \"-Dkey=value value\"");
   }
 
-  static class SimpleConverter implements Converter<String> {
-
-    @Override
-    public boolean supports(Class<?> type, String optionContext) {
-      if (type.isAssignableFrom(String.class)) {
-        return true;
-      }
-      return false;
-    }
-
-    @Override
-    public String convertFromText(String value, Class<?> targetType, String optionContext) {
-      return value;
-    }
-
-    @Override
-    public boolean getAllPossibleValues(List<Completion> completions, Class<?> targetType,
-        String existingData, String context, MethodTarget target) {
-      if (context.equals(ARGUMENT1_CONTEXT)) {
-        for (Completion completion : ARGUMENT1_COMPLETIONS) {
-          completions.add(completion);
-        }
-      } else if (context.equals(ARGUMENT2_CONTEXT)) {
-        for (Completion completion : ARGUMENT2_COMPLETIONS) {
-          completions.add(completion);
-        }
-      } else if (context.equals(OPTION1_CONTEXT)) {
-        for (Completion completion : OPTION1_COMPLETIONS) {
-          completions.add(completion);
-        }
-      }
-      return true;
-    }
+  @Test
+  public void testGetSimpleParserInputWithJWithDoubleQuotes() throws Exception {
+    String[] strings = {"command", "--J", "\"-Dkey=value value\""};
+    Arrays.stream(strings).forEach(tokens::add);
+    assertThat(GfshParser.getSimpleParserInputFromTokens(tokens))
+        .isEqualTo("command --J \"-Dkey=value value\"");
   }
 
-  public static class AvailabilityCommands implements CommandMarker {
-
-    static final String C1_NAME = "C1";
-    static final String C1_PROP = C1_NAME + "-available";
-    static final String C1_MSG_UNAVAILABLE = "Requires " + C1_PROP + "=true";
-    static final String C1_MSG_AVAILABLE = C1_NAME + " is available.";
-
-    static final String C2_NAME = "C2";
-    static final String C2_PROP = C2_NAME + "-available";
-    static final String C2_MSG_UNAVAILABLE =
-        CliStrings.AVAILABILITYTARGET_MSG_DEFAULT_UNAVAILABILITY_DESCRIPTION;
-    static final String C2_MSG_AVAILABLE = C2_NAME + " is available.";
-
-    @CliCommand(value = {C1_NAME})
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public Result command1() {
-      return ResultBuilder.createInfoResult(C1_MSG_AVAILABLE);
-    }
-
-    @CliCommand(value = {C2_NAME})
-    @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-    public Result command2() {
-      return ResultBuilder.createInfoResult(C2_MSG_AVAILABLE);
-    }
-
-    @CliAvailabilityIndicator(C1_NAME)
-    public boolean isCommand1Available() {
-      return Boolean.getBoolean(C1_PROP);
-    }
-
-    @CliAvailabilityIndicator(C2_NAME)
-    public boolean isCommand2Available() {
-      return Boolean.getBoolean(C2_PROP);
-    }
+  @Test
+  public void testGetSimpleParserInputWithJAtTheEnd() throws Exception {
+    String[] strings =
+        {"command", "--option", "'test value'", "--J", "-Dkey=value", "--J", "-Dkey2=value2"};
+    Arrays.stream(strings).forEach(tokens::add);
+    assertThat(GfshParser.getSimpleParserInputFromTokens(tokens))
+        .isEqualTo("command --option 'test value' --J \"-Dkey=value,-Dkey2=value2\"");
   }
 }