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

[02/10] geode git commit: GEODE-2267: enhance error output for gfsh.

GEODE-2267: enhance error output for gfsh.

* correctly output error message if gfsh execution has an error
* export logs should output correct log message over http connection as well


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

Branch: refs/heads/feature/GEODE-2645
Commit: 55a2a3ce0b3c30f7386096d0ff3a8c4eae0c08b3
Parents: f4701a1
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu Mar 9 22:33:13 2017 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Mar 13 16:40:18 2017 -0700

----------------------------------------------------------------------
 .../internal/cli/commands/ExportLogCommand.java |   9 +-
 .../cli/shell/GfshExecutionStrategy.java        |   6 +-
 .../web/controllers/ExportLogController.java    |  44 +++++-
 .../web/shell/AbstractHttpOperationInvoker.java |  19 ++-
 .../cli/commands/ExportLogsIntegrationTest.java |  71 +++++++++
 .../cli/commands/ExportLogsStatsDUnitTest.java  | 150 ++++++++++++++++++
 .../cli/commands/ExportStatsDUnitTest.java      | 153 -------------------
 .../controllers/ExportLogControllerTest.java    |  56 +++++++
 .../dunit/rules/GfshShellConnectionRule.java    |   4 +-
 .../commands/ExportLogsOverHttpDUnitTest.java   |  71 ---------
 .../ExportLogsOverHttpIntegrationTest.java      |  30 ++++
 .../ExportLogsStatsOverHttpDUnitTest.java       |  71 +++++++++
 12 files changed, 440 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogCommand.java
index 36d071c..678fdaf 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogCommand.java
@@ -98,6 +98,10 @@ public class ExportLogCommand implements CommandMarker {
       Set<DistributedMember> targetMembers =
           CliUtil.findMembersIncludingLocators(groups, memberIds);
 
+      if (targetMembers.isEmpty()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+      }
+
       Map<String, Path> zipFilesFromMembers = new HashMap<>();
       for (DistributedMember server : targetMembers) {
         Region region = ExportLogsFunction.createOrGetExistingExportLogsRegion(true, cache);
@@ -120,8 +124,11 @@ public class ExportLogCommand implements CommandMarker {
         }
       }
 
-      Path tempDir = Files.createTempDirectory("exportedLogs");
+      if (zipFilesFromMembers.isEmpty()) {
+        return ResultBuilder.createUserErrorResult("No files to be exported.");
+      }
 
+      Path tempDir = Files.createTempDirectory("exportedLogs");
       // make sure the directory is created, so that even if there is no files unzipped to this dir,
       // we can
       // still zip it and send an empty zip file back to the client

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
index ad78efd..d74f5d6 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
@@ -284,12 +284,10 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
       if (postExecResult != null) {
         if (Status.ERROR.equals(postExecResult.getStatus())) {
           if (logWrapper.infoEnabled()) {
-            logWrapper
-                .info("Post execution Result :: " + ResultBuilder.resultAsString(postExecResult));
+            logWrapper.info("Post execution Result :: " + postExecResult);
           }
         } else if (logWrapper.fineEnabled()) {
-          logWrapper
-              .fine("Post execution Result :: " + ResultBuilder.resultAsString(postExecResult));
+          logWrapper.fine("Post execution Result :: " + postExecResult);
         }
         commandResult = postExecResult;
       }

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java
index 0351573..527e059 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java
@@ -16,13 +16,16 @@
 package org.apache.geode.management.internal.web.controllers;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.management.cli.Result;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.ResultBuilder;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.springframework.core.io.InputStreamResource;
 import org.springframework.http.HttpHeaders;
 import org.springframework.http.HttpStatus;
+import org.springframework.http.MediaType;
 import org.springframework.http.ResponseEntity;
 import org.springframework.stereotype.Controller;
 import org.springframework.web.bind.annotation.RequestMapping;
@@ -85,21 +88,50 @@ public class ExportLogController extends AbstractCommandsController {
       command.addOption(CliStrings.EXPORT_LOGS__ENDTIME, endTime);
     }
 
-    // the result is json string from CommandResult
     String result = processCommand(command.toString());
+    return getResponse(result);
 
-    // parse the result to get the file path. This file Path should always exist in the file system
-    String filePath = ResultBuilder.fromJson(result).nextLine().trim();
+  }
+
+  ResponseEntity<InputStreamResource> getResponse(String result) {
+    // the result is json string from CommandResul
+    Result commandResult = ResultBuilder.fromJson(result);
+    if (commandResult.getStatus().equals(Result.Status.OK)) {
+      return getOKResponse(commandResult);
+
+    } else {
+      return getErrorResponse(result);
+    }
+  }
+
+  private ResponseEntity<InputStreamResource> getErrorResponse(String result) {
+    HttpHeaders respHeaders = new HttpHeaders();
+    InputStreamResource isr;// if the command is successful, the output is the filepath,
+    // else we need to send the orignal result back so that the receiver will know to turn it
+    // into a Result object
+    try {
+      isr = new InputStreamResource(IOUtils.toInputStream(result, "UTF-8"));
+      respHeaders.set(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON_VALUE);
+      return new ResponseEntity<InputStreamResource>(isr, respHeaders, HttpStatus.OK);
+    } catch (Exception e) {
+      throw new RuntimeException("IO Error writing file to output stream", e);
+    }
+  }
 
+  private ResponseEntity<InputStreamResource> getOKResponse(Result commandResult) {
     HttpHeaders respHeaders = new HttpHeaders();
+    InputStreamResource isr;// if the command is successful, the output is the filepath,
+    String filePath = commandResult.nextLine().trim();
     File zipFile = new File(filePath);
     try {
-      InputStreamResource isr = new InputStreamResource(new FileInputStream(zipFile));
+      isr = new InputStreamResource(new FileInputStream(zipFile));
+      respHeaders.set(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_OCTET_STREAM_VALUE);
       return new ResponseEntity<InputStreamResource>(isr, respHeaders, HttpStatus.OK);
-    } catch (Exception ex) {
-      throw new RuntimeException("IOError writing file to output stream", ex);
+    } catch (Exception e) {
+      throw new RuntimeException("IO Error writing file to output stream", e);
     } finally {
       FileUtils.deleteQuietly(zipFile);
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/AbstractHttpOperationInvoker.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/AbstractHttpOperationInvoker.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/AbstractHttpOperationInvoker.java
index f60aabc..25f972e 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/AbstractHttpOperationInvoker.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/shell/AbstractHttpOperationInvoker.java
@@ -633,7 +633,7 @@ public abstract class AbstractHttpOperationInvoker implements HttpOperationInvok
     return response.getBody();
   }
 
-  protected Path downloadResponseToTempFile(ClientHttpRequest request,
+  protected Object downloadResponseToTempFile(ClientHttpRequest request,
       Map<String, ?> uriVariables) {
     final URI url = request.getURL(uriVariables);
 
@@ -645,13 +645,18 @@ public abstract class AbstractHttpOperationInvoker implements HttpOperationInvok
     };
 
     // Streams the response instead of loading it all in memory
-    ResponseExtractor<Path> responseExtractor = resp -> {
-      Path tempFile = Files.createTempFile("fileDownload", "");
-      if (tempFile.toFile().exists()) {
-        FileUtils.deleteQuietly(tempFile.toFile());
+    ResponseExtractor<Object> responseExtractor = resp -> {
+      MediaType mediaType = resp.getHeaders().getContentType();
+      if (mediaType.equals(MediaType.APPLICATION_JSON)) {
+        return org.apache.commons.io.IOUtils.toString(resp.getBody(), "UTF-8");
+      } else {
+        Path tempFile = Files.createTempFile("fileDownload", "");
+        if (tempFile.toFile().exists()) {
+          FileUtils.deleteQuietly(tempFile.toFile());
+        }
+        Files.copy(resp.getBody(), tempFile);
+        return tempFile;
       }
-      Files.copy(resp.getBody(), tempFile);
-      return tempFile;
     };
     return getRestTemplate().execute(url, org.springframework.http.HttpMethod.GET, requestCallback,
         responseExtractor);

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsIntegrationTest.java
new file mode 100644
index 0000000..46a07ad
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsIntegrationTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Properties;
+
+@Category(IntegrationTest.class)
+public class ExportLogsIntegrationTest {
+
+  @ClassRule
+  public static LocatorStarterRule locator = new LocatorStarterRule();
+
+  @Rule
+  public GfshShellConnectionRule gfsh = new GfshShellConnectionRule();
+
+  private static int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+  protected static int httpPort = ports[0];
+  protected static int jmxPort = ports[1];
+
+  @BeforeClass
+  public static void before() throws Exception {
+    Properties properties = new Properties();
+    properties.setProperty(ConfigurationProperties.HTTP_SERVICE_PORT, httpPort + "");
+    properties.setProperty(ConfigurationProperties.JMX_MANAGER_PORT, jmxPort + "");
+    locator.startLocator(properties);
+  }
+
+  protected void connect() throws Exception {
+    gfsh.connectAndVerify(locator);
+  }
+
+  @Test
+  public void testInvalidMember() throws Exception {
+    connect();
+    gfsh.executeCommand("export logs --member=member1,member2");
+    assertThat(gfsh.getGfshOutput()).contains("No Members Found");
+  }
+
+  @Test
+  public void testNothingToExport() throws Exception {
+    connect();
+    gfsh.executeCommand("export logs --stats-only");
+    assertThat(gfsh.getGfshOutput()).contains("No files to be exported.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsStatsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsStatsDUnitTest.java
new file mode 100644
index 0000000..de2ea64
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsStatsDUnitTest.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.STATISTIC_ARCHIVE_FILE;
+import static org.apache.geode.management.internal.cli.commands.ExportLogCommand.ONLY_DATE_FORMAT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import com.google.common.collect.Sets;
+
+import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+@Category(DistributedTest.class)
+public class ExportLogsStatsDUnitTest {
+  @ClassRule
+  public static LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
+
+  @ClassRule
+  public static GfshShellConnectionRule connector = new GfshShellConnectionRule();
+
+  protected static int jmxPort, httpPort;
+  protected static Set<String> expectedZipEntries = new HashSet<>();
+  protected static MemberVM locator;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+    httpPort = ports[0];
+    jmxPort = ports[1];
+    Properties locatorProperties = new Properties();
+    locatorProperties.setProperty(HTTP_SERVICE_BIND_ADDRESS, "localhost");
+    locatorProperties.setProperty(HTTP_SERVICE_PORT, httpPort + "");
+    locatorProperties.setProperty(JMX_MANAGER_PORT, jmxPort + "");
+
+    // start the locator in vm0 and then connect to it over http
+    locator = lsRule.startLocatorVM(0, locatorProperties);
+
+    Properties serverProperties = new Properties();
+    serverProperties.setProperty(ConfigurationProperties.STATISTIC_SAMPLING_ENABLED, "true");
+    serverProperties.setProperty(STATISTIC_ARCHIVE_FILE, "statistics.gfs");
+    lsRule.startServerVM(1, serverProperties, locator.getPort());
+
+    expectedZipEntries = Sets.newHashSet("locator-0/locator-0.log", "server-1/server-1.log",
+        "server-1/statistics.gfs");
+  }
+
+  protected void connectIfNeeded() throws Exception {
+    if (!connector.isConnected()) {
+      connector.connect(locator);
+    }
+  }
+
+  @Test
+  public void testExportLogsAndStats() throws Exception {
+    connectIfNeeded();
+    connector.executeAndVerifyCommand("export logs");
+    String zipPath = getZipPathFromCommandResult(connector.getGfshOutput());
+    Set<String> actualZipEnries = getZipEntries(zipPath);
+
+    Set<String> expectedFiles = Sets.newHashSet("locator-0/locator-0.log", "server-1/server-1.log",
+        "server-1/statistics.gfs");
+    assertThat(actualZipEnries).isEqualTo(expectedFiles);
+  }
+
+  @Test
+  public void testExportLogsOnly() throws Exception {
+    connectIfNeeded();
+    connector.executeAndVerifyCommand("export logs --logs-only");
+    String zipPath = getZipPathFromCommandResult(connector.getGfshOutput());
+    Set<String> actualZipEnries = getZipEntries(zipPath);
+
+    Set<String> expectedFiles = Sets.newHashSet("locator-0/locator-0.log", "server-1/server-1.log");
+    assertThat(actualZipEnries).isEqualTo(expectedFiles);
+  }
+
+  @Test
+  public void testExportStatsOnly() throws Exception {
+    connectIfNeeded();
+    connector.executeAndVerifyCommand("export logs --stats-only");
+    String zipPath = getZipPathFromCommandResult(connector.getGfshOutput());
+    Set<String> actualZipEnries = getZipEntries(zipPath);
+
+    Set<String> expectedFiles = Sets.newHashSet("server-1/statistics.gfs");
+    assertThat(actualZipEnries).isEqualTo(expectedFiles);
+  }
+
+  @Test
+  public void startAndEndDateCanExcludeLogs() throws Exception {
+    connectIfNeeded();
+    ZonedDateTime now = LocalDateTime.now().atZone(ZoneId.systemDefault());
+    ZonedDateTime yesterday = now.minusDays(1);
+    ZonedDateTime twoDaysAgo = now.minusDays(2);
+
+    DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(ONLY_DATE_FORMAT);
+
+    CommandStringBuilder commandStringBuilder = new CommandStringBuilder("export logs");
+    commandStringBuilder.addOption("start-time", dateTimeFormatter.format(twoDaysAgo));
+    commandStringBuilder.addOption("end-time", dateTimeFormatter.format(yesterday));
+    commandStringBuilder.addOption("log-level", "debug");
+
+    String output = connector.execute(commandStringBuilder.toString());
+    assertThat(output).contains("No files to be exported");
+  }
+
+  protected String getZipPathFromCommandResult(String message) {
+    return message.replaceAll("Logs exported to the connected member's file system: ", "").trim();
+  }
+
+  private static Set<String> getZipEntries(String zipFilePath) throws IOException {
+    return new ZipFile(zipFilePath).stream().map(ZipEntry::getName).collect(Collectors.toSet());
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportStatsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportStatsDUnitTest.java
deleted file mode 100644
index b7f8c2a..0000000
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/ExportStatsDUnitTest.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.management.internal.cli.commands;
-
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.STATISTIC_ARCHIVE_FILE;
-import static org.apache.geode.management.internal.cli.commands.ExportLogCommand.ONLY_DATE_FORMAT;
-import static org.assertj.core.api.Assertions.assertThat;
-
-import com.google.common.collect.Sets;
-
-import org.apache.geode.distributed.ConfigurationProperties;
-import org.apache.geode.internal.AvailablePortHelper;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
-import org.apache.geode.test.dunit.rules.MemberVM;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.io.IOException;
-import java.time.LocalDateTime;
-import java.time.ZoneId;
-import java.time.ZonedDateTime;
-import java.time.format.DateTimeFormatter;
-import java.util.HashSet;
-import java.util.Properties;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipFile;
-
-@Category(DistributedTest.class)
-public class ExportStatsDUnitTest {
-  @ClassRule
-  public static LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
-
-  @ClassRule
-  public static GfshShellConnectionRule connector = new GfshShellConnectionRule();
-
-  protected static int jmxPort, httpPort;
-  protected static Set<String> expectedZipEntries = new HashSet<>();
-  protected static MemberVM locator;
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
-    httpPort = ports[0];
-    jmxPort = ports[1];
-    Properties locatorProperties = new Properties();
-    locatorProperties.setProperty(HTTP_SERVICE_BIND_ADDRESS, "localhost");
-    locatorProperties.setProperty(HTTP_SERVICE_PORT, httpPort + "");
-    locatorProperties.setProperty(JMX_MANAGER_PORT, jmxPort + "");
-
-    // start the locator in vm0 and then connect to it over http
-    locator = lsRule.startLocatorVM(0, locatorProperties);
-
-    Properties serverProperties = new Properties();
-    serverProperties.setProperty(ConfigurationProperties.STATISTIC_SAMPLING_ENABLED, "true");
-    serverProperties.setProperty(STATISTIC_ARCHIVE_FILE, "statistics.gfs");
-    lsRule.startServerVM(1, serverProperties, locator.getPort());
-
-    expectedZipEntries = Sets.newHashSet("locator-0/locator-0.log", "server-1/server-1.log",
-        "server-1/statistics.gfs");
-  }
-
-  protected void connectIfNeeded() throws Exception {
-    if (!connector.isConnected()) {
-      connector.connect(locator);
-    }
-  }
-
-  @Test
-  public void testExportLogsAndStats() throws Exception {
-    connectIfNeeded();
-    connector.executeAndVerifyCommand("export logs");
-    String zipPath = getZipPathFromCommandResult(connector.getGfshOutput());
-    Set<String> actualZipEnries = getZipEntries(zipPath);
-
-    Set<String> expectedFiles = Sets.newHashSet("locator-0/locator-0.log", "server-1/server-1.log",
-        "server-1/statistics.gfs");
-    assertThat(actualZipEnries).isEqualTo(expectedFiles);
-  }
-
-  @Test
-  public void testExportLogsOnly() throws Exception {
-    connectIfNeeded();
-    connector.executeAndVerifyCommand("export logs --logs-only");
-    String zipPath = getZipPathFromCommandResult(connector.getGfshOutput());
-    Set<String> actualZipEnries = getZipEntries(zipPath);
-
-    Set<String> expectedFiles = Sets.newHashSet("locator-0/locator-0.log", "server-1/server-1.log");
-    assertThat(actualZipEnries).isEqualTo(expectedFiles);
-  }
-
-  @Test
-  public void testExportStatsOnly() throws Exception {
-    connectIfNeeded();
-    connector.executeAndVerifyCommand("export logs --stats-only");
-    String zipPath = getZipPathFromCommandResult(connector.getGfshOutput());
-    Set<String> actualZipEnries = getZipEntries(zipPath);
-
-    Set<String> expectedFiles = Sets.newHashSet("server-1/statistics.gfs");
-    assertThat(actualZipEnries).isEqualTo(expectedFiles);
-  }
-
-  @Test
-  public void startAndEndDateCanExcludeLogs() throws Exception {
-    connectIfNeeded();
-    ZonedDateTime now = LocalDateTime.now().atZone(ZoneId.systemDefault());
-    ZonedDateTime yesterday = now.minusDays(1);
-    ZonedDateTime twoDaysAgo = now.minusDays(2);
-
-    DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(ONLY_DATE_FORMAT);
-
-    CommandStringBuilder commandStringBuilder = new CommandStringBuilder("export logs");
-    commandStringBuilder.addOption("start-time", dateTimeFormatter.format(twoDaysAgo));
-    commandStringBuilder.addOption("end-time", dateTimeFormatter.format(yesterday));
-    commandStringBuilder.addOption("log-level", "debug");
-
-    connector.executeAndVerifyCommand(commandStringBuilder.toString());
-    String zipPath = getZipPathFromCommandResult(connector.getGfshOutput());
-
-    Set<String> actualZipEnries = getZipEntries(zipPath);
-    assertThat(actualZipEnries).hasSize(0);
-  }
-
-  protected String getZipPathFromCommandResult(String message) {
-    return message.replaceAll("Logs exported to the connected member's file system: ", "").trim();
-  }
-
-  private static Set<String> getZipEntries(String zipFilePath) throws IOException {
-    return new ZipFile(zipFilePath).stream().map(ZipEntry::getName).collect(Collectors.toSet());
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ExportLogControllerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ExportLogControllerTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ExportLogControllerTest.java
new file mode 100644
index 0000000..bee7db2
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/web/controllers/ExportLogControllerTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.web.controllers;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.geode.management.internal.cli.CommandResponseBuilder;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.springframework.core.io.InputStreamResource;
+import org.springframework.http.HttpHeaders;
+import org.springframework.http.MediaType;
+import org.springframework.http.ResponseEntity;
+
+@Category(UnitTest.class)
+public class ExportLogControllerTest {
+  private ExportLogController controller;
+
+  @Before
+  public void before() throws Exception {
+    controller = new ExportLogController();
+  }
+
+  @Test
+  public void testErrorResponse() throws Exception {
+    String message = "No Members Found";
+    CommandResult result = (CommandResult) ResultBuilder.createUserErrorResult(message);
+    String responseJson = CommandResponseBuilder.createCommandResponseJson("memberName", result);
+
+    ResponseEntity<InputStreamResource> resp = controller.getResponse(responseJson);
+    HttpHeaders headers = resp.getHeaders();
+    assertThat(headers.get(HttpHeaders.CONTENT_TYPE).get(0))
+        .isEqualTo(MediaType.APPLICATION_JSON_VALUE);
+
+    InputStreamResource body = resp.getBody();
+    assertThat(IOUtils.toString(body.getInputStream(), "utf-8")).isEqualTo(responseJson);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/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 f367458..3b1c99a 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
@@ -86,11 +86,11 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
 
   }
 
-  public void connect(MemberVM locator, String... options) throws Exception {
+  public void connect(Member locator, String... options) throws Exception {
     connect(locator.getPort(), PortType.locator, options);
   }
 
-  public void connectAndVerify(MemberVM locator, String... options) throws Exception {
+  public void connectAndVerify(Member locator, String... options) throws Exception {
     connect(locator.getPort(), PortType.locator, options);
     assertThat(this.connected).isTrue();
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpDUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpDUnitTest.java
deleted file mode 100644
index cc4ae28..0000000
--- a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpDUnitTest.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.geode.management.internal.cli.commands;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.util.Arrays;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipFile;
-
-@Category(DistributedTest.class)
-public class ExportLogsOverHttpDUnitTest extends ExportStatsDUnitTest {
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
-  @Override
-  public void connectIfNeeded() throws Exception {
-    if (!connector.isConnected())
-      connector.connect(httpPort, GfshShellConnectionRule.PortType.http);
-  }
-
-  @Test
-  public void testExportWithDir() throws Exception {
-    connectIfNeeded();
-    File dir = temporaryFolder.newFolder();
-    // export the logs
-    connector.executeCommand("export logs --dir=" + dir.getAbsolutePath());
-    // verify that the message contains a path to the user.dir
-    String message = connector.getGfshOutput();
-    assertThat(message).contains("Logs exported to: ");
-    assertThat(message).contains(dir.getAbsolutePath());
-
-    String zipPath = getZipPathFromCommandResult(message);
-    Set<String> actualZipEntries =
-        new ZipFile(zipPath).stream().map(ZipEntry::getName).collect(Collectors.toSet());
-
-    assertThat(actualZipEntries).isEqualTo(expectedZipEntries);
-
-    // also verify that the zip file on locator is deleted
-    assertThat(Arrays.stream(locator.getWorkingDir().listFiles())
-        .filter(file -> file.getName().endsWith(".zip")).collect(Collectors.toSet())).isEmpty();
-  }
-
-  protected String getZipPathFromCommandResult(String message) {
-    return message.replaceAll("Logs exported to: ", "").trim();
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpIntegrationTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpIntegrationTest.java
new file mode 100644
index 0000000..420f2dd
--- /dev/null
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpIntegrationTest.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class ExportLogsOverHttpIntegrationTest extends ExportLogsIntegrationTest {
+
+  @Override
+  protected void connect() throws Exception {
+    gfsh.connectAndVerify(httpPort, GfshShellConnectionRule.PortType.http);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/55a2a3ce/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsStatsOverHttpDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsStatsOverHttpDUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsStatsOverHttpDUnitTest.java
new file mode 100644
index 0000000..c0b08cf
--- /dev/null
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsStatsOverHttpDUnitTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+@Category(DistributedTest.class)
+public class ExportLogsStatsOverHttpDUnitTest extends ExportLogsStatsDUnitTest {
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Override
+  public void connectIfNeeded() throws Exception {
+    if (!connector.isConnected())
+      connector.connect(httpPort, GfshShellConnectionRule.PortType.http);
+  }
+
+  @Test
+  public void testExportWithDir() throws Exception {
+    connectIfNeeded();
+    File dir = temporaryFolder.newFolder();
+    // export the logs
+    connector.executeCommand("export logs --dir=" + dir.getAbsolutePath());
+    // verify that the message contains a path to the user.dir
+    String message = connector.getGfshOutput();
+    assertThat(message).contains("Logs exported to: ");
+    assertThat(message).contains(dir.getAbsolutePath());
+
+    String zipPath = getZipPathFromCommandResult(message);
+    Set<String> actualZipEntries =
+        new ZipFile(zipPath).stream().map(ZipEntry::getName).collect(Collectors.toSet());
+
+    assertThat(actualZipEntries).isEqualTo(expectedZipEntries);
+
+    // also verify that the zip file on locator is deleted
+    assertThat(Arrays.stream(locator.getWorkingDir().listFiles())
+        .filter(file -> file.getName().endsWith(".zip")).collect(Collectors.toSet())).isEmpty();
+  }
+
+  protected String getZipPathFromCommandResult(String message) {
+    return message.replaceAll("Logs exported to: ", "").trim();
+  }
+}