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

geode git commit: GEODE-3277: Fix error path constructors of inner State classes of the Launchers

Repository: geode
Updated Branches:
  refs/heads/develop b775c8451 -> 6fb2ca83f


GEODE-3277: Fix error path constructors of inner State classes of the Launchers

Updated tests for changes in the error constructors for ServerState and
LocatorState.

Minor refactorings and spelling corrections to improve code clarity.


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

Branch: refs/heads/develop
Commit: 6fb2ca83f53bef0d9fa32e14c058280e894b8ac2
Parents: b775c84
Author: Ken Howe <kh...@pivotal.io>
Authored: Tue Jul 25 08:26:28 2017 -0700
Committer: Ken Howe <kh...@pivotal.io>
Committed: Tue Aug 8 14:39:40 2017 -0700

----------------------------------------------------------------------
 .../cli/commands/StatusLocatorRealGfshTest.java | 45 +++++++++-
 .../geode/distributed/LocatorLauncher.java      | 66 +++------------
 .../geode/distributed/ServerLauncher.java       |  6 +-
 .../lifecycle/StatusLocatorCommand.java         |  5 +-
 .../commands/lifecycle/StatusServerCommand.java |  2 +-
 .../LocatorLauncherRemoteIntegrationTest.java   | 65 +++++++--------
 .../GfshStatusCommandsIntegrationTest.java      | 88 ++++++++++++++++++++
 .../dunit/rules/GfshShellConnectionRule.java    |  4 +-
 8 files changed, 181 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/6fb2ca83/geode-assembly/src/test/java/org/apache/geode/management/internal/cli/commands/StatusLocatorRealGfshTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/management/internal/cli/commands/StatusLocatorRealGfshTest.java b/geode-assembly/src/test/java/org/apache/geode/management/internal/cli/commands/StatusLocatorRealGfshTest.java
index 3a98373..a6d3a6f 100644
--- a/geode-assembly/src/test/java/org/apache/geode/management/internal/cli/commands/StatusLocatorRealGfshTest.java
+++ b/geode-assembly/src/test/java/org/apache/geode/management/internal/cli/commands/StatusLocatorRealGfshTest.java
@@ -14,30 +14,67 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import static org.assertj.core.api.Assertions.assertThat;
+
+import static org.apache.geode.internal.AvailablePort.SOCKET;
+import static org.apache.geode.internal.AvailablePort.getRandomAvailablePort;
+
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 
+import org.apache.geode.test.dunit.rules.gfsh.GfshExecution;
 import org.apache.geode.test.dunit.rules.gfsh.GfshRule;
 import org.apache.geode.test.dunit.rules.gfsh.GfshScript;
 import org.apache.geode.test.junit.categories.AcceptanceTest;
+import java.util.concurrent.TimeUnit;
 
 @Category(AcceptanceTest.class)
 public class StatusLocatorRealGfshTest {
+  private int port;
+
+  @Rule
+  public TestName testName = new TestName();
+
   @Rule
   public GfshRule gfshRule = new GfshRule();
 
+  @Before
+  public void setup() {
+    port = getRandomAvailablePort(SOCKET);
+  }
+
   @Test
   public void statusLocatorSucceedsWhenConnected() throws Exception {
-    GfshScript.of("start locator --name=locator1").execute(gfshRule);
+    GfshScript.of("start locator --name=locator1 --port=" + Integer.valueOf(port))
+        .execute(gfshRule);
 
-    GfshScript.of("connect", "status locator --name=locator1").execute(gfshRule);
+    GfshScript.of("connect --locator=192.168.1.66[" + Integer.valueOf(port) + "]",
+        "status locator --name=locator1").execute(gfshRule);
   }
 
   @Test
-  public void statusLocatorFailsWhenNotConnected() throws Exception {
-    GfshScript.of("start locator --name=locator1").withName("start locator").execute(gfshRule);
+  public void multilineStatusLocatorByNameWhenNotConnected() throws Exception {
+    CommandStringBuilder csb =
+        new CommandStringBuilder("status locator").addNewLine().addOption("name", "locator1");
+
+    GfshScript.of("start locator --name=locator1 --port=" + Integer.valueOf(port))
+        .awaitAtMost(1, TimeUnit.MINUTES).execute(gfshRule);
+
+    GfshScript.of(csb.toString()).awaitAtMost(1, TimeUnit.MINUTES).expectFailure()
+        .execute(gfshRule);
 
+  }
+
+  @Test
+  public void statusLocatorByNameFailsWhenNotConnected() throws Exception {
+    GfshExecution gfshExecution =
+        GfshScript.of("start locator --name=locator1 --port=" + Integer.valueOf(port))
+            .withName("start locator").execute(gfshRule);
     GfshScript.of("status locator --name=locator1").withName("status locator").expectFailure()
         .execute(gfshRule);
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/6fb2ca83/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..9400088 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
@@ -55,7 +55,6 @@ 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;
@@ -684,8 +683,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
 
   @Override
   protected Properties getDistributedSystemProperties() {
-    Properties properties = super.getDistributedSystemProperties(getProperties());
-    return properties;
+    return super.getDistributedSystemProperties(getProperties());
   }
 
   /**
@@ -742,9 +740,6 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       Thread.currentThread().interrupt();
       t = e;
       debug(e);
-    } catch (RuntimeException e) {
-      t = e;
-      throw e;
     } catch (Throwable e) {
       t = e;
       throw e;
@@ -895,24 +890,15 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     } catch (ConnectionFailedException e) {
       // failed to attach to locator JVM
       return createNoResponseState(e, "Failed to connect to locator with process id " + getPid());
-    } catch (IOException e) {
+    } catch (IOException | MBeanInvocationFailedException | UnableToControlProcessException
+        | TimeoutException 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,
-          "Failed to communicate with locator with process id " + getPid());
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       return createNoResponseState(e,
           "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());
     }
   }
 
@@ -951,28 +937,19 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       // could not find pid file
       return createNoResponseState(e, "Failed to find process file "
           + ProcessType.LOCATOR.getPidFileName() + " in " + getWorkingDirectory());
-    } catch (IOException e) {
+    } catch (IOException | MBeanInvocationFailedException | UnableToControlProcessException
+        | TimeoutException 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,
-          "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) {
       Thread.currentThread().interrupt();
       return createNoResponseState(e,
           "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);
     }
   }
 
@@ -1056,17 +1033,10 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     } catch (ConnectionFailedException e) {
       // failed to attach to locator JVM
       return createNoResponseState(e, "Failed to connect to locator with process id " + getPid());
-    } catch (IOException e) {
+    } catch (IOException | MBeanInvocationFailedException | UnableToControlProcessException 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,
-          "Failed to communicate with locator with process id " + getPid());
     }
   }
 
@@ -1096,7 +1066,7 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       // could not find pid file
       return createNoResponseState(e, "Failed to find process file "
           + ProcessType.LOCATOR.getPidFileName() + " in " + getWorkingDirectory());
-    } catch (IOException e) {
+    } catch (IOException | MBeanInvocationFailedException | UnableToControlProcessException e) {
       // failed to open or read file or dir
       return createNoResponseState(e,
           "Failed to communicate with locator with process id " + parsedPid);
@@ -1104,10 +1074,6 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       Thread.currentThread().interrupt();
       return createNoResponseState(e,
           "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) {
       // couldn't determine pid from within locator JVM
       return createNoResponseState(e, "Failed to find usable process id within file "
@@ -1115,15 +1081,11 @@ public class LocatorLauncher extends AbstractLauncher<String> {
     } catch (TimeoutException e) {
       return createNoResponseState(e, "Timed out trying 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);
     }
   }
 
   private LocatorState createNoResponseState(final Exception cause, final String errorMessage) {
     debug(cause);
-    // info(errorMessage);
     return new LocatorState(this, Status.NOT_RESPONDING, errorMessage);
   }
 
@@ -1167,9 +1129,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 e) {
         return null;
       }
     }
@@ -2014,17 +1974,17 @@ public class LocatorLauncher extends AbstractLauncher<String> {
       this(status, // status
           errorMessage, // statusMessage
           System.currentTimeMillis(), // timestamp
-          null, // locatorLocation
+          launcher.getId(), // locatorLocation
           null, // pid
           0L, // uptime
           launcher.getWorkingDirectory(), // workingDirectory
-          Collections.<String>emptyList(), // jvmArguments
+          ManagementFactory.getRuntimeMXBean().getInputArguments(), // jvmArguments
           null, // classpath
           GemFireVersion.getGemFireVersion(), // gemfireVersion
-          null, // javaVersion
+          System.getProperty("java.version"), // javaVersion
           null, // logFile
-          null, // host
-          null, // port
+          launcher.getBindAddressAsString(), // host
+          launcher.getPortAsString(), // port
           null);// memberName
     }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/6fb2ca83/geode-core/src/main/java/org/apache/geode/distributed/ServerLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/ServerLauncher.java b/geode-core/src/main/java/org/apache/geode/distributed/ServerLauncher.java
index 158e7bf..4f96942 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/ServerLauncher.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/ServerLauncher.java
@@ -2644,7 +2644,7 @@ public class ServerLauncher extends AbstractLauncher<String> {
       this(status, // status
           errorMessage, // statusMessage
           System.currentTimeMillis(), // timestamp
-          null, // serverLocation
+          launcher.getId(), // serverLocation
           null, // pid
           0L, // uptime
           launcher.getWorkingDirectory(), // workingDirectory
@@ -2653,8 +2653,8 @@ public class ServerLauncher extends AbstractLauncher<String> {
           GemFireVersion.getGemFireVersion(), // gemfireVersion
           null, // javaVersion
           null, // logFile
-          null, // host
-          null, // port
+          launcher.getServerBindAddressAsString(), // host
+          launcher.getServerPortAsString(), // port
           null);// memberName
     }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/6fb2ca83/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusLocatorCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusLocatorCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusLocatorCommand.java
index 06f8350..f2281e9 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusLocatorCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusLocatorCommand.java
@@ -26,7 +26,6 @@ import org.springframework.shell.core.annotation.CliOption;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.distributed.AbstractLauncher;
 import org.apache.geode.distributed.LocatorLauncher;
-import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.internal.lang.SystemUtils;
 import org.apache.geode.management.MemberMXBean;
 import org.apache.geode.management.cli.CliMetaData;
@@ -38,6 +37,8 @@ import org.apache.geode.management.internal.cli.result.InfoResultData;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.configuration.utils.ClusterConfigurationStatusRetriever;
 
+import org.apache.commons.lang.StringUtils;
+
 public class StatusLocatorCommand implements GfshCommand {
   @CliCommand(value = CliStrings.STATUS_LOCATOR, help = CliStrings.STATUS_LOCATOR__HELP)
   @CliMetaData(shellOnly = true,
@@ -85,7 +86,7 @@ public class StatusLocatorCommand implements GfshCommand {
         return createStatusLocatorResult(status);
       }
     } catch (IllegalArgumentException | IllegalStateException e) {
-      return ResultBuilder.createUserErrorResult(e.getMessage());
+      return ResultBuilder.createUserErrorResult(e.toString());
     } catch (VirtualMachineError e) {
       SystemFailure.initiateFailure(e);
       throw e;

http://git-wip-us.apache.org/repos/asf/geode/blob/6fb2ca83/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusServerCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusServerCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusServerCommand.java
index 43374ab..6f859cf 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusServerCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/lifecycle/StatusServerCommand.java
@@ -80,7 +80,7 @@ public class StatusServerCommand implements GfshCommand {
       }
     } catch (IllegalArgumentException | IllegalStateException e) {
 
-      return ResultBuilder.createUserErrorResult(e.getMessage());
+      return ResultBuilder.createUserErrorResult(e.toString());
     } catch (VirtualMachineError e) {
       SystemFailure.initiateFailure(e);
       throw e;

http://git-wip-us.apache.org/repos/asf/geode/blob/6fb2ca83/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteIntegrationTest.java
index 47e512a..989b10e 100755
--- a/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/LocatorLauncherRemoteIntegrationTest.java
@@ -30,7 +30,6 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
 import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -458,18 +457,7 @@ public class LocatorLauncherRemoteIntegrationTest
       this.errorCollector.addError(e);
     }
 
-    try {
-      // check the status
-      final LocatorState locatorState = dirLauncher.status();
-      assertNotNull(locatorState);
-      assertEquals(Status.NOT_RESPONDING, locatorState.getStatus());
-
-      final String logFileName = getUniqueName() + ".log";
-      assertFalse("Log file should exist: " + logFileName,
-          new File(this.temporaryFolder.getRoot(), logFileName).exists());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+    checkLocatorStatus(dirLauncher);
 
     // if the following fails, then the SHORTER_TIMEOUT is too short for slow machines
     // or this test needs to use MainLauncher in ProcessWrapper
@@ -489,6 +477,21 @@ public class LocatorLauncherRemoteIntegrationTest
         is(equalTo(getExpectedStopStatusForNotRunning())));
   }
 
+  private void checkLocatorStatus(LocatorLauncher dirLauncher) {
+    try {
+      // check the status
+      final LocatorState locatorState = dirLauncher.status();
+      assertNotNull(locatorState);
+      assertEquals(Status.NOT_RESPONDING, locatorState.getStatus());
+
+      final String logFileName = getUniqueName() + ".log";
+      assertFalse("Log file should exist: " + logFileName,
+          new File(this.temporaryFolder.getRoot(), logFileName).exists());
+    } catch (Throwable e) {
+      this.errorCollector.addError(e);
+    }
+  }
+
   @Test
   public void testStartWithDefaultPortInUseFails() throws Throwable {
     String expectedString = "java.net.BindException";
@@ -541,19 +544,7 @@ public class LocatorLauncherRemoteIntegrationTest
       this.errorCollector.addError(e);
     }
 
-    try {
-      // check the status
-      final LocatorState locatorState = dirLauncher.status();
-      assertNotNull(locatorState);
-      assertEquals(Status.NOT_RESPONDING, locatorState.getStatus());
-
-      // creation of log file seems to be random -- look into why sometime
-      final String logFileName = getUniqueName() + ".log";
-      assertFalse("Log file should exist: " + logFileName,
-          new File(this.temporaryFolder.getRoot(), logFileName).exists());
-    } catch (Throwable e) {
-      this.errorCollector.addError(e);
-    }
+    checkLocatorStatus(dirLauncher);
 
     // if the following fails, then the SHORTER_TIMEOUT might be too short for slow machines
     // or this test needs to use MainLauncher in ProcessWrapper
@@ -644,7 +635,7 @@ public class LocatorLauncherRemoteIntegrationTest
   public void testStatusUsingPid() throws Throwable {
     final List<String> jvmArguments = getJvmArguments();
 
-    final List<String> command = new ArrayList<String>();
+    final List<String> command = new ArrayList<>();
     command
         .add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
     for (String jvmArgument : jvmArguments) {
@@ -705,7 +696,9 @@ public class LocatorLauncherRemoteIntegrationTest
       assertEquals(System.getProperty("java.version"), actualStatus.getJavaVersion());
       assertEquals(this.temporaryFolder.getRoot().getCanonicalPath() + File.separator
           + getUniqueName() + ".log", actualStatus.getLogFile());
-      assertEquals(InetAddress.getLocalHost().getCanonicalHostName(), actualStatus.getHost());
+      // Deleting the assertion on host. Assertions using InetAddress.getLocalHost can be flaky
+      // because some systems return "localhost" instead of the localhost's IP address as a string.
+      // assertEquals(InetAddress.getLocalHost().getCanonicalHostName(), actualStatus.getHost());
       assertEquals(getUniqueName(), actualStatus.getMemberName());
     } catch (Throwable e) {
       this.errorCollector.addError(e);
@@ -731,7 +724,7 @@ public class LocatorLauncherRemoteIntegrationTest
   public void testStatusUsingWorkingDirectory() throws Throwable {
     final List<String> jvmArguments = getJvmArguments();
 
-    final List<String> command = new ArrayList<String>();
+    final List<String> command = new ArrayList<>();
     command
         .add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
     for (String jvmArgument : jvmArguments) {
@@ -788,7 +781,9 @@ public class LocatorLauncherRemoteIntegrationTest
       assertEquals(System.getProperty("java.version"), actualStatus.getJavaVersion());
       assertEquals(this.temporaryFolder.getRoot().getCanonicalPath() + File.separator
           + getUniqueName() + ".log", actualStatus.getLogFile());
-      assertEquals(InetAddress.getLocalHost().getCanonicalHostName(), actualStatus.getHost());
+      // Deleting the assertion on host. Assertions using InetAddress.getLocalHost can be flaky
+      // because some systems return "localhost" instead of the localhost's IP address as a string.
+      // assertEquals(InetAddress.getLocalHost().getCanonicalHostName(), actualStatus.getHost());
       assertEquals(getUniqueName(), actualStatus.getMemberName());
     } catch (Throwable e) {
       this.errorCollector.addError(e);
@@ -819,7 +814,7 @@ public class LocatorLauncherRemoteIntegrationTest
         is(equalTo(this.temporaryFolder.getRoot().getCanonicalPath())));
     assertThat(actualStatus.getClasspath(), is(nullValue()));
     assertThat(actualStatus.getGemFireVersion(), is(equalTo(GemFireVersion.getGemFireVersion())));
-    assertThat(actualStatus.getJavaVersion(), is(nullValue()));
+    assertThat(actualStatus.getJavaVersion(), is(equalTo(System.getProperty("java.version"))));
     assertThat(actualStatus.getLogFile(), is(nullValue()));
     assertThat(actualStatus.getHost(), is(nullValue()));
     assertThat(actualStatus.getMemberName(), is(nullValue()));
@@ -851,7 +846,7 @@ public class LocatorLauncherRemoteIntegrationTest
         is(equalTo(this.temporaryFolder.getRoot().getCanonicalPath())));
     assertThat(actualStatus.getClasspath(), is(nullValue()));
     assertThat(actualStatus.getGemFireVersion(), is(equalTo(GemFireVersion.getGemFireVersion())));
-    assertThat(actualStatus.getJavaVersion(), is(nullValue()));
+    assertThat(actualStatus.getJavaVersion(), is(equalTo(System.getProperty("java.version"))));
     assertThat(actualStatus.getLogFile(), is(nullValue()));
     assertThat(actualStatus.getHost(), is(nullValue()));
     assertThat(actualStatus.getMemberName(), is(nullValue()));
@@ -861,7 +856,7 @@ public class LocatorLauncherRemoteIntegrationTest
   public void testStopUsingPid() throws Throwable {
     final List<String> jvmArguments = getJvmArguments();
 
-    final List<String> command = new ArrayList<String>();
+    final List<String> command = new ArrayList<>();
     command
         .add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
     for (String jvmArgument : jvmArguments) {
@@ -942,7 +937,7 @@ public class LocatorLauncherRemoteIntegrationTest
   public void testStopUsingWorkingDirectory() throws Throwable {
     final List<String> jvmArguments = getJvmArguments();
 
-    final List<String> command = new ArrayList<String>();
+    final List<String> command = new ArrayList<>();
     command
         .add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
     for (String jvmArgument : jvmArguments) {
@@ -1013,7 +1008,7 @@ public class LocatorLauncherRemoteIntegrationTest
         final int port = Integer.parseInt(args[0]);
 
         // launch LocatorLauncher
-        List<String> command = new ArrayList<String>();
+        List<String> command = new ArrayList<>();
         command.add(
             new File(new File(System.getProperty("java.home"), "bin"), "java").getAbsolutePath());
         command.add("-cp");

http://git-wip-us.apache.org/repos/asf/geode/blob/6fb2ca83/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/lifecycle/GfshStatusCommandsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/lifecycle/GfshStatusCommandsIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/lifecycle/GfshStatusCommandsIntegrationTest.java
new file mode 100644
index 0000000..dd5841f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/lifecycle/GfshStatusCommandsIntegrationTest.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.management.internal.cli.commands.lifecycle;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class GfshStatusCommandsIntegrationTest {
+  final private static String LOCATOR_NAME = "locator1";
+  // private int port;
+
+  @Rule
+  public LocatorStarterRule locator =
+      new LocatorStarterRule().withJMXManager().withName(LOCATOR_NAME).withAutoStart();
+
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void connect() throws Exception {
+    // port = getRandomAvailablePort(SOCKET);
+    gfsh.connectAndVerify(locator);
+  }
+
+  @Test
+  public void statusLocatorWithBadPortReportsNotResponding() throws Exception {
+    CommandResult result = gfsh.executeCommand("status locator --host=localhost --port="
+        + String.valueOf(locator.getLocator().getPort() - 1));
+    assertThat(result.getContent().getString("message")).contains("not responding");
+  }
+
+  @Test
+  public void statusLocatorWithActivePortReportsOnline() throws Exception {
+    CommandResult result = gfsh.executeCommand(
+        "status locator --host=localhost --port=" + String.valueOf(locator.getLocator().getPort()));
+    assertThat(result.getContent().getString("message")).contains("is currently online");
+  }
+
+  @Test
+  public void statusServerWithWithNoOptions() throws Exception {
+    File serverDir = new File(temporaryFolder.getRoot(), "serverDir");
+    serverDir.mkdirs();
+    CommandResult result = gfsh.executeCommand("status server");
+    assertThat(result.getContent().getString("message")).contains("not responding");
+  }
+
+  @Test
+  public void statusServerWithInvalidDirReturnsMeangingfulMessage() throws Exception {
+    File serverDir = new File(temporaryFolder.getRoot(), "serverDir");
+    serverDir.mkdirs();
+    CommandResult result = gfsh.executeCommand("status server --dir=" + serverDir.toString());
+    assertThat(result.getContent().getString("message")).contains("not responding");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/6fb2ca83/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
index e7f17ef..dc17d03 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
@@ -44,12 +44,12 @@ import org.apache.geode.test.junit.rules.DescribedExternalResource;
  *
  * you can use this as Rule
  * 
- * @Rule GfshShellConnectionRule rule = new GfshSheelConnectionRule(); then after you connect to a
+ * @Rule GfshShellConnectionRule rule = new GfshShellConnectionRule(); then after you connect to a
  *       locator, you don't have to call disconnect() or close() at all, since the rule's after
  *       takes care of it for you.
  *
  *       Or as a ClassRule
- * @ClassRule GfshShellConnectionRule rule = new GfshSheelConnectionRule(); When using as a
+ * @ClassRule GfshShellConnectionRule rule = new GfshShellConnectionRule(); When using as a
  *            ClassRule, if you call connect in a test, you will need to call disconnect after the
  *            test as well. See NetstatDUnitTest for example.
  *