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:47 UTC

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

http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTest.java
index b856361..a5c20f4 100755
--- a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTest.java
@@ -17,14 +17,21 @@ package org.apache.geode.distributed;
 import static com.googlecode.catchexception.apis.BDDCatchException.caughtException;
 import static com.googlecode.catchexception.apis.BDDCatchException.when;
 import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.apache.geode.distributed.DistributedSystem.PROPERTIES_FILE_PROPERTY;
+import static org.apache.geode.distributed.internal.DistributionConfig.GEMFIRE_PREFIX;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPort;
+import static org.apache.geode.internal.DistributionLocator.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY;
 import static org.assertj.core.api.BDDAssertions.assertThat;
 import static org.assertj.core.api.BDDAssertions.then;
 
-import org.apache.geode.distributed.LocatorLauncher.Builder;
-import org.apache.geode.distributed.LocatorLauncher.Command;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.test.junit.categories.IntegrationTest;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.InetAddress;
+import java.util.Properties;
+
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
@@ -32,19 +39,19 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.util.Properties;
+import org.apache.geode.distributed.LocatorLauncher.Builder;
+import org.apache.geode.distributed.LocatorLauncher.Command;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.test.junit.categories.IntegrationTest;
 
 /**
- * Integration tests for LocatorLauncher. These tests require file system I/O.
+ * Integration tests for using {@link LocatorLauncher} as an in-process API within an existing JVM.
  */
 @Category(IntegrationTest.class)
 public class LocatorLauncherIntegrationTest {
 
+  private static final String CURRENT_DIRECTORY = System.getProperty("user.dir");
+
   @Rule
   public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
 
@@ -55,177 +62,242 @@ public class LocatorLauncherIntegrationTest {
   public final TestName testName = new TestName();
 
   @Test
-  public void testBuilderParseArgumentsWithValuesSeparatedWithCommas() throws Exception {
-    // given: a new builder and working directory
-    String expectedWorkingDirectory = this.temporaryFolder.getRoot().getCanonicalPath();
+  public void buildWithMemberNameSetInGemFireProperties() throws Exception {
+    // given: gemfire.properties with a name
+    givenGemFirePropertiesFile(withMemberName());
+
+    // when: starting with null MemberName
+    LocatorLauncher launcher = new Builder().setCommand(Command.START).build();
+
+    // then: name in gemfire.properties file should be used for MemberName
+    assertThat(launcher.getMemberName()).isNull(); // name will be read during start()
+  }
+
+  @Test
+  public void buildWithNoMemberNameThrowsIllegalStateException() throws Exception {
+    // given: gemfire.properties with no name
+    givenGemFirePropertiesFile(withoutMemberName());
+
+    // when: no MemberName is specified
+    when(new Builder().setCommand(Command.START)).build();
+
+    // then: throw IllegalStateException
+    then(caughtException()).isExactlyInstanceOf(IllegalStateException.class)
+        .hasMessage(memberNameValidationErrorMessage());
+  }
+
+  @Test
+  public void buildWithWorkingDirectoryNotEqualToCurrentDirectoryThrowsIllegalStateException()
+      throws Exception {
+    // given: using LocatorLauncher in-process
+
+    // when: setting WorkingDirectory to non-current directory
+    when(new Builder().setCommand(Command.START).setMemberName("memberOne")
+        .setWorkingDirectory(getWorkingDirectoryPath())).build();
+
+    // then: throw IllegalStateException
+    then(caughtException()).isExactlyInstanceOf(IllegalStateException.class)
+        .hasMessage(workingDirectoryOptionNotValidErrorMessage());
+  }
+
+  @Test
+  public void parseArgumentsParsesValuesSeparatedByCommas() throws Exception {
+    // given: a new builder
     Builder builder = new Builder();
 
     // when: parsing many arguments
     builder.parseArguments("start", "memberOne", "--bind-address",
-        InetAddress.getLocalHost().getHostAddress(), "--dir", expectedWorkingDirectory,
+        InetAddress.getLocalHost().getHostAddress(), "--dir", getWorkingDirectoryPath(),
         "--hostname-for-clients", "Tucows", "--pid", "1234", "--port", "11235", "--redirect-output",
         "--force", "--debug");
 
     // then: the getters should return properly parsed values
     assertThat(builder.getCommand()).isEqualTo(Command.START);
     assertThat(builder.getBindAddress()).isEqualTo(InetAddress.getLocalHost());
-    assertThat(builder.getWorkingDirectory()).isEqualTo(expectedWorkingDirectory);
+    assertThat(builder.getDebug()).isTrue();
+    assertThat(builder.getForce()).isTrue();
     assertThat(builder.getHostnameForClients()).isEqualTo("Tucows");
     assertThat(builder.getPid().intValue()).isEqualTo(1234);
     assertThat(builder.getPort().intValue()).isEqualTo(11235);
     assertThat(builder.getRedirectOutput()).isTrue();
-    assertThat(builder.getForce()).isTrue();
-    assertThat(builder.getDebug()).isTrue();
+    assertThat(builder.getWorkingDirectory()).isEqualTo(getWorkingDirectoryPath());
   }
 
   @Test
-  public void testBuilderParseArgumentsWithValuesSeparatedWithEquals() throws Exception {
-    // given: a new builder and a directory
-    String expectedWorkingDirectory = this.temporaryFolder.getRoot().getCanonicalPath();
+  public void parseArgumentsParsesValuesSeparatedByEquals() throws Exception {
+    // given: a new builder
     Builder builder = new Builder();
 
     // when: parsing arguments with values separated by equals
-    builder.parseArguments("start", "--dir=" + expectedWorkingDirectory, "--port=" + "12345",
+    builder.parseArguments("start", "--dir=" + getWorkingDirectoryPath(), "--port=" + "12345",
         "memberOne");
 
     // then: the getters should return properly parsed values
+    assertThat(builder.getBindAddress()).isNull();
     assertThat(builder.getCommand()).isEqualTo(Command.START);
     assertThat(builder.getDebug()).isFalse();
     assertThat(builder.getForce()).isFalse();
     assertThat(builder.getHelp()).isFalse();
-    assertThat(builder.getBindAddress()).isNull();
     assertThat(builder.getHostnameForClients()).isNull();
     assertThat(builder.getMemberName()).isEqualTo("memberOne");
     assertThat(builder.getPid()).isNull();
-    assertThat(builder.getWorkingDirectory()).isEqualTo(expectedWorkingDirectory);
     assertThat(builder.getPort().intValue()).isEqualTo(12345);
+    assertThat(builder.getWorkingDirectory()).isEqualTo(getWorkingDirectoryPath());
   }
 
   @Test
-  public void testBuildWithMemberNameSetInGemFirePropertiesOnStart() throws Exception {
-    // given: gemfire.properties with a name
-    Properties gemfireProperties = new Properties();
-    gemfireProperties.setProperty(NAME, "locator123");
-    useGemFirePropertiesFileInTemporaryFolder(DistributionConfig.GEMFIRE_PREFIX + "properties",
-        gemfireProperties);
+  public void getWorkingDirectoryReturnsCurrentDirectoryByDefault() throws Exception {
+    // given:
 
-    // when: starting with null MemberName
-    LocatorLauncher launcher = new Builder().setCommand(Command.START).build();
+    // when: not setting WorkingDirectory
 
-    // then: name in gemfire.properties file should be used for MemberName
-    assertThat(launcher).isNotNull();
-    assertThat(launcher.getCommand()).isEqualTo(Command.START);
-    assertThat(launcher.getMemberName()).isNull();
+    // then: getDirectory returns default
+    assertThat(new Builder().getWorkingDirectory()).isEqualTo(CURRENT_DIRECTORY);
   }
 
   @Test
-  public void testBuildWithNoMemberNameOnStart() throws Exception {
-    // given: gemfire.properties with no name
-    useGemFirePropertiesFileInTemporaryFolder(DistributionConfig.GEMFIRE_PREFIX + "properties",
-        new Properties());
+  public void setWorkingDirectoryToNullUsesCurrentDirectory() throws Exception {
+    // given: a new builder
+    Builder builder = new Builder();
 
-    // when: no MemberName is specified
-    when(new Builder().setCommand(Command.START)).build();
+    // when: setting WorkingDirectory to null
+    assertThat(builder.setWorkingDirectory(null)).isSameAs(builder);
 
-    // then: throw IllegalStateException
-    then(caughtException()).isExactlyInstanceOf(IllegalStateException.class)
-        .hasMessage(LocalizedStrings.Launcher_Builder_MEMBER_NAME_VALIDATION_ERROR_MESSAGE
-            .toLocalizedString("Locator"));
+    // then: getDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(CURRENT_DIRECTORY);
   }
 
   @Test
-  public void testBuilderSetAndGetWorkingDirectory() throws Exception {
-    // given: a new builder and a directory
-    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+  public void setWorkingDirectoryToEmptyStringUsesCurrentDirectory() throws Exception {
+    // given: a new builder
     Builder builder = new Builder();
 
-    // when: not setting WorkingDirectory
-    // then: getWorkingDirectory returns default
-    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
-
-    // when: setting WorkingDirectory to null
-    assertThat(builder.setWorkingDirectory(null)).isSameAs(builder);
-    // then: getWorkingDirectory returns default
-    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
-
     // when: setting WorkingDirectory to empty string
     assertThat(builder.setWorkingDirectory("")).isSameAs(builder);
-    // then: getWorkingDirectory returns default
-    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+
+    // then: getDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(CURRENT_DIRECTORY);
+  }
+
+  @Test
+  public void setWorkingDirectoryToBlankStringUsesCurrentDirectory() throws Exception {
+    // given: a new builder
+    Builder builder = new Builder();
 
     // when: setting WorkingDirectory to white space
     assertThat(builder.setWorkingDirectory("  ")).isSameAs(builder);
-    // then: getWorkingDirectory returns default
-    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+
+    // then: getDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(CURRENT_DIRECTORY);
+  }
+
+  @Test
+  public void setWorkingDirectoryToExistingDirectory() throws Exception {
+    // given: a new builder
+    Builder builder = new Builder();
 
     // when: setting WorkingDirectory to a directory
-    assertThat(builder.setWorkingDirectory(rootFolder)).isSameAs(builder);
-    // then: getWorkingDirectory returns that directory
-    assertThat(builder.getWorkingDirectory()).isEqualTo(rootFolder);
+    assertThat(builder.setWorkingDirectory(getWorkingDirectoryPath())).isSameAs(builder);
 
-    // when: setting WorkingDirectory to null (again)
-    assertThat(builder.setWorkingDirectory(null)).isSameAs(builder);
-    // then: getWorkingDirectory returns default
-    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+    // then: getDirectory returns that directory
+    assertThat(builder.getWorkingDirectory()).isEqualTo(getWorkingDirectoryPath());
   }
 
   @Test
-  public void testBuilderSetWorkingDirectoryToFile() throws IOException {
+  public void setWorkingDirectoryToExistingFileThrowsIllegalArgumentException() throws Exception {
     // given: a file instead of a directory
-    File tmpFile = this.temporaryFolder.newFile();
+    File nonDirectory = temporaryFolder.newFile();
 
     // when: setting WorkingDirectory to that file
-    when(new Builder()).setWorkingDirectory(tmpFile.getCanonicalPath());
+    when(new Builder()).setWorkingDirectory(nonDirectory.getCanonicalPath());
 
     // then: throw IllegalArgumentException
     then(caughtException()).isExactlyInstanceOf(IllegalArgumentException.class)
-        .hasMessage(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE
-            .toLocalizedString("Locator"))
-        .hasCause(new FileNotFoundException(tmpFile.getCanonicalPath()));
+        .hasMessage(workingDirectoryNotFoundErrorMessage())
+        .hasCause(new FileNotFoundException(nonDirectory.getCanonicalPath()));
   }
 
   @Test
-  public void testBuildSetWorkingDirectoryToNonCurrentDirectoryOnStart() throws Exception {
-    // given: using LocatorLauncher in-process
-
-    // when: setting WorkingDirectory to non-current directory
-    when(new Builder().setCommand(Command.START).setMemberName("memberOne")
-        .setWorkingDirectory(this.temporaryFolder.getRoot().getCanonicalPath())).build();
-
-    // then: throw IllegalStateException
-    then(caughtException()).isExactlyInstanceOf(IllegalStateException.class).hasMessage(
-        LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_OPTION_NOT_VALID_ERROR_MESSAGE
-            .toLocalizedString("Locator"));
-  }
+  public void setWorkingDirectoryToNonExistingDirectory() throws Exception {
+    // given:
 
-  @Test
-  public void testBuilderSetWorkingDirectoryToNonExistingDirectory() {
     // when: setting WorkingDirectory to non-existing directory
     when(new Builder()).setWorkingDirectory("/path/to/non_existing/directory");
 
     // then: throw IllegalArgumentException
     then(caughtException()).isExactlyInstanceOf(IllegalArgumentException.class)
-        .hasMessage(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE
-            .toLocalizedString("Locator"))
+        .hasMessage(workingDirectoryNotFoundErrorMessage())
         .hasCause(new FileNotFoundException("/path/to/non_existing/directory"));
   }
 
+  @Test
+  public void portCanBeOverriddenBySystemProperty() throws Exception {
+    // given: overridden default port
+    int overriddenPort = getRandomAvailableTCPPort();
+    System.setProperty(TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, String.valueOf(overriddenPort));
+
+    // when: creating new LocatorLauncher
+    LocatorLauncher launcher = new Builder().build();
+
+    // then: locator port should be the overridden default port
+    assertThat(launcher.getPort()).isEqualTo(overriddenPort);
+  }
+
+  private String memberNameValidationErrorMessage() {
+    return LocalizedStrings.Launcher_Builder_MEMBER_NAME_VALIDATION_ERROR_MESSAGE
+        .toLocalizedString("Locator");
+  }
+
+  private String workingDirectoryOptionNotValidErrorMessage() {
+    return LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_OPTION_NOT_VALID_ERROR_MESSAGE
+        .toLocalizedString("Locator");
+  }
+
+  private String workingDirectoryNotFoundErrorMessage() {
+    return LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE
+        .toLocalizedString("Locator");
+  }
+
+  private File getWorkingDirectory() {
+    return temporaryFolder.getRoot();
+  }
+
+  private String getWorkingDirectoryPath() {
+    try {
+      return getWorkingDirectory().getCanonicalPath();
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  private Properties withoutMemberName() {
+    return new Properties();
+  }
+
+  private Properties withMemberName() {
+    Properties properties = new Properties();
+    properties.setProperty(NAME, "locator123");
+    return properties;
+  }
+
   /**
    * Creates a gemfire properties file in temporaryFolder:
    * <ol>
-   * <li>creates <code>fileName</code> in <code>temporaryFolder</code></li>
+   * <li>creates gemfire.properties in <code>temporaryFolder</code></li>
+   * <li>writes config to the file</li>
    * <li>sets "gemfirePropertyFile" system property</li>
-   * <li>writes <code>gemfireProperties</code> to the file</li>
    * </ol>
    */
-  private void useGemFirePropertiesFileInTemporaryFolder(final String fileName,
-      final Properties gemfireProperties) throws Exception {
-    File propertiesFile = new File(this.temporaryFolder.getRoot().getCanonicalPath(), fileName);
-    System.setProperty(DistributedSystem.PROPERTIES_FILE_PROPERTY,
-        propertiesFile.getCanonicalPath());
-
-    gemfireProperties.store(new FileWriter(propertiesFile, false), this.testName.getMethodName());
-    assertThat(propertiesFile.isFile()).isTrue();
-    assertThat(propertiesFile.exists()).isTrue();
+  private void givenGemFirePropertiesFile(final Properties config) {
+    try {
+      String name = GEMFIRE_PREFIX + "properties";
+      File file = new File(getWorkingDirectory(), name);
+      config.store(new FileWriter(file, false), testName.getMethodName());
+      assertThat(file).isFile().exists();
+
+      System.setProperty(PROPERTIES_FILE_PROPERTY, file.getCanonicalPath());
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTestCase.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTestCase.java
new file mode 100755
index 0000000..c1d2505
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherIntegrationTestCase.java
@@ -0,0 +1,163 @@
+/*
+ * 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 static java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.geode.distributed.AbstractLauncher.Status.STOPPED;
+import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_CONFIGURATION_DIR;
+import static org.apache.geode.distributed.ConfigurationProperties.DISABLE_AUTO_RECONNECT;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.distributed.internal.ClusterConfigurationService.CLUSTER_CONFIG_DISK_DIR_PREFIX;
+import static org.apache.geode.distributed.internal.DistributionConfig.GEMFIRE_PREFIX;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.apache.geode.internal.DistributionLocator.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.ErrorCollector;
+
+import org.apache.geode.distributed.AbstractLauncher.Status;
+import org.apache.geode.distributed.LocatorLauncher.Builder;
+import org.apache.geode.distributed.LocatorLauncher.LocatorState;
+import org.apache.geode.internal.process.ProcessType;
+
+/**
+ * Abstract base class for integration tests of {@link LocatorLauncher}.
+ *
+ * @since GemFire 8.0
+ */
+public abstract class LocatorLauncherIntegrationTestCase extends LauncherIntegrationTestCase {
+
+  protected volatile int defaultLocatorPort;
+  protected volatile int nonDefaultLocatorPort;
+  protected volatile LocatorLauncher launcher;
+
+  private volatile File clusterConfigDirectory;
+
+  @Rule
+  public ErrorCollector errorCollector = new ErrorCollector();
+
+  @Before
+  public void setUpAbstractLocatorLauncherIntegrationTestCase() throws Exception {
+    System.setProperty(GEMFIRE_PREFIX + MCAST_PORT, Integer.toString(0));
+
+    clusterConfigDirectory =
+        temporaryFolder.newFolder(CLUSTER_CONFIG_DISK_DIR_PREFIX + getUniqueName());
+
+    int[] ports = getRandomAvailableTCPPorts(2);
+    defaultLocatorPort = ports[0];
+    nonDefaultLocatorPort = ports[1];
+    System.setProperty(TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, String.valueOf(defaultLocatorPort));
+  }
+
+  @After
+  public void tearDownAbstractLocatorLauncherIntegrationTestCase() throws Exception {
+    if (launcher != null) {
+      launcher.stop();
+    }
+  }
+
+  @Override
+  protected ProcessType getProcessType() {
+    return ProcessType.LOCATOR;
+  }
+
+  @Override
+  protected void givenEmptyWorkingDirectory() {
+    File[] files = getWorkingDirectory().listFiles();
+    assertThat(files).hasSize(1);
+    assertThat(files[0]).isDirectory().isEqualTo(getClusterConfigDirectory());
+  }
+
+  protected LocatorLauncher givenLocatorLauncher() {
+    return givenLocatorLauncher(newBuilder());
+  }
+
+  private LocatorLauncher givenLocatorLauncher(final Builder builder) {
+    return builder.build();
+  }
+
+  protected LocatorLauncher givenRunningLocator() {
+    return givenRunningLocator(newBuilder());
+  }
+
+  protected LocatorLauncher givenRunningLocator(final Builder builder) {
+    return awaitStart(builder);
+  }
+
+  protected LocatorLauncher awaitStart(final LocatorLauncher launcher) {
+    await().atMost(2, MINUTES).until(() -> assertThat(isLauncherOnline()).isTrue());
+    return launcher;
+  }
+
+  protected Locator getLocator() {
+    return launcher.getLocator();
+  }
+
+  /**
+   * Returns a new Builder with helpful defaults for safe testing. If you need a Builder in a test
+   * without any of these defaults then simply use {@code new Builder()} instead.
+   */
+  protected Builder newBuilder() {
+    return new Builder().setMemberName(getUniqueName()).setRedirectOutput(true)
+        .setWorkingDirectory(getWorkingDirectoryPath())
+        .set(CLUSTER_CONFIGURATION_DIR, getClusterConfigDirectoryPath())
+        .set(DISABLE_AUTO_RECONNECT, "true").set(LOG_LEVEL, "config").set(MCAST_PORT, "0");
+  }
+
+  protected LocatorLauncher startLocator() {
+    return awaitStart(newBuilder());
+  }
+
+  protected LocatorLauncher startLocator(final Builder builder) {
+    return awaitStart(builder);
+  }
+
+  protected void stopLocator() {
+    assertThat(launcher.stop().getStatus()).isEqualTo(STOPPED);
+  }
+
+  private LocatorLauncher awaitStart(final Builder builder) {
+    launcher = builder.build();
+    assertThat(launcher.start().getStatus()).isEqualTo(Status.ONLINE);
+    return awaitStart(launcher);
+  }
+
+  private File getClusterConfigDirectory() {
+    return clusterConfigDirectory;
+  }
+
+  private String getClusterConfigDirectoryPath() {
+    try {
+      return clusterConfigDirectory.getCanonicalPath();
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  private boolean isLauncherOnline() {
+    LocatorState locatorState = launcher.status();
+    assertNotNull(locatorState);
+    return Status.ONLINE.equals(locatorState.getStatus());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherJmxManagerLocalRegressionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherJmxManagerLocalRegressionTest.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherJmxManagerLocalRegressionTest.java
new file mode 100644
index 0000000..a465070
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherJmxManagerLocalRegressionTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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 static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.Set;
+
+import org.awaitility.Awaitility;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.internal.process.ProcessType;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+/**
+ * Regression tests for stopping a JMX Manager process launched with {@link LocatorLauncher}.
+ *
+ * <p>
+ * Confirms fix for <bold>GEODE-528: Locator not stopping correctly if jmx-manager-port=0</bold>
+ *
+ * <p>
+ * Refactored from LocatorLauncherAssemblyIntegrationTest which used to be in geode-assembly.
+ */
+@Category(IntegrationTest.class)
+public class LocatorLauncherJmxManagerLocalRegressionTest
+    extends LocatorLauncherIntegrationTestCase {
+
+  /**
+   * Using Awaitility will increase total thread count by 2: ConditionAwaiter and
+   * ConditionAwaiter$ConditionPoller.
+   */
+  private static final int AWAITILITY_USAGE_THREAD_COUNT = 2;
+
+  private Set<Thread> initialThreads;
+  private int jmxManagerPort;
+
+  @Before
+  public void setUpLocatorLauncherJmxManagerLocalIntegrationTest() throws Exception {
+    disconnectFromDS();
+    System.setProperty(ProcessType.PROPERTY_TEST_PREFIX, getUniqueName() + "-");
+
+    int[] ports = getRandomAvailableTCPPorts(3);
+    this.defaultLocatorPort = ports[0];
+    this.nonDefaultLocatorPort = ports[1];
+    this.jmxManagerPort = ports[2];
+
+    this.initialThreads = Thread.getAllStackTraces().keySet();
+  }
+
+  @Test
+  public void locatorWithZeroJmxPortCleansUpWhenStopped() throws Exception {
+    startLocator(newBuilder().setDeletePidFileOnStop(true).setMemberName(getUniqueName())
+        .setPort(this.defaultLocatorPort).setRedirectOutput(false)
+        .setWorkingDirectory(getWorkingDirectoryPath()).set(LOG_LEVEL, "config")
+        .set(ENABLE_CLUSTER_CONFIGURATION, "false").set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_START, "true").set(JMX_MANAGER_PORT, "0"));
+
+    stopLocator();
+
+    assertDeletionOf(getPidFile());
+    assertThatThreadsStopped();
+  }
+
+  @Test
+  public void locatorWithNonZeroJmxPortCleansUpWhenStopped() throws Exception {
+    startLocator(newBuilder().setDeletePidFileOnStop(true).setMemberName(getUniqueName())
+        .setPort(this.defaultLocatorPort).setRedirectOutput(false)
+        .setWorkingDirectory(getWorkingDirectoryPath()).set(LOG_LEVEL, "config")
+        .set(ENABLE_CLUSTER_CONFIGURATION, "false").set(JMX_MANAGER, "true")
+        .set(JMX_MANAGER_START, "true").set(JMX_MANAGER_PORT, String.valueOf(jmxManagerPort)));
+
+    stopLocator();
+
+    assertDeletionOf(getPidFile());
+    assertThatThreadsStopped();
+  }
+
+  private void assertThatThreadsStopped() {
+    Awaitility.await().atMost(30, SECONDS).until(
+        () -> assertThat(currentThreadCount()).isEqualTo(initialThreadCountPlusAwaitility()));
+  }
+
+  private int currentThreadCount() {
+    return Thread.getAllStackTraces().keySet().size();
+  }
+
+  private int initialThreadCountPlusAwaitility() {
+    return initialThreads.size() + AWAITILITY_USAGE_THREAD_COUNT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherJmxManagerRemoteRegressionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherJmxManagerRemoteRegressionTest.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherJmxManagerRemoteRegressionTest.java
new file mode 100644
index 0000000..ded6a72
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherJmxManagerRemoteRegressionTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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 static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.internal.AvailablePortHelper.getRandomAvailableTCPPorts;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+/**
+ * Regression tests for stopping a JMX Manager process launched with {@link LocatorLauncher}.
+ *
+ * <p>
+ * Confirms fix for <bold>GEODE-528: Locator not stopping correctly if jmx-manager-port=0</bold>
+ *
+ * <p>
+ * Refactored from LocatorLauncherAssemblyIntegrationTest which used to be in geode-assembly.
+ */
+@Category(IntegrationTest.class)
+public class LocatorLauncherJmxManagerRemoteRegressionTest
+    extends LocatorLauncherRemoteIntegrationTestCase {
+
+  private int jmxManagerPort;
+
+  @Before
+  public void before() throws Exception {
+    int[] ports = getRandomAvailableTCPPorts(3);
+    this.defaultLocatorPort = ports[0];
+    this.nonDefaultLocatorPort = ports[1];
+    this.jmxManagerPort = ports[2];
+  }
+
+  @Test
+  public void locatorProcessWithZeroJmxPortExitsWhenStopped() throws Exception {
+    givenRunningLocator(addJvmArgument("-D" + JMX_MANAGER + "=true")
+        .addJvmArgument("-D" + JMX_MANAGER_START + "=true")
+        .addJvmArgument("-D" + JMX_MANAGER_PORT + "=0"));
+
+    new LocatorLauncher.Builder().setWorkingDirectory(getWorkingDirectoryPath()).build().stop();
+
+    assertStopOf(getLocatorProcess());
+  }
+
+  @Test
+  public void locatorProcessWithNonZeroJmxPortExitsWhenStopped() throws Exception {
+    givenRunningLocator(addJvmArgument("-D" + JMX_MANAGER + "=true")
+        .addJvmArgument("-D" + JMX_MANAGER_START + "=true")
+        .addJvmArgument("-D" + JMX_MANAGER_PORT + "=" + jmxManagerPort));
+
+    new LocatorLauncher.Builder().setWorkingDirectory(getWorkingDirectoryPath()).build().stop();
+
+    assertStopOf(getLocatorProcess());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalFileIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalFileIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalFileIntegrationTest.java
index 86374f1..3ae64c1 100755
--- a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalFileIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalFileIntegrationTest.java
@@ -14,42 +14,29 @@
  */
 package org.apache.geode.distributed;
 
-import org.apache.geode.internal.process.ProcessControllerFactory;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-import org.junit.After;
+import static org.assertj.core.api.Assertions.assertThat;
+
 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 static org.junit.Assert.assertFalse;
+import org.apache.geode.internal.process.ProcessControllerFactory;
+import org.apache.geode.test.junit.categories.IntegrationTest;
 
 /**
- * Subclass of LocatorLauncherLocalDUnitTest which forces the code to not find the Attach API which
- * is in the JDK tools.jar. As a result LocatorLauncher ends up using the FileProcessController
- * implementation.
+ * Integration tests for using {@link LocatorLauncher} as an in-process API within an existing JVM
+ * without the Attach API.
+ *
+ * Sets {@link ProcessControllerFactory#PROPERTY_DISABLE_ATTACH_API} to force
+ * {@code LocatorLauncher} to use the FileProcessController implementation.
  *
  * @since GemFire 8.0
  */
 @Category(IntegrationTest.class)
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class LocatorLauncherLocalFileIntegrationTest extends LocatorLauncherLocalIntegrationTest {
 
   @Before
-  public final void setUpLocatorLauncherLocalFileIntegrationTest() throws Exception {
+  public void setUpLocatorLauncherLocalFileIntegrationTest() throws Exception {
     System.setProperty(ProcessControllerFactory.PROPERTY_DISABLE_ATTACH_API, "true");
-  }
-
-  @After
-  public final void tearDownLocatorLauncherLocalFileIntegrationTest() throws Exception {}
-
-  @Override
-  @Test
-  public void testIsAttachAPIFound() throws Exception {
-    final ProcessControllerFactory factory = new ProcessControllerFactory();
-    assertFalse(factory.isAttachAPIFound());
+    assertThat(new ProcessControllerFactory().isAttachAPIFound()).isFalse();
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalIntegrationTest.java
index 79314aa..9fce94e 100755
--- a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherLocalIntegrationTest.java
@@ -14,745 +14,269 @@
  */
 package org.apache.geode.distributed;
 
-import org.apache.geode.distributed.AbstractLauncher.Status;
+import static org.apache.geode.distributed.AbstractLauncher.Status.NOT_RESPONDING;
+import static org.apache.geode.distributed.AbstractLauncher.Status.ONLINE;
+import static org.apache.geode.distributed.AbstractLauncher.Status.STOPPED;
+import static org.apache.geode.distributed.ConfigurationProperties.DISABLE_AUTO_RECONNECT;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.File;
+import java.net.BindException;
+import java.net.InetAddress;
+import java.util.Collections;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.distributed.LocatorLauncher.Builder;
 import org.apache.geode.distributed.LocatorLauncher.LocatorState;
 import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.internal.*;
-import org.apache.geode.internal.net.SocketCreatorFactory;
+import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.process.ProcessControllerFactory;
 import org.apache.geode.internal.process.ProcessType;
-import org.apache.geode.internal.process.ProcessUtils;
-import org.apache.geode.internal.security.SecurableCommunicationChannel;
 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.Ignore;
-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 java.lang.management.ManagementFactory;
-import java.net.BindException;
-import java.net.InetAddress;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
 
 /**
- * Tests usage of LocatorLauncher as a local API in existing JVM.
+ * Integration tests for using {@link LocatorLauncher} as an in-process API within an existing JVM.
  *
  * @since GemFire 8.0
  */
 @Category(IntegrationTest.class)
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class LocatorLauncherLocalIntegrationTest
-    extends AbstractLocatorLauncherIntegrationTestCase {
+public class LocatorLauncherLocalIntegrationTest extends LocatorLauncherIntegrationTestCase {
 
   @Before
-  public final void setUpLocatorLauncherLocalIntegrationTest() throws Exception {
+  public void setUpLocatorLauncherLocalIntegrationTest() throws Exception {
     disconnectFromDS();
-    System.setProperty(ProcessType.TEST_PREFIX_PROPERTY, getUniqueName() + "-");
+    System.setProperty(ProcessType.PROPERTY_TEST_PREFIX, getUniqueName() + "-");
+    assertThat(new ProcessControllerFactory().isAttachAPIFound()).isTrue();
   }
 
   @After
-  public final void tearDownLocatorLauncherLocalIntegrationTest() throws Exception {
+  public void tearDownLocatorLauncherLocalIntegrationTest() throws Exception {
     disconnectFromDS();
   }
 
   @Test
-  public void testBuilderSetProperties() throws Throwable {
-    this.launcher = new Builder().setForce(true).setMemberName(getUniqueName())
-        .setPort(this.locatorPort).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory)
-        .set(DISABLE_AUTO_RECONNECT, "true").set(LOG_LEVEL, "config").set(MCAST_PORT, "0").build();
-
-    try {
-      assertEquals(Status.ONLINE, this.launcher.start().getStatus());
-      waitForLocatorToStart(this.launcher, true);
-
-      final InternalLocator locator = this.launcher.getLocator();
-      assertNotNull(locator);
-
-      final DistributedSystem distributedSystem = locator.getDistributedSystem();
-
-      assertNotNull(distributedSystem);
-      assertEquals("true", distributedSystem.getProperties().getProperty(DISABLE_AUTO_RECONNECT));
-      assertEquals("0", distributedSystem.getProperties().getProperty(MCAST_PORT));
-      assertEquals("config", distributedSystem.getProperties().getProperty(LOG_LEVEL));
-      assertEquals(getUniqueName(), distributedSystem.getProperties().getProperty(NAME));
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
-      assertNull(this.launcher.getLocator());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  public void usesLocatorPortAsDefaultPort() throws Exception {
+    launcher = givenLocatorLauncher();
+
+    assertThat(launcher.getPort()).isEqualTo(defaultLocatorPort);
   }
 
   @Test
-  public void testIsAttachAPIFound() throws Exception {
-    final ProcessControllerFactory factory = new ProcessControllerFactory();
-    assertTrue(factory.isAttachAPIFound());
+  public void startReturnsOnline() throws Exception {
+    launcher = givenLocatorLauncher();
+
+    assertThat(launcher.start().getStatus()).isEqualTo(ONLINE);
   }
 
   @Test
-  public void testStartCreatesPidFile() throws Throwable {
-    this.launcher = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(true).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config")
-        .build();
-
-    try {
-      this.launcher.start();
-      waitForLocatorToStart(this.launcher);
-      assertEquals(Status.ONLINE, this.launcher.status().getStatus());
-
-      // validate the pid file and its contents
-      this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-      assertTrue(this.pidFile.exists());
-      final int pid = readPid(this.pidFile);
-      assertTrue(pid > 0);
-      assertTrue(ProcessUtils.isProcessAlive(pid));
-      assertEquals(getPid(), pid);
-
-      assertEquals(Status.ONLINE, this.launcher.status().getStatus());
-
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
-      waitForFileToDelete(this.pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  public void startWithPortUsesPort() throws Exception {
+    LocatorLauncher launcher = startLocator(newBuilder().setPort(defaultLocatorPort));
+
+    assertThat(launcher.getLocator().getPort()).isEqualTo(defaultLocatorPort);
   }
 
   @Test
-  public void testStartDeletesStaleControlFiles() throws Throwable {
-    // create existing control files
-    this.stopRequestFile =
-        new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getStopRequestFileName());
-    this.stopRequestFile.createNewFile();
-    assertTrue(this.stopRequestFile.exists());
-
-    this.statusRequestFile =
-        new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getStatusRequestFileName());
-    this.statusRequestFile.createNewFile();
-    assertTrue(this.statusRequestFile.exists());
-
-    this.statusFile =
-        new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getStatusFileName());
-    this.statusFile.createNewFile();
-    assertTrue(this.statusFile.exists());
-
-    // build and start the locator
-    final Builder builder = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(true).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config");
-
-    assertFalse(builder.getForce());
-    this.launcher = builder.build();
-    assertFalse(this.launcher.isForcing());
-    this.launcher.start();
-
-    try {
-      waitForLocatorToStart(this.launcher);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      // validate the pid file and its contents
-      this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-      assertTrue(this.pidFile.exists());
-      final int pid = readPid(this.pidFile);
-      assertTrue(pid > 0);
-      assertTrue(ProcessUtils.isProcessAlive(pid));
-      assertEquals(getPid(), pid);
-
-      // validate stale control files were deleted
-      assertFalse(stopRequestFile.exists());
-      assertFalse(statusRequestFile.exists());
-      assertFalse(statusFile.exists());
-
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
-      waitForFileToDelete(this.pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  public void startWithPortZeroUsesAnEphemeralPort() throws Exception {
+    LocatorLauncher launcher = startLocator(newBuilder().setPort(0));
+
+    assertThat(launcher.getLocator().getPort()).isGreaterThan(0);
+    assertThat(launcher.getLocator().isPeerLocator()).isTrue();
   }
 
   @Test
-  public void testStartOverwritesStalePidFile() throws Throwable {
-    // create existing pid file
-    this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-    assertFalse("Integer.MAX_VALUE shouldn't be the same as local pid " + Integer.MAX_VALUE,
-        Integer.MAX_VALUE == ProcessUtils.identifyPid());
-    writePid(this.pidFile, Integer.MAX_VALUE);
-
-    // build and start the locator
-    final Builder builder = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(true).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config");
-
-    assertFalse(builder.getForce());
-    this.launcher = builder.build();
-    assertFalse(this.launcher.isForcing());
-    this.launcher.start();
-
-    try {
-      waitForLocatorToStart(this.launcher);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      // validate the pid file and its contents
-      assertTrue(this.pidFile.exists());
-      final int pid = readPid(this.pidFile);
-      assertTrue(pid > 0);
-      assertTrue(ProcessUtils.isProcessAlive(pid));
-      assertEquals(getPid(), pid);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
-      waitForFileToDelete(this.pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  public void startUsesBuilderValues() throws Exception {
+    LocatorLauncher launcher = startLocator(newBuilder().setPort(nonDefaultLocatorPort));
+
+    InternalLocator locator = launcher.getLocator();
+    assertThat(locator.getPort()).isEqualTo(nonDefaultLocatorPort);
+
+    DistributedSystem system = locator.getDistributedSystem();
+    assertThat(system.getProperties().getProperty(DISABLE_AUTO_RECONNECT)).isEqualTo("true");
+    assertThat(system.getProperties().getProperty(LOG_LEVEL)).isEqualTo("config");
+    assertThat(system.getProperties().getProperty(MCAST_PORT)).isEqualTo("0");
+    assertThat(system.getProperties().getProperty(NAME)).isEqualTo(getUniqueName());
   }
 
   @Test
-  @Ignore("Need to rewrite this without using dunit.Host")
-  public void testStartUsingForceOverwritesExistingPidFile() throws Throwable {}
-  /*
-   * assertTrue(getUniqueName() + " is broken if PID == Integer.MAX_VALUE",
-   * ProcessUtils.identifyPid() != Integer.MAX_VALUE);
-   * 
-   * // create existing pid file this.pidFile = new File(ProcessType.LOCATOR.getPidFileName());
-   * final int realPid = Host.getHost(0).getVM(3).invoke(() -> ProcessUtils.identifyPid());
-   * assertFalse(realPid == ProcessUtils.identifyPid()); writePid(this.pidFile, realPid);
-   * 
-   * // build and start the locator final Builder builder = new Builder() .setForce(true)
-   * .setMemberName(getUniqueName()) .setPort(this.locatorPort) .setRedirectOutput(true)
-   * 
-   * assertTrue(builder.getForce()); this.launcher = builder.build();
-   * assertTrue(this.launcher.isForcing()); this.launcher.start();
-   * 
-   * // collect and throw the FIRST failure Throwable failure = null;
-   * 
-   * try { waitForLocatorToStart(this.launcher);
-   * 
-   * // validate the pid file and its contents assertTrue(this.pidFile.exists()); final int pid =
-   * readPid(this.pidFile); assertTrue(pid > 0); assertTrue(ProcessUtils.isProcessAlive(pid));
-   * assertIndexDetailsEquals(getPid(), pid);
-   * 
-   * // validate log file was created final String logFileName = getUniqueName()+".log";
-   * assertTrue("Log file should exist: " + logFileName, new File(logFileName).exists());
-   * 
-   * } catch (Throwable e) { logger.error(e); if (failure == null) { failure = e; } }
-   * 
-   * try { assertIndexDetailsEquals(Status.STOPPED, this.launcher.stop().getStatus());
-   * waitForFileToDelete(this.pidFile); } catch (Throwable e) { logger.error(e); if (failure ==
-   * null) { failure = e; } }
-   * 
-   * if (failure != null) { throw failure; } } // testStartUsingForceOverwritesExistingPidFile
-   */
+  public void startCreatesPidFile() throws Exception {
+    startLocator();
+
+    assertThat(getPidFile()).exists();
+  }
 
   @Test
-  public void testStartWithDefaultPortInUseFails() throws Throwable {
-    // Test makes no sense in this case
-    if (this.locatorPort == 0) {
-      return;
-    }
-
-    this.socket =
-        SocketCreatorFactory.getSocketCreatorForComponent(SecurableCommunicationChannel.CLUSTER)
-            .createServerSocket(this.locatorPort, 50, null, -1);
-    assertTrue(this.socket.isBound());
-    assertFalse(this.socket.isClosed());
-    assertFalse(AvailablePort.isPortAvailable(this.locatorPort, AvailablePort.SOCKET));
-
-    assertNotNull(System.getProperty(DistributionLocator.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY));
-    assertEquals(this.locatorPort,
-        Integer.valueOf(System.getProperty(DistributionLocator.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY))
-            .intValue());
-    assertFalse(AvailablePort.isPortAvailable(this.locatorPort, AvailablePort.SOCKET));
-
-    this.launcher = new Builder().setMemberName(getUniqueName()).setRedirectOutput(true)
-        .setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config")
-        .build();
-
-    assertEquals(this.locatorPort, this.launcher.getPort().intValue());
-
-    RuntimeException expected = null;
-    try {
-      this.launcher.start();
-
-      // why did it not fail like it's supposed to?
-      final String property =
-          System.getProperty(DistributionLocator.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY);
-      assertNotNull(property);
-      assertEquals(this.locatorPort, Integer.valueOf(property).intValue());
-      assertFalse(AvailablePort.isPortAvailable(this.locatorPort, AvailablePort.SOCKET));
-      assertEquals(this.locatorPort, this.launcher.getPort().intValue());
-      assertEquals(this.locatorPort, this.socket.getLocalPort());
-      assertTrue(this.socket.isBound());
-      assertFalse(this.socket.isClosed());
-
-      fail("LocatorLauncher start should have thrown RuntimeException caused by BindException");
-    } catch (RuntimeException e) {
-      expected = e;
-      assertNotNull(expected.getMessage());
-      // BindException text varies by platform
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      assertNotNull(expected);
-      final Throwable cause = expected.getCause();
-      assertNotNull(cause);
-      assertTrue(cause instanceof BindException);
-      // BindException string varies by platform
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-      assertFalse("Pid file should not exist: " + this.pidFile, this.pidFile.exists());
-
-      // creation of log file seems to be random -- look into why sometime
-      final String logFileName = getUniqueName() + ".log";
-      assertFalse("Log file should not exist: " + logFileName,
-          new File(this.temporaryFolder.getRoot(), logFileName).exists());
-
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    // just in case the launcher started...
-    LocatorState status = null;
-    try {
-      status = this.launcher.stop();
-    } catch (Throwable t) {
-      // ignore
-    }
-
-    try {
-      waitForFileToDelete(this.pidFile);
-      assertEquals(getExpectedStopStatusForNotRunning(), status.getStatus());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  public void pidFileContainsServerPid() throws Exception {
+    startLocator();
+
+    assertThat(getLocatorPid()).isEqualTo(localPid);
   }
 
   @Test
-  @Ignore("Need to rewrite this without using dunit.Host")
-  public void testStartWithExistingPidFileFails()
-      throws Throwable {}/*
-                          * // create existing pid file final int realPid =
-                          * Host.getHost(0).getVM(3).invoke(() -> ProcessUtils.identifyPid());
-                          * assertFalse("Remote pid shouldn't be the same as local pid " + realPid,
-                          * realPid == ProcessUtils.identifyPid());
-                          * 
-                          * this.pidFile = new File(ProcessType.LOCATOR.getPidFileName());
-                          * writePid(this.pidFile, realPid);
-                          * 
-                          * // build and start the locator final Builder builder = new Builder()
-                          * .setMemberName(getUniqueName()) .setPort(this.locatorPort)
-                          * .setRedirectOutput(true) .set(logLevel, "config");
-                          * 
-                          * assertFalse(builder.getForce()); this.launcher = builder.build();
-                          * assertFalse(this.launcher.isForcing());
-                          * 
-                          * // collect and throw the FIRST failure Throwable failure = null;
-                          * RuntimeException expected = null;
-                          * 
-                          * try { this.launcher.start();
-                          * fail("LocatorLauncher start should have thrown RuntimeException caused by FileAlreadyExistsException"
-                          * ); } catch (RuntimeException e) { expected = e;
-                          * assertNotNull(expected.getMessage()); assertTrue(expected.getMessage(),
-                          * expected.getMessage().
-                          * contains("A PID file already exists and a Locator may be running in"));
-                          * assertIndexDetailsEquals(RuntimeException.class, expected.getClass()); }
-                          * catch (Throwable e) { logger.error(e); if (failure == null) { failure =
-                          * e; } }
-                          * 
-                          * // just in case the launcher started... LocatorState status = null; try
-                          * { status = this.launcher.stop(); } catch (Throwable t) { // ignore }
-                          * 
-                          * try { assertNotNull(expected); final Throwable cause =
-                          * expected.getCause(); assertNotNull(cause); assertTrue(cause instanceof
-                          * FileAlreadyExistsException);
-                          * assertTrue(cause.getMessage().contains("Pid file already exists: "));
-                          * assertTrue(cause.getMessage().contains("vf.gf.locator.pid for process "
-                          * + realPid)); } catch (Throwable e) { logger.error(e); if (failure ==
-                          * null) { failure = e; } }
-                          * 
-                          * try { delete(this.pidFile); final Status theStatus = status.getStatus();
-                          * assertFalse(theStatus == Status.STARTING); assertFalse(theStatus ==
-                          * Status.ONLINE); } catch (Throwable e) { logger.error(e); if (failure ==
-                          * null) { failure = e; } }
-                          * 
-                          * if (failure != null) { throw failure; } } //
-                          * testStartWithExistingPidFileFails
-                          */
+  public void startDeletesStaleControlFiles() throws Exception {
+    File stopRequestFile = givenControlFile(getProcessType().getStopRequestFileName());
+    File statusRequestFile = givenControlFile(getProcessType().getStatusRequestFileName());
+    File statusFile = givenControlFile(getProcessType().getStatusFileName());
+
+    startLocator();
+
+    assertDeletionOf(stopRequestFile);
+    assertDeletionOf(statusRequestFile);
+    assertDeletionOf(statusFile);
+  }
 
   @Test
-  public void testStartUsingPort() throws Throwable {
-    // generate one free port and then use it instead of default
-    final int freeTCPPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    assertTrue(AvailablePort.isPortAvailable(freeTCPPort, AvailablePort.SOCKET));
-
-    this.launcher = new Builder().setMemberName(getUniqueName()).setPort(freeTCPPort)
-        .setRedirectOutput(true).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config")
-        .build();
-
-    int pid = 0;
-    try {
-      // if start succeeds without throwing exception then #47778 is fixed
-      this.launcher.start();
-      waitForLocatorToStart(this.launcher);
-
-      // validate the pid file and its contents
-      this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-      assertTrue(pidFile.exists());
-      pid = readPid(pidFile);
-      assertTrue(pid > 0);
-      assertTrue(ProcessUtils.isProcessAlive(pid));
-      assertEquals(getPid(), pid);
-
-      // verify locator did not use default port
-      assertTrue(AvailablePort.isPortAvailable(this.locatorPort, AvailablePort.SOCKET));
-
-      final LocatorState status = this.launcher.status();
-      final String portString = status.getPort();
-      assertEquals("Port should be \"" + freeTCPPort + "\" instead of " + portString,
-          String.valueOf(freeTCPPort), portString);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    // stop the locator
-    try {
-      assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
-      waitForFileToDelete(this.pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  public void startOverwritesStalePidFile() throws Exception {
+    givenPidFile(fakePid);
+
+    startLocator();
+
+    assertThat(getLocatorPid()).isNotEqualTo(fakePid);
   }
 
   @Test
-  public void testStartUsingPortInUseFails() throws Throwable {
-    // Test makes no sense in this case
-    if (this.locatorPort == 0) {
-      return;
-    }
-
-    // generate one free port and then use it instead of default
-    this.socket =
-        SocketCreatorFactory.getSocketCreatorForComponent(SecurableCommunicationChannel.CLUSTER)
-            .createServerSocket(this.locatorPort, 50, null, -1);
-
-    this.launcher = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(true).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config")
-        .build();
-
-    RuntimeException expected = null;
-    try {
-      this.launcher.start();
-      fail("LocatorLauncher start should have thrown RuntimeException caused by BindException");
-    } catch (RuntimeException e) {
-      expected = e;
-      assertNotNull(expected.getMessage());
-      // BindException string varies by platform
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      assertNotNull(expected);
-      final Throwable cause = expected.getCause();
-      assertNotNull(cause);
-      assertTrue(cause instanceof BindException);
-      // BindException string varies by platform
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-      assertFalse("Pid file should not exist: " + this.pidFile, this.pidFile.exists());
-
-      // creation of log file seems to be random -- look into why sometime
-      final String logFileName = getUniqueName() + ".log";
-      assertFalse("Log file should not exist: " + logFileName,
-          new File(this.temporaryFolder.getRoot(), logFileName).exists());
-
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    // just in case the launcher started...
-    LocatorState status = null;
-    try {
-      status = this.launcher.stop();
-    } catch (Throwable t) {
-      // ignore
-    }
-
-    try {
-      waitForFileToDelete(this.pidFile);
-      assertEquals(getExpectedStopStatusForNotRunning(), status.getStatus());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  public void startWithDefaultPortInUseFailsWithBindException() throws Exception {
+    givenLocatorPortInUse(defaultLocatorPort);
+
+    launcher = new Builder().build();
+
+    assertThatThrownBy(() -> launcher.start()).isInstanceOf(RuntimeException.class)
+        .hasCauseInstanceOf(BindException.class);
+  }
+
+  @Test
+  public void startWithLocatorPortInUseFailsWithBindException() throws Exception {
+    givenServerPortInUse(nonDefaultLocatorPort);
+
+    launcher = new Builder().setPort(nonDefaultLocatorPort).build();
+
+    assertThatThrownBy(() -> this.launcher.start()).isInstanceOf(RuntimeException.class)
+        .hasCauseInstanceOf(BindException.class);
+  }
+
+  @Test
+  public void statusWithPidReturnsOnlineWithDetails() throws Exception {
+    givenRunningLocator();
+
+    LocatorState locatorState = new Builder().setPid(localPid).build().status();
+
+    assertThat(locatorState.getStatus()).isEqualTo(ONLINE);
+    assertThat(locatorState.getClasspath()).isEqualTo(getClassPath());
+    assertThat(locatorState.getGemFireVersion()).isEqualTo(GemFireVersion.getGemFireVersion());
+    assertThat(locatorState.getHost()).isEqualTo(InetAddress.getLocalHost().getCanonicalHostName());
+    assertThat(locatorState.getJavaVersion()).isEqualTo(System.getProperty("java.version"));
+    assertThat(locatorState.getJvmArguments()).isEqualTo(getJvmArguments());
+    assertThat(locatorState.getLogFile()).isEqualTo(getLogFilePath());
+    assertThat(locatorState.getMemberName()).isEqualTo(getUniqueName());
+    assertThat(locatorState.getPid().intValue()).isEqualTo(localPid);
+    assertThat(locatorState.getUptime()).isGreaterThan(0);
+    assertThat(locatorState.getWorkingDirectory()).isEqualTo(new File(".").getCanonicalPath());
+  }
+
+  @Test
+  public void statusWithWorkingDirectoryReturnsOnlineWithDetails() throws Exception {
+    givenRunningLocator();
+
+    LocatorState locatorState =
+        new Builder().setWorkingDirectory(getWorkingDirectoryPath()).build().status();
+
+    assertThat(locatorState.getStatus()).isEqualTo(ONLINE);
+    assertThat(locatorState.getClasspath()).isEqualTo(getClassPath());
+    assertThat(locatorState.getGemFireVersion()).isEqualTo(GemFireVersion.getGemFireVersion());
+    assertThat(locatorState.getHost()).isEqualTo(InetAddress.getLocalHost().getCanonicalHostName());
+    assertThat(locatorState.getJavaVersion()).isEqualTo(System.getProperty("java.version"));
+    assertThat(locatorState.getJvmArguments()).isEqualTo(getJvmArguments());
+    assertThat(locatorState.getLogFile()).isEqualTo(getLogFilePath());
+    assertThat(locatorState.getMemberName()).isEqualTo(getUniqueName());
+    assertThat(locatorState.getPid().intValue()).isEqualTo(readPidFile());
+    assertThat(locatorState.getUptime()).isGreaterThan(0);
+    assertThat(locatorState.getWorkingDirectory()).isEqualTo(new File(".").getCanonicalPath());
+  }
+
+  @Test
+  public void statusWithEmptyPidFileThrowsIllegalArgumentException() throws Exception {
+    givenEmptyPidFile();
+
+    LocatorLauncher launcher = new Builder().setWorkingDirectory(getWorkingDirectoryPath()).build();
+
+    assertThatThrownBy(() -> launcher.status()).isInstanceOf(IllegalArgumentException.class)
+        .hasMessageContaining("Invalid pid 'null' found in");
   }
 
   @Test
-  public void testStatusUsingPid() throws Throwable {
-    // build and start the locator
-    final Builder builder = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(true).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config");
-
-    assertFalse(builder.getForce());
-    this.launcher = builder.build();
-    assertFalse(this.launcher.isForcing());
-
-    LocatorLauncher pidLauncher = null;
-    try {
-      this.launcher.start();
-      waitForLocatorToStart(this.launcher);
-
-      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);
-
-      pidLauncher = new Builder().setPid(pid).build();
-      assertNotNull(pidLauncher);
-      assertFalse(pidLauncher.isRunning());
-
-      final LocatorState actualStatus = pidLauncher.status();
-      assertNotNull(actualStatus);
-      assertEquals(Status.ONLINE, actualStatus.getStatus());
-      assertEquals(pid, actualStatus.getPid().intValue());
-      assertTrue(actualStatus.getUptime() > 0);
-      // getWorkingDirectory returns user.dir instead of rootFolder because test is starting Locator
-      // in this process (to move logFile and pidFile into temp dir)
-      assertEquals(ManagementFactory.getRuntimeMXBean().getClassPath(),
-          actualStatus.getClasspath());
-      assertEquals(GemFireVersion.getGemFireVersion(), actualStatus.getGemFireVersion());
-      assertEquals(System.getProperty("java.version"), actualStatus.getJavaVersion());
-      assertEquals(this.workingDirectory + File.separator + getUniqueName() + ".log",
-          actualStatus.getLogFile());
-      assertEquals(InetAddress.getLocalHost().getCanonicalHostName(), actualStatus.getHost());
-      assertEquals(getUniqueName(), actualStatus.getMemberName());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    if (pidLauncher == null) {
-      try {
-        assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
-        waitForFileToDelete(this.pidFile);
-      } catch (Throwable e) {
-        this.errorCollector.addError(e);
-      }
-
-    } else {
-      try {
-        assertEquals(Status.STOPPED, pidLauncher.stop().getStatus());
-        waitForFileToDelete(this.pidFile);
-      } catch (Throwable e) {
-        this.errorCollector.addError(e);
-      }
-    }
+  public void statusWithEmptyWorkingDirectoryReturnsNotRespondingWithDetails() throws Exception {
+    givenEmptyWorkingDirectory();
+
+    LocatorState locatorState =
+        new Builder().setWorkingDirectory(getWorkingDirectoryPath()).build().status();
+
+    assertThat(locatorState.getStatus()).isEqualTo(NOT_RESPONDING);
+    assertThat(locatorState.getClasspath()).isNull();
+    assertThat(locatorState.getGemFireVersion()).isEqualTo(GemFireVersion.getGemFireVersion());
+    assertThat(locatorState.getHost()).isNull();
+    assertThat(locatorState.getJavaVersion()).isNull();
+    assertThat(locatorState.getJvmArguments()).isEqualTo(Collections.emptyList());
+    assertThat(locatorState.getLogFile()).isNull();
+    assertThat(locatorState.getMemberName()).isNull();
+    assertThat(locatorState.getPid()).isNull();
+    assertThat(locatorState.getUptime().intValue()).isEqualTo(0);
+    assertThat(locatorState.getWorkingDirectory()).isEqualTo(getWorkingDirectoryPath());
   }
 
+  /**
+   * This test takes > 1 minute to run in {@link LocatorLauncherLocalFileIntegrationTest}.
+   */
   @Test
-  public void testStatusUsingWorkingDirectory() throws Throwable {
-    final Builder builder = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(true).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config");
-
-    assertFalse(builder.getForce());
-    this.launcher = builder.build();
-    assertFalse(this.launcher.isForcing());
-
-    LocatorLauncher dirLauncher = null;
-    try {
-      this.launcher.start();
-      waitForLocatorToStart(this.launcher);
-
-      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(this.workingDirectory).build();
-      assertNotNull(dirLauncher);
-      assertFalse(dirLauncher.isRunning());
-
-      final LocatorState actualStatus = dirLauncher.status();
-      assertNotNull(actualStatus);
-      assertEquals(Status.ONLINE, actualStatus.getStatus());
-      assertEquals(pid, actualStatus.getPid().intValue());
-      assertTrue(actualStatus.getUptime() > 0);
-      // getWorkingDirectory returns user.dir instead of rootFolder because test is starting Locator
-      // in this process (to move logFile and pidFile into temp dir)
-      assertEquals(ManagementFactory.getRuntimeMXBean().getClassPath(),
-          actualStatus.getClasspath());
-      assertEquals(GemFireVersion.getGemFireVersion(), actualStatus.getGemFireVersion());
-      assertEquals(System.getProperty("java.version"), actualStatus.getJavaVersion());
-      assertEquals(this.workingDirectory + File.separator + getUniqueName() + ".log",
-          actualStatus.getLogFile());
-      assertEquals(InetAddress.getLocalHost().getCanonicalHostName(), actualStatus.getHost());
-      assertEquals(getUniqueName(), actualStatus.getMemberName());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    if (dirLauncher == null) {
-      try {
-        assertEquals(Status.STOPPED, this.launcher.stop().getStatus());
-        waitForFileToDelete(this.pidFile);
-      } catch (Throwable e) {
-        this.errorCollector.addError(e);
-      }
-
-    } else {
-      try {
-        assertEquals(Status.STOPPED, dirLauncher.stop().getStatus());
-        waitForFileToDelete(this.pidFile);
-      } catch (Throwable e) {
-        this.errorCollector.addError(e);
-      }
-    }
+  public void statusWithStalePidFileReturnsNotResponding() throws Exception {
+    givenPidFile(fakePid);
+
+    LocatorState locatorState =
+        new Builder().setWorkingDirectory(getWorkingDirectoryPath()).build().status();
+
+    assertThat(locatorState.getStatus()).isEqualTo(NOT_RESPONDING);
   }
 
   @Test
-  public void testStopUsingPid() throws Throwable {
-    final Builder builder = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(true).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config");
-
-    assertFalse(builder.getForce());
-    this.launcher = builder.build();
-    assertFalse(this.launcher.isForcing());
-
-    LocatorLauncher pidLauncher = null;
-    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(this.pidFile.exists());
-      final int pid = readPid(this.pidFile);
-      assertTrue(pid > 0);
-      assertEquals(ProcessUtils.identifyPid(), pid);
-
-      pidLauncher = new Builder().setPid(pid).build();
-      assertNotNull(pidLauncher);
-      assertFalse(pidLauncher.isRunning());
-
-      // stop the locator
-      final LocatorState locatorState = pidLauncher.stop();
-      assertNotNull(locatorState);
-      assertEquals(Status.STOPPED, locatorState.getStatus());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      this.launcher.stop();
-    } catch (Throwable e) {
-      // ignore
-    }
-
-    try {
-      // verify the PID file was deleted
-      waitForFileToDelete(this.pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  public void stopWithPidReturnsStopped() throws Exception {
+    givenRunningLocator();
+
+    LocatorState locatorState = new Builder().setPid(localPid).build().stop();
+
+    assertThat(locatorState.getStatus()).isEqualTo(STOPPED);
   }
 
   @Test
-  public void testStopUsingWorkingDirectory() throws Throwable {
-    final Builder builder = new Builder().setMemberName(getUniqueName()).setPort(this.locatorPort)
-        .setRedirectOutput(true).setWorkingDirectory(this.workingDirectory)
-        .set(CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory).set(LOG_LEVEL, "config");
-
-    assertFalse(builder.getForce());
-    this.launcher = builder.build();
-    assertFalse(this.launcher.isForcing());
-
-    LocatorLauncher dirLauncher = null;
-    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(this.workingDirectory).build();
-      assertNotNull(dirLauncher);
-      assertFalse(dirLauncher.isRunning());
-
-      // stop the locator
-      final LocatorState locatorState = dirLauncher.stop();
-      assertNotNull(locatorState);
-      assertEquals(Status.STOPPED, locatorState.getStatus());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    try {
-      this.launcher.stop();
-    } catch (Throwable e) {
-      // ignore
-    }
-
-    try {
-      // verify the PID file was deleted
-      waitForFileToDelete(this.pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  public void stopWithPidDeletesPidFile() throws Exception {
+    givenRunningLocator(newBuilder().setDeletePidFileOnStop(true));
+
+    new Builder().setPid(localPid).build().stop();
+
+    assertDeletionOf(getPidFile());
+  }
+
+  @Test
+  public void stopWithWorkingDirectoryReturnsStopped() throws Exception {
+    givenRunningLocator();
+
+    LocatorState locatorState =
+        new Builder().setWorkingDirectory(getWorkingDirectoryPath()).build().stop();
+
+    assertThat(locatorState.getStatus()).isEqualTo(STOPPED);
+  }
+
+  @Test
+  public void stopWithWorkingDirectoryDeletesPidFile() throws Exception {
+    givenRunningLocator(newBuilder().setDeletePidFileOnStop(true));
+
+    new Builder().setWorkingDirectory(getWorkingDirectoryPath()).build().stop();
+
+    assertDeletionOf(getPidFile());
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/894f3ee7/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteFileIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteFileIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteFileIntegrationTest.java
index 9e86c8c..6bfba2d 100755
--- a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteFileIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteFileIntegrationTest.java
@@ -14,214 +14,69 @@
  */
 package org.apache.geode.distributed;
 
-import static org.junit.Assert.*;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
-
-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.apache.geode.distributed.AbstractLauncher.Status;
 import org.apache.geode.distributed.LocatorLauncher.Builder;
 import org.apache.geode.internal.process.ProcessControllerFactory;
-import org.apache.geode.internal.process.ProcessStreamReader;
-import org.apache.geode.internal.process.ProcessType;
-import org.apache.geode.internal.process.ProcessUtils;
 import org.apache.geode.lang.AttachAPINotFoundException;
 import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 /**
- * Subclass of LocatorLauncherRemoteDUnitTest which forces the code to not find the Attach API which
- * is in the JDK tools.jar. As a result LocatorLauncher ends up using the FileProcessController
- * implementation.
- * 
+ * Integration tests for using {@code LocatorLauncher} as an application main in a forked JVM
+ * without the Attach API.
+ *
+ * Sets {@link ProcessControllerFactory#PROPERTY_DISABLE_ATTACH_API} to force
+ * {@code LocatorLauncher} to use the FileProcessController implementation.
+ *
  * @since GemFire 8.0
  */
 @Category(IntegrationTest.class)
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class LocatorLauncherRemoteFileIntegrationTest extends LocatorLauncherRemoteIntegrationTest {
 
   @Before
-  public final void setUpLocatorLauncherRemoteFileIntegrationTest() throws Exception {
+  public void setUpLocatorLauncherRemoteFileIntegrationTest() throws Exception {
     System.setProperty(ProcessControllerFactory.PROPERTY_DISABLE_ATTACH_API, "true");
+    assertThat(new ProcessControllerFactory().isAttachAPIFound()).isFalse();
   }
 
-  @After
-  public final void tearDownLocatorLauncherRemoteFileIntegrationTest() throws Exception {}
-
-  /**
-   * Override and assert Attach API is NOT found
-   */
-  @Override
   @Test
-  public void testIsAttachAPIFound() throws Exception {
-    final ProcessControllerFactory factory = new ProcessControllerFactory();
-    assertFalse(factory.isAttachAPIFound());
-  }
-
-  /**
-   * Override because FileProcessController cannot request status with PID
-   */
   @Override
-  @Test
-  public void testStatusUsingPid() throws Throwable {
-    final List<String> jvmArguments = getJvmArguments();
-
-    final List<String> command = new ArrayList<String>();
-    command
-        .add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
-    for (String jvmArgument : jvmArguments) {
-      command.add(jvmArgument);
-    }
-    command.add("-cp");
-    command.add(System.getProperty("java.class.path"));
-    command.add(LocatorLauncher.class.getName());
-    command.add(LocatorLauncher.Command.START.getName());
-    command.add(getUniqueName());
-    command.add("--port=" + this.locatorPort);
-    command.add("--redirect-output");
-
-    this.process = new ProcessBuilder(command).directory(this.temporaryFolder.getRoot()).start();
-    this.processOutReader = new ProcessStreamReader.Builder(this.process)
-        .inputStream(this.process.getInputStream()).build().start();
-    this.processErrReader = new ProcessStreamReader.Builder(this.process)
-        .inputStream(this.process.getErrorStream()).build().start();
-
-    // wait for locator to start
-    int pid = 0;
-    LocatorLauncher pidLauncher = null;
-    final LocatorLauncher dirLauncher = new LocatorLauncher.Builder()
-        .setWorkingDirectory(this.temporaryFolder.getRoot().getCanonicalPath()).build();
-    try {
-      waitForLocatorToStart(dirLauncher);
-
-      // validate the pid file and its contents
-      final File pidFile =
-          new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-      assertTrue(pidFile.exists());
-      pid = readPid(pidFile);
-      assertTrue(pid > 0);
-      assertTrue(ProcessUtils.isProcessAlive(pid));
-
-      // validate log file was created
-      final String logFileName = getUniqueName() + ".log";
-      assertTrue("Log file should exist: " + logFileName,
-          new File(this.temporaryFolder.getRoot(), logFileName).exists());
-
-      // use launcher with pid
-      pidLauncher = new Builder().setPid(pid).build();
-
-      assertNotNull(pidLauncher);
-      assertFalse(pidLauncher.isRunning());
+  public void statusWithPidReturnsOnlineWithDetails() throws Exception {
+    givenRunningLocator();
 
-      // status with pid only should throw AttachAPINotFoundException
-      try {
-        pidLauncher.status();
-        fail("FileProcessController should have thrown AttachAPINotFoundException");
-      } catch (AttachAPINotFoundException e) {
-        // passed
-      }
-
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
-
-    // stop the locator
-    try {
-      assertEquals(Status.STOPPED, dirLauncher.stop().getStatus());
-      waitForPidToStop(pid, true);
-      waitForFileToDelete(this.pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+    assertThatThrownBy(() -> new Builder().setPid(getLocatorPid()).build().status())
+        .isInstanceOf(AttachAPINotFoundException.class);
   }
 
-  /**
-   * Override because FileProcessController cannot request stop with PID
-   */
-  @Override
   @Test
-  public void testStopUsingPid() throws Throwable {
-    final List<String> jvmArguments = getJvmArguments();
-
-    final List<String> command = new ArrayList<String>();
-    command
-        .add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
-    for (String jvmArgument : jvmArguments) {
-      command.add(jvmArgument);
-    }
-    command.add("-cp");
-    command.add(System.getProperty("java.class.path"));
-    command.add(LocatorLauncher.class.getName());
-    command.add(LocatorLauncher.Command.START.getName());
-    command.add(getUniqueName());
-    command.add("--port=" + this.locatorPort);
-    command.add("--redirect-output");
-
-    this.process = new ProcessBuilder(command).directory(this.temporaryFolder.getRoot()).start();
-    this.processOutReader =
-        new ProcessStreamReader.Builder(this.process).inputStream(this.process.getInputStream())
-            .inputListener(createLoggingListener("sysout", getUniqueName() + "#sysout")).build()
-            .start();
-    this.processErrReader =
-        new ProcessStreamReader.Builder(this.process).inputStream(this.process.getErrorStream())
-            .inputListener(createLoggingListener("syserr", getUniqueName() + "#syserr")).build()
-            .start();
-
-    // wait for locator to start
-    int pid = 0;
-    File pidFile = null;
-    LocatorLauncher pidLauncher = null;
-    final LocatorLauncher dirLauncher = new LocatorLauncher.Builder()
-        .setWorkingDirectory(this.temporaryFolder.getRoot().getCanonicalPath()).build();
-    try {
-      waitForLocatorToStart(dirLauncher);
-
-      // validate the pid file and its contents
-      pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
-      assertTrue(pidFile.exists());
-      pid = readPid(pidFile);
-      assertTrue(pid > 0);
-      assertTrue(ProcessUtils.isProcessAlive(pid));
-
-      // validate log file was created
-      final String logFileName = getUniqueName() + ".log";
-      assertTrue("Log file should exist: " + logFileName,
-          new File(this.temporaryFolder.getRoot(), logFileName).exists());
+  @Override
+  public void stopWithPidDeletesPidFile() throws Exception {
+    givenRunningLocator();
 
-      // use launcher with pid
-      pidLauncher = new Builder().setPid(pid).build();
+    assertThatThrownBy(() -> new Builder().setPid(getLocatorPid()).build().stop())
+        .isInstanceOf(AttachAPINotFoundException.class);
+  }
 
-      assertNotNull(pidLauncher);
-      assertFalse(pidLauncher.isRunning());
+  @Test
+  @Override
+  public void stopWithPidReturnsStopped() throws Exception {
+    givenRunningLocator();
 
-      // stop with pid only should throw AttachAPINotFoundException
-      try {
-        pidLauncher.stop();
-        fail("FileProcessController should have thrown AttachAPINotFoundException");
-      } catch (AttachAPINotFoundException e) {
-        // passed
-      }
+    assertThatThrownBy(() -> new Builder().setPid(getLocatorPid()).build().stop())
+        .isInstanceOf(AttachAPINotFoundException.class);
+  }
 
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+  @Test
+  @Override
+  public void stopWithPidStopsLocatorProcess() throws Exception {
+    givenRunningLocator();
 
-    try {
-      // stop the locator
-      assertEquals(Status.STOPPED, dirLauncher.stop().getStatus());
-      waitForPidToStop(pid);
-      waitForFileToDelete(pidFile);
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+    assertThatThrownBy(() -> new Builder().setPid(getLocatorPid()).build().stop())
+        .isInstanceOf(AttachAPINotFoundException.class);
   }
 }