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

[40/50] [abbrv] geode git commit: GEODE-2971: Introduce ExitCode to resolve inconsistency of shell exit values.

GEODE-2971: Introduce ExitCode to resolve inconsistency of shell exit values.

* this closes #652

* Corrected inconsistency of error handling and subsequent exit status in member status commands
* 'gfsh help' now returns 0 exit code as base behavior (previously returned 1).


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

Branch: refs/heads/feature/GEODE-3299
Commit: 9db5c8e58ac6c53ce21a6c0dca3f8c64555f6cfc
Parents: 3acb47e
Author: Patrick Rhomberg <pr...@pivotal.io>
Authored: Thu Jul 20 12:36:36 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Jul 31 11:23:04 2017 -0700

----------------------------------------------------------------------
 .../modules/session/installer/Installer.java    |  31 +-
 .../shell/GfshExitCodeStatusCommandsTest.java   | 391 +++++++++++++++++++
 .../geode/test/dunit/rules/gfsh/GfshRule.java   |  35 +-
 .../geode/test/dunit/rules/gfsh/GfshScript.java |  15 +
 .../java/org/apache/geode/SystemFailure.java    | 225 ++---------
 .../geode/admin/jmx/internal/AgentImpl.java     |   6 +-
 .../geode/admin/jmx/internal/AgentLauncher.java |  26 +-
 .../admin/jmx/internal/GenerateMBeanHTML.java   |  23 +-
 .../internal/deadlock/DeadlockDetector.java     |   4 +-
 .../apache/geode/internal/AvailablePort.java    |  24 +-
 .../geode/internal/DistributionLocator.java     |  27 +-
 .../org/apache/geode/internal/ExitCode.java     |  49 +++
 .../apache/geode/internal/MigrationClient.java  |   7 +-
 .../apache/geode/internal/MigrationServer.java  |   7 +-
 .../org/apache/geode/internal/SystemAdmin.java  |  12 +-
 .../internal/cache/CacheServerLauncher.java     |  28 +-
 .../cache/persistence/DiskInitFileParser.java   |   3 +-
 .../internal/cache/snapshot/GFSnapshot.java     |   3 +-
 .../geode/internal/logging/LogFileParser.java   |  16 +-
 .../geode/internal/logging/MergeLogFiles.java   |  13 +-
 .../geode/internal/logging/SortLogFile.java     |  23 +-
 .../sequencelog/io/GemfireLogConverter.java     |   4 +-
 .../internal/statistics/ArchiveSplitter.java    |  24 +-
 .../internal/statistics/StatArchiveReader.java  | 203 ++--------
 .../geode/management/internal/cli/Launcher.java |   4 +-
 .../lifecycle/StatusLocatorCommand.java         |  18 +-
 .../commands/lifecycle/StatusServerCommand.java |  11 +-
 .../com/main/WANBootStrapping_Site1_Add.java    |  15 +-
 .../com/main/WANBootStrapping_Site1_Remove.java |  14 +-
 .../com/main/WANBootStrapping_Site2_Add.java    |  14 +-
 .../com/main/WANBootStrapping_Site2_Remove.java |  12 +-
 .../codeAnalysis/decode/CompiledClass.java      |  25 +-
 .../DistributedSystemConnectPerf.java           |  10 +-
 .../LocatorLauncherRemoteIntegrationTest.java   |  31 +-
 .../ServerLauncherRemoteIntegrationTest.java    |  23 +-
 .../jta/functional/TestXACacheLoader.java       |  20 +-
 .../geode/sequence/GemfireSequenceDisplay.java  |  40 +-
 .../geode/test/dunit/standalone/ChildVM.java    |   8 +-
 .../geode/codeAnalysis/excludedClasses.txt      |   1 +
 39 files changed, 900 insertions(+), 545 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/extensions/geode-modules-session/src/main/java/org/apache/geode/modules/session/installer/Installer.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-session/src/main/java/org/apache/geode/modules/session/installer/Installer.java b/extensions/geode-modules-session/src/main/java/org/apache/geode/modules/session/installer/Installer.java
index b2d0bd0..151ebb3 100644
--- a/extensions/geode-modules-session/src/main/java/org/apache/geode/modules/session/installer/Installer.java
+++ b/extensions/geode-modules-session/src/main/java/org/apache/geode/modules/session/installer/Installer.java
@@ -15,15 +15,10 @@
 
 package org.apache.geode.modules.session.installer;
 
-import org.apache.geode.modules.session.installer.args.Argument;
-import org.apache.geode.modules.session.installer.args.ArgumentProcessor;
-import org.apache.geode.modules.session.installer.args.ArgumentValues;
-import org.apache.geode.modules.session.installer.args.UnknownArgumentHandler;
-import org.apache.geode.modules.session.installer.args.UsageException;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
+import java.io.ByteArrayOutputStream;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -32,10 +27,18 @@ import javax.xml.transform.Transformer;
 import javax.xml.transform.TransformerFactory;
 import javax.xml.transform.dom.DOMSource;
 import javax.xml.transform.stream.StreamResult;
-import java.io.ByteArrayOutputStream;
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.io.OutputStream;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+import org.apache.geode.internal.ExitCode;
+import org.apache.geode.modules.session.installer.args.Argument;
+import org.apache.geode.modules.session.installer.args.ArgumentProcessor;
+import org.apache.geode.modules.session.installer.args.ArgumentValues;
+import org.apache.geode.modules.session.installer.args.UnknownArgumentHandler;
+import org.apache.geode.modules.session.installer.args.UsageException;
 
 /**
  *
@@ -116,7 +119,7 @@ public class Installer {
         error.append(ux.getUsage());
       }
       log(error.toString());
-      System.exit(2);
+      ExitCode.INSTALL_FAILURE.doSystemExit();
     }
 
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-assembly/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExitCodeStatusCommandsTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExitCodeStatusCommandsTest.java b/geode-assembly/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExitCodeStatusCommandsTest.java
new file mode 100755
index 0000000..bc309dd
--- /dev/null
+++ b/geode-assembly/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExitCodeStatusCommandsTest.java
@@ -0,0 +1,391 @@
+/*
+ * 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 java.util.concurrent.TimeUnit.MINUTES;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.internal.AvailablePort;
+import org.apache.geode.internal.ExitCode;
+import org.apache.geode.internal.process.PidFile;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.management.internal.cli.util.ThreePhraseGenerator;
+import org.apache.geode.test.dunit.rules.gfsh.GfshExecution;
+import org.apache.geode.test.dunit.rules.gfsh.GfshRule;
+import org.apache.geode.test.dunit.rules.gfsh.GfshScript;
+import org.apache.geode.test.junit.categories.AcceptanceTest;
+
+// Originally created in response to GEODE-2971
+
+@Category(AcceptanceTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class GfshExitCodeStatusCommandsTest {
+  private static File toolsJar;
+  private static final ThreePhraseGenerator nameGenerator = new ThreePhraseGenerator();
+  private static final String memberControllerName = "member-controller";
+
+  @Rule
+  public GfshRule gfsh = new GfshRule();
+  private String locatorName;
+  private String serverName;
+
+  private int locatorPort;
+
+  // Some test configuration shorthands
+  private static final TestConfiguration LOCATOR_ONLINE_BUT_NOT_CONNECTED =
+      new TestConfiguration(true, false, false);
+  private static final TestConfiguration LOCATOR_ONLINE_AND_CONNECTED =
+      new TestConfiguration(true, false, true);
+  private static final TestConfiguration BOTH_ONLINE_BUT_NOT_CONNECTED =
+      new TestConfiguration(true, true, false);
+  private static final TestConfiguration BOTH_ONLINE_AND_CONNECTED =
+      new TestConfiguration(true, true, true);
+
+  @BeforeClass
+  public static void classSetup() {
+    File javaHome = new File(System.getProperty("java.home"));
+    String toolsPath =
+        javaHome.getName().equalsIgnoreCase("jre") ? "../lib/tools.jar" : "lib/tools.jar";
+    toolsJar = new File(javaHome, toolsPath);
+  }
+
+  @Before
+  public void setup() {
+    locatorName = "locator-" + nameGenerator.generate('-');
+    serverName = "server-" + nameGenerator.generate('-');
+    locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+  }
+
+  @Test
+  @Parameters(
+      value = {"status locator --port=-10", "status locator --pid=-1", "status server --pid=-1"})
+  public void statusCommandWithInvalidOptionValueShouldFail(String cmd) {
+    GfshScript.of(cmd).withName("test-frame").awaitAtMost(1, MINUTES)
+        .expectExitCode(ExitCode.FATAL.getValue()).execute(gfsh);
+  }
+
+
+  @Test
+  @Parameters(value = {"status locator --host=somehostname", "status locator --port=10334",
+      "status locator --dir=.", "status server --dir=.", "status locator --name=some-locator-name",
+      "status server --name=some-server-name", "status locator --pid=100",
+      "status server --pid=100"})
+  public void statusCommandWithValidOptionValueShouldFailWithNoMembers(String cmd) {
+    GfshScript.of(cmd).withName("test-frame").awaitAtMost(1, MINUTES)
+        .expectExitCode(ExitCode.FATAL.getValue()).execute(gfsh);
+  }
+
+
+  @Test
+  public void onlineStatusCommandShouldFailWhenNotConnected_locator_name() {
+    TestConfiguration config = LOCATOR_ONLINE_BUT_NOT_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByName();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.FATAL);
+  }
+
+
+  @Test
+  public void onlineStatusCommandShouldFailWhenNotConnected_server_name() {
+    TestConfiguration config = BOTH_ONLINE_BUT_NOT_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusServerCommandByName();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.FATAL);
+  }
+
+  @Test
+  public void offlineStatusCommandShouldSucceedWhenNotConnected_locator_port() {
+    // --host defaults to localhost, so `status locator --port=xxx` should still succeed.
+    TestConfiguration config = LOCATOR_ONLINE_BUT_NOT_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByPort();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+  @Test
+  public void offlineStatusCommandShouldSucceedWhenNotConnected_locator_host_and_port() {
+    // Since this is still local to the testing VM's machine, `status locator --host=localhost
+    // --port=xxx` should succeed
+    TestConfiguration config = LOCATOR_ONLINE_BUT_NOT_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByHostAndPort();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+
+
+  @Test
+  public void onlineStatusCommandShouldSucceedWhenConnected_locator_name() {
+    TestConfiguration config = LOCATOR_ONLINE_AND_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByName();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+
+  @Test
+  public void onlineStatusCommandShouldSucceedWhenConnected_server_name() {
+    TestConfiguration config = BOTH_ONLINE_AND_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusServerCommandByName();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+  @Test
+  public void onlineStatusCommandShouldSucceedWhenConnected_locator_port() {
+    TestConfiguration config = LOCATOR_ONLINE_AND_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByPort();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+  @Test
+  public void onlineStatusCommandShouldSucceedWhenConnected_locator_host_and_port() {
+    TestConfiguration config = LOCATOR_ONLINE_AND_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByHostAndPort();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+
+
+  @Test
+  public void offlineStatusCommandShouldSucceedWhenConnected_locator_dir() {
+    TestConfiguration config = LOCATOR_ONLINE_AND_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByDir();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+  @Test
+  public void offlineStatusCommandShouldSucceedWhenConnected_server_dir() {
+    TestConfiguration config = BOTH_ONLINE_AND_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusServerCommandByDir();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+  @Test
+  public void offlineStatusCommandShouldSucceedWhenConnected_locator_pid() throws IOException {
+    Assume.assumeTrue(toolsJar.exists());
+    TestConfiguration config = LOCATOR_ONLINE_AND_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByPid();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+  @Test
+  public void offlineStatusCommandShouldSucceedWhenConnected_server_pid() throws IOException {
+    Assume.assumeTrue(toolsJar.exists());
+    TestConfiguration config = BOTH_ONLINE_AND_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusServerCommandByPid();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+
+
+  @Test
+  public void offlineStatusCommandShouldSucceedEvenWhenNotConnected_locator_dir() {
+    TestConfiguration config = LOCATOR_ONLINE_BUT_NOT_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByDir();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+  @Test
+  public void offlineStatusCommandShouldSucceedEvenWhenNotConnected_server_dir() {
+    TestConfiguration config = BOTH_ONLINE_BUT_NOT_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusServerCommandByDir();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+  @Test
+  public void offlineStatusCommandShouldSucceedEvenWhenNotConnected_locator_pid()
+      throws IOException {
+    Assume.assumeTrue(toolsJar.exists());
+    TestConfiguration config = LOCATOR_ONLINE_BUT_NOT_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusLocatorCommandByPid();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+  @Test
+  public void offlineStatusCommandShouldSucceedEvenWhenNotConnected_server_pid()
+      throws IOException {
+    Assume.assumeTrue(toolsJar.exists());
+    TestConfiguration config = BOTH_ONLINE_BUT_NOT_CONNECTED;
+    config.startNecessaryMembers(startLocatorCommand(), startServerCommand(), gfsh);
+
+    String statusCommand = statusServerCommandByPid();
+    executeScriptWithExpectedExitCode(statusCommand, config, ExitCode.NORMAL);
+  }
+
+
+
+  private String startLocatorCommand() {
+    return new CommandStringBuilder("start locator").addOption("name", locatorName)
+        .addOption("port", String.valueOf(locatorPort)).toString();
+  }
+
+
+  private String startServerCommand() {
+    return new CommandStringBuilder("start server").addOption("name", serverName).toString();
+  }
+
+
+  private String connectCommand() {
+    return new CommandStringBuilder("connect")
+        .addOption("locator", String.format("localhost[%d]", locatorPort)).toString();
+  }
+
+
+  private String statusServerCommandByName() {
+    return new CommandStringBuilder("status server").addOption("name", serverName).toString();
+  }
+
+  private String statusServerCommandByDir() {
+    String serverDir = gfsh.getTemporaryFolder().getRoot().toPath().resolve(memberControllerName)
+        .resolve(serverName).toAbsolutePath().toString();
+    return new CommandStringBuilder("status server").addOption("dir", serverDir).toString();
+  }
+
+
+  private String statusServerCommandByPid() throws IOException {
+    int serverPid = snoopMemberFile(serverName, "server.pid");
+    return new CommandStringBuilder("status server").addOption("pid", String.valueOf(serverPid))
+        .toString();
+  }
+
+  private String statusLocatorCommandByName() {
+    return new CommandStringBuilder("status locator").addOption("name", locatorName).toString();
+  }
+
+  private String statusLocatorCommandByPort() {
+    return new CommandStringBuilder("status locator").addOption("port", String.valueOf(locatorPort))
+        .toString();
+  }
+
+  private String statusLocatorCommandByHostAndPort() {
+    return new CommandStringBuilder("status locator").addOption("host", "localhost")
+        .addOption("port", String.valueOf(locatorPort)).toString();
+  }
+
+  private String statusLocatorCommandByDir() {
+    String locatorDir = gfsh.getTemporaryFolder().getRoot().toPath().resolve(memberControllerName)
+        .resolve(locatorName).toAbsolutePath().toString();
+    return new CommandStringBuilder("status locator").addOption("dir", locatorDir).toString();
+  }
+
+
+  private String statusLocatorCommandByPid() throws IOException {
+    int locatorPid = snoopMemberFile(locatorName, "locator.pid");
+    return new CommandStringBuilder("status locator").addOption("pid", String.valueOf(locatorPid))
+        .toString();
+  }
+
+  private int snoopMemberFile(String memberName, String pidFileEndsWith) throws IOException {
+    File directory = gfsh.getTemporaryFolder().getRoot().toPath().resolve(memberControllerName)
+        .resolve(memberName).toFile();
+    File[] files = directory.listFiles();
+    if (files == null) {
+      throw new RuntimeException(String.format(
+          "Expected directory ('%s') for member '%s' either does not denote a directory, or an I/O error occurred.",
+          directory.toString(), memberName));
+    }
+    File pidFile = Arrays.stream(files).filter(file -> file.getName().endsWith(pidFileEndsWith))
+        .findFirst().orElseThrow(() -> new RuntimeException(String
+            .format("Expected member '%s' to have pid file but could not find it.", memberName)));
+    return new PidFile(pidFile).readPid();
+  }
+
+  private void executeScriptWithExpectedExitCode(String statusCommand, TestConfiguration config,
+      ExitCode expectedExit) {
+
+    String[] gfshScriptCommands = config.connectedToLocator
+        ? new String[] {connectCommand(), statusCommand} : new String[] {statusCommand};
+    GfshScript gfshScript = GfshScript.of(gfshScriptCommands).withName("test-frame")
+        .awaitAtMost(1, MINUTES).expectExitCode(expectedExit.getValue());
+    if (toolsJar.exists()) {
+      gfshScript.addToClasspath(toolsJar.getAbsolutePath());
+    }
+    gfshScript.execute(gfsh);
+  }
+
+
+  private static class TestConfiguration {
+    TestConfiguration(boolean locatorStarted, boolean serverStarted, boolean connectedToLocator) {
+      this.locatorStarted = locatorStarted;
+      this.serverStarted = serverStarted;
+      this.connectedToLocator = connectedToLocator;
+    }
+
+    private boolean locatorStarted;
+    private boolean serverStarted;
+    private boolean connectedToLocator;
+
+    void startNecessaryMembers(String startLocator, String startServer, GfshRule gfsh) {
+      if (!locatorStarted && !serverStarted) {
+        return;
+      }
+
+      List<String> commands = new ArrayList<>();
+      if (locatorStarted) {
+        commands.add(startLocator);
+      }
+      if (serverStarted) {
+        commands.add(startServer);
+      }
+
+      GfshExecution exec = GfshScript.of(commands.toArray(new String[] {}))
+          .withName(memberControllerName).awaitAtMost(1, MINUTES).execute(gfsh);
+      if (exec.getProcess().exitValue() != 0) {
+        throw new RuntimeException(
+            "The locator and server launcher exited with non-zero exit code.  This failure is beyond the scope of this test.");
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshRule.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshRule.java b/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshRule.java
index 23445bd..fa25f14 100644
--- a/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshRule.java
+++ b/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshRule.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.test.dunit.rules.gfsh;
 
+import static org.apache.commons.lang.SystemUtils.PATH_SEPARATOR;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.File;
@@ -22,6 +23,7 @@ import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
@@ -29,6 +31,7 @@ import java.util.stream.Collectors;
 import org.junit.rules.ExternalResource;
 import org.junit.rules.TemporaryFolder;
 
+import org.apache.geode.internal.lang.SystemUtils;
 import org.apache.geode.management.internal.cli.commands.StatusLocatorRealGfshTest;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.apache.geode.test.dunit.rules.RequiresGeodeHome;
@@ -40,6 +43,10 @@ import org.apache.geode.test.dunit.rules.RequiresGeodeHome;
  * {@link GfshRule#after()} method will attempt to clean up all forked JVMs.
  */
 public class GfshRule extends ExternalResource {
+  public TemporaryFolder getTemporaryFolder() {
+    return temporaryFolder;
+  }
+
   private TemporaryFolder temporaryFolder = new TemporaryFolder();
   private List<GfshExecution> gfshExecutions;
   private Path gfsh;
@@ -102,11 +109,25 @@ public class GfshRule extends ExternalResource {
       commandsToExecute.add("-e " + command);
     }
 
-    return new ProcessBuilder(commandsToExecute).directory(workingDir);
+    ProcessBuilder processBuilder = new ProcessBuilder(commandsToExecute);
+    processBuilder.directory(workingDir);
+
+    List<String> extendedClasspath = gfshScript.getExtendedClasspath();
+    if (!extendedClasspath.isEmpty()) {
+      Map<String, String> environmentMap = processBuilder.environment();
+      String classpathKey = "CLASSPATH";
+      String existingJavaArgs = environmentMap.get(classpathKey);
+      String specified = String.join(PATH_SEPARATOR, extendedClasspath);
+      String newValue =
+          String.format("%s%s", existingJavaArgs == null ? "" : existingJavaArgs + ":", specified);
+      environmentMap.put(classpathKey, newValue);
+    }
+
+    return processBuilder;
   }
 
   private void stopMembersQuietly(File parentDirectory) {
-    File[] potentalMemberDirectories = parentDirectory.listFiles(File::isDirectory);
+    File[] potentialMemberDirectories = parentDirectory.listFiles(File::isDirectory);
 
     Predicate<File> isServerDir = (File directory) -> Arrays.stream(directory.list())
         .anyMatch(filename -> filename.endsWith("server.pid"));
@@ -114,15 +135,16 @@ public class GfshRule extends ExternalResource {
     Predicate<File> isLocatorDir = (File directory) -> Arrays.stream(directory.list())
         .anyMatch(filename -> filename.endsWith("locator.pid"));
 
-    Arrays.stream(potentalMemberDirectories).filter(isServerDir).forEach(this::stopServerInDir);
-    Arrays.stream(potentalMemberDirectories).filter(isLocatorDir).forEach(this::stopLocatorInDir);
+    Arrays.stream(potentialMemberDirectories).filter(isServerDir).forEach(this::stopServerInDir);
+    Arrays.stream(potentialMemberDirectories).filter(isLocatorDir).forEach(this::stopLocatorInDir);
   }
 
   private void stopServerInDir(File dir) {
     String stopServerCommand =
         new CommandStringBuilder("stop server").addOption("dir", dir).toString();
 
-    GfshScript stopServerScript = new GfshScript(stopServerCommand).awaitQuietly();
+    GfshScript stopServerScript =
+        new GfshScript(stopServerCommand).withName("stop-server-teardown").awaitQuietly();
     execute(stopServerScript);
   }
 
@@ -130,7 +152,8 @@ public class GfshRule extends ExternalResource {
     String stopLocatorCommand =
         new CommandStringBuilder("stop locator").addOption("dir", dir).toString();
 
-    GfshScript stopServerScript = new GfshScript(stopLocatorCommand).awaitQuietly();
+    GfshScript stopServerScript =
+        new GfshScript(stopLocatorCommand).withName("stop-locator-teardown").awaitQuietly();
     execute(stopServerScript);
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshScript.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshScript.java b/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshScript.java
index 30c7140..52ef0d3 100644
--- a/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshScript.java
+++ b/geode-assembly/src/test/java/org/apache/geode/test/dunit/rules/gfsh/GfshScript.java
@@ -16,6 +16,9 @@ package org.apache.geode.test.dunit.rules.gfsh;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.geode.management.internal.cli.util.ThreePhraseGenerator;
@@ -27,6 +30,7 @@ public class GfshScript {
   private int timeout = 1;
   private boolean awaitQuietly = false;
   private int expectedExitValue = 0;
+  private List<String> extendedClasspath = new ArrayList<>();
 
   public GfshScript(String... commands) {
     this.commands = commands;
@@ -41,6 +45,7 @@ public class GfshScript {
 
   public GfshScript withName(String name) {
     this.name = name;
+
     return this;
   }
 
@@ -69,6 +74,16 @@ public class GfshScript {
     return this;
   }
 
+  public List<String> getExtendedClasspath() {
+    return extendedClasspath;
+  }
+
+  public GfshScript addToClasspath(String classpath) {
+    extendedClasspath.add(classpath);
+
+    return this;
+  }
+
   /**
    * Will cause the thread that executes {@link GfshScript#awaitIfNecessary} to wait, if necessary,
    * until the subprocess executing this Gfsh script has terminated, or the specified waiting time

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/SystemFailure.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/SystemFailure.java b/geode-core/src/main/java/org/apache/geode/SystemFailure.java
index 99b0671..13c8f60 100644
--- a/geode-core/src/main/java/org/apache/geode/SystemFailure.java
+++ b/geode-core/src/main/java/org/apache/geode/SystemFailure.java
@@ -12,12 +12,10 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-/**
- * 
- */
 package org.apache.geode;
 
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.SystemFailureTestHook;
 import org.apache.geode.internal.admin.remote.RemoteGfManagerAgent;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
@@ -204,11 +202,9 @@ public final class SystemFailure {
    * 
    * @see #setFailureAction(Runnable)
    */
-  private static volatile Runnable failureAction = new Runnable() {
-    public void run() {
-      System.err.println(JVM_CORRUPTION);
-      failure.printStackTrace();
-    }
+  private static volatile Runnable failureAction = () -> {
+    System.err.println(JVM_CORRUPTION);
+    failure.printStackTrace();
   };
 
   /**
@@ -241,15 +237,11 @@ public final class SystemFailure {
     return result;
   }
 
-  // merge42180: Added this method while merging 42180. It should have already be here through
-  // different merges or will come later
   /**
    * Returns true if the given Error is a fatal to the JVM and it should be shut down. Code should
    * call {@link #initiateFailure(Error)} or {@link #setFailure(Error)} if this returns true.
    */
   public static boolean isJVMFailureError(Error err) {
-    // all VirtualMachineErrors are not fatal to the JVM, in particular
-    // StackOverflowError is not
     return err instanceof OutOfMemoryError || err instanceof UnknownError;
   }
 
@@ -289,13 +281,8 @@ public final class SystemFailure {
   private final static ThreadGroup tg;
   static {
     tg = new ThreadGroup("SystemFailure Watchdog Threads") {
-      // If the watchdog is correctly written, this will never get executed.
-      // However, there's no reason for us not to eat our own dog food
-      // (har, har) -- see the javadoc above.
       @Override
       public void uncaughtException(Thread t, Throwable e) {
-        // Uhhh...if the watchdog is running, we *know* there's some
-        // sort of serious error, no need to check for it here.
         System.err.println("Internal error in SystemFailure watchdog:" + e);
         e.printStackTrace();
       }
@@ -350,18 +337,13 @@ public final class SystemFailure {
    */
   private static void startWatchDog() {
     if (failureActionCompleted) {
-      // Our work is done, don't restart
       return;
     }
     synchronized (failureSync) {
       if (watchDog != null && watchDog.isAlive()) {
         return;
       }
-      watchDog = new Thread(tg, new Runnable() {
-        public void run() {
-          runWatchDog();
-        }
-      }, "SystemFailure WatchDog");
+      watchDog = new Thread(tg, SystemFailure::runWatchDog, "SystemFailure WatchDog");
       watchDog.setDaemon(true);
       watchDog.start();
     }
@@ -402,13 +384,13 @@ public final class SystemFailure {
     try {
       basicLoadEmergencyClasses();
     } catch (ExceptionInInitializerError e) {
-      // Uhhh...are we shutting down?
+      // Determine if we're shutting down...
       boolean noSurprise = false;
       Throwable cause = e.getCause();
       if (cause != null) {
         if (cause instanceof IllegalStateException) {
           String msg = cause.getMessage();
-          if (msg.indexOf("Shutdown in progress") >= 0) {
+          if (msg.contains("Shutdown in progress")) {
             noSurprise = true;
           }
         }
@@ -416,7 +398,6 @@ public final class SystemFailure {
       if (!noSurprise) {
         logWarning(WATCHDOG_NAME, "Unable to load GemFire classes: ", e);
       }
-      // In any event, we're toast
       return;
     } catch (CancelException e) {
       // ignore this because we are shutting down anyway
@@ -447,31 +428,27 @@ public final class SystemFailure {
             return;
           }
         }
-        // Poke nose in the air, take a sniff...
+
+        // Perform watchdog sentinel duties.
 
         if (failureActionCompleted) {
-          // early out, for testing
           logInfo(WATCHDOG_NAME, "all actions completed; exiting");
         }
         if (failure == null) {
-          // Tail wag. Go back to sleep.
           logFine(WATCHDOG_NAME, "no failure detected");
           continue;
         }
-        // BOW WOW WOW WOW WOW! Corrupted system.
         if (!warned) {
           warned = logWarning(WATCHDOG_NAME, "failure detected", failure);
         }
 
-        // If any of the following fail, we will go back to sleep and
-        // retry.
         if (!gemfireCloseCompleted) {
           logInfo(WATCHDOG_NAME, "closing GemFire");
           try {
             emergencyClose();
           } catch (Throwable t) {
             logWarning(WATCHDOG_NAME, "trouble closing GemFire", t);
-            continue; // go back to sleep
+            continue;
           }
           gemfireCloseCompleted = true;
         }
@@ -485,7 +462,7 @@ public final class SystemFailure {
               r.run();
             } catch (Throwable t) {
               logWarning(WATCHDOG_NAME, "trouble running user's runnable", t);
-              continue; // go back to sleep
+              continue;
             }
           }
           failureActionCompleted = true;
@@ -495,32 +472,26 @@ public final class SystemFailure {
         stopProctor();
 
         if (exitOK) {
-          logWarning(WATCHDOG_NAME,
-              // No "+" in this long message, we're out of memory!
-              CALLING_SYSTEM_EXIT, exitExcuse);
+          logWarning(WATCHDOG_NAME, CALLING_SYSTEM_EXIT, exitExcuse);
 
           // ATTENTION: there are VERY FEW places in GemFire where it is
           // acceptable to call System.exit. This is one of those
           // places...
-          System.exit(1);
+          ExitCode.FATAL.doSystemExit();
         }
 
 
-        // Our job here is done
         logInfo(WATCHDOG_NAME, "exiting");
         return;
       } catch (Throwable t) {
-        // We *never* give up. NEVER EVER!
         logWarning(WATCHDOG_NAME, "thread encountered a problem: " + t, t);
       }
-    } // for
+    }
   }
 
   /**
    * Spies on system statistics looking for low memory threshold
    * 
-   * Well, if you're gonna have a watchdog, why not a watch CAT????
-   * 
    * @guarded.By {@link #failureSync}
    * @see #minimumMemoryThreshold
    */
@@ -542,9 +513,8 @@ public final class SystemFailure {
    * @see #setFailureMemoryThreshold(long)
    * @guarded.By {@link #memorySync}
    */
-  static long minimumMemoryThreshold =
-      Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "SystemFailure.chronic_memory_threshold",
-          1048576).longValue();
+  static long minimumMemoryThreshold = Long.getLong(
+      DistributionConfig.GEMFIRE_PREFIX + "SystemFailure.chronic_memory_threshold", 1048576);
 
   /**
    * This is the interval, in seconds, that the proctor thread will awaken and poll system free
@@ -556,8 +526,7 @@ public final class SystemFailure {
    * @see #setFailureMemoryThreshold(long)
    */
   static final public long MEMORY_POLL_INTERVAL =
-      Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "SystemFailure.MEMORY_POLL_INTERVAL", 1)
-          .longValue();
+      Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "SystemFailure.MEMORY_POLL_INTERVAL", 1);
 
   /**
    * This is the maximum amount of time, in seconds, that the proctor thread will tolerate seeing
@@ -569,8 +538,8 @@ public final class SystemFailure {
    * 
    * @see #setFailureMemoryThreshold(long)
    */
-  static final public long MEMORY_MAX_WAIT = Long
-      .getLong(DistributionConfig.GEMFIRE_PREFIX + "SystemFailure.MEMORY_MAX_WAIT", 15).longValue();
+  static final public long MEMORY_MAX_WAIT =
+      Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "SystemFailure.MEMORY_MAX_WAIT", 15);
 
   /**
    * Flag that determines whether or not we monitor memory on our own. If this flag is set, we will
@@ -592,7 +561,6 @@ public final class SystemFailure {
    */
   private static void startProctor() {
     if (failure != null) {
-      // no point!
       notifyWatchDog();
       return;
     }
@@ -600,18 +568,14 @@ public final class SystemFailure {
       if (proctor != null && proctor.isAlive()) {
         return;
       }
-      proctor = new Thread(tg, new Runnable() {
-        public void run() {
-          runProctor();
-        }
-      }, "SystemFailure Proctor");
+      proctor = new Thread(tg, SystemFailure::runProctor, "SystemFailure Proctor");
       proctor.setDaemon(true);
       proctor.start();
     }
   }
 
   private static void stopProctor() {
-    Thread proctorSnapshot = null;
+    Thread proctorSnapshot;
     synchronized (failureSync) {
       stopping = true;
       proctorSnapshot = proctor;
@@ -644,21 +608,18 @@ public final class SystemFailure {
   static private long lastTotalMemory = 0;
 
   /**
-   * This is the run loop for the proctor thread (formally known as the "watchcat" (grin)
+   * This is the run loop for the proctor thread
    */
   static protected void runProctor() {
     // Note that the javadocs say this can return Long.MAX_VALUE.
-    // If it does, the proctor will never do its job...
     final long maxMemory = Runtime.getRuntime().maxMemory();
 
-    // Allocate this error in advance, since it's too late once
-    // it's been detected!
+    // Allocate this error in advance, since it's too late once it's been detected!
     final OutOfMemoryError oome = new OutOfMemoryError(
         LocalizedStrings.SystemFailure_0_MEMORY_HAS_REMAINED_CHRONICALLY_BELOW_1_BYTES_OUT_OF_A_MAXIMUM_OF_2_FOR_3_SEC
             .toLocalizedString(new Object[] {PROCTOR_NAME, Long.valueOf(minimumMemoryThreshold),
                 Long.valueOf(maxMemory), Integer.valueOf(WATCHDOG_WAIT)}));
 
-    // Catenation, but should be OK when starting up
     logFine(PROCTOR_NAME,
         "Starting, threshold = " + minimumMemoryThreshold + "; max = " + maxMemory);
     for (;;) {
@@ -670,7 +631,6 @@ public final class SystemFailure {
       }
 
       try {
-        // *** catnap...
         try {
           Thread.sleep(MEMORY_POLL_INTERVAL * 1000);
         } catch (InterruptedException e) {
@@ -681,13 +641,11 @@ public final class SystemFailure {
           return;
         }
 
-        // *** Twitch ear, take a bath...
         if (failureActionCompleted) {
-          // it's all over, we're late
           return;
         }
         if (failure != null) {
-          notifyWatchDog(); // wake the dog, just in case
+          notifyWatchDog();
           logFine(PROCTOR_NAME, "Failure has been reported, exiting");
           return;
         }
@@ -696,13 +654,9 @@ public final class SystemFailure {
           continue;
         }
 
-        // *** Sit up, stretch...
         long totalMemory = Runtime.getRuntime().totalMemory();
         if (totalMemory < maxMemory) {
-          // We haven't finished growing the heap, so no worries...yet
           if (DEBUG) {
-            // This message has catenation, we don't want this in
-            // production code :-)
             logFine(PROCTOR_NAME,
                 "totalMemory (" + totalMemory + ") < maxMemory (" + maxMemory + ")");
           }
@@ -710,24 +664,17 @@ public final class SystemFailure {
           continue;
         }
         if (lastTotalMemory < totalMemory) {
-          // Don't get too impatient if the heap just now grew
-          lastTotalMemory = totalMemory; // now we're maxed
-          firstStarveTime = NEVER_STARVED; // reset the clock
+          lastTotalMemory = totalMemory;
+          firstStarveTime = NEVER_STARVED;
           continue;
         }
-        lastTotalMemory = totalMemory; // make a note of this
+        lastTotalMemory = totalMemory;
 
-        // *** Hey, is that the food bowl?
-
-        // At this point, freeMemory really indicates how much
-        // trouble we're in.
         long freeMemory = Runtime.getRuntime().freeMemory();
         if (freeMemory == 0) {
-          /*
-           * This is to workaround X bug #41821 in JRockit. Often, Jrockit returns 0 from
-           * Runtime.getRuntime().freeMemory() Allocating this one object and calling again seems to
-           * workaround the problem.
-           */
+          // This is to workaround X bug #41821 in JRockit. Often, Jrockit returns 0 from
+          // Runtime.getRuntime().freeMemory() Allocating this one object and calling again seems to
+          // workaround the problem.
           new Object();
           freeMemory = Runtime.getRuntime().freeMemory();
         }
@@ -740,12 +687,9 @@ public final class SystemFailure {
           lastStarveTime = firstStarveTime;
         }
 
-        if (freeMemory >= curThreshold /* enough memory */
-            || curThreshold == 0 /* disabled */) {
+        if (freeMemory >= curThreshold || curThreshold == 0) {
           // Memory is FINE, reset everything
           if (DEBUG) {
-            // This message has catenation, we don't want this in
-            // production code :-)
             logFine(PROCTOR_NAME, "Current free memory is: " + freeMemory);
           }
 
@@ -757,14 +701,11 @@ public final class SystemFailure {
           }
           continue;
         }
-        // Memory is low
 
-        // *** Leap to feet, nose down, tail switching...
+        // Memory is low
         long now = System.currentTimeMillis();
         if (lastStarveTime == NEVER_STARVED) {
-          // first sighting
           if (DEBUG) {
-            // Catenation in this message, don't put in production
             logFine(PROCTOR_NAME,
                 "Noting current memory " + freeMemory + " is less than threshold " + curThreshold);
           } else {
@@ -776,15 +717,12 @@ public final class SystemFailure {
           synchronized (memorySync) {
             firstStarveTime = now;
           }
-          System.gc(); // at least TRY...
+          System.gc(); // Attempt to free memory and avoid overflow
           continue;
         }
 
-        // *** squirm, wait for the right moment...wait...wait...
         if (now - lastStarveTime < MEMORY_MAX_WAIT * 1000) {
-          // Very recent; problem may correct itself.
           if (DEBUG) {
-            // catenation
             logFine(PROCTOR_NAME, "...memory is still below threshold: " + freeMemory);
           } else {
             logWarning(PROCTOR_NAME,
@@ -795,18 +733,14 @@ public final class SystemFailure {
           continue;
         }
 
-        // *** Meow! Meow! MEOWWWW!!!!!
-
-        // Like any smart cat, let the Dog do all the work.
         logWarning(PROCTOR_NAME, "Memory is chronically low; setting failure!", null);
         SystemFailure.setFailure(oome);
         notifyWatchDog();
-        return; // we're done!
+        return;
       } catch (Throwable t) {
         logWarning(PROCTOR_NAME, "thread encountered a problem", t);
-        // We *never* give up. NEVER EVER!
       }
-    } // for
+    }
   }
 
   /**
@@ -837,10 +771,6 @@ public final class SystemFailure {
    * Just make sure to do it while you still have memory to succeed!
    */
   public static void loadEmergencyClasses() {
-    // This method was called to basically load this class
-    // and invoke its static initializers. Now that we don't
-    // use statics to start the threads all we need to do is
-    // call startThreads. The watchdog thread will call basicLoadEmergencyClasses.
     startThreads();
   }
 
@@ -870,22 +800,18 @@ public final class SystemFailure {
    * system.
    */
   public static void emergencyClose() {
-    // Make the cache (more) useless and inaccessible...
     if (TRACE_CLOSE) {
       System.err.println("SystemFailure: closing GemFireCache");
     }
     GemFireCacheImpl.emergencyClose();
 
-    // Arcane strange DS's exist in this class:
     if (TRACE_CLOSE) {
       System.err.println("SystemFailure: closing admins");
     }
     RemoteGfManagerAgent.emergencyClose();
 
-    // If memory was the problem, make an explicit attempt at
-    // this point to clean up.
-
-    System.gc(); // This will fail if we're out of memory?/
+    // If memory was the problem, make an explicit attempt at this point to clean up.
+    System.gc();
 
     if (TRACE_CLOSE) {
       System.err.println("SystemFailure: end of emergencyClose");
@@ -904,7 +830,6 @@ public final class SystemFailure {
    * @throws Error
    */
   static private void throwFailure() throws InternalGemFireError, Error {
-    // Do not return normally...
     if (failure != null)
       throw failure;
   }
@@ -913,7 +838,7 @@ public final class SystemFailure {
    * Notifies the watchdog thread (assumes that {@link #failure} has been set)
    */
   private static void notifyWatchDog() {
-    startWatchDog(); // just in case
+    startWatchDog();
     synchronized (failureSync) {
       failureSync.notifyAll();
     }
@@ -968,10 +893,6 @@ public final class SystemFailure {
     if (SystemFailureTestHook.errorIsExpected(failure)) {
       return;
     }
-    // created (OutOfMemoryError), and no stack frames are created
-    // (StackOverflowError). There is a slight chance that the
-    // very first error may get overwritten, but this avoids the
-    // potential of object creation via a fat lock
     SystemFailure.failure = failure;
     notifyWatchDog();
   }
@@ -1038,47 +959,13 @@ public final class SystemFailure {
     synchronized (memorySync) {
       result = minimumMemoryThreshold;
       minimumMemoryThreshold = newVal;
-      firstStarveTime = NEVER_STARVED; // reset
+      firstStarveTime = NEVER_STARVED;
     }
-    startProctor(); // just in case
+    startProctor();
     return result;
   }
 
-  // /**
-  // * For use by GemStone Quality Assurance Only
-  // *
-  // * @deprecated TODO remove this
-  // */
-  // public static void reset() {
-  // System.gc();
-  // logWarning("DJP", "do not commit SystemFailure#reset", null);
-  // failure = null;
-  // failureAction = new Runnable() {
-  // public void run() {
-  // System.err.println("(SystemFailure) JVM corruption has been detected!");
-  // failure.printStackTrace();
-  // }
-  // };
-  // gemfireCloseCompleted = false;
-  // failureActionCompleted = false;
-  // synchronized (failureSync) {
-  // if (watchDog != null) {
-  // watchDog.interrupt();
-  // }
-  // watchDog = null;
-  // if (watchCat != null) {
-  // watchCat.interrupt();
-  // }
-  // watchCat = null;
-  // }
-  //
-  // startWatchDog();
-  // startWatchCat();
-  // }
-
   static private boolean logStdErr(String kind, String name, String s, Throwable t) {
-    // As far as I can tell, this code path doesn't allocate
-    // any objects!!!!
     try {
       System.err.print(name);
       System.err.print(": [");
@@ -1105,16 +992,6 @@ public final class SystemFailure {
    */
   static protected boolean logWarning(String name, String s, Throwable t) {
     return logStdErr("warning", name, s, t);
-    // if (PREFER_STDERR) {
-    // return logStdErr("warning", name, s, t);
-    // }
-    // try {
-    // log.warning(name + ": " + s, t);
-    // return true;
-    // }
-    // catch (Throwable t2) {
-    // return logStdErr("warning", name, s, t);
-    // }
   }
 
   /**
@@ -1125,16 +1002,6 @@ public final class SystemFailure {
    */
   static protected void logInfo(String name, String s) {
     logStdErr("info", name, s, null);
-    // if (PREFER_STDERR) {
-    // logStdErr("info", name, s, null);
-    // return;
-    // }
-    // try {
-    // log.info(name + ": " + s);
-    // }
-    // catch (Throwable t) {
-    // logStdErr("info", name, s, t);
-    // }
   }
 
   /**
@@ -1147,18 +1014,6 @@ public final class SystemFailure {
     if (DEBUG) {
       logStdErr("fine", name, s, null);
     }
-    // if (DEBUG && PREFER_STDERR) {
-    // logStdErr("fine", name, s, null);
-    // return;
-    // }
-    // try {
-    // log.fine(name + ": " + s);
-    // }
-    // catch (Throwable t) {
-    // if (DEBUG) {
-    // logStdErr("fine", name, s, null);
-    // }
-    // }
   }
 
   private static volatile boolean stopping;

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentImpl.java b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentImpl.java
index e55c3f1..9eb741f 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentImpl.java
@@ -43,7 +43,6 @@ import javax.management.remote.rmi.RMIConnectorServer;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
 import mx4j.tools.adaptor.http.HttpAdaptor;
-
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.GemFireException;
@@ -58,6 +57,7 @@ import org.apache.geode.admin.jmx.AgentFactory;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.i18n.StringId;
 import org.apache.geode.internal.Banner;
+import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.admin.remote.TailLogResponse;
 import org.apache.geode.internal.i18n.LocalizedStrings;
@@ -1054,7 +1054,7 @@ public class AgentImpl implements org.apache.geode.admin.jmx.Agent,
     } catch (RuntimeException ex) {
       System.err
           .println(LocalizedStrings.AgentImpl_FAILED_READING_CONFIGURATION_0.toLocalizedString(ex));
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
       return;
     }
 
@@ -1075,7 +1075,7 @@ public class AgentImpl implements org.apache.geode.admin.jmx.Agent,
       // is still usable:
       SystemFailure.checkFailure();
       t.printStackTrace();
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentLauncher.java b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentLauncher.java
index 6a1bd63..7731293 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentLauncher.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentLauncher.java
@@ -44,11 +44,12 @@ import org.apache.geode.admin.jmx.Agent;
 import org.apache.geode.admin.jmx.AgentConfig;
 import org.apache.geode.admin.jmx.AgentFactory;
 import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.PureJavaMode;
-import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.util.IOUtils;
 import org.apache.geode.internal.util.JavaCommandBuilder;
 
@@ -176,7 +177,7 @@ public class AgentLauncher {
     }
     out.println("");
 
-    System.exit(1);
+    ExitCode.FATAL.doSystemExit();
   }
 
   /**
@@ -263,7 +264,7 @@ public class AgentLauncher {
     // the status file was not successfully written to
     pollAgentUntilRunning();
 
-    System.exit(0);
+    ExitCode.NORMAL.doSystemExit();
   }
 
   private void verifyAndClearStatus() throws Exception {
@@ -448,7 +449,7 @@ public class AgentLauncher {
           OSProcess.getId(), message, cause));
     } catch (Exception e) {
       logger.fatal(e.getMessage(), e);
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
   }
 
@@ -459,9 +460,10 @@ public class AgentLauncher {
 
       if (isStatus(SHUTDOWN_PENDING, SHUTDOWN_PENDING_AFTER_FAILED_STARTUP)) {
         agent.stop();
-        final int exitCode = (isStatus(SHUTDOWN_PENDING_AFTER_FAILED_STARTUP) ? 1 : 0);
+        final ExitCode exitCode =
+            (isStatus(SHUTDOWN_PENDING_AFTER_FAILED_STARTUP) ? ExitCode.FATAL : ExitCode.NORMAL);
         writeStatus(createStatus(this.status, SHUTDOWN));
-        System.exit(exitCode);
+        exitCode.doSystemExit();
       }
     }
   }
@@ -497,7 +499,7 @@ public class AgentLauncher {
 
     workingDirectory = IOUtils.tryGetCanonicalFileElseGetAbsoluteFile((File) options.get(DIR));
 
-    int exitStatus = 1;
+    ExitCode exitCode = ExitCode.FATAL;
 
     if (new File(workingDirectory, statusFileName).exists()) {
       spinReadStatus();
@@ -512,7 +514,7 @@ public class AgentLauncher {
         System.out
             .println(LocalizedStrings.AgentLauncher_0_HAS_STOPPED.toLocalizedString(this.basename));
         deleteStatus();
-        exitStatus = 0;
+        exitCode = ExitCode.NORMAL;
       } else {
         System.out
             .println(LocalizedStrings.AgentLauncher_TIMEOUT_WAITING_FOR_0_TO_SHUTDOWN_STATUS_IS_1
@@ -524,7 +526,7 @@ public class AgentLauncher {
               .toLocalizedString(workingDirectory));
     }
 
-    System.exit(exitStatus);
+    exitCode.doSystemExit();
   }
 
   private void pollAgentForShutdown() throws InterruptedException {
@@ -545,7 +547,7 @@ public class AgentLauncher {
     this.workingDirectory =
         IOUtils.tryGetCanonicalFileElseGetAbsoluteFile((File) getStopOptions(args).get(DIR));
     System.out.println(getStatus());
-    System.exit(0);
+    ExitCode.NORMAL.doSystemExit();
   }
 
   /**
@@ -779,7 +781,7 @@ public class AgentLauncher {
     out.println("\t" + LocalizedStrings.AgentLauncher_DIR.toLocalizedString());
     out.println();
 
-    System.exit(1);
+    ExitCode.FATAL.doSystemExit();
   }
 
   /**
@@ -833,7 +835,7 @@ public class AgentLauncher {
       t.printStackTrace();
       System.err.println(
           LocalizedStrings.AgentLauncher_ERROR_0.toLocalizedString(t.getLocalizedMessage()));
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/GenerateMBeanHTML.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/GenerateMBeanHTML.java b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/GenerateMBeanHTML.java
index 5375a19..2c33215 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/GenerateMBeanHTML.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/GenerateMBeanHTML.java
@@ -14,14 +14,27 @@
  */
 package org.apache.geode.admin.jmx.internal;
 
-import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.internal.i18n.LocalizedStrings;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.io.PrintWriter;
 
 import javax.xml.parsers.SAXParser;
 import javax.xml.parsers.SAXParserFactory;
-import org.xml.sax.*;
+
+import org.xml.sax.Attributes;
+import org.xml.sax.InputSource;
+import org.xml.sax.Locator;
+import org.xml.sax.SAXException;
+import org.xml.sax.SAXNotRecognizedException;
+import org.xml.sax.SAXParseException;
 import org.xml.sax.helpers.DefaultHandler;
-import java.io.*;
+
+import org.apache.geode.internal.ClassPathLoader;
+import org.apache.geode.internal.ExitCode;
+import org.apache.geode.internal.i18n.LocalizedStrings;
 // import java.util.*;
 
 /**
@@ -464,7 +477,7 @@ public class GenerateMBeanHTML extends DefaultHandler {
 
     err.println("");
 
-    System.exit(1);
+    ExitCode.FATAL.doSystemExit();
   }
 
   public static void main(String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/distributed/internal/deadlock/DeadlockDetector.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/deadlock/DeadlockDetector.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/deadlock/DeadlockDetector.java
index 6806288..385065e 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/deadlock/DeadlockDetector.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/deadlock/DeadlockDetector.java
@@ -35,6 +35,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.geode.internal.ExitCode;
+
 /**
  * A class used for detecting deadlocks. The static method
  * {@link #collectAllDependencies(Serializable)} will find all dependencies between threads and
@@ -304,7 +306,7 @@ public class DeadlockDetector {
     File file = new File(filename);
     if (!file.exists()) {
       System.err.println("unable to find " + filename);
-      System.exit(-1);
+      ExitCode.DEPENDENCY_GRAPH_FAILURE.doSystemExit();
     }
 
     ObjectInputStream ois =

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/AvailablePort.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/AvailablePort.java b/geode-core/src/main/java/org/apache/geode/internal/AvailablePort.java
index f22872f..b0c401e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/AvailablePort.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/AvailablePort.java
@@ -14,17 +14,27 @@
  */
 package org.apache.geode.internal;
 
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.internal.net.SocketCreator;
-
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.Serializable;
-import java.net.*;
+import java.net.DatagramPacket;
+import java.net.Inet6Address;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.MulticastSocket;
+import java.net.NetworkInterface;
+import java.net.ServerSocket;
+import java.net.SocketAddress;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.net.UnknownHostException;
 import java.util.Enumeration;
 import java.util.Random;
 
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.net.SocketCreator;
+
 /**
  * This class determines whether or not a given port is available and can also provide a randomly
  * selected available port.
@@ -493,7 +503,7 @@ public class AvailablePort {
         LocalizedStrings.AvailablePort_THIS_PROGRAM_EITHER_PRINTS_WHETHER_OR_NOT_A_PORT_IS_AVAILABLE_FOR_A_GIVEN_PROTOCOL_OR_IT_PRINTS_OUT_AN_AVAILABLE_PORT_FOR_A_GIVEN_PROTOCOL
             .toLocalizedString());
     err.println("");
-    System.exit(1);
+    ExitCode.FATAL.doSystemExit();
   }
 
   public static void main(String[] args) {
@@ -538,7 +548,7 @@ public class AvailablePort {
         addr = InetAddress.getByName(addrString);
       } catch (Exception e) {
         e.printStackTrace();
-        System.exit(1);
+        ExitCode.FATAL.doSystemExit();
       }
     }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java b/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
index e190d0b..a81932b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DistributionLocator.java
@@ -14,21 +14,22 @@
  */
 package org.apache.geode.internal;
 
+import java.io.File;
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.InetAddress;
+import java.util.Properties;
+
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.SystemFailure;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.distributed.internal.tcpserver.*;
+import org.apache.geode.distributed.internal.tcpserver.TcpClient;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
-import org.apache.logging.log4j.Logger;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.ConnectException;
-import java.net.InetAddress;
-import java.util.Properties;
 
 /**
  * This class is used to work with a managed VM that hosts a
@@ -114,7 +115,7 @@ public class DistributionLocator {
       System.err.println(
           LocalizedStrings.DistributionLocator_A_ZEROLENGTH_HOSTNAMEFORCLIENTS_WILL_DEFAULT_TO_BINDADDRESS
               .toLocalizedString());
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     } else {
       shutdown = false;
     }
@@ -132,7 +133,7 @@ public class DistributionLocator {
           System.err.println(
               LocalizedStrings.DistributionLocator__0_IS_NOT_A_VALID_IP_ADDRESS_FOR_THIS_MACHINE
                   .toLocalizedString(args[1]));
-          System.exit(1);
+          ExitCode.FATAL.doSystemExit();
         }
         address = InetAddress.getByName(args[1]);
       } else {
@@ -183,14 +184,14 @@ public class DistributionLocator {
     } catch (java.net.BindException ex) {
       logger.fatal(LocalizedMessage.create(
           LocalizedStrings.DistributionLocator_COULD_NOT_BIND_LOCATOR_TO__0__1,
-          new Object[] {address, Integer.valueOf(port)}));
-      System.exit(1);
+          new Object[] {address, port}));
+      ExitCode.FATAL.doSystemExit();
 
     } catch (Exception ex) {
       logger.fatal(
           LocalizedMessage.create(LocalizedStrings.DistributionLocator_COULD_NOT_START_LOCATOR),
           ex);
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/ExitCode.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/ExitCode.java b/geode-core/src/main/java/org/apache/geode/internal/ExitCode.java
new file mode 100644
index 0000000..a7d7d07
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/ExitCode.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal;
+
+import java.util.Arrays;
+
+import org.springframework.shell.core.ExitShellRequest;
+
+
+public enum ExitCode {
+
+  // JVM_TERMINATED_EXIT(99) exists for coverage of Spring's ExitShellRequest values in fromSpring.
+  DEPENDENCY_GRAPH_FAILURE(-1), NORMAL(0), FATAL(1), INSTALL_FAILURE(2), JVM_TERMINATED_EXIT(99);
+
+  private final int shellReturnValue;
+
+  ExitCode(final int shellReturnValue) {
+    this.shellReturnValue = shellReturnValue;
+  }
+
+  public int getValue() {
+    return shellReturnValue;
+  }
+
+  public void doSystemExit() {
+    System.exit(this.shellReturnValue);
+  }
+
+  public static ExitCode fromValue(int i) {
+    return Arrays.stream(ExitCode.values()).filter(c -> c.getValue() == i).findFirst().orElseThrow(
+        () -> new IllegalArgumentException("No ExitCode exists with shell exit value: " + i));
+  }
+
+  public static ExitCode fromSpring(ExitShellRequest s) {
+    return fromValue(s.getExitCode());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/MigrationClient.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/MigrationClient.java b/geode-core/src/main/java/org/apache/geode/internal/MigrationClient.java
index 1f3ef2a..d7f6b44 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/MigrationClient.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/MigrationClient.java
@@ -14,7 +14,10 @@
  */
 package org.apache.geode.internal;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.distributed.ConfigurationProperties.CACHE_XML_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -97,7 +100,7 @@ public class MigrationClient {
       instance = new MigrationClient(cacheXmlFileName, bindAddressName, serverPort);
     } catch (IllegalArgumentException e) {
       System.err.println(e.getMessage());
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
     instance.createDistributedSystem();
     instance.createCache();

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/MigrationServer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/MigrationServer.java b/geode-core/src/main/java/org/apache/geode/internal/MigrationServer.java
index 1252eb0..7266c89 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/MigrationServer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/MigrationServer.java
@@ -14,7 +14,10 @@
  */
 package org.apache.geode.internal;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.distributed.ConfigurationProperties.CACHE_XML_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -147,7 +150,7 @@ public class MigrationServer {
       instance = new MigrationServer(cacheXmlFileName, bindAddressName, listenPort);
     } catch (IllegalArgumentException e) {
       System.err.println(e.getMessage());
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
     instance.createDistributedSystem();
     instance.createCache();

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java b/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
index ccc64e3..e431012 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
@@ -1416,7 +1416,7 @@ public class SystemAdmin {
   protected void usage(String cmd) {
     System.err.println(
         LocalizedStrings.SystemAdmin_USAGE.toLocalizedString() + " " + getUsageString(cmd));
-    System.exit(1);
+    ExitCode.FATAL.doSystemExit();
   }
 
   private final static String[] validCommands = new String[] {"version", "stats", START_LOCATOR,
@@ -2006,7 +2006,7 @@ public class SystemAdmin {
       if (debug) {
         ex.printStackTrace(System.err);
       }
-      System.exit(1); // fix for bug 28351
+      ExitCode.FATAL.doSystemExit(); // fix for bug 28351
     }
     if (cmdLine.size() == 0) {
       if (help) {
@@ -2041,7 +2041,7 @@ public class SystemAdmin {
       if (debug) {
         ex.printStackTrace(System.err);
       }
-      System.exit(1); // fix for bug 28351
+      ExitCode.FATAL.doSystemExit(); // fix for bug 28351
     }
     if (needsSysDir(cmd) && !help) {
       if (sysDirName != null && sysDirName.length() > 0) {
@@ -2237,7 +2237,7 @@ public class SystemAdmin {
       if (debug) {
         ex.printStackTrace(System.err);
       }
-      System.exit(1); // fix for bug 28351
+      ExitCode.FATAL.doSystemExit(); // fix for bug 28351
     } catch (IllegalArgumentException ex) {
       System.err.println(LocalizedStrings.SystemAdmin_ERROR_OPERATION_0_FAILED_BECAUSE_1
           .toLocalizedString(new Object[] {cmd, getExceptionMessage(ex)}));
@@ -2245,14 +2245,14 @@ public class SystemAdmin {
       if (debug) {
         ex.printStackTrace(System.err);
       }
-      System.exit(1); // fix for bug 28351
+      ExitCode.FATAL.doSystemExit(); // fix for bug 28351
     } catch (Exception ex) {
       System.err.println(LocalizedStrings.SystemAdmin_ERROR_OPERATION_0_FAILED_BECAUSE_1
           .toLocalizedString(new Object[] {cmd, getExceptionMessage(ex)}));
       if (debug) {
         ex.printStackTrace(System.err);
       }
-      System.exit(1); // fix for bug 28351
+      ExitCode.FATAL.doSystemExit(); // fix for bug 28351
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerLauncher.java b/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerLauncher.java
index ab53ca4..5faa72b 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerLauncher.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerLauncher.java
@@ -15,7 +15,8 @@
 
 package org.apache.geode.internal.cache;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.SERVER_BIND_ADDRESS;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -46,6 +47,7 @@ import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionConfigImpl;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.i18n.LogWriterI18n;
+import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.PureJavaMode;
 import org.apache.geode.internal.cache.tier.sockets.CacheServerHelper;
@@ -175,7 +177,7 @@ public class CacheServerLauncher {
   protected void status(final String[] args) throws Exception {
     workingDir = (File) getStopOptions(args).get(DIR);
     System.out.println(getStatus());
-    System.exit(0);
+    ExitCode.NORMAL.doSystemExit();
   }
 
   /**
@@ -216,11 +218,11 @@ public class CacheServerLauncher {
           launcher.status(args);
         } else {
           launcher.usage();
-          System.exit(1);
+          ExitCode.FATAL.doSystemExit();
         }
       } else {
         launcher.usage();
-        System.exit(1);
+        ExitCode.FATAL.doSystemExit();
       }
 
       throw new Exception(LocalizedStrings.CacheServerLauncher_INTERNAL_ERROR_SHOULDNT_REACH_HERE
@@ -251,7 +253,7 @@ public class CacheServerLauncher {
         System.out.println(
             LocalizedStrings.CacheServerLauncher_ERROR_0.toLocalizedString(t.getMessage()));
       }
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
   }
 
@@ -500,7 +502,7 @@ public class CacheServerLauncher {
       return;
     }
 
-    System.exit(0);
+    ExitCode.NORMAL.doSystemExit();
   }
 
   private void verifyAndClearStatus() throws Exception {
@@ -765,7 +767,7 @@ public class CacheServerLauncher {
         }
         if (!reconnected) {
           // shutdown-all disconnected the DS
-          System.exit(0);
+          ExitCode.NORMAL.doSystemExit();
         }
       }
     }
@@ -902,7 +904,7 @@ public class CacheServerLauncher {
 
     // determine the current state of the Cache Server process...
     final File statusFile = new File(this.workingDir, this.statusName);
-    int exitStatus = 1;
+    ExitCode exitCode = ExitCode.FATAL;
 
     if (statusFile.exists()) {
       this.status = spinReadStatus();
@@ -925,7 +927,7 @@ public class CacheServerLauncher {
         System.out.println(
             LocalizedStrings.CacheServerLauncher_0_STOPPED.toLocalizedString(this.baseName));
         deleteStatus();
-        exitStatus = 0;
+        exitCode = ExitCode.NORMAL;
       } else {
         System.out.println(
             LocalizedStrings.CacheServerLauncher_TIMEOUT_WAITING_FOR_0_TO_SHUTDOWN_STATUS_IS_1
@@ -941,7 +943,7 @@ public class CacheServerLauncher {
       return;
     }
 
-    System.exit(exitStatus);
+    exitCode.doSystemExit();
   }
 
   private void pollCacheServerForShutdown() throws InterruptedException {
@@ -1037,7 +1039,7 @@ public class CacheServerLauncher {
       } else {
         e.printStackTrace();
       }
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
   }
 
@@ -1178,7 +1180,7 @@ public class CacheServerLauncher {
           }
           if (status.state == SHUTDOWN) {
             System.out.println(status);
-            System.exit(1);
+            ExitCode.FATAL.doSystemExit();
           }
           break;
         default:
@@ -1203,7 +1205,7 @@ public class CacheServerLauncher {
       } catch (IOException io) {
         // throw new GemFireIOException("Failed reading " + url, io);
         System.out.println("Failed reading " + url);
-        System.exit(1);
+        ExitCode.FATAL.doSystemExit();
       }
       final String logFile = gfprops.getProperty(LOG_FILE);
       if (logFile == null || logFile.length() == 0) {

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/DiskInitFileParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/DiskInitFileParser.java b/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/DiskInitFileParser.java
index e97e882..ed0cec4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/DiskInitFileParser.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/DiskInitFileParser.java
@@ -33,6 +33,7 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.DiskAccessException;
 import org.apache.geode.cache.UnsupportedVersionException;
+import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.CountingDataInputStream;
@@ -583,7 +584,7 @@ public class DiskInitFileParser {
   public static void main(String[] args) throws IOException, ClassNotFoundException {
     if (args.length != 1) {
       System.err.println("Usage: parse filename");
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
     dump(new File(args[0]));
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/cache/snapshot/GFSnapshot.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/snapshot/GFSnapshot.java b/geode-core/src/main/java/org/apache/geode/internal/cache/snapshot/GFSnapshot.java
index 5ba8800..aa72bd9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/snapshot/GFSnapshot.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/snapshot/GFSnapshot.java
@@ -31,6 +31,7 @@ import java.util.NoSuchElementException;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.snapshot.SnapshotIterator;
+import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
@@ -75,7 +76,7 @@ public class GFSnapshot {
   public static void main(String[] args) throws Exception {
     if (args.length != 1) {
       System.out.println("Usage: GFSnapshot <file>");
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
 
     GFSnapshotImporter imp = new GFSnapshotImporter(new File(args[0]));

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/logging/LogFileParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/LogFileParser.java b/geode-core/src/main/java/org/apache/geode/internal/logging/LogFileParser.java
index af8baeb..665d967 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/LogFileParser.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/LogFileParser.java
@@ -14,12 +14,18 @@
  */
 package org.apache.geode.internal.logging;
 
+import java.io.BufferedReader;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.util.Date;
+
+import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
-import java.io.*;
-import java.text.*;
-import java.util.*;
-
 /**
  * Parses a log file written by a {@link org.apache.geode.i18n.LogWriterI18n} into
  * {@link LogFileParser.LogEntry}s. It behaves sort of like an {@link java.util.StringTokenizer}.
@@ -381,7 +387,7 @@ public class LogFileParser {
   public static void main(String[] args) throws Throwable {
     if (args.length < 1) {
       System.err.println(LocalizedStrings.LogFileParser_MISSING_LOG_FILE_NAME.toLocalizedString());
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
 
     String logFileName = args[0];

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/logging/MergeLogFiles.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/MergeLogFiles.java b/geode-core/src/main/java/org/apache/geode/internal/logging/MergeLogFiles.java
index 27f2116..0c1306e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/MergeLogFiles.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/MergeLogFiles.java
@@ -14,10 +14,6 @@
  */
 package org.apache.geode.internal.logging;
 
-import org.apache.geode.SystemFailure;
-import org.apache.geode.internal.Assert;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
@@ -41,6 +37,11 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.regex.Pattern;
 import java.util.zip.GZIPInputStream;
 
+import org.apache.geode.SystemFailure;
+import org.apache.geode.internal.Assert;
+import org.apache.geode.internal.ExitCode;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+
 /**
  * This program merges entries from multiple GemFire log files (those written using a
  * {@link org.apache.geode.i18n.LogWriterI18n} together, sorting them by their timestamp. Note that
@@ -178,7 +179,7 @@ public class MergeLogFiles {
         LocalizedStrings.MergeLogFiles_IF_A_DIRECTORY_IS_SPECIFIED_ALL_LOG_FILES_IN_THAT_DIRECTORY_ARE_MERGED
             .toLocalizedString());
     err.println("");
-    System.exit(1);
+    ExitCode.FATAL.doSystemExit();
   }
 
   /**
@@ -340,7 +341,7 @@ public class MergeLogFiles {
     mergeLogFiles(logFiles, logFileNames, mergedFile, tabOut, suppressBlanks, multithreaded,
         patterns);
 
-    System.exit(0);
+    ExitCode.NORMAL.doSystemExit();
   }
 
   ////////////////////// Inner Classes //////////////////////

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/logging/SortLogFile.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/SortLogFile.java b/geode-core/src/main/java/org/apache/geode/internal/logging/SortLogFile.java
index af7010c..bc70bab 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/SortLogFile.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/SortLogFile.java
@@ -14,10 +14,23 @@
  */
 package org.apache.geode.internal.logging;
 
-import org.apache.geode.internal.i18n.LocalizedStrings;
 
-import java.io.*;
-import java.util.*;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.geode.internal.ExitCode;
+import org.apache.geode.internal.i18n.LocalizedStrings;
 
 /**
  * This program sorts the entries in a GemFire log file (one written using a
@@ -88,7 +101,7 @@ public class SortLogFile {
         LocalizedStrings.SortLogFile_SORTS_A_GEMFIRE_LOG_FILE_BY_TIMESTAMP_THE_MERGED_LOG_FILE_IS_WRITTEN_TO_SYSTEM_OUT_OR_A_FILE
             .toLocalizedString());
     err.println("");
-    System.exit(1);
+    ExitCode.FATAL.doSystemExit();
   }
 
   public static void main(String[] args) throws IOException {
@@ -135,7 +148,7 @@ public class SortLogFile {
 
     sortLogFile(logFileStream, pw);
 
-    System.exit(0);
+    ExitCode.NORMAL.doSystemExit();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/sequencelog/io/GemfireLogConverter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/sequencelog/io/GemfireLogConverter.java b/geode-core/src/main/java/org/apache/geode/internal/sequencelog/io/GemfireLogConverter.java
index 8a2f3a9..b4a8d66 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/sequencelog/io/GemfireLogConverter.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/sequencelog/io/GemfireLogConverter.java
@@ -23,12 +23,12 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.text.DateFormat;
 import java.text.ParsePosition;
-import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.logging.DateFormatter;
 import org.apache.geode.internal.sequencelog.GraphType;
 import org.apache.geode.internal.sequencelog.Transition;
@@ -79,7 +79,7 @@ public class GemfireLogConverter {
 
     if (args.length == 0) {
       usage();
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
 
     File outputFile = new File(args[0]);

http://git-wip-us.apache.org/repos/asf/geode/blob/9db5c8e5/geode-core/src/main/java/org/apache/geode/internal/statistics/ArchiveSplitter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/ArchiveSplitter.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/ArchiveSplitter.java
index 4d1d597..2336608 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/ArchiveSplitter.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/ArchiveSplitter.java
@@ -14,13 +14,25 @@
  */
 package org.apache.geode.internal.statistics;
 
-import org.apache.geode.*;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
+import org.apache.geode.GemFireIOException;
+import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
-import java.io.*;
-// import java.text.*;
-// import java.util.*;
-import java.util.zip.*;
 
 /**
  * ArchiveSplitter provides APIs to read statistic snapshots from an archive file.
@@ -535,7 +547,7 @@ public class ArchiveSplitter implements StatArchiveFormat {
     if (args.length != 1) {
       System.err.println(LocalizedStrings.ArchiveSplitter_USAGE.toLocalizedString()
           + ": org.apache.geode.internal.statistics.ArchiveSplitter <archive.gfs>");
-      System.exit(1);
+      ExitCode.FATAL.doSystemExit();
     }
     ArchiveSplitter as = new ArchiveSplitter(new File(args[0]));
     as.split();