You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by zh...@apache.org on 2017/08/11 17:26:52 UTC

[22/23] geode git commit: GEODE-3413: overhaul launcher and process classes and tests

GEODE-3413: overhaul launcher and process classes and tests

This is primarily an overall of all ServerLauncher and LocatorLauncher
tests and org.apache.geode.internal.process tests. The main classes in
org.apachage.geode.internal.process package are also cleaned up.

In addition, several bugs involving these classes and tests are fixed.

Here is the complete list of tickets that are resolved in this overhaul:

* GEODE-1229: LocatorLauncherRemoteJUnitTest.testStartOverwritesStalePidFile
* GEODE-2791: LocatorLauncherAssemblyIntegrationTest.testLocatorStopsWhenJmxPortIsNonZero fails intermittently with AssertionError
* GEODE-1308: CI failure: LocatorLauncherTest.testSetBindAddressToNonLocalHost
* GEODE-1309: CI failure: ServerLauncherTest.testSetServerBindAddressToNonLocalHost
* GEODE-3193: locator pid file is removed even if there was a problem while shutting down
* GEODE-3413: Overhaul launcher tests and process tests
* GEODE-3414: Cleanup org.apache.geode.internal.process package

Note I moved all useful tests from LocatorLauncherAssemblyIntegrationTest
into the other launcher tests in geode-core.


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

Branch: refs/heads/feature/GEODE-3304
Commit: 894f3ee710e500c83f06be660b852d389cbb94cb
Parents: beebb65
Author: Kirk Lund <kl...@apache.org>
Authored: Mon Jul 10 12:30:09 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Aug 11 10:00:16 2017 -0700

----------------------------------------------------------------------
 .../LocatorLauncherAssemblyIntegrationTest.java |  150 --
 .../geode/distributed/AbstractLauncher.java     |  139 +-
 .../geode/distributed/LocatorLauncher.java      |  377 +++--
 .../geode/distributed/ServerLauncher.java       |  420 ++---
 .../distributed/internal/InternalLocator.java   |    5 +-
 .../cache/ClusterConfigurationLoader.java       |   35 +-
 .../geode/internal/cache/GemFireCacheImpl.java  |    9 +-
 ...usterConfigurationNotAvailableException.java |   29 +
 .../geode/internal/io/TeePrintStream.java       |    2 +-
 .../internal/process/AttachProcessUtils.java    |    9 +-
 .../process/BlockingProcessStreamReader.java    |   33 +-
 ...usterConfigurationNotAvailableException.java |    8 +-
 .../process/ConnectionFailedException.java      |   14 +-
 .../internal/process/ControlFileWatchdog.java   |  124 +-
 .../process/ControlNotificationHandler.java     |    5 +-
 .../internal/process/ControllableProcess.java   |  196 ++-
 .../process/FileAlreadyExistsException.java     |   14 +-
 .../process/FileControllerParameters.java       |    5 +-
 .../internal/process/FileProcessController.java |  111 +-
 .../process/LocalProcessController.java         |  478 ------
 .../internal/process/LocalProcessLauncher.java  |  101 +-
 .../process/MBeanControllerParameters.java      |   13 +-
 .../process/MBeanInvocationFailedException.java |   14 +-
 .../process/MBeanProcessController.java         |  128 +-
 .../internal/process/NativeProcessUtils.java    |   14 +-
 .../process/NonBlockingProcessStreamReader.java |   88 +-
 .../apache/geode/internal/process/PidFile.java  |  113 +-
 .../process/PidUnavailableException.java        |   14 +-
 .../internal/process/ProcessController.java     |   17 +-
 .../process/ProcessControllerFactory.java       |   72 +-
 .../process/ProcessControllerParameters.java    |    6 +-
 .../process/ProcessLauncherContext.java         |   68 +-
 .../internal/process/ProcessStreamReader.java   |  161 +-
 .../ProcessTerminatedAbnormallyException.java   |   97 --
 .../geode/internal/process/ProcessType.java     |   24 +-
 .../geode/internal/process/ProcessUtils.java    |   66 +-
 .../geode/internal/process/StartupStatus.java   |   20 +-
 .../internal/process/StartupStatusListener.java |    2 +-
 .../UnableToControlProcessException.java        |   14 +-
 .../AbstractSignalNotificationHandler.java      |   81 +-
 .../geode/internal/process/signal/Signal.java   |   20 +-
 .../internal/process/signal/SignalEvent.java    |   11 +-
 .../internal/process/signal/SignalListener.java |    8 +-
 .../internal/process/signal/SignalType.java     |    7 +-
 .../cli/commands/StartServerCommand.java        |    3 -
 .../AbstractLauncherIntegrationTest.java        |   49 +-
 .../AbstractLauncherIntegrationTestCase.java    |  268 ---
 .../AbstractLauncherServiceStateTest.java       |  224 +++
 .../AbstractLauncherServiceStatusTest.java      |  224 ---
 .../geode/distributed/AbstractLauncherTest.java |  462 +++--
 ...tractLocatorLauncherIntegrationTestCase.java |  135 --
 ...ocatorLauncherRemoteIntegrationTestCase.java |   65 -
 ...stractServerLauncherIntegrationTestCase.java |   98 --
 ...ServerLauncherRemoteIntegrationTestCase.java |   95 --
 .../LauncherIntegrationTestCase.java            |  314 ++++
 .../LauncherMemberMXBeanIntegrationTest.java    |  174 +-
 .../geode/distributed/LocatorCommand.java       |  107 ++
 .../distributed/LocatorLauncherBuilderTest.java |  507 ++++++
 .../LocatorLauncherIntegrationTest.java         |  278 +--
 .../LocatorLauncherIntegrationTestCase.java     |  163 ++
 ...orLauncherJmxManagerLocalRegressionTest.java |  111 ++
 ...rLauncherJmxManagerRemoteRegressionTest.java |   72 +
 ...LocatorLauncherLocalFileIntegrationTest.java |   35 +-
 .../LocatorLauncherLocalIntegrationTest.java    |  886 +++-------
 ...ocatorLauncherRemoteFileIntegrationTest.java |  211 +--
 .../LocatorLauncherRemoteIntegrationTest.java   | 1092 ++----------
 ...ocatorLauncherRemoteIntegrationTestCase.java |  234 +++
 ...rRemoteWithCustomLoggingIntegrationTest.java |  104 +-
 .../geode/distributed/LocatorLauncherTest.java  |  344 +---
 .../geode/distributed/LocatorStateTest.java     |  185 +-
 .../MockServerLauncherCacheProvider.java        |   38 -
 .../apache/geode/distributed/ServerCommand.java |  120 ++
 .../distributed/ServerLauncherBuilderTest.java  |  845 ++++++++++
 .../ServerLauncherIntegrationTest.java          |  349 ++--
 .../ServerLauncherIntegrationTestCase.java      |  204 +++
 .../ServerLauncherLocalFileIntegrationTest.java |   24 +-
 .../ServerLauncherLocalIntegrationTest.java     | 1124 +++----------
 .../ServerLauncherLocalIntegrationTestCase.java |   26 +
 ...ServerLauncherRemoteFileIntegrationTest.java |  207 +--
 .../ServerLauncherRemoteIntegrationTest.java    | 1579 +++---------------
 ...ServerLauncherRemoteIntegrationTestCase.java |  236 +++
 ...rRemoteWithCustomLoggingIntegrationTest.java |  107 +-
 .../geode/distributed/ServerLauncherTest.java   |  968 ++---------
 ...erLauncherWaitOnServerMultiThreadedTest.java |  111 ++
 ...rverLauncherWithProviderIntegrationTest.java |   81 -
 ...erverLauncherWithProviderRegressionTest.java |   68 +
 .../TestServerLauncherCacheProvider.java        |   42 +
 .../geode/distributed/UsesLocatorCommand.java   |   28 +
 .../geode/distributed/UsesServerCommand.java    |   30 +
 .../support/DistributedSystemAdapter.java       |  272 ---
 ...tractProcessStreamReaderIntegrationTest.java |  309 ++++
 .../process/AttachProcessUtilsTest.java         |   85 +
 .../BaseProcessStreamReaderIntegrationTest.java |   51 +
 ...ckingProcessStreamReaderIntegrationTest.java |  133 ++
 .../BlockingProcessStreamReaderJUnitTest.java   |  443 -----
 .../BlockingProcessStreamReaderWindowsTest.java |   93 ++
 .../ControlFileWatchdogIntegrationTest.java     |  241 +++
 .../ControllableProcessIntegrationTest.java     |  196 +++
 ...leProcessControllerIntegrationJUnitTest.java |  151 --
 .../FileProcessControllerIntegrationTest.java   |  249 +++
 .../process/FileProcessControllerTest.java      |   90 +
 .../LocalProcessControllerJUnitTest.java        |  121 --
 .../process/LocalProcessLauncherDUnitTest.java  |  154 --
 .../LocalProcessLauncherDistributedTest.java    |   92 +
 .../LocalProcessLauncherIntegrationTest.java    |  160 ++
 .../process/LocalProcessLauncherJUnitTest.java  |  177 --
 .../NativeProcessUtilsIntegrationTest.java      |  119 ++
 .../process/NativeProcessUtilsTest.java         |   76 +
 ...ckingProcessStreamReaderIntegrationTest.java |  130 ++
 ...NonBlockingProcessStreamReaderJUnitTest.java |  365 ----
 .../process/PidFileIntegrationTest.java         |  187 +++
 .../internal/process/PidFileJUnitTest.java      |  275 ---
 ...ProcessControllerFactoryIntegrationTest.java |  135 ++
 .../ProcessControllerFactoryJUnitTest.java      |  176 --
 .../process/ProcessControllerFactoryTest.java   |  116 ++
 .../process/ProcessLauncherContextTest.java     |  208 +++
 .../process/ProcessStreamReaderTestCase.java    |  254 ---
 .../internal/process/StartupStatusTest.java     |  176 ++
 .../internal/process/io/EmptyFileWriter.java    |   40 +
 .../internal/process/io/IntegerFileReader.java  |   38 +
 .../internal/process/io/IntegerFileWriter.java  |   33 +
 .../internal/process/io/StringFileWriter.java   |   43 +
 .../internal/process/lang/AvailablePid.java     |  106 ++
 .../internal/process/lang/AvailablePidTest.java |  107 ++
 .../geode/internal/process/mbean/Process.java   |   57 -
 .../internal/process/mbean/ProcessMBean.java    |   27 -
 ...tractSignalNotificationHandlerJUnitTest.java |  467 ------
 .../AbstractSignalNotificationHandlerTest.java  |  406 +++++
 ...eode.distributed.ServerLauncherCacheProvider |    2 +-
 .../codeAnalysis/sanctionedSerializables.txt    |    9 +-
 130 files changed, 10256 insertions(+), 12262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-assembly/src/test/java/org/apache/geode/distributed/LocatorLauncherAssemblyIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/distributed/LocatorLauncherAssemblyIntegrationTest.java b/geode-assembly/src/test/java/org/apache/geode/distributed/LocatorLauncherAssemblyIntegrationTest.java
deleted file mode 100644
index 4e8d389..0000000
--- a/geode-assembly/src/test/java/org/apache/geode/distributed/LocatorLauncherAssemblyIntegrationTest.java
+++ /dev/null
@@ -1,150 +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.distributed;
-
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.AbstractLauncher.Status;
-import org.apache.geode.distributed.LocatorLauncher.Builder;
-import org.apache.geode.internal.AvailablePortHelper;
-import org.apache.geode.internal.process.ProcessType;
-import org.apache.geode.internal.process.ProcessUtils;
-import org.apache.geode.management.ManagementService;
-import org.apache.geode.management.ManagerMXBean;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.io.File;
-
-import static org.junit.Assert.*;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-
-/**
- * These tests are part of assembly as they require the REST war file to be present.
- */
-@Category(IntegrationTest.class)
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class LocatorLauncherAssemblyIntegrationTest
-    extends AbstractLocatorLauncherIntegrationTestCase {
-
-  @Before
-  public final void setUpLocatorLauncherLocalTest() throws Exception {
-    disconnectFromDS();
-    System.setProperty(ProcessType.TEST_PREFIX_PROPERTY, getUniqueName() + "-");
-  }
-
-  @After
-  public final void tearDownLocatorLauncherLocalTest() throws Exception {
-    disconnectFromDS();
-  }
-
-  /*
-   * This test addresses GEODE-528
-   */
-  @Test
-  public void testLocatorStopsWhenJmxPortIsZero() throws Throwable {
-    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
-
-    final Builder builder = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(false).setWorkingDirectory(rootFolder).set(LOG_LEVEL, "config")
-        .set(ENABLE_CLUSTER_CONFIGURATION, "false").set(JMX_MANAGER, "true")
-        .set(JMX_MANAGER_START, "true").set(JMX_MANAGER_PORT, "0");
-
-    performTest(builder);
-  }
-
-  /*
-   * This test addresses GEODE-528
-   */
-  @Test
-  public void testLocatorStopsWhenJmxPortIsNonZero() throws Throwable {
-    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
-    final int jmxPort = AvailablePortHelper.getRandomAvailableTCPPorts(1)[0];
-
-    final Builder builder = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(false).setWorkingDirectory(rootFolder).set(LOG_LEVEL, "config")
-        .set(ENABLE_CLUSTER_CONFIGURATION, "false").set(JMX_MANAGER, "true")
-        .set(JMX_MANAGER_START, "true").set(JMX_MANAGER_PORT, Integer.toString(jmxPort));
-
-    performTest(builder);
-  }
-
-  private void performTest(Builder builder) {
-    assertFalse(builder.getForce());
-    this.launcher = builder.build();
-    assertFalse(this.launcher.isForcing());
-
-    LocatorLauncher dirLauncher = null;
-    int initialThreadCount = Thread.activeCount();
-
-    try {
-      this.launcher.start();
-      waitForLocatorToStart(this.launcher);
-
-      // validate the pid file and its contents
-      this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-      assertTrue("Pid file " + this.pidFile.getCanonicalPath().toString() + " should exist",
-          this.pidFile.exists());
-      final int pid = readPid(this.pidFile);
-      assertTrue(pid > 0);
-      assertEquals(ProcessUtils.identifyPid(), pid);
-
-      dirLauncher = new Builder().setWorkingDirectory(builder.getWorkingDirectory()).build();
-      assertNotNull(dirLauncher);
-      assertFalse(dirLauncher.isRunning());
-
-      // Stop the manager
-      Cache cache = CacheFactory.getAnyInstance();
-      ManagerMXBean managerBean = ManagementService.getManagementService(cache).getManagerMXBean();
-      managerBean.stop();
-
-      // stop the locator
-      final LocatorLauncher.LocatorState locatorState = dirLauncher.stop();
-      assertNotNull(locatorState);
-      assertEquals(Status.STOPPED, locatorState.getStatus());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      // verify the PID file was deleted
-      waitForFileToDelete(this.pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    int finalThreadCount = Integer.MAX_VALUE;
-
-    // Spin for up to 5 seconds waiting for threads to finish
-    for (int i = 0; i < 50 && finalThreadCount > initialThreadCount; i++) {
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException ex) {
-        // ignored
-      }
-      finalThreadCount = Thread.activeCount();
-    }
-
-    assertEquals(initialThreadCount, finalThreadCount);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-core/src/main/java/org/apache/geode/distributed/AbstractLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/AbstractLauncher.java b/geode-core/src/main/java/org/apache/geode/distributed/AbstractLauncher.java
index 28a156a..952afa8 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/AbstractLauncher.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/AbstractLauncher.java
@@ -12,33 +12,24 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.distributed;
 
+import static org.apache.commons.lang.StringUtils.isNotBlank;
+import static org.apache.commons.lang.StringUtils.join;
+import static org.apache.commons.lang.StringUtils.lowerCase;
 import static org.apache.geode.distributed.ConfigurationProperties.NAME;
-
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.unsafe.RegisterSignalHandlerSupport;
-import org.apache.geode.internal.AvailablePort;
-import org.apache.geode.internal.GemFireVersion;
-import org.apache.geode.internal.OSProcess;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.internal.lang.ClassUtils;
-import org.apache.geode.internal.lang.ObjectUtils;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.internal.lang.SystemUtils;
-import org.apache.geode.internal.process.PidUnavailableException;
-import org.apache.geode.internal.process.ProcessUtils;
-import org.apache.geode.internal.util.ArgumentRedactor;
-import org.apache.geode.internal.util.SunAPINotFoundException;
-import org.apache.geode.management.internal.cli.json.GfJsonObject;
+import static org.apache.geode.internal.lang.ClassUtils.forName;
+import static org.apache.geode.internal.lang.ObjectUtils.defaultIfNull;
+import static org.apache.geode.internal.lang.StringUtils.defaultString;
+import static org.apache.geode.internal.lang.SystemUtils.CURRENT_DIRECTORY;
 
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.net.BindException;
 import java.net.InetAddress;
+import java.net.URISyntaxException;
 import java.net.URL;
 import java.sql.Timestamp;
 import java.text.SimpleDateFormat;
@@ -50,10 +41,21 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.logging.FileHandler;
-import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.unsafe.RegisterSignalHandlerSupport;
+import org.apache.geode.internal.AvailablePort;
+import org.apache.geode.internal.GemFireVersion;
+import org.apache.geode.internal.OSProcess;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.process.PidUnavailableException;
+import org.apache.geode.internal.process.ProcessUtils;
+import org.apache.geode.internal.util.ArgumentRedactor;
+import org.apache.geode.internal.util.SunAPINotFoundException;
+import org.apache.geode.management.internal.cli.json.GfJsonObject;
+
 /**
  * The AbstractLauncher class is a base class for implementing various launchers to construct and
  * run different GemFire processes, like Cache Servers, Locators, Managers, HTTP servers and so on.
@@ -67,9 +69,13 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
 
   protected static final Boolean DEFAULT_FORCE = Boolean.FALSE;
 
+  /**
+   * @deprecated This timeout is no longer needed.
+   */
+  @Deprecated
   protected static final long READ_PID_FILE_TIMEOUT_MILLIS = 2 * 1000;
 
-  public static final String DEFAULT_WORKING_DIRECTORY = SystemUtils.CURRENT_DIRECTORY;
+  public static final String DEFAULT_WORKING_DIRECTORY = CURRENT_DIRECTORY;
 
   public static final String SIGNAL_HANDLER_REGISTRATION_SYSTEM_PROPERTY =
       DistributionConfig.GEMFIRE_PREFIX + "launcher.registerSignalHandlers";
@@ -88,12 +94,12 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
   public AbstractLauncher() {
     try {
       if (Boolean.getBoolean(SIGNAL_HANDLER_REGISTRATION_SYSTEM_PROPERTY)) {
-        ClassUtils.forName(SUN_SIGNAL_API_CLASS_NAME, new SunAPINotFoundException(
+        forName(SUN_SIGNAL_API_CLASS_NAME, new SunAPINotFoundException(
             "WARNING!!! Not running a Sun JVM.  Could not find the sun.misc.Signal class; Signal handling disabled."));
         RegisterSignalHandlerSupport.registerSignalHandlers();
       }
-    } catch (SunAPINotFoundException e) {
-      info(e.getMessage());
+    } catch (SunAPINotFoundException handled) {
+      info(handled.getMessage());
     }
   }
 
@@ -102,7 +108,6 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
    *
    * @param port an integer indicating the network port to listen for client network requests.
    * @throws BindException if the network port is not available.
-   * @see #assertPortAvailable
    */
   protected static void assertPortAvailable(final int port) throws BindException {
     assertPortAvailable(null, port);
@@ -124,7 +129,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
     if (!AvailablePort.isPortAvailable(port, AvailablePort.SOCKET, bindAddress)) {
       throw new BindException(
           String.format("Network is unreachable; port (%1$d) is not available on %2$s.", port,
-              (bindAddress != null ? bindAddress.getCanonicalHostName() : "localhost")));
+              bindAddress != null ? bindAddress.getCanonicalHostName() : "localhost"));
     }
   }
 
@@ -141,7 +146,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
    * @see java.util.Properties
    */
   protected static boolean isSet(final Properties properties, final String propertyName) {
-    return StringUtils.isNotBlank(properties.getProperty(propertyName));
+    return isNotBlank(properties.getProperty(propertyName));
   }
 
   /**
@@ -152,34 +157,27 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
    * @see java.net.URL
    */
   protected static Properties loadGemFireProperties(final URL url) {
-    final Properties properties = new Properties();
+    if (url == null) {
+      return new Properties();
+    }
+    Properties properties = new Properties();
 
-    if (url != null) {
-      try {
-        properties.load(new FileReader(new File(url.toURI())));
-      } catch (Exception e) {
-        try {
-          // not in the file system, try the classpath
-          properties.load(
-              AbstractLauncher.class.getResourceAsStream(DistributedSystem.getPropertiesFile()));
-        } catch (Exception ignore) {
-          // not in the file system or the classpath; gemfire.properties does not exist
-        }
-      }
+    try {
+      properties.load(new FileReader(new File(url.toURI())));
+    } catch (IOException | URISyntaxException handled) {
+      // not in the file system, try the classpath
+      loadGemFirePropertiesFromClassPath(properties);
     }
 
     return properties;
   }
 
-  void initLogger() {
+  private static void loadGemFirePropertiesFromClassPath(Properties properties) {
     try {
-      this.logger.addHandler(new FileHandler(SystemUtils.CURRENT_DIRECTORY.concat("debug.log")));
-      this.logger.setLevel(Level.ALL);
-      this.logger.setUseParentHandlers(true);
-    } catch (IOException e) {
-      e.printStackTrace(System.err);
-      System.err.flush();
-      throw new RuntimeException(e);
+      properties
+          .load(AbstractLauncher.class.getResourceAsStream(DistributedSystem.getPropertiesFile()));
+    } catch (IOException | NullPointerException handled) {
+      // leave the properties empty
     }
   }
 
@@ -240,7 +238,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
       distributedSystemProperties.putAll(defaults);
     }
 
-    if (StringUtils.isNotBlank(getMemberName())) {
+    if (isNotBlank(getMemberName())) {
       distributedSystemProperties.setProperty(NAME, getMemberName());
     }
 
@@ -264,8 +262,8 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
   protected String getLogFileCanonicalPath() {
     try {
       return getLogFile().getCanonicalPath();
-    } catch (IOException e) {
-      return getLogFileName(); // TODO: or return null?
+    } catch (IOException handled) {
+      return getLogFileName();
     }
   }
 
@@ -288,10 +286,10 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
    * @see #getMemberId()
    */
   public String getMember() {
-    if (StringUtils.isNotBlank(getMemberName())) {
+    if (isNotBlank(getMemberName())) {
       return getMemberName();
     }
-    if (StringUtils.isNotBlank(getMemberId())) {
+    if (isNotBlank(getMemberId())) {
       return getMemberId();
     }
     return null;
@@ -307,7 +305,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
   public String getMemberId() {
     final InternalDistributedSystem distributedSystem =
         InternalDistributedSystem.getConnectedInstance();
-    return (distributedSystem != null ? distributedSystem.getMemberId() : null);
+    return distributedSystem != null ? distributedSystem.getMemberId() : null;
   }
 
   /**
@@ -319,7 +317,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
   public String getMemberName() {
     final InternalDistributedSystem distributedSystem =
         InternalDistributedSystem.getConnectedInstance();
-    return (distributedSystem != null ? distributedSystem.getConfig().getName() : null);
+    return distributedSystem != null ? distributedSystem.getConfig().getName() : null;
   }
 
   /**
@@ -430,7 +428,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
   int identifyPidOrNot() {
     try {
       return identifyPid();
-    } catch (PidUnavailableException e) {
+    } catch (PidUnavailableException handled) {
       return -1;
     }
   }
@@ -488,21 +486,19 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
 
     private final Timestamp timestamp;
 
-    // TODO refactor the logic in this method into a DateTimeFormatUtils class
     protected static String format(final Date timestamp) {
-      return (timestamp == null ? ""
-          : new SimpleDateFormat(DATE_TIME_FORMAT_PATTERN).format(timestamp));
+      return timestamp == null ? ""
+          : new SimpleDateFormat(DATE_TIME_FORMAT_PATTERN).format(timestamp);
     }
 
     protected static Integer identifyPid() {
       try {
         return ProcessUtils.identifyPid();
-      } catch (PidUnavailableException ignore) {
+      } catch (PidUnavailableException handled) {
         return null;
       }
     }
 
-    // TODO refactor the logic in this method into a DateTimeFormatUtils class
     protected static String toDaysHoursMinutesSeconds(final Long milliseconds) {
       final StringBuilder buffer = new StringBuilder();
 
@@ -555,7 +551,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
       this.pid = pid;
       this.uptime = uptime;
       this.workingDirectory = workingDirectory;
-      this.jvmArguments = ObjectUtils.defaultIfNull(Collections.unmodifiableList(jvmArguments),
+      this.jvmArguments = defaultIfNull(Collections.unmodifiableList(jvmArguments),
           Collections.<String>emptyList());
       this.classpath = classpath;
       this.gemfireVersion = gemfireVersion;
@@ -592,12 +588,11 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
     }
 
     public static boolean isStartingNotRespondingOrNull(final ServiceState serviceState) {
-      return (serviceState == null || serviceState.isStartingOrNotResponding());
+      return serviceState == null || serviceState.isStartingOrNotResponding();
     }
 
     public boolean isStartingOrNotResponding() {
-      return (Status.NOT_RESPONDING.equals(this.getStatus())
-          || Status.STARTING.equals(this.getStatus()));
+      return Status.NOT_RESPONDING == getStatus() || Status.STARTING == getStatus();
     }
 
     public boolean isVmWithProcessIdRunning() {
@@ -729,7 +724,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
      * @return a String value indicating the GemFire service's working (running) directory.
      */
     public String getWorkingDirectory() {
-      return ObjectUtils.defaultIfNull(workingDirectory, DEFAULT_WORKING_DIRECTORY);
+      return defaultIfNull(workingDirectory, DEFAULT_WORKING_DIRECTORY);
     }
 
     /**
@@ -795,17 +790,17 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
 
     // the value of a Number as a String, or "" if null
     protected String toString(final Number value) {
-      return StringUtils.defaultString(value);
+      return defaultString(value);
     }
 
     // a String concatenation of all values separated by " "
     protected String toString(final Object... values) {
-      return values == null ? "" : StringUtils.join(values, " ");
+      return values == null ? "" : join(values, " ");
     }
 
     // the value of the String, or "" if value is null
     protected String toString(final String value) {
-      return ObjectUtils.defaultIfNull(value, "");
+      return defaultIfNull(value, "");
     }
   }
 
@@ -813,7 +808,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
    * The Status enumerated type represents the various lifecycle states of a GemFire service (such
    * as a Cache Server, a Locator or a Manager).
    */
-  public static enum Status {
+  public enum Status {
     NOT_RESPONDING(LocalizedStrings.Launcher_Status_NOT_RESPONDING.toLocalizedString()),
     ONLINE(LocalizedStrings.Launcher_Status_ONLINE.toLocalizedString()),
     STARTING(LocalizedStrings.Launcher_Status_STARTING.toLocalizedString()),
@@ -822,8 +817,8 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
     private final String description;
 
     Status(final String description) {
-      assert StringUtils.isNotBlank(description) : "The Status description must be specified!";
-      this.description = StringUtils.lowerCase(description);
+      assert isNotBlank(description) : "The Status description must be specified!";
+      this.description = lowerCase(description);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java b/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
index c5a2de8..83c1ab5 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/LocatorLauncher.java
@@ -12,11 +12,44 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.distributed;
 
+import static org.apache.commons.lang.StringUtils.defaultIfBlank;
+import static org.apache.commons.lang.StringUtils.isBlank;
+import static org.apache.commons.lang.StringUtils.isNotBlank;
+import static org.apache.commons.lang.StringUtils.lowerCase;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
 import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.apache.geode.internal.lang.ObjectUtils.defaultIfNull;
+import static org.apache.geode.internal.lang.StringUtils.wrap;
+import static org.apache.geode.internal.lang.SystemUtils.CURRENT_DIRECTORY;
+import static org.apache.geode.internal.util.IOUtils.tryGetCanonicalPathElseGetAbsolutePath;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.logging.Level;
+
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import joptsimple.OptionException;
+import joptsimple.OptionParser;
+import joptsimple.OptionSet;
 
 import org.apache.geode.cache.client.internal.locator.LocatorStatusRequest;
 import org.apache.geode.cache.client.internal.locator.LocatorStatusResponse;
@@ -28,8 +61,6 @@ import org.apache.geode.internal.DistributionLocator;
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.lang.ObjectUtils;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.internal.lang.SystemUtils;
 import org.apache.geode.internal.net.SocketCreator;
 import org.apache.geode.internal.process.ConnectionFailedException;
 import org.apache.geode.internal.process.ControlNotificationHandler;
@@ -45,37 +76,11 @@ import org.apache.geode.internal.process.ProcessType;
 import org.apache.geode.internal.process.ProcessUtils;
 import org.apache.geode.internal.process.StartupStatusListener;
 import org.apache.geode.internal.process.UnableToControlProcessException;
-import org.apache.geode.internal.util.IOUtils;
 import org.apache.geode.lang.AttachAPINotFoundException;
 import org.apache.geode.management.internal.cli.json.GfJsonArray;
 import org.apache.geode.management.internal.cli.json.GfJsonException;
 import org.apache.geode.management.internal.cli.json.GfJsonObject;
 
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.net.ConnectException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.logging.Level;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import joptsimple.OptionException;
-import joptsimple.OptionParser;
-import joptsimple.OptionSet;
-
 /**
  * The LocatorLauncher class is a launcher for a GemFire Locator.
  * 
@@ -104,6 +109,8 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     helpMap.put("bind-address",
         LocalizedStrings.LocatorLauncher_LOCATOR_BIND_ADDRESS_HELP.toLocalizedString());
     helpMap.put("debug", LocalizedStrings.LocatorLauncher_LOCATOR_DEBUG_HELP.toLocalizedString());
+    helpMap.put("delete-pid-file-on-stop",
+        "Specifies that this Locator's PID file should be deleted on stop.  The default is to not delete this Locator's PID file until JVM exit if --delete-pid-file-on-stop is not specified.");
     helpMap.put("dir", LocalizedStrings.LocatorLauncher_LOCATOR_DIR_HELP.toLocalizedString());
     helpMap.put("force", LocalizedStrings.LocatorLauncher_LOCATOR_FORCE_HELP.toLocalizedString());
     helpMap.put("help",
@@ -143,6 +150,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
 
   private final AtomicBoolean starting = new AtomicBoolean(false);
 
+  private final boolean deletePidFileOnStop;
   private final boolean force;
   private final boolean help;
   private final boolean redirectOutput;
@@ -181,8 +189,8 @@ public class LocatorLauncher extends AbstractLauncher<String> {
   public static void main(final String... args) {
     try {
       new Builder(args).build().run();
-    } catch (AttachAPINotFoundException e) {
-      System.err.println(e.getMessage());
+    } catch (AttachAPINotFoundException handled) {
+      System.err.println(handled.getMessage());
     }
   }
 
@@ -223,12 +231,13 @@ public class LocatorLauncher extends AbstractLauncher<String> {
    */
   private LocatorLauncher(final Builder builder) {
     this.command = builder.getCommand();
-    setDebug(Boolean.TRUE.equals(builder.getDebug()));
-    this.force = Boolean.TRUE.equals(builder.getForce());
     this.help = Boolean.TRUE.equals(builder.getHelp());
     this.bindAddressSpecified = builder.isBindAddressSpecified();
     this.bindAddress = builder.getBindAddress();
+    setDebug(Boolean.TRUE.equals(builder.getDebug()));
+    this.deletePidFileOnStop = Boolean.TRUE.equals(builder.getDeletePidFileOnStop());
     this.distributedSystemProperties = builder.getDistributedSystemProperties();
+    this.force = Boolean.TRUE.equals(builder.getForce());
     this.hostnameForClients = builder.getHostnameForClients();
     this.memberName = builder.getMemberName();
     this.pid = builder.getPid();
@@ -370,12 +379,9 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       InetAddress localhost = SocketCreator.getLocalHost();
 
       return localhost.getCanonicalHostName();
-    } catch (UnknownHostException ignore) {
-      // TODO determine a better value for the host on which the Locator is running to return
-      // here...
+    } catch (UnknownHostException handled) {
       // NOTE returning localhost/127.0.0.1 implies the bindAddress was null and no IP address for
-      // localhost
-      // could be found
+      // localhost could be found
       return "localhost/127.0.0.1";
     }
   }
@@ -394,8 +400,9 @@ public class LocatorLauncher extends AbstractLauncher<String> {
    * 
    * @return a String value indicating the name of this Locator's log file.
    */
+  @Override
   public String getLogFileName() {
-    return StringUtils.defaultIfBlank(getMemberName(), DEFAULT_LOCATOR_LOG_NAME)
+    return defaultIfBlank(getMemberName(), DEFAULT_LOCATOR_LOG_NAME)
         .concat(DEFAULT_LOCATOR_LOG_EXT);
   }
 
@@ -405,10 +412,10 @@ public class LocatorLauncher extends AbstractLauncher<String> {
    * 
    * @return a String indicating the name of the member (this Locator) in the GemFire distributed
    *         system.
-   * @see AbstractLauncher#getMemberName()
    */
+  @Override
   public String getMemberName() {
-    return StringUtils.defaultIfBlank(this.memberName, super.getMemberName());
+    return defaultIfBlank(this.memberName, super.getMemberName());
   }
 
   /**
@@ -444,7 +451,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
    * @see #getPort()
    */
   public String getPortAsString() {
-    return ObjectUtils.defaultIfNull(getPort(), getDefaultLocatorPort()).toString();
+    return defaultIfNull(getPort(), getDefaultLocatorPort()).toString();
   }
 
   /**
@@ -463,6 +470,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
    * 
    * @return a String indicating the name for a GemFire Locator.
    */
+  @Override
   public String getServiceName() {
     return LOCATOR_SERVICE_NAME;
   }
@@ -488,14 +496,13 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     if (Command.isUnspecified(command)) {
       usage();
     } else {
-      info(StringUtils.wrap(helpMap.get(command.getName()), 80, ""));
+      info(wrap(helpMap.get(command.getName()), 80, ""));
       info("\n\nusage: \n\n");
-      info(StringUtils.wrap("> java ... " + getClass().getName() + " " + usageMap.get(command), 80,
-          "\t\t"));
+      info(wrap("> java ... " + getClass().getName() + " " + usageMap.get(command), 80, "\t\t"));
       info("\n\noptions: \n\n");
 
       for (String option : command.getOptions()) {
-        info(StringUtils.wrap("--" + option + ": " + helpMap.get(option) + "\n", 80, "\t"));
+        info(wrap("--" + option + ": " + helpMap.get(option) + "\n", 80, "\t"));
       }
 
       info("\n\n");
@@ -509,7 +516,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
    * @see #help(org.apache.geode.distributed.LocatorLauncher.Command)
    */
   public void usage() {
-    info(StringUtils.wrap(helpMap.get("launcher"), 80, "\t"));
+    info(wrap(helpMap.get("launcher"), 80, "\t"));
     info("\n\nSTART\n\n");
     help(Command.START);
     info("STATUS\n\n");
@@ -533,6 +540,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
    * @see LocatorLauncher#help(org.apache.geode.distributed.LocatorLauncher.Command)
    * @see LocatorLauncher#usage()
    */
+  @Override
   public void run() {
     if (!isHelping()) {
       switch (getCommand()) {
@@ -633,7 +641,6 @@ public class LocatorLauncher extends AbstractLauncher<String> {
               }
             });
 
-        // TODO : remove the extra param for loadFromSharedConfigDir
         try {
           this.locator = InternalLocator.startLocator(getPort(), getLogFile(), null, null, null,
               getBindAddress(), true, getDistributedSystemProperties(), getHostnameForClients());
@@ -663,10 +670,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
             LocalizedStrings.Launcher_Command_START_PID_UNAVAILABLE_ERROR_MESSAGE.toLocalizedString(
                 getServiceName(), getId(), getWorkingDirectory(), e.getMessage()),
             e);
-      } catch (Error e) {
-        failOnStart(e);
-        throw e;
-      } catch (RuntimeException e) {
+      } catch (Error | RuntimeException e) {
         failOnStart(e);
         throw e;
       } catch (Exception e) {
@@ -707,7 +711,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       this.locator = null;
     }
     if (this.process != null) {
-      this.process.stop();
+      this.process.stop(this.deletePidFileOnStop);
       this.process = null;
     }
 
@@ -738,13 +742,10 @@ public class LocatorLauncher extends AbstractLauncher<String> {
 
       // prevent the JVM from exiting by joining the Locator Thread
       getLocator().waitToStop();
-    } catch (InterruptedException e) {
+    } catch (InterruptedException handled) {
       Thread.currentThread().interrupt();
-      t = e;
-      debug(e);
-    } catch (RuntimeException e) {
-      t = e;
-      throw e;
+      t = handled;
+      debug(handled);
     } catch (Throwable e) {
       t = e;
       throw e;
@@ -789,14 +790,8 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       try {
         LocatorStatusResponse response = statusLocator(getPort(), getBindAddress());
         return new LocatorState(this, Status.ONLINE, response);
-      } catch (Exception ignore) {
-        try {
-          synchronized (this) {
-            timeUnit.timedWait(this, interval);
-          }
-        } catch (InterruptedException ignoreInterrupt) {
-          // NOTE just go and send another status request to the Locator...
-        }
+      } catch (Exception handled) {
+        timedWait(interval, timeUnit);
       }
     }
 
@@ -808,6 +803,16 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     return new LocatorState(this, Status.NOT_RESPONDING);
   }
 
+  private void timedWait(final long interval, final TimeUnit timeUnit) {
+    try {
+      synchronized (this) {
+        timeUnit.timedWait(this, interval);
+      }
+    } catch (InterruptedException handled) {
+      // NOTE just go and send another status request to the Locator...
+    }
+  }
+
   /**
    * Attempts to determine the state of the Locator. The Locator's status will be in only 1 of 2
    * possible states, either ONLINE or OFFLINE. This method behaves differently depending on which
@@ -892,27 +897,18 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       controller.checkPidSupport();
       final String statusJson = controller.status();
       return LocatorState.fromJson(statusJson);
-    } catch (ConnectionFailedException e) {
+    } catch (ConnectionFailedException handled) {
       // failed to attach to locator JVM
-      return createNoResponseState(e, "Failed to connect to locator with process id " + getPid());
-    } catch (IOException e) {
-      // failed to open or read file or dir
-      return createNoResponseState(e,
+      return createNoResponseState(handled,
+          "Failed to connect to locator with process id " + getPid());
+    } catch (IOException | MBeanInvocationFailedException | UnableToControlProcessException
+        | TimeoutException handled) {
+      return createNoResponseState(handled,
           "Failed to communicate with locator with process id " + getPid());
-    } catch (MBeanInvocationFailedException e) {
-      // MBean either doesn't exist or method or attribute don't exist
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + getPid());
-    } catch (UnableToControlProcessException e) {
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + getPid());
-    } catch (InterruptedException e) {
+    } catch (InterruptedException handled) {
       Thread.currentThread().interrupt();
-      return createNoResponseState(e,
+      return createNoResponseState(handled,
           "Interrupted while trying to communicate with locator with process id " + getPid());
-    } catch (TimeoutException e) {
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + getPid());
     }
   }
 
@@ -920,8 +916,8 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     try {
       LocatorStatusResponse response = statusLocator(getPort(), getBindAddress());
       return new LocatorState(this, Status.ONLINE, response);
-    } catch (Exception e) {
-      return createNoResponseState(e, "Failed to connect to locator " + getId());
+    } catch (Exception handled) {
+      return createNoResponseState(handled, "Failed to connect to locator " + getId());
     }
   }
 
@@ -930,8 +926,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     try {
       final ProcessController controller =
           new ProcessControllerFactory().createProcessController(this.controllerParameters,
-              new File(getWorkingDirectory()), ProcessType.LOCATOR.getPidFileName(),
-              READ_PID_FILE_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+              new File(getWorkingDirectory()), ProcessType.LOCATOR.getPidFileName());
       parsedPid = controller.getProcessId();
 
       // note: in-process request will go infinite loop unless we do the following
@@ -944,35 +939,26 @@ public class LocatorLauncher extends AbstractLauncher<String> {
 
       final String statusJson = controller.status();
       return LocatorState.fromJson(statusJson);
-    } catch (ConnectionFailedException e) {
+    } catch (ConnectionFailedException handled) {
       // failed to attach to locator JVM
-      return createNoResponseState(e, "Failed to connect to locator with process id " + parsedPid);
-    } catch (FileNotFoundException e) {
+      return createNoResponseState(handled,
+          "Failed to connect to locator with process id " + parsedPid);
+    } catch (FileNotFoundException handled) {
       // could not find pid file
-      return createNoResponseState(e, "Failed to find process file "
+      return createNoResponseState(handled, "Failed to find process file "
           + ProcessType.LOCATOR.getPidFileName() + " in " + getWorkingDirectory());
-    } catch (IOException e) {
-      // failed to open or read file or dir
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + parsedPid);
-    } catch (MBeanInvocationFailedException e) {
-      // MBean either doesn't exist or method or attribute don't exist
-      return createNoResponseState(e,
+    } catch (IOException | MBeanInvocationFailedException | UnableToControlProcessException
+        | TimeoutException handled) {
+      return createNoResponseState(handled,
           "Failed to communicate with locator with process id " + parsedPid);
     } catch (PidUnavailableException e) {
       // couldn't determine pid from within locator JVM
       return createNoResponseState(e, "Failed to find usable process id within file "
           + ProcessType.LOCATOR.getPidFileName() + " in " + getWorkingDirectory());
-    } catch (UnableToControlProcessException e) {
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + parsedPid);
-    } catch (InterruptedException e) {
+    } catch (InterruptedException handled) {
       Thread.currentThread().interrupt();
-      return createNoResponseState(e,
+      return createNoResponseState(handled,
           "Interrupted while trying to communicate with locator with process id " + parsedPid);
-    } catch (TimeoutException e) {
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + parsedPid);
     }
   }
 
@@ -1036,7 +1022,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     if (isStoppable()) {
       this.locator.stop();
       this.locator = null;
-      this.process.stop();
+      this.process.stop(this.deletePidFileOnStop);
       this.process = null;
       INSTANCE.compareAndSet(this, null); // note: other thread may return Status.NOT_RESPONDING now
       this.running.set(false);
@@ -1053,19 +1039,13 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       controller.checkPidSupport();
       controller.stop();
       return new LocatorState(this, Status.STOPPED);
-    } catch (ConnectionFailedException e) {
+    } catch (ConnectionFailedException handled) {
       // failed to attach to locator JVM
-      return createNoResponseState(e, "Failed to connect to locator with process id " + getPid());
-    } catch (IOException e) {
-      // failed to open or read file or dir
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + getPid());
-    } catch (MBeanInvocationFailedException e) {
-      // MBean either doesn't exist or method or attribute don't exist
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + getPid());
-    } catch (UnableToControlProcessException e) {
-      return createNoResponseState(e,
+      return createNoResponseState(handled,
+          "Failed to connect to locator with process id " + getPid());
+    } catch (IOException | MBeanInvocationFailedException
+        | UnableToControlProcessException handled) {
+      return createNoResponseState(handled,
           "Failed to communicate with locator with process id " + getPid());
     }
   }
@@ -1075,8 +1055,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     try {
       final ProcessController controller =
           new ProcessControllerFactory().createProcessController(this.controllerParameters,
-              new File(getWorkingDirectory()), ProcessType.LOCATOR.getPidFileName(),
-              READ_PID_FILE_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+              new File(getWorkingDirectory()), ProcessType.LOCATOR.getPidFileName());
       parsedPid = controller.getProcessId();
 
       // NOTE in-process request will go infinite loop unless we do the following
@@ -1089,49 +1068,43 @@ public class LocatorLauncher extends AbstractLauncher<String> {
 
       controller.stop();
       return new LocatorState(this, Status.STOPPED);
-    } catch (ConnectionFailedException e) {
+    } catch (ConnectionFailedException handled) {
       // failed to attach to locator JVM
-      return createNoResponseState(e, "Failed to connect to locator with process id " + parsedPid);
-    } catch (FileNotFoundException e) {
+      return createNoResponseState(handled,
+          "Failed to connect to locator with process id " + parsedPid);
+    } catch (FileNotFoundException handled) {
       // could not find pid file
-      return createNoResponseState(e, "Failed to find process file "
+      return createNoResponseState(handled, "Failed to find process file "
           + ProcessType.LOCATOR.getPidFileName() + " in " + getWorkingDirectory());
-    } catch (IOException e) {
-      // failed to open or read file or dir
-      return createNoResponseState(e,
+    } catch (IOException | MBeanInvocationFailedException
+        | UnableToControlProcessException handled) {
+      return createNoResponseState(handled,
           "Failed to communicate with locator with process id " + parsedPid);
-    } catch (InterruptedException e) {
+    } catch (InterruptedException handled) {
       Thread.currentThread().interrupt();
-      return createNoResponseState(e,
+      return createNoResponseState(handled,
           "Interrupted while trying to communicate with locator with process id " + parsedPid);
-    } catch (MBeanInvocationFailedException e) {
-      // MBean either doesn't exist or method or attribute don't exist
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + parsedPid);
-    } catch (PidUnavailableException e) {
+    } catch (PidUnavailableException handled) {
       // couldn't determine pid from within locator JVM
-      return createNoResponseState(e, "Failed to find usable process id within file "
-          + ProcessType.LOCATOR.getPidFileName() + " in " + getWorkingDirectory());
-    } catch (TimeoutException e) {
-      return createNoResponseState(e, "Timed out trying to find usable process id within file "
+      return createNoResponseState(handled, "Failed to find usable process id within file "
           + ProcessType.LOCATOR.getPidFileName() + " in " + getWorkingDirectory());
-    } catch (UnableToControlProcessException e) {
-      return createNoResponseState(e,
-          "Failed to communicate with locator with process id " + parsedPid);
+    } catch (TimeoutException handled) {
+      return createNoResponseState(handled,
+          "Timed out trying to find usable process id within file "
+              + ProcessType.LOCATOR.getPidFileName() + " in " + getWorkingDirectory());
     }
   }
 
   private LocatorState createNoResponseState(final Exception cause, final String errorMessage) {
     debug(cause);
-    // info(errorMessage);
     return new LocatorState(this, Status.NOT_RESPONDING, errorMessage);
   }
 
-  private Properties getOverriddenDefaults() {
+  private Properties getOverriddenDefaults() throws IOException {
     Properties overriddenDefaults = new Properties();
 
     overriddenDefaults.put(ProcessLauncherContext.OVERRIDDEN_DEFAULTS_PREFIX.concat(LOG_FILE),
-        getLogFileName());
+        getLogFile().getCanonicalPath());
 
     for (String key : System.getProperties().stringPropertyNames()) {
       if (key.startsWith(ProcessLauncherContext.OVERRIDDEN_DEFAULTS_PREFIX)) {
@@ -1149,7 +1122,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     }
 
     @Override
-    public File getWorkingDirectory() {
+    public File getDirectory() {
       return new File(LocatorLauncher.this.getWorkingDirectory());
     }
 
@@ -1167,9 +1140,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     public ObjectName getNamePattern() {
       try {
         return ObjectName.getInstance("GemFire:type=Member,*");
-      } catch (MalformedObjectNameException e) {
-        return null;
-      } catch (NullPointerException e) {
+      } catch (MalformedObjectNameException | NullPointerException handled) {
         return null;
       }
     }
@@ -1210,6 +1181,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     protected static final Command DEFAULT_COMMAND = Command.UNSPECIFIED;
 
     private Boolean debug;
+    private Boolean deletePidFileOnStop;
     private Boolean force;
     private Boolean help;
     private Boolean redirectOutput;
@@ -1255,6 +1227,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
 
       parser.accepts("bind-address").withRequiredArg().ofType(String.class);
       parser.accepts("debug");
+      parser.accepts("delete-pid-file-on-stop");
       parser.accepts("dir").withRequiredArg().ofType(String.class);
       parser.accepts("force");
       parser.accepts("help");
@@ -1283,6 +1256,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
         final OptionSet options = getParser().parse(args);
 
         setDebug(options.has("debug"));
+        setDeletePidFileOnStop(options.has("delete-pid-file-on-stop"));
         setForce(options.has("force"));
         setHelp(options.has("help"));
         setRedirectOutput(options.has("redirect-output"));
@@ -1333,11 +1307,13 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       // search the list of arguments for the command; technically, the command should be the first
       // argument in the
       // list, but does it really matter? stop after we find one valid command.
-      for (String arg : args) {
-        final Command command = Command.valueOfName(arg);
-        if (command != null) {
-          setCommand(command);
-          break;
+      if (args != null) {
+        for (String arg : args) {
+          final Command command = Command.valueOfName(arg);
+          if (command != null) {
+            setCommand(command);
+            break;
+          }
         }
       }
     }
@@ -1352,11 +1328,13 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see org.apache.geode.distributed.LocatorLauncher.Command#isCommand(String)
      * @see #parseArguments(String...)
      */
-    protected void parseMemberName(final String[] args) {
-      for (String arg : args) {
-        if (!(arg.startsWith(OPTION_PREFIX) || Command.isCommand(arg))) {
-          setMemberName(arg);
-          break;
+    protected void parseMemberName(final String... args) {
+      if (args != null) {
+        for (String arg : args) {
+          if (!(arg.startsWith(OPTION_PREFIX) || Command.isCommand(arg))) {
+            setMemberName(arg);
+            break;
+          }
         }
       }
     }
@@ -1369,7 +1347,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see LocatorLauncher.Command
      */
     public Command getCommand() {
-      return ObjectUtils.defaultIfNull(this.command, DEFAULT_COMMAND);
+      return defaultIfNull(this.command, DEFAULT_COMMAND);
     }
 
     /**
@@ -1401,7 +1379,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * 
      * @param debug a boolean value indicating whether debug mode is to be enabled or disabled.
      * @return this Builder instance.
-     * @see #getHelp()
+     * @see #getDebug()
      */
     public Builder setDebug(final Boolean debug) {
       this.debug = debug;
@@ -1409,6 +1387,32 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     }
 
     /**
+     * Determines whether the Geode Locator should delete the pid file when its service stops or
+     * when the JVM exits.
+     *
+     * @return a boolean value indicating if the pid file should be deleted when this service stops
+     *         or when the JVM exits.
+     * @see #setDeletePidFileOnStop(Boolean)
+     */
+    public Boolean getDeletePidFileOnStop() {
+      return this.deletePidFileOnStop;
+    }
+
+    /**
+     * Sets whether the Geode Locator should delete the pid file when its service stops or when the
+     * JVM exits.
+     *
+     * @param deletePidFileOnStop a boolean value indicating if the pid file should be deleted when
+     *        this service stops or when the JVM exits.
+     * @return this Builder instance.
+     * @see #getDeletePidFileOnStop()
+     */
+    public Builder setDeletePidFileOnStop(final Boolean deletePidFileOnStop) {
+      this.deletePidFileOnStop = deletePidFileOnStop;
+      return this;
+    }
+
+    /**
      * Gets the GemFire Distributed System (cluster) Properties configuration.
      *
      * @return a Properties object containing configuration settings for the GemFire Distributed
@@ -1425,11 +1429,10 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * 
      * @return the boolean value specifying whether or not to overwrite the PID file if it already
      *         exists.
-     * @see org.apache.geode.internal.process.LocalProcessLauncher
      * @see #setForce(Boolean)
      */
     public Boolean getForce() {
-      return ObjectUtils.defaultIfNull(this.force, DEFAULT_FORCE);
+      return defaultIfNull(this.force, DEFAULT_FORCE);
     }
 
     /**
@@ -1439,7 +1442,6 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @param force a boolean value indicating whether to overwrite the PID file when it already
      *        exists.
      * @return this Builder instance.
-     * @see org.apache.geode.internal.process.LocalProcessLauncher
      * @see #getForce()
      */
     public Builder setForce(final Boolean force) {
@@ -1513,7 +1515,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see java.net.InetAddress
      */
     public Builder setBindAddress(final String bindAddress) {
-      if (StringUtils.isBlank(bindAddress)) {
+      if (isBlank(bindAddress)) {
         this.bindAddress = null;
         return this;
       } else {
@@ -1557,7 +1559,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see #getHostnameForClients()
      */
     public Builder setHostnameForClients(final String hostnameForClients) {
-      if (StringUtils.isBlank(hostnameForClients)) {
+      if (isBlank(hostnameForClients)) {
         throw new IllegalArgumentException(
             LocalizedStrings.LocatorLauncher_Builder_INVALID_HOSTNAME_FOR_CLIENTS_ERROR_MESSAGE
                 .toLocalizedString());
@@ -1585,7 +1587,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see #getMemberName()
      */
     public Builder setMemberName(final String memberName) {
-      if (StringUtils.isBlank(memberName)) {
+      if (isBlank(memberName)) {
         throw new IllegalArgumentException(
             LocalizedStrings.Launcher_Builder_MEMBER_NAME_ERROR_MESSAGE
                 .toLocalizedString("Locator"));
@@ -1638,7 +1640,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see #setPort(Integer)
      */
     public Integer getPort() {
-      return ObjectUtils.defaultIfNull(port, getDefaultLocatorPort());
+      return defaultIfNull(port, getDefaultLocatorPort());
     }
 
     /**
@@ -1701,7 +1703,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     }
 
     boolean isWorkingDirectorySpecified() {
-      return StringUtils.isNotBlank(this.workingDirectory);
+      return isNotBlank(this.workingDirectory);
     }
 
     /**
@@ -1712,8 +1714,8 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see #setWorkingDirectory(String)
      */
     public String getWorkingDirectory() {
-      return IOUtils.tryGetCanonicalPathElseGetAbsolutePath(
-          new File(StringUtils.defaultIfBlank(this.workingDirectory, DEFAULT_WORKING_DIRECTORY)));
+      return tryGetCanonicalPathElseGetAbsolutePath(
+          new File(defaultIfBlank(this.workingDirectory, DEFAULT_WORKING_DIRECTORY)));
     }
 
     /**
@@ -1730,8 +1732,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see java.io.FileNotFoundException
      */
     public Builder setWorkingDirectory(final String workingDirectory) {
-      if (!new File(StringUtils.defaultIfBlank(workingDirectory, DEFAULT_WORKING_DIRECTORY))
-          .isDirectory()) {
+      if (!new File(defaultIfBlank(workingDirectory, DEFAULT_WORKING_DIRECTORY)).isDirectory()) {
         throw new IllegalArgumentException(
             LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE
                 .toLocalizedString("Locator"),
@@ -1781,8 +1782,8 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see org.apache.geode.distributed.LocatorLauncher.Command#START
      */
     protected void validateOnStart() {
-      if (Command.START.equals(getCommand())) {
-        if (StringUtils.isBlank(getMemberName())
+      if (Command.START == getCommand()) {
+        if (isBlank(getMemberName())
             && !isSet(System.getProperties(), DistributionConfig.GEMFIRE_PREFIX + NAME)
             && !isSet(getDistributedSystemProperties(), NAME)
             && !isSet(loadGemFireProperties(DistributedSystem.getPropertyFileURL()), NAME)) {
@@ -1791,7 +1792,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
                   .toLocalizedString("Locator"));
         }
 
-        if (!SystemUtils.CURRENT_DIRECTORY.equals(getWorkingDirectory())) {
+        if (!CURRENT_DIRECTORY.equals(getWorkingDirectory())) {
           throw new IllegalStateException(
               LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_OPTION_NOT_VALID_ERROR_MESSAGE
                   .toLocalizedString("Locator"));
@@ -1805,7 +1806,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see org.apache.geode.distributed.LocatorLauncher.Command#STATUS
      */
     protected void validateOnStatus() {
-      if (Command.STATUS.equals(getCommand())) {
+      if (Command.STATUS == getCommand()) {
       }
     }
 
@@ -1815,7 +1816,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see org.apache.geode.distributed.LocatorLauncher.Command#STOP
      */
     protected void validateOnStop() {
-      if (Command.STOP.equals(getCommand())) {
+      if (Command.STOP == getCommand()) {
       }
     }
 
@@ -1848,8 +1849,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     private final String name;
 
     Command(final String name, final String... options) {
-      assert !StringUtils
-          .isBlank(name) : "The name of the locator launcher command must be specified!";
+      assert isNotBlank(name) : "The name of the locator launcher command must be specified!";
       this.name = name;
       this.options = (options != null ? Collections.unmodifiableList(Arrays.asList(options))
           : Collections.<String>emptyList());
@@ -1925,7 +1925,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      *         option.
      */
     public boolean hasOption(final String option) {
-      return getOptions().contains(StringUtils.lowerCase(option));
+      return getOptions().contains(lowerCase(option));
     }
 
     /**
@@ -1935,7 +1935,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
      * @see #UNSPECIFIED
      */
     public boolean isUnspecified() {
-      return UNSPECIFIED.equals(this);
+      return UNSPECIFIED == this;
     }
 
     /**
@@ -2033,7 +2033,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
         final InternalLocator locator = InternalLocator.getLocator();
         final InetAddress bindAddress = locator.getBindAddress();
         if (bindAddress != null) {
-          if (StringUtils.isBlank(bindAddress.getHostAddress())) {
+          if (isBlank(bindAddress.getHostAddress())) {
             return bindAddress.getHostAddress();
           }
         }
@@ -2047,10 +2047,9 @@ public class LocatorLauncher extends AbstractLauncher<String> {
         final File logFile = locator.getLogFile();
 
         if (logFile != null && logFile.isFile()) {
-          final String logFileCanonicalPath =
-              IOUtils.tryGetCanonicalPathElseGetAbsolutePath(logFile);
-          if (StringUtils.isNotBlank(logFileCanonicalPath)) { // this is probably not need but a
-                                                              // safe
+          final String logFileCanonicalPath = tryGetCanonicalPathElseGetAbsolutePath(logFile);
+          if (isNotBlank(logFileCanonicalPath)) { // this is probably not need but a
+                                                  // safe
             // check none-the-less.
             return logFileCanonicalPath;
           }
@@ -2063,7 +2062,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       if (InternalLocator.hasLocator()) {
         final InternalLocator locator = InternalLocator.getLocator();
         final String portAsString = String.valueOf(locator.getPort());
-        if (StringUtils.isNotBlank(portAsString)) {
+        if (isNotBlank(portAsString)) {
           return portAsString;
         }
       }