You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2017/08/15 23:10:21 UTC

[10/50] [abbrv] geode git commit: GEODE-3436: revert recent refactoring of GFSH commands

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartMemberUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartMemberUtils.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartMemberUtils.java
new file mode 100644
index 0000000..3e1053a
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartMemberUtils.java
@@ -0,0 +1,251 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.management.internal.cli.shell.MXBeanProvider.getDistributedSystemMXBean;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Properties;
+
+import javax.management.MalformedObjectNameException;
+
+import org.apache.commons.lang.ArrayUtils;
+
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.internal.GemFireVersion;
+import org.apache.geode.internal.process.ProcessLauncherContext;
+import org.apache.geode.internal.util.IOUtils;
+import org.apache.geode.management.DistributedSystemMXBean;
+import org.apache.geode.management.internal.cli.util.ThreePhraseGenerator;
+
+/**
+ * Encapsulates methods used by StartServerCommand and StartLocatorCommand and their associated
+ * tests.
+ * 
+ * @see StartLocatorCommand
+ * @see StartServerCommand
+ */
+public class StartMemberUtils {
+  public static final String GEODE_HOME = System.getenv("GEODE_HOME");
+
+  private static final String JAVA_HOME = System.getProperty("java.home");
+  static final int CMS_INITIAL_OCCUPANCY_FRACTION = 60;
+  private static final ThreePhraseGenerator nameGenerator = new ThreePhraseGenerator();
+
+  static final String CORE_DEPENDENCIES_JAR_PATHNAME =
+      IOUtils.appendToPath(GEODE_HOME, "lib", "geode-dependencies.jar");
+  static final String GEODE_JAR_PATHNAME =
+      IOUtils.appendToPath(GEODE_HOME, "lib", GemFireVersion.getGemFireJarFileName());
+  static final long PROCESS_STREAM_READER_ASYNC_STOP_TIMEOUT_MILLIS = 5 * 1000;
+  static final int INVALID_PID = -1;
+
+  static ThreePhraseGenerator getNameGenerator() {
+    return nameGenerator;
+  }
+
+  static void setPropertyIfNotNull(Properties properties, String key, Object value) {
+    if (key != null && value != null) {
+      properties.setProperty(key, value.toString());
+    }
+  }
+
+  static String resolveWorkingDir(String userSpecifiedDir, String memberName) {
+    File workingDir =
+        (userSpecifiedDir == null) ? new File(memberName) : new File(userSpecifiedDir);
+    String workingDirPath = IOUtils.tryGetCanonicalPathElseGetAbsolutePath(workingDir);
+    if (!workingDir.exists()) {
+      if (!workingDir.mkdirs()) {
+        throw new IllegalStateException(String.format(
+            "Could not create directory %s. Please verify directory path or user permissions.",
+            workingDirPath));
+      }
+    }
+    return workingDirPath;
+  }
+
+  static void addGemFirePropertyFile(final List<String> commandLine,
+      final File gemfirePropertiesFile) {
+    if (gemfirePropertiesFile != null) {
+      commandLine.add("-DgemfirePropertyFile=" + gemfirePropertiesFile.getAbsolutePath());
+    }
+  }
+
+  static void addGemFireSecurityPropertyFile(final List<String> commandLine,
+      final File gemfireSecurityPropertiesFile) {
+    if (gemfireSecurityPropertiesFile != null) {
+      commandLine
+          .add("-DgemfireSecurityPropertyFile=" + gemfireSecurityPropertiesFile.getAbsolutePath());
+    }
+  }
+
+  static void addGemFireSystemProperties(final List<String> commandLine,
+      final Properties gemfireProperties) {
+    for (final Object property : gemfireProperties.keySet()) {
+      final String propertyName = property.toString();
+      final String propertyValue = gemfireProperties.getProperty(propertyName);
+      if (org.apache.geode.internal.lang.StringUtils.isNotBlank(propertyValue)) {
+        commandLine.add(
+            "-D" + DistributionConfig.GEMFIRE_PREFIX + "" + propertyName + "=" + propertyValue);
+      }
+    }
+  }
+
+  static void addJvmArgumentsAndOptions(final List<String> commandLine,
+      final String[] jvmArgsOpts) {
+    if (jvmArgsOpts != null) {
+      commandLine.addAll(Arrays.asList(jvmArgsOpts));
+    }
+  }
+
+  static void addInitialHeap(final List<String> commandLine, final String initialHeap) {
+    if (org.apache.geode.internal.lang.StringUtils.isNotBlank(initialHeap)) {
+      commandLine.add("-Xms" + initialHeap);
+    }
+  }
+
+  static void addMaxHeap(final List<String> commandLine, final String maxHeap) {
+    if (org.apache.geode.internal.lang.StringUtils.isNotBlank(maxHeap)) {
+      commandLine.add("-Xmx" + maxHeap);
+      commandLine.add("-XX:+UseConcMarkSweepGC");
+      commandLine.add("-XX:CMSInitiatingOccupancyFraction=" + CMS_INITIAL_OCCUPANCY_FRACTION);
+      // commandLine.add("-XX:MinHeapFreeRatio=" + MINIMUM_HEAP_FREE_RATIO);
+    }
+  }
+
+  static void addCurrentLocators(GfshCommand gfshCommand, final List<String> commandLine,
+      final Properties gemfireProperties) throws MalformedObjectNameException {
+    if (org.apache.geode.internal.lang.StringUtils
+        .isBlank(gemfireProperties.getProperty(LOCATORS))) {
+      String currentLocators = getCurrentLocators(gfshCommand);
+      if (org.apache.geode.internal.lang.StringUtils.isNotBlank(currentLocators)) {
+        commandLine.add("-D".concat(ProcessLauncherContext.OVERRIDDEN_DEFAULTS_PREFIX)
+            .concat(LOCATORS).concat("=").concat(currentLocators));
+      }
+    }
+  }
+
+  private static String getCurrentLocators(GfshCommand gfshCommand)
+      throws MalformedObjectNameException {
+    String delimitedLocators = "";
+    try {
+      if (gfshCommand.isConnectedAndReady()) {
+        final DistributedSystemMXBean dsMBeanProxy = getDistributedSystemMXBean();
+        if (dsMBeanProxy != null) {
+          final String[] locators = dsMBeanProxy.listLocators();
+          if (locators != null && locators.length > 0) {
+            final StringBuilder sb = new StringBuilder();
+            for (int i = 0; i < locators.length; i++) {
+              if (i > 0) {
+                sb.append(",");
+              }
+              sb.append(locators[i]);
+            }
+            delimitedLocators = sb.toString();
+          }
+        }
+      }
+    } catch (IOException e) { // thrown by getDistributedSystemMXBean
+      // leave delimitedLocators = ""
+      gfshCommand.getGfsh().logWarning("DistributedSystemMXBean is unavailable\n", e);
+    }
+    return delimitedLocators;
+  }
+
+  public static int readPid(final File pidFile) {
+    assert pidFile != null : "The file from which to read the process ID (pid) cannot be null!";
+    if (pidFile.isFile()) {
+      BufferedReader fileReader = null;
+      try {
+        fileReader = new BufferedReader(new FileReader(pidFile));
+        return Integer.parseInt(fileReader.readLine());
+      } catch (IOException | NumberFormatException ignore) {
+      } finally {
+        IOUtils.close(fileReader);
+      }
+    }
+    return INVALID_PID;
+  }
+
+  static String getJavaPath() {
+    return new File(new File(JAVA_HOME, "bin"), "java").getPath();
+  }
+
+  static String getSystemClasspath() {
+    return System.getProperty("java.class.path");
+  }
+
+  static String toClasspath(final boolean includeSystemClasspath, String[] jarFilePathnames,
+      String... userClasspaths) {
+    // gemfire jar must absolutely be the first JAR file on the CLASSPATH!!!
+    StringBuilder classpath = new StringBuilder(getGemFireJarPath());
+
+    userClasspaths = (userClasspaths != null ? userClasspaths : ArrayUtils.EMPTY_STRING_ARRAY);
+
+    // Then, include user-specified classes on CLASSPATH to enable the user to override GemFire JAR
+    // dependencies
+    // with application-specific versions; this logic/block corresponds to classes/jar-files
+    // specified with the
+    // --classpath option to the 'start locator' and 'start server commands'; also this will
+    // override any
+    // System CLASSPATH environment variable setting, which is consistent with the Java platform
+    // behavior...
+    for (String userClasspath : userClasspaths) {
+      if (org.apache.geode.internal.lang.StringUtils.isNotBlank(userClasspath)) {
+        classpath.append((classpath.length() == 0)
+            ? org.apache.geode.internal.lang.StringUtils.EMPTY : File.pathSeparator);
+        classpath.append(userClasspath);
+      }
+    }
+
+    // Now, include any System-specified CLASSPATH environment variable setting...
+    if (includeSystemClasspath) {
+      classpath.append(File.pathSeparator);
+      classpath.append(getSystemClasspath());
+    }
+
+    jarFilePathnames =
+        (jarFilePathnames != null ? jarFilePathnames : ArrayUtils.EMPTY_STRING_ARRAY);
+
+    // And finally, include all GemFire dependencies on the CLASSPATH...
+    for (String jarFilePathname : jarFilePathnames) {
+      if (org.apache.geode.internal.lang.StringUtils.isNotBlank(jarFilePathname)) {
+        classpath.append((classpath.length() == 0)
+            ? org.apache.geode.internal.lang.StringUtils.EMPTY : File.pathSeparator);
+        classpath.append(jarFilePathname);
+      }
+    }
+    return classpath.toString();
+  }
+
+  static String getGemFireJarPath() {
+    String classpath = getSystemClasspath();
+    String gemfireJarPath = GEODE_JAR_PATHNAME;
+    for (String classpathElement : classpath.split(File.pathSeparator)) {
+      // MUST CHANGE THIS TO REGEX SINCE VERSION CHANGES IN JAR NAME
+      if (classpathElement.endsWith("gemfire-core-8.2.0.0-SNAPSHOT.jar")) {
+        gemfireJarPath = classpathElement;
+        break;
+      }
+    }
+    return gemfireJarPath;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
index 035a8c1..8080e3f 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
@@ -48,7 +48,7 @@ import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.security.ResourceConstants;
 
-public class StartServerCommand extends StartMemberCommand implements GfshCommand {
+public class StartServerCommand implements GfshCommand {
   private static final String SERVER_TERM_NAME = "Server";
 
   @CliCommand(value = CliStrings.START_SERVER, help = CliStrings.START_SERVER__HELP)
@@ -181,7 +181,7 @@ public class StartServerCommand extends StartMemberCommand implements GfshComman
     try {
       if (StringUtils.isBlank(memberName)) {
         // when the user doesn't give us a name, we make one up!
-        memberName = getNameGenerator().generate('-');
+        memberName = StartMemberUtils.getNameGenerator().generate('-');
       }
 
       // prompt for password is username is specified in the command
@@ -195,7 +195,7 @@ public class StartServerCommand extends StartMemberCommand implements GfshComman
         }
       }
 
-      workingDirectory = resolveWorkingDir(workingDirectory, memberName);
+      workingDirectory = StartMemberUtils.resolveWorkingDir(workingDirectory, memberName);
 
       cacheXmlPathname = CliUtil.resolvePathname(cacheXmlPathname);
 
@@ -219,49 +219,56 @@ public class StartServerCommand extends StartMemberCommand implements GfshComman
 
       File serverPidFile = new File(workingDirectory, ProcessType.SERVER.getPidFileName());
 
-      readPid(serverPidFile);
+      final int oldPid = StartMemberUtils.readPid(serverPidFile);
 
       Properties gemfireProperties = new Properties();
 
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.BIND_ADDRESS, bindAddress);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.CACHE_XML_FILE,
-          cacheXmlPathname);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.ENABLE_TIME_STATISTICS,
-          enableTimeStatistics);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.GROUPS, group);
-      setPropertyIfNotNull(gemfireProperties,
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.BIND_ADDRESS,
+          bindAddress);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.CACHE_XML_FILE, cacheXmlPathname);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.ENABLE_TIME_STATISTICS, enableTimeStatistics);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.GROUPS,
+          group);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
           ConfigurationProperties.JMX_MANAGER_HOSTNAME_FOR_CLIENTS, jmxManagerHostnameForClients);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.LOCATORS, locators);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.LOCATOR_WAIT_TIME,
-          locatorWaitTime);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.LOG_LEVEL, logLevel);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.MCAST_ADDRESS,
-          mcastBindAddress);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.MCAST_PORT, mcastPort);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.MEMCACHED_PORT,
-          memcachedPort);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.MEMCACHED_PROTOCOL,
-          memcachedProtocol);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.MEMCACHED_BIND_ADDRESS,
-          memcachedBindAddress);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.REDIS_PORT, redisPort);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.REDIS_BIND_ADDRESS,
-          redisBindAddress);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.REDIS_PASSWORD,
-          redisPassword);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.STATISTIC_ARCHIVE_FILE,
-          statisticsArchivePathname);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.USE_CLUSTER_CONFIGURATION,
-          requestSharedConfiguration);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.LOCK_MEMORY, lockMemory);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.OFF_HEAP_MEMORY_SIZE,
-          offHeapMemorySize);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.START_DEV_REST_API,
-          startRestApi);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.HTTP_SERVICE_PORT,
-          httpServicePort);
-      setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS,
-          httpServiceBindAddress);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.LOCATORS,
+          locators);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.LOCATOR_WAIT_TIME, locatorWaitTime);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.LOG_LEVEL,
+          logLevel);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.MCAST_ADDRESS, mcastBindAddress);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.MCAST_PORT,
+          mcastPort);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.MEMCACHED_PORT, memcachedPort);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.MEMCACHED_PROTOCOL, memcachedProtocol);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.MEMCACHED_BIND_ADDRESS, memcachedBindAddress);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.REDIS_PORT,
+          redisPort);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.REDIS_BIND_ADDRESS, redisBindAddress);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.REDIS_PASSWORD, redisPassword);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.STATISTIC_ARCHIVE_FILE, statisticsArchivePathname);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.USE_CLUSTER_CONFIGURATION, requestSharedConfiguration);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.LOCK_MEMORY,
+          lockMemory);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.OFF_HEAP_MEMORY_SIZE, offHeapMemorySize);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.START_DEV_REST_API, startRestApi);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.HTTP_SERVICE_PORT, httpServicePort);
+      StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
+          ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS, httpServiceBindAddress);
       // if username is specified in the command line, it will overwrite what's set in the
       // properties file
       if (StringUtils.isNotBlank(userName)) {
@@ -370,9 +377,9 @@ public class StartServerCommand extends StartMemberCommand implements GfshComman
         } while (!(registeredServerSignalListener && serverSignalListener.isSignaled())
             && serverState.isStartingOrNotResponding());
       } finally {
-        stderrReader.stopAsync(PROCESS_STREAM_READER_ASYNC_STOP_TIMEOUT_MILLIS); // stop
-                                                                                 // will
-                                                                                 // close
+        stderrReader.stopAsync(StartMemberUtils.PROCESS_STREAM_READER_ASYNC_STOP_TIMEOUT_MILLIS); // stop
+                                                                                                  // will
+                                                                                                  // close
         // ErrorStream
         getGfsh().getSignalHandler().unregisterListener(serverSignalListener);
       }
@@ -417,16 +424,16 @@ public class StartServerCommand extends StartMemberCommand implements GfshComman
       throws MalformedObjectNameException {
     List<String> commandLine = new ArrayList<>();
 
-    commandLine.add(getJavaPath());
+    commandLine.add(StartMemberUtils.getJavaPath());
     commandLine.add("-server");
     commandLine.add("-classpath");
     commandLine.add(getServerClasspath(Boolean.TRUE.equals(includeSystemClasspath), userClasspath));
 
-    addCurrentLocators(this, commandLine, gemfireProperties);
-    addGemFirePropertyFile(commandLine, gemfirePropertiesFile);
-    addGemFireSecurityPropertyFile(commandLine, gemfireSecurityPropertiesFile);
-    addGemFireSystemProperties(commandLine, gemfireProperties);
-    addJvmArgumentsAndOptions(commandLine, jvmArgsOpts);
+    StartMemberUtils.addCurrentLocators(this, commandLine, gemfireProperties);
+    StartMemberUtils.addGemFirePropertyFile(commandLine, gemfirePropertiesFile);
+    StartMemberUtils.addGemFireSecurityPropertyFile(commandLine, gemfireSecurityPropertiesFile);
+    StartMemberUtils.addGemFireSystemProperties(commandLine, gemfireProperties);
+    StartMemberUtils.addJvmArgumentsAndOptions(commandLine, jvmArgsOpts);
 
     // NOTE asserting not equal to true rather than equal to false handles the null case and ensures
     // the user
@@ -435,8 +442,8 @@ public class StartServerCommand extends StartMemberCommand implements GfshComman
       addJvmOptionsForOutOfMemoryErrors(commandLine);
     }
 
-    addInitialHeap(commandLine, initialHeap);
-    addMaxHeap(commandLine, maxHeap);
+    StartMemberUtils.addInitialHeap(commandLine, initialHeap);
+    StartMemberUtils.addMaxHeap(commandLine, maxHeap);
 
     commandLine.add(
         "-D".concat(AbstractLauncher.SIGNAL_HANDLER_REGISTRATION_SYSTEM_PROPERTY.concat("=true")));
@@ -544,9 +551,9 @@ public class StartServerCommand extends StartMemberCommand implements GfshComman
   String getServerClasspath(final boolean includeSystemClasspath, final String userClasspath) {
     List<String> jarFilePathnames = new ArrayList<>();
 
-    jarFilePathnames.add(CORE_DEPENDENCIES_JAR_PATHNAME);
+    jarFilePathnames.add(StartMemberUtils.CORE_DEPENDENCIES_JAR_PATHNAME);
 
-    return toClasspath(includeSystemClasspath,
+    return StartMemberUtils.toClasspath(includeSystemClasspath,
         jarFilePathnames.toArray(new String[jarFilePathnames.size()]), userClasspath);
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StatusClusterConfigServiceCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StatusClusterConfigServiceCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StatusClusterConfigServiceCommand.java
deleted file mode 100644
index 6a0fc1e..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StatusClusterConfigServiceCommand.java
+++ /dev/null
@@ -1,82 +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 java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.springframework.shell.core.annotation.CliCommand;
-
-import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.cli.Result.Status;
-import org.apache.geode.management.internal.cli.CliUtil;
-import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
-import org.apache.geode.management.internal.cli.functions.FetchSharedConfigurationStatusFunction;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.result.TabularResultData;
-import org.apache.geode.management.internal.configuration.domain.SharedConfigurationStatus;
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-public class StatusClusterConfigServiceCommand implements GfshCommand {
-  private static final FetchSharedConfigurationStatusFunction fetchSharedConfigStatusFunction =
-      new FetchSharedConfigurationStatusFunction();
-
-  @SuppressWarnings("unchecked")
-  @CliCommand(value = CliStrings.STATUS_SHARED_CONFIG, help = CliStrings.STATUS_SHARED_CONFIG_HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_LOCATOR)
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-  public Result statusSharedConfiguration() {
-    final InternalCache cache = GemFireCacheImpl.getInstance();
-    final Set<DistributedMember> locators = new HashSet<DistributedMember>(
-        cache.getDistributionManager().getAllHostedLocatorsWithSharedConfiguration().keySet());
-    if (locators.isEmpty()) {
-      return ResultBuilder.createInfoResult(CliStrings.NO_LOCATORS_WITH_SHARED_CONFIG);
-    } else {
-      return ResultBuilder.buildResult(getSharedConfigurationStatus(locators));
-    }
-  }
-
-  private TabularResultData getSharedConfigurationStatus(Set<DistributedMember> locators) {
-    boolean isSharedConfigRunning = false;
-    ResultCollector<?, ?> rc =
-        CliUtil.executeFunction(fetchSharedConfigStatusFunction, null, locators);
-    List<CliFunctionResult> results = (List<CliFunctionResult>) rc.getResult();
-    TabularResultData table = ResultBuilder.createTabularResultData();
-    table.setHeader("Status of shared configuration on locators");
-
-    for (CliFunctionResult result : results) {
-      table.accumulate(CliStrings.STATUS_SHARED_CONFIG_NAME_HEADER, result.getMemberIdOrName());
-      String status = (String) result.getSerializables()[0];
-      table.accumulate(CliStrings.STATUS_SHARED_CONFIG_STATUS, status);
-      if (SharedConfigurationStatus.RUNNING.name().equals(status)) {
-        isSharedConfigRunning = true;
-      }
-    }
-
-    if (!isSharedConfigRunning) {
-      table.setStatus(Status.ERROR);
-    }
-    return table;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StatusCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StatusCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StatusCommands.java
new file mode 100644
index 0000000..0b0b78b
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/StatusCommands.java
@@ -0,0 +1,82 @@
+/*
+ * 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.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.cli.Result.Status;
+import org.apache.geode.management.internal.cli.CliUtil;
+import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
+import org.apache.geode.management.internal.cli.functions.FetchSharedConfigurationStatusFunction;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.result.TabularResultData;
+import org.apache.geode.management.internal.configuration.domain.SharedConfigurationStatus;
+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.CliCommand;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class StatusCommands implements GfshCommand {
+  static final FetchSharedConfigurationStatusFunction fetchSharedConfigStatusFunction =
+      new FetchSharedConfigurationStatusFunction();
+
+  @SuppressWarnings("unchecked")
+  @CliCommand(value = CliStrings.STATUS_SHARED_CONFIG, help = CliStrings.STATUS_SHARED_CONFIG_HELP)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_LOCATOR)
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
+  public Result statusSharedConfiguration() {
+    final InternalCache cache = GemFireCacheImpl.getInstance();
+    final Set<DistributedMember> locators = new HashSet<DistributedMember>(
+        cache.getDistributionManager().getAllHostedLocatorsWithSharedConfiguration().keySet());
+    if (locators.isEmpty()) {
+      return ResultBuilder.createInfoResult(CliStrings.NO_LOCATORS_WITH_SHARED_CONFIG);
+    } else {
+      return ResultBuilder.buildResult(getSharedConfigurationStatus(locators));
+    }
+  }
+
+  private TabularResultData getSharedConfigurationStatus(Set<DistributedMember> locators) {
+    boolean isSharedConfigRunning = false;
+    ResultCollector<?, ?> rc =
+        CliUtil.executeFunction(fetchSharedConfigStatusFunction, null, locators);
+    List<CliFunctionResult> results = (List<CliFunctionResult>) rc.getResult();
+    TabularResultData table = ResultBuilder.createTabularResultData();
+    table.setHeader("Status of shared configuration on locators");
+
+    for (CliFunctionResult result : results) {
+      table.accumulate(CliStrings.STATUS_SHARED_CONFIG_NAME_HEADER, result.getMemberIdOrName());
+      String status = (String) result.getSerializables()[0];
+      table.accumulate(CliStrings.STATUS_SHARED_CONFIG_STATUS, status);
+      if (SharedConfigurationStatus.RUNNING.name().equals(status)) {
+        isSharedConfigRunning = true;
+      }
+    }
+
+    if (!isSharedConfigRunning) {
+      table.setStatus(Status.ERROR);
+    }
+    return table;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UndeployCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UndeployCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UndeployCommand.java
deleted file mode 100644
index 5df7ac7..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UndeployCommand.java
+++ /dev/null
@@ -1,114 +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 java.util.List;
-import java.util.Set;
-
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import org.apache.geode.SystemFailure;
-import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.ConverterHint;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.CliUtil;
-import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
-import org.apache.geode.management.internal.cli.functions.UndeployFunction;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.result.TabularResultData;
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission;
-
-public class UndeployCommand implements GfshCommand {
-  private final UndeployFunction undeployFunction = new UndeployFunction();
-
-  /**
-   * Undeploy one or more JAR files from members of a group or all members.
-   *
-   * @param groups Group(s) to undeploy the JAR from or null for all members
-   * @param jars JAR(s) to undeploy (separated by comma)
-   * @return The result of the attempt to undeploy
-   */
-  @CliCommand(value = {CliStrings.UNDEPLOY}, help = CliStrings.UNDEPLOY__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
-  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
-      operation = ResourcePermission.Operation.MANAGE, target = ResourcePermission.Target.JAR)
-  public Result undeploy(
-      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
-          help = CliStrings.UNDEPLOY__GROUP__HELP,
-          optionContext = ConverterHint.MEMBERGROUP) String[] groups,
-      @CliOption(key = {CliStrings.JAR, CliStrings.JARS},
-          help = CliStrings.UNDEPLOY__JAR__HELP) String[] jars) {
-
-    try {
-      TabularResultData tabularData = ResultBuilder.createTabularResultData();
-      boolean accumulatedData = false;
-
-      Set<DistributedMember> targetMembers = CliUtil.findMembers(groups, null);
-
-      if (targetMembers.isEmpty()) {
-        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
-      }
-
-      ResultCollector<?, ?> rc =
-          CliUtil.executeFunction(this.undeployFunction, new Object[] {jars}, targetMembers);
-      List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
-
-      for (CliFunctionResult result : results) {
-
-        if (result.getThrowable() != null) {
-          tabularData.accumulate("Member", result.getMemberIdOrName());
-          tabularData.accumulate("Un-Deployed JAR", "");
-          tabularData.accumulate("Un-Deployed JAR Location",
-              "ERROR: " + result.getThrowable().getClass().getName() + ": "
-                  + result.getThrowable().getMessage());
-          accumulatedData = true;
-          tabularData.setStatus(Result.Status.ERROR);
-        } else {
-          String[] strings = (String[]) result.getSerializables();
-          for (int i = 0; i < strings.length; i += 2) {
-            tabularData.accumulate("Member", result.getMemberIdOrName());
-            tabularData.accumulate("Un-Deployed JAR", strings[i]);
-            tabularData.accumulate("Un-Deployed From JAR Location", strings[i + 1]);
-            accumulatedData = true;
-          }
-        }
-      }
-
-      if (!accumulatedData) {
-        return ResultBuilder.createInfoResult(CliStrings.UNDEPLOY__NO_JARS_FOUND_MESSAGE);
-      }
-
-      Result result = ResultBuilder.buildResult(tabularData);
-      if (tabularData.getStatus().equals(Result.Status.OK)) {
-        persistClusterConfiguration(result,
-            () -> getSharedConfiguration().removeJars(jars, groups));
-      }
-      return result;
-    } catch (VirtualMachineError e) {
-      SystemFailure.initiateFailure(e);
-      throw e;
-    } catch (Throwable th) {
-      SystemFailure.checkFailure();
-      return ResultBuilder.createGemFireErrorResult("Exception while attempting to un-deploy: "
-          + th.getClass().getName() + ": " + th.getMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UpgradeOfflineDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UpgradeOfflineDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UpgradeOfflineDiskStoreCommand.java
deleted file mode 100644
index 85b86db..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/UpgradeOfflineDiskStoreCommand.java
+++ /dev/null
@@ -1,177 +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 java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import org.apache.geode.GemFireIOException;
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.CliUtil;
-import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.LogWrapper;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.cli.util.DiskStoreUpgrader;
-
-public class UpgradeOfflineDiskStoreCommand implements GfshCommand {
-  @CliCommand(value = CliStrings.UPGRADE_OFFLINE_DISK_STORE,
-      help = CliStrings.UPGRADE_OFFLINE_DISK_STORE__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
-  public Result upgradeOfflineDiskStore(
-      @CliOption(key = CliStrings.UPGRADE_OFFLINE_DISK_STORE__NAME, mandatory = true,
-          help = CliStrings.UPGRADE_OFFLINE_DISK_STORE__NAME__HELP) String diskStoreName,
-      @CliOption(key = CliStrings.UPGRADE_OFFLINE_DISK_STORE__DISKDIRS, mandatory = true,
-          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,
-          help = CliStrings.UPGRADE_OFFLINE_DISK_STORE__J__HELP) String[] jvmProps)
-      throws InterruptedException {
-
-    Result result;
-    LogWrapper logWrapper = LogWrapper.getInstance();
-
-    StringBuilder output = new StringBuilder();
-    StringBuilder error = new StringBuilder();
-    StringBuilder errorMessage = new StringBuilder();
-    Process upgraderProcess = null;
-
-    try {
-      String validatedDirectories = DiskStoreCommandsUtils.validatedDirectories(diskDirs);
-      if (validatedDirectories != null) {
-        throw new IllegalArgumentException(
-            "Could not find " + CliStrings.UPGRADE_OFFLINE_DISK_STORE__DISKDIRS + ": \""
-                + validatedDirectories + "\"");
-      }
-
-      List<String> commandList = new ArrayList<>();
-      commandList.add(System.getProperty("java.home") + File.separatorChar + "bin"
-          + File.separatorChar + "java");
-
-      DiskStoreCommandsUtils.configureLogging(commandList);
-
-      if (jvmProps != null && jvmProps.length != 0) {
-        commandList.addAll(Arrays.asList(jvmProps));
-      }
-      commandList.add("-classpath");
-      commandList.add(System.getProperty("java.class.path", "."));
-      commandList.add(DiskStoreUpgrader.class.getName());
-
-      commandList.add(CliStrings.UPGRADE_OFFLINE_DISK_STORE__NAME + "=" + diskStoreName);
-
-      if (diskDirs != null && diskDirs.length != 0) {
-        StringBuilder builder = new StringBuilder();
-        int arrayLength = diskDirs.length;
-        for (int i = 0; i < arrayLength; i++) {
-          if (File.separatorChar == '\\') {
-            builder.append(diskDirs[i].replace("\\", "/")); // see 46120
-          } else {
-            builder.append(diskDirs[i]);
-          }
-          if (i + 1 != arrayLength) {
-            builder.append(',');
-          }
-        }
-        commandList.add(CliStrings.UPGRADE_OFFLINE_DISK_STORE__DISKDIRS + "=" + builder.toString());
-      }
-      // -1 is ignore as maxOplogSize
-      commandList.add(CliStrings.UPGRADE_OFFLINE_DISK_STORE__MAXOPLOGSIZE + "=" + maxOplogSize);
-
-      ProcessBuilder procBuilder = new ProcessBuilder(commandList);
-      // procBuilder.redirectErrorStream(true);
-      upgraderProcess = procBuilder.start();
-      InputStream inputStream = upgraderProcess.getInputStream();
-      InputStream errorStream = upgraderProcess.getErrorStream();
-      BufferedReader inputReader = new BufferedReader(new InputStreamReader(inputStream));
-      BufferedReader errorReader = new BufferedReader(new InputStreamReader(errorStream));
-
-      String line;
-      while ((line = inputReader.readLine()) != null) {
-        output.append(line).append(GfshParser.LINE_SEPARATOR);
-      }
-
-      boolean switchToStackTrace = false;
-      while ((line = errorReader.readLine()) != null) {
-        if (!switchToStackTrace && DiskStoreUpgrader.STACKTRACE_START.equals(line)) {
-          switchToStackTrace = true;
-        } else if (switchToStackTrace) {
-          error.append(line).append(GfshParser.LINE_SEPARATOR);
-        } else {
-          errorMessage.append(line);
-        }
-      }
-
-      if (errorMessage.length() > 0) {
-        throw new GemFireIOException(errorMessage.toString());
-      }
-
-      upgraderProcess.destroy();
-      result = ResultBuilder.createInfoResult(output.toString());
-    } catch (IOException e) {
-      if (output.length() != 0) {
-        Gfsh.println(output.toString());
-      }
-      String fieldsMessage = (maxOplogSize != -1
-          ? CliStrings.UPGRADE_OFFLINE_DISK_STORE__MAXOPLOGSIZE + "=" + maxOplogSize + "," : "");
-      fieldsMessage += CliUtil.arrayToString(diskDirs);
-      String errorString = CliStrings.format(
-          CliStrings.UPGRADE_OFFLINE_DISK_STORE__MSG__ERROR_WHILE_COMPACTING_DISKSTORE_0_WITH_1_REASON_2,
-          diskStoreName, fieldsMessage);
-      result = ResultBuilder.createUserErrorResult(errorString);
-      if (logWrapper.fineEnabled()) {
-        logWrapper.fine(e.getMessage(), e);
-      }
-    } catch (GemFireIOException e) {
-      if (output.length() != 0) {
-        Gfsh.println(output.toString());
-      }
-      result = ResultBuilder.createUserErrorResult(errorMessage.toString());
-      if (logWrapper.fineEnabled()) {
-        logWrapper.fine(error.toString());
-      }
-    } catch (IllegalArgumentException e) {
-      if (output.length() != 0) {
-        Gfsh.println(output.toString());
-      }
-      result = ResultBuilder.createUserErrorResult(e.getMessage());
-    } finally {
-      if (upgraderProcess != null) {
-        try {
-          // just to check whether the process has exited
-          // Process.exitValue() throws IllegalStateException if Process is alive
-          upgraderProcess.exitValue();
-        } catch (IllegalThreadStateException itse) {
-          // not yet terminated, destroy the process
-          upgraderProcess.destroy();
-        }
-      }
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ValidateDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ValidateDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ValidateDiskStoreCommand.java
deleted file mode 100644
index 26954ba..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ValidateDiskStoreCommand.java
+++ /dev/null
@@ -1,105 +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 java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.util.DiskStoreValidater;
-
-public class ValidateDiskStoreCommand implements GfshCommand {
-  @CliCommand(value = CliStrings.VALIDATE_DISK_STORE, help = CliStrings.VALIDATE_DISK_STORE__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE}) // offline
-  // command
-  public Result validateDiskStore(
-      @CliOption(key = CliStrings.VALIDATE_DISK_STORE__NAME, mandatory = true,
-          help = CliStrings.VALIDATE_DISK_STORE__NAME__HELP) String diskStoreName,
-      @CliOption(key = CliStrings.VALIDATE_DISK_STORE__DISKDIRS, mandatory = true,
-          help = CliStrings.VALIDATE_DISK_STORE__DISKDIRS__HELP) String[] diskDirs,
-      @CliOption(key = CliStrings.VALIDATE_DISK_STORE__J,
-          help = CliStrings.VALIDATE_DISK_STORE__J__HELP) String[] jvmProps) {
-    try {
-      // create a new process ...bug 46075
-      StringBuilder dirList = new StringBuilder();
-      for (String diskDir : diskDirs) {
-        dirList.append(diskDir);
-        dirList.append(";");
-      }
-
-      List<String> commandList = new ArrayList<>();
-      commandList.add(System.getProperty("java.home") + File.separatorChar + "bin"
-          + File.separatorChar + "java");
-
-      DiskStoreCommandsUtils.configureLogging(commandList);
-
-      if (jvmProps != null && jvmProps.length != 0) {
-        commandList.addAll(Arrays.asList(jvmProps));
-      }
-
-      // Pass any java options on to the command
-      String opts = System.getenv("JAVA_OPTS");
-      if (opts != null) {
-        commandList.add(opts);
-      }
-      commandList.add("-classpath");
-      commandList.add(System.getProperty("java.class.path", "."));
-      commandList.add(DiskStoreValidater.class.getName());
-      commandList.add(diskStoreName);
-      commandList.add(dirList.toString());
-
-      ProcessBuilder procBuilder = new ProcessBuilder(commandList);
-      StringBuilder output = new StringBuilder();
-      String errorString = "";
-
-      Process validateDiskStoreProcess = procBuilder.redirectErrorStream(true).start();
-      InputStream inputStream = validateDiskStoreProcess.getInputStream();
-      BufferedReader br = new BufferedReader(new InputStreamReader(inputStream));
-      String line;
-
-      while ((line = br.readLine()) != null) {
-        output.append(line).append(GfshParser.LINE_SEPARATOR);
-      }
-      validateDiskStoreProcess.destroy();
-
-      output.append(errorString).append(GfshParser.LINE_SEPARATOR);
-      String resultString =
-          "Validating " + diskStoreName + GfshParser.LINE_SEPARATOR + output.toString();
-      return ResultBuilder.createInfoResult(resultString);
-    } catch (IOException ex) {
-      return ResultBuilder.createGemFireErrorResult(CliStrings
-          .format(CliStrings.VALIDATE_DISK_STORE__MSG__IO_ERROR, diskStoreName, ex.getMessage()));
-    } catch (Exception ex) {
-      // StringPrintWriter s = new StringPrintWriter();
-      // ex.printStackTrace(s);
-      return ResultBuilder.createGemFireErrorResult(CliStrings
-          .format(CliStrings.VALIDATE_DISK_STORE__MSG__ERROR, diskStoreName, ex.getMessage()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StopLocatorCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StopLocatorCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StopLocatorCommand.java
index da50912..5dfb54e 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StopLocatorCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StopLocatorCommand.java
@@ -15,12 +15,7 @@
 package org.apache.geode.management.internal.cli.commands.lifecycle;
 
 import static org.apache.geode.management.internal.cli.i18n.CliStrings.LOCATOR_TERM_NAME;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
+import static org.apache.geode.management.internal.cli.shell.MXBeanProvider.getMemberMXBean;
 
 import org.apache.geode.SystemFailure;
 import org.apache.geode.distributed.AbstractLauncher;
@@ -35,7 +30,10 @@ import org.apache.geode.management.internal.cli.commands.GfshCommand;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.cli.shell.MXBeanProvider;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import java.util.concurrent.TimeUnit;
 
 public class StopLocatorCommand implements GfshCommand {
   private static final long WAITING_FOR_STOP_TO_MAKE_PID_GO_AWAY_TIMEOUT_MILLIS = 30 * 1000;
@@ -123,8 +121,4 @@ public class StopLocatorCommand implements GfshCommand {
       Gfsh.redirectInternalJavaLoggers();
     }
   }
-
-  MemberMXBean getMemberMXBean(String member) throws IOException {
-    return MXBeanProvider.getMemberMXBean(member);
-  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/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
index 502eddd..88fd758 100644
--- 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
@@ -14,20 +14,19 @@
  */
 package org.apache.geode.management.internal.cli.converters;
 
-import java.util.List;
-import java.util.Set;
-
+import org.apache.geode.management.cli.ConverterHint;
+import org.apache.geode.management.internal.cli.CommandManager;
+import org.apache.geode.management.internal.cli.CommandManagerAware;
+import org.apache.geode.management.internal.cli.commands.GfshHelpCommands;
 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;
-import org.apache.geode.management.internal.cli.CommandManagerAware;
-import org.apache.geode.management.internal.cli.commands.GfshHelpCommand;
+import java.util.List;
+import java.util.Set;
 
 /**
- * {@link Converter} for {@link GfshHelpCommand#obtainHelp(String)}
+ * {@link Converter} for {@link GfshHelpCommands#obtainHelp(String)}
  * 
  *
  * @since GemFire 7.0
@@ -49,12 +48,18 @@ public class HelpConverter implements Converter<String>, CommandManagerAware {
       completionCandidates.add(new Completion(string));
     }
 
-    return completionCandidates.size() > 0;
+    if (completionCandidates.size() > 0) {
+      return true;
+    }
+    return false;
   }
 
   @Override
   public boolean supports(Class<?> arg0, String optionContext) {
-    return String.class.isAssignableFrom(arg0) && optionContext.contains(ConverterHint.HELP);
+    if (String.class.isAssignableFrom(arg0) && optionContext.contains(ConverterHint.HELP)) {
+      return true;
+    }
+    return false;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/CreateDefinedIndexesFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/CreateDefinedIndexesFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/CreateDefinedIndexesFunction.java
index 47cdb27..742840c 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/CreateDefinedIndexesFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/CreateDefinedIndexesFunction.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.management.internal.cli.functions;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -37,7 +38,7 @@ public class CreateDefinedIndexesFunction extends FunctionAdapter implements Int
   public void execute(FunctionContext context) {
     String memberId = null;
     List<Index> indexes = null;
-    Cache cache;
+    Cache cache = null;
     try {
       cache = CacheFactory.getAnyInstance();
       memberId = cache.getDistributedSystem().getDistributedMember().getId();

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/DataCommandFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/DataCommandFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/DataCommandFunction.java
index d0a6e72..96f8815 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/DataCommandFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/DataCommandFunction.java
@@ -14,8 +14,10 @@
  */
 package org.apache.geode.management.internal.cli.functions;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -24,6 +26,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.Logger;
+import org.apache.shiro.subject.Subject;
 import org.json.JSONArray;
 
 import org.apache.geode.cache.CacheFactory;
@@ -499,7 +502,6 @@ public class DataCommandFunction extends FunctionAdapter implements InternalEnti
       return DataCommandResult.createLocateEntryResult(key, null, null,
           CliStrings.LOCATE_ENTRY__MSG__KEY_EMPTY, false);
     }
-
     List<Region> listOfRegionsStartingWithRegionPath = new ArrayList<>();
 
     if (recursive) {
@@ -631,8 +633,8 @@ public class DataCommandFunction extends FunctionAdapter implements InternalEnti
       try {
         keyObject = getClassObject(key, keyClass);
       } catch (ClassNotFoundException e) {
-        return DataCommandResult.createPutResult(key, null, e,
-            CliStrings.format(CliStrings.PUT__MSG__KEY_NOT_FOUND_REGION, keyClass), false);
+        return DataCommandResult.createPutResult(key, null, null,
+            "ClassNotFoundException " + keyClass, false);
       } catch (IllegalArgumentException e) {
         return DataCommandResult.createPutResult(key, null, null,
             "Error in converting JSON " + e.getMessage(), false);

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GetSubscriptionQueueSizeFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GetSubscriptionQueueSizeFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GetSubscriptionQueueSizeFunction.java
index 4fb295f..70b649c 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GetSubscriptionQueueSizeFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/GetSubscriptionQueueSizeFunction.java
@@ -85,7 +85,7 @@ public class GetSubscriptionQueueSizeFunction extends FunctionAdapter implements
           }
         } else {
           result.setErrorMessage(
-              CliStrings.format(CliStrings.DURABLE_CQ_CLIENT_NOT_FOUND, durableClientId));
+              CliStrings.format(CliStrings.NO_CLIENT_FOUND_WITH_CLIENT_ID, durableClientId));
         }
       } else {
         result.setErrorMessage(CliStrings.NO_CLIENT_FOUND);

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
index ee9520d..6ae10a3 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/RegionCreateFunction.java
@@ -43,7 +43,7 @@ import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.internal.cli.CliUtil;
-import org.apache.geode.management.internal.cli.commands.CreateRegionCommand;
+import org.apache.geode.management.internal.cli.commands.CreateAlterDestroyRegionCommands;
 import org.apache.geode.management.internal.cli.exceptions.CreateSubregionException;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.util.RegionPath;
@@ -100,9 +100,9 @@ public class RegionCreateFunction extends FunctionAdapter implements InternalEnt
       String localizedString =
           LocalizedStrings.DiskStore_IS_USED_IN_NONPERSISTENT_REGION.toLocalizedString();
       if (localizedString.equals(e.getMessage())) {
-        exceptionMsg = exceptionMsg + " "
-            + CliStrings.format(CliStrings.CREATE_REGION__MSG__USE_ONE_OF_THESE_SHORTCUTS_0,
-                new Object[] {String.valueOf(CreateRegionCommand.PERSISTENT_OVERFLOW_SHORTCUTS)});
+        exceptionMsg = exceptionMsg + " " + CliStrings
+            .format(CliStrings.CREATE_REGION__MSG__USE_ONE_OF_THESE_SHORTCUTS_0, new Object[] {
+                String.valueOf(CreateAlterDestroyRegionCommands.PERSISTENT_OVERFLOW_SHORTCUTS)});
       }
       resultSender.lastResult(handleException(memberNameOrId, exceptionMsg, null/* do not log */));
     } catch (IllegalArgumentException e) {

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/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
index ba6d146..3525013 100644
--- 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
@@ -14,6 +14,15 @@
  */
 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;
@@ -26,16 +35,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
 
-import org.apache.commons.lang.StringUtils;
-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 org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.internal.cli.GfshParser;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-
 /**
  * 
  * 
@@ -86,7 +85,6 @@ public class Helper {
     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_SHARED_CONFIGURATION, CliStrings.TOPIC_SHARED_CONFIGURATION_DESC);
     initTopic(CliStrings.TOPIC_LOGS, CliStrings.TOPIC_LOGS__DESC);
     initTopic(CliStrings.TOPIC_CLIENT, CliStrings.TOPIC_CLIENT__DESC);
   }