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

[13/50] [abbrv] geode git commit: GEODE-2267: add validation to the arguments and include export stats in the command

GEODE-2267:  add validation to the arguments and include export stats in the command

  * use the config to determine where the logs and stats are


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

Branch: refs/heads/feature/GEODE-1969
Commit: bf788176bc479006dc196bedd04904d8858b26bb
Parents: 92582fb
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Feb 24 15:45:58 2017 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri Mar 3 16:03:50 2017 -0800

----------------------------------------------------------------------
 .../geode/internal/logging/GemFireLevel.java    |   6 +-
 .../internal/logging/InternalLogWriter.java     |   3 +
 .../geode/internal/logging/LogService.java      |  20 +-
 .../internal/statistics/StatArchiveHandler.java |   2 +-
 .../internal/cli/CliAroundInterceptor.java      |  12 +-
 .../management/internal/cli/CommandRequest.java |   8 +-
 .../internal/cli/commands/ConfigCommands.java   |   3 +-
 .../internal/cli/commands/DeployCommands.java   |   5 -
 ...xportImportClusterConfigurationCommands.java |   7 +-
 .../internal/cli/commands/ExportLogCommand.java | 229 ++++++++++++
 .../internal/cli/commands/FunctionCommands.java |  24 +-
 .../cli/commands/MiscellaneousCommands.java     | 139 -------
 .../cli/functions/ExportLogsFunction.java       | 141 ++++---
 .../internal/cli/functions/LogFileFunction.java | 315 ----------------
 .../internal/cli/i18n/CliStrings.java           |   6 +-
 .../cli/remote/RemoteExecutionStrategy.java     |   2 +-
 .../cli/shell/GfshExecutionStrategy.java        |   5 -
 .../cli/util/ExportLogsCacheWriter.java         |   6 +-
 .../internal/cli/util/LogExporter.java          |  64 ++--
 .../management/internal/cli/util/LogFilter.java |  77 ++--
 .../internal/cli/util/LogLevelExtractor.java    |  35 +-
 .../web/controllers/ExportLogController.java    | 101 +++++
 .../MiscellaneousCommandsController.java        |  70 ----
 .../SimpleStatSamplerIntegrationTest.java       |  25 +-
 .../ConnectToLocatorSSLDUnitTest.java           |   7 +-
 .../AbstractCliAroundInterceptorJUnitTest.java  |   5 +-
 .../management/internal/cli/HeadlessGfsh.java   |  10 +-
 .../cli/commands/CliCommandTestBase.java        |  41 ++-
 .../internal/cli/commands/ExportLogsDUnit.java  | 364 -------------------
 .../cli/commands/ExportLogsDUnitTest.java       | 342 +++++++++++++++++
 .../ExportLogsInterceptorJUnitTest.java         |  89 +++++
 .../cli/commands/ExportStatsDUnitTest.java      | 153 ++++++++
 .../ExportLogsFunctionIntegrationTest.java      |  67 ++--
 .../cli/util/ExportLogsCacheWriterUnitTest.java |  46 +++
 .../cli/util/LogExporterIntegrationTest.java    | 164 +++++++++
 .../internal/cli/util/LogExporterTest.java      |  67 +++-
 .../internal/cli/util/LogFilterTest.java        | 149 ++++++--
 .../cli/util/LogLevelExtractorTest.java         |  62 +++-
 .../internal/security/MultiUserDUnitTest.java   |  30 +-
 .../dunit/rules/GfshShellConnectionRule.java    |  16 +-
 .../dunit/rules/LocatorServerStartupRule.java   |   9 +-
 .../codeAnalysis/sanctionedSerializables.txt    |   4 +-
 .../cli/commands/golden-help-offline.properties |  17 +-
 .../commands/ExportLogsOverHttpDUnitTest.java   |  72 +---
 44 files changed, 1755 insertions(+), 1264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/internal/logging/GemFireLevel.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/GemFireLevel.java b/geode-core/src/main/java/org/apache/geode/internal/logging/GemFireLevel.java
index cffd162..0e1dc6e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/GemFireLevel.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/GemFireLevel.java
@@ -14,13 +14,15 @@
  */
 package org.apache.geode.internal.logging;
 
-import java.util.logging.Level;
-
 import org.apache.geode.i18n.LogWriterI18n;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
+import java.util.logging.Level;
+
 /**
  * Extension that adds ERROR to the standard JDK logging level class.
+ * 
+ * @deprecated use log4j level
  */
 public class GemFireLevel extends Level {
   private static final long serialVersionUID = -8123818329485173242L;

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/internal/logging/InternalLogWriter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/InternalLogWriter.java b/geode-core/src/main/java/org/apache/geode/internal/logging/InternalLogWriter.java
index 661fce9..4c04ca6 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/InternalLogWriter.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/InternalLogWriter.java
@@ -33,7 +33,10 @@ import org.apache.geode.i18n.StringId;
  * <li>{@link #SEVERE_LEVEL}
  * <li>{@link #NONE_LEVEL}
  * </ol>
+ * 
+ * @deprecated use log4j api instead
  */
+@Deprecated
 public interface InternalLogWriter extends LogWriter, LogWriterI18n {
 
   /**

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/internal/logging/LogService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/LogService.java b/geode-core/src/main/java/org/apache/geode/internal/logging/LogService.java
index 0ad7d84..1f8a564 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/LogService.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/LogService.java
@@ -14,10 +14,12 @@
  */
 package org.apache.geode.internal.logging;
 
-import java.beans.PropertyChangeEvent;
-import java.beans.PropertyChangeListener;
-import java.io.File;
-
+import org.apache.geode.internal.logging.log4j.AppenderContext;
+import org.apache.geode.internal.logging.log4j.ConfigLocator;
+import org.apache.geode.internal.logging.log4j.Configurator;
+import org.apache.geode.internal.logging.log4j.FastLogger;
+import org.apache.geode.internal.logging.log4j.LogWriterLogger;
+import org.apache.geode.internal.logging.log4j.message.GemFireParameterizedMessageFactory;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -29,12 +31,9 @@ import org.apache.logging.log4j.core.lookup.StrLookup;
 import org.apache.logging.log4j.core.lookup.StrSubstitutor;
 import org.apache.logging.log4j.status.StatusLogger;
 
-import org.apache.geode.internal.logging.log4j.AppenderContext;
-import org.apache.geode.internal.logging.log4j.ConfigLocator;
-import org.apache.geode.internal.logging.log4j.Configurator;
-import org.apache.geode.internal.logging.log4j.FastLogger;
-import org.apache.geode.internal.logging.log4j.LogWriterLogger;
-import org.apache.geode.internal.logging.log4j.message.GemFireParameterizedMessageFactory;
+import java.beans.PropertyChangeEvent;
+import java.beans.PropertyChangeListener;
+import java.io.File;
 
 /**
  * Centralizes log configuration and initialization.
@@ -47,6 +46,7 @@ public class LogService extends LogManager {
   public static final String BASE_LOGGER_NAME = "org.apache.geode";
   public static final String MAIN_LOGGER_NAME = "org.apache.geode";
   public static final String SECURITY_LOGGER_NAME = "org.apache.geode.security";
+  public static final String DEFAULT_LOG_LEVEL = "INFO";
 
   public static final String GEODE_VERBOSE_FILTER = "{GEODE_VERBOSE}";
   public static final String GEMFIRE_VERBOSE_FILTER = "{GEMFIRE_VERBOSE}";

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/internal/statistics/StatArchiveHandler.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatArchiveHandler.java b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatArchiveHandler.java
index 03b0f89..c96e672 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/statistics/StatArchiveHandler.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/statistics/StatArchiveHandler.java
@@ -383,7 +383,7 @@ public class StatArchiveHandler implements SampleHandler {
       }
       try {
         StatArchiveDescriptor archiveDescriptor = new StatArchiveDescriptor.Builder()
-            .setArchiveName(newFile.getPath()).setSystemId(this.config.getSystemId())
+            .setArchiveName(newFile.getAbsolutePath()).setSystemId(this.config.getSystemId())
             .setSystemStartTime(this.config.getSystemStartTime())
             .setSystemDirectoryPath(this.config.getSystemDirectoryPath())
             .setProductDescription(this.config.getProductDescription()).build();

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliAroundInterceptor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliAroundInterceptor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliAroundInterceptor.java
index 2e26efd..99114f0 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliAroundInterceptor.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CliAroundInterceptor.java
@@ -32,24 +32,18 @@ public interface CliAroundInterceptor {
   /**
    * called by the OperationInvoker before the command is executed
    */
-  default public Result preExecution(GfshParseResult parseResult) {
+  default Result preExecution(GfshParseResult parseResult) {
     return ResultBuilder.createInfoResult("");
   }
 
-  @Deprecated
-  default public Result postExecution(GfshParseResult parseResult, Result commandResult) {
-    return commandResult;
-  }
-
   /**
    * called by the OperationInvoker after the command is executed
    * 
    * @param tempFile: if the command's isFileDownloadOverHttp is true, the is the File downloaded
    *        after the http response is processed.
    */
-  default public Result postExecution(GfshParseResult parseResult, Result commandResult,
-      Path tempFile) {
-    return postExecution(parseResult, commandResult);
+  default Result postExecution(GfshParseResult parseResult, Result commandResult, Path tempFile) {
+    return commandResult;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandRequest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandRequest.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandRequest.java
index 046499e..00a0587 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/CommandRequest.java
@@ -40,7 +40,7 @@ public class CommandRequest {
 
   private final Map<String, String> customParameters = new HashMap<String, String>();
   private final Map<String, String> env;
-  private boolean downloadFile = false;
+  private final boolean downloadFile;
 
   private String customInput;
 
@@ -48,12 +48,14 @@ public class CommandRequest {
     this.env = env;
     this.fileData = null;
     this.parseResult = null;
+    downloadFile = false;
   }
 
   public CommandRequest(final Map<String, String> env, final byte[][] fileData) {
     this.env = env;
     this.fileData = fileData;
     this.parseResult = null;
+    downloadFile = false;
   }
 
   public CommandRequest(final GfshParseResult parseResult, final Map<String, String> env) {
@@ -69,9 +71,7 @@ public class CommandRequest {
     this.parseResult = parseResult;
 
     CliMetaData metaData = parseResult.getMethod().getDeclaredAnnotation(CliMetaData.class);
-    if (metaData != null && metaData.isFileDownloadOverHttp()) {
-      downloadFile = true;
-    }
+    this.downloadFile = (metaData != null && metaData.isFileDownloadOverHttp());
   }
 
   public String getName() {

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
index 12a0a64..8c63382 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
@@ -51,6 +51,7 @@ import org.springframework.shell.core.annotation.CliOption;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -500,7 +501,7 @@ public class ConfigCommands extends AbstractCommandsSupport {
     }
 
     @Override
-    public Result postExecution(GfshParseResult parseResult, Result commandResult) {
+    public Result postExecution(GfshParseResult parseResult, Result commandResult, Path tempFile) {
       if (commandResult.hasIncomingFiles()) {
         try {
           commandResult.saveIncomingFiles(saveDirString);

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java
index bd67728..1444088 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DeployCommands.java
@@ -340,10 +340,5 @@ public final class DeployCommands extends AbstractCommandsSupport {
 
       return fileResult;
     }
-
-    @Override
-    public Result postExecution(GfshParseResult parseResult, Result commandResult) {
-      return null;
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
index e94af4f..7afb0db 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportImportClusterConfigurationCommands.java
@@ -245,7 +245,7 @@ public class ExportImportClusterConfigurationCommands extends AbstractCommandsSu
     }
 
     @Override
-    public Result postExecution(GfshParseResult parseResult, Result commandResult) {
+    public Result postExecution(GfshParseResult parseResult, Result commandResult, Path tempFile) {
       if (commandResult.hasIncomingFiles()) {
         try {
           commandResult.saveIncomingFiles(System.getProperty("user.dir"));
@@ -293,11 +293,6 @@ public class ExportImportClusterConfigurationCommands extends AbstractCommandsSu
 
       return fileResult;
     }
-
-    @Override
-    public Result postExecution(GfshParseResult parseResult, Result commandResult) {
-      return null;
-    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/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
new file mode 100644
index 0000000..36d071c
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportLogCommand.java
@@ -0,0 +1,229 @@
+/*
+ * 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.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.geode.cache.Region;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.cli.ConverterHint;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.AbstractCliAroundInterceptor;
+import org.apache.geode.management.internal.cli.CliUtil;
+import org.apache.geode.management.internal.cli.GfshParseResult;
+import org.apache.geode.management.internal.cli.functions.ExportLogsFunction;
+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.ExportLogsCacheWriter;
+import org.apache.geode.management.internal.configuration.utils.ZipUtils;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.Logger;
+import org.springframework.shell.core.CommandMarker;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.LocalDateTime;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+public class ExportLogCommand implements CommandMarker {
+  public final static String FORMAT = "yyyy/MM/dd/HH/mm/ss/SSS/z";
+  public final static String ONLY_DATE_FORMAT = "yyyy/MM/dd";
+  private final static Logger logger = LogService.getLogger();
+
+  @CliCommand(value = CliStrings.EXPORT_LOGS, help = CliStrings.EXPORT_LOGS__HELP)
+  @CliMetaData(shellOnly = false, isFileDownloadOverHttp = true,
+      interceptor = "org.apache.geode.management.internal.cli.commands.ExportLogCommand$ExportLogsInterceptor",
+      relatedTopic = {CliStrings.TOPIC_GEODE_SERVER, CliStrings.TOPIC_GEODE_DEBUG_UTIL})
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.READ)
+  public Result exportLogs(
+      @CliOption(key = CliStrings.EXPORT_LOGS__DIR, help = CliStrings.EXPORT_LOGS__DIR__HELP,
+          mandatory = false) String dirName,
+      @CliOption(key = CliStrings.EXPORT_LOGS__GROUP,
+          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
+          optionContext = ConverterHint.MEMBERGROUP,
+          help = CliStrings.EXPORT_LOGS__GROUP__HELP) String[] groups,
+      @CliOption(key = CliStrings.EXPORT_LOGS__MEMBER,
+          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
+          optionContext = ConverterHint.ALL_MEMBER_IDNAME,
+          help = CliStrings.EXPORT_LOGS__MEMBER__HELP) String[] memberIds,
+      @CliOption(key = CliStrings.EXPORT_LOGS__LOGLEVEL,
+          unspecifiedDefaultValue = LogService.DEFAULT_LOG_LEVEL,
+          optionContext = ConverterHint.LOG_LEVEL,
+          help = CliStrings.EXPORT_LOGS__LOGLEVEL__HELP) String logLevel,
+      @CliOption(key = CliStrings.EXPORT_LOGS__UPTO_LOGLEVEL, unspecifiedDefaultValue = "false",
+          help = CliStrings.EXPORT_LOGS__UPTO_LOGLEVEL__HELP) boolean onlyLogLevel,
+      @CliOption(key = CliStrings.EXPORT_LOGS__MERGELOG, unspecifiedDefaultValue = "false",
+          help = CliStrings.EXPORT_LOGS__MERGELOG__HELP) boolean mergeLog,
+      @CliOption(key = CliStrings.EXPORT_LOGS__STARTTIME,
+          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
+          help = CliStrings.EXPORT_LOGS__STARTTIME__HELP) String start,
+      @CliOption(key = CliStrings.EXPORT_LOGS__ENDTIME,
+          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
+          help = CliStrings.EXPORT_LOGS__ENDTIME__HELP) String end,
+      @CliOption(key = CliStrings.EXPORT_LOGS__LOGSONLY, unspecifiedDefaultValue = "false",
+          specifiedDefaultValue = "true",
+          help = CliStrings.EXPORT_LOGS__LOGSONLY__HELP) boolean logsOnly,
+      @CliOption(key = CliStrings.EXPORT_LOGS__STATSONLY, unspecifiedDefaultValue = "false",
+          specifiedDefaultValue = "true",
+          help = CliStrings.EXPORT_LOGS__STATSONLY__HELP) boolean statsOnly) {
+    Result result = null;
+    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+    try {
+      Set<DistributedMember> targetMembers =
+          CliUtil.findMembersIncludingLocators(groups, memberIds);
+
+      Map<String, Path> zipFilesFromMembers = new HashMap<>();
+      for (DistributedMember server : targetMembers) {
+        Region region = ExportLogsFunction.createOrGetExistingExportLogsRegion(true, cache);
+
+        ExportLogsCacheWriter cacheWriter =
+            (ExportLogsCacheWriter) region.getAttributes().getCacheWriter();
+
+        cacheWriter.startFile(server.getName());
+
+        CliUtil.executeFunction(new ExportLogsFunction(),
+            new ExportLogsFunction.Args(start, end, logLevel, onlyLogLevel, logsOnly, statsOnly),
+            server).getResult();
+        Path zipFile = cacheWriter.endFile();
+        ExportLogsFunction.destroyExportLogsRegion(cache);
+
+        // only put the zipfile in the map if it is not null
+        if (zipFile != null) {
+          logger.info("Received zip file from member {}: {}", server.getId(), zipFile);
+          zipFilesFromMembers.put(server.getId(), zipFile);
+        }
+      }
+
+      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
+      Path exportedLogsDir = tempDir.resolve("exportedLogs");
+      FileUtils.forceMkdir(exportedLogsDir.toFile());
+
+      for (Path zipFile : zipFilesFromMembers.values()) {
+        Path unzippedMemberDir =
+            exportedLogsDir.resolve(zipFile.getFileName().toString().replace(".zip", ""));
+        ZipUtils.unzip(zipFile.toAbsolutePath().toString(), unzippedMemberDir.toString());
+        FileUtils.deleteQuietly(zipFile.toFile());
+      }
+
+      Path workingDir = Paths.get(System.getProperty("user.dir"));
+      Path exportedLogsZipFile = workingDir
+          .resolve("exportedLogs_" + System.currentTimeMillis() + ".zip").toAbsolutePath();
+
+      logger.info("Zipping into: " + exportedLogsZipFile.toString());
+      ZipUtils.zipDirectory(exportedLogsDir, exportedLogsZipFile);
+      FileUtils.deleteDirectory(tempDir.toFile());
+      result = ResultBuilder.createInfoResult(exportedLogsZipFile.toString());
+    } catch (Exception ex) {
+      logger.error(ex, ex);
+      result = ResultBuilder.createGemFireErrorResult(ex.getMessage());
+    } finally {
+      ExportLogsFunction.destroyExportLogsRegion(cache);
+    }
+    logger.debug("Exporting logs returning = {}", result);
+    return result;
+  }
+
+  /**
+   * after the export logs, will need to copy the tempFile to the desired location and delete the
+   * temp file.
+   */
+  public static class ExportLogsInterceptor extends AbstractCliAroundInterceptor {
+    @Override
+    public Result preExecution(GfshParseResult parseResult) {
+      // the arguments are in the order of it's being declared
+      Map<String, String> arguments = parseResult.getParamValueStrings();
+
+      // validates groupId and memberIds not both set
+      if (arguments.get("group") != null && arguments.get("member") != null) {
+        return ResultBuilder.createUserErrorResult("Can't specify both group and member.");
+      }
+
+      // validate log level
+      String logLevel = arguments.get("log-level");
+      if (StringUtils.isBlank(logLevel) || Level.getLevel(logLevel.toUpperCase()) == null) {
+        return ResultBuilder.createUserErrorResult("Invalid log level: " + logLevel);
+      }
+
+      // validate start date and end date
+      String start = arguments.get("start-time");
+      String end = arguments.get("end-time");
+      if (start != null && end != null) {
+        // need to make sure end is later than start
+        LocalDateTime startTime = ExportLogsFunction.parseTime(start);
+        LocalDateTime endTime = ExportLogsFunction.parseTime(end);
+        if (startTime.isAfter(endTime)) {
+          return ResultBuilder.createUserErrorResult("start-time has to be earlier than end-time.");
+        }
+      }
+
+      // validate onlyLogs and onlyStats
+      boolean onlyLogs = Boolean.parseBoolean(arguments.get("logs-only"));
+      boolean onlyStats = Boolean.parseBoolean(arguments.get("stats-only"));
+      if (onlyLogs && onlyStats) {
+        return ResultBuilder.createUserErrorResult("logs-only and stats-only can't both be true");
+      }
+
+      return ResultBuilder.createInfoResult("");
+    }
+
+    @Override
+    public Result postExecution(GfshParseResult parseResult, Result commandResult, Path tempFile) {
+      // in the command over http case, the command result is in the downloaded temp file
+      if (tempFile != null) {
+        Path dirPath;
+        String dirName = parseResult.getParamValueStrings().get("dir");
+        if (StringUtils.isBlank(dirName)) {
+          dirPath = Paths.get(System.getProperty("user.dir"));
+        } else {
+          dirPath = Paths.get(dirName);
+        }
+        String fileName = "exportedLogs_" + System.currentTimeMillis() + ".zip";
+        File exportedLogFile = dirPath.resolve(fileName).toFile();
+        try {
+          FileUtils.copyFile(tempFile.toFile(), exportedLogFile);
+          FileUtils.deleteQuietly(tempFile.toFile());
+          commandResult = ResultBuilder
+              .createInfoResult("Logs exported to: " + exportedLogFile.getAbsolutePath());
+        } catch (IOException e) {
+          logger.error(e.getMessage(), e);
+          commandResult = ResultBuilder.createGemFireErrorResult(e.getMessage());
+        }
+      } else if (commandResult.getStatus() == Result.Status.OK) {
+        commandResult = ResultBuilder.createInfoResult(
+            "Logs exported to the connected member's file system: " + commandResult.nextLine());
+      }
+      return commandResult;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java
index 91810e8..c2fb88f 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/FunctionCommands.java
@@ -14,15 +14,6 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
@@ -60,12 +51,20 @@ import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
-
 import org.springframework.shell.core.CommandMarker;
 import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
 /**
  * 
  * @since GemFire 7.0
@@ -509,11 +508,6 @@ public class FunctionCommands implements CommandMarker {
             .createInfoResult("Destroying " + paramValueMap.get(CliStrings.DESTROY_FUNCTION__ID));
       }
     }
-
-    @Override
-    public Result postExecution(GfshParseResult parseResult, Result commandResult) {
-      return commandResult;
-    }
   }
 
   Result executeFunction(Cache cache, Set<DistributedMember> DsMembers, String functionId) {

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java
index e921031..1fc6027 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MiscellaneousCommands.java
@@ -14,12 +14,9 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang.StringUtils;
 import org.apache.geode.LogWriter;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.Region;
 import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionException;
@@ -55,7 +52,6 @@ import org.apache.geode.management.internal.cli.GfshParser;
 import org.apache.geode.management.internal.cli.LogWrapper;
 import org.apache.geode.management.internal.cli.domain.StackTracesPerMember;
 import org.apache.geode.management.internal.cli.functions.ChangeLogLevelFunction;
-import org.apache.geode.management.internal.cli.functions.ExportLogsFunction;
 import org.apache.geode.management.internal.cli.functions.GarbageCollectionFunction;
 import org.apache.geode.management.internal.cli.functions.GetStackTracesFunction;
 import org.apache.geode.management.internal.cli.functions.NetstatFunction;
@@ -74,8 +70,6 @@ import org.apache.geode.management.internal.cli.result.ResultData;
 import org.apache.geode.management.internal.cli.result.ResultDataException;
 import org.apache.geode.management.internal.cli.result.TabularResultData;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.cli.util.ExportLogsCacheWriter;
-import org.apache.geode.management.internal.configuration.utils.ZipUtils;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
@@ -92,9 +86,6 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.PrintWriter;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -122,8 +113,6 @@ import javax.management.ObjectName;
  */
 public class MiscellaneousCommands implements CommandMarker {
   public static final String NETSTAT_FILE_REQUIRED_EXTENSION = ".txt";
-  public final static String FORMAT = "yyyy/MM/dd/HH/mm/ss/SSS/z";
-  public final static String ONLY_DATE_FORMAT = "yyyy/MM/dd";
   public final static String DEFAULT_TIME_OUT = "10";
   private final static Logger logger = LogService.getLogger();
 
@@ -308,11 +297,6 @@ public class MiscellaneousCommands implements CommandMarker {
         return ResultBuilder.createInfoResult(CliStrings.SHUTDOWN__MSG__SHUTDOWN_ENTIRE_DS);
       }
     }
-
-    @Override
-    public Result postExecution(GfshParseResult parseResult, Result commandResult) {
-      return commandResult;
-    }
   }
 
   @CliCommand(value = CliStrings.GC, help = CliStrings.GC__HELP)
@@ -684,124 +668,6 @@ public class MiscellaneousCommands implements CommandMarker {
     return result;
   }
 
-  @CliCommand(value = CliStrings.EXPORT_LOGS, help = CliStrings.EXPORT_LOGS__HELP)
-  @CliMetaData(shellOnly = false, isFileDownloadOverHttp = true,
-      interceptor = "org.apache.geode.management.internal.cli.commands.MiscellaneousCommands$ExportLogsInterceptor",
-      relatedTopic = {CliStrings.TOPIC_GEODE_SERVER, CliStrings.TOPIC_GEODE_DEBUG_UTIL})
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-  public Result exportLogs(
-      @CliOption(key = CliStrings.EXPORT_LOGS__DIR, help = CliStrings.EXPORT_LOGS__DIR__HELP,
-          mandatory = false) String dirName,
-      @CliOption(key = CliStrings.EXPORT_LOGS__GROUP,
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.EXPORT_LOGS__GROUP__HELP) String[] groups,
-      @CliOption(key = CliStrings.EXPORT_LOGS__MEMBER,
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-          help = CliStrings.EXPORT_LOGS__MEMBER__HELP) String[] memberIds,
-      @CliOption(key = CliStrings.EXPORT_LOGS__LOGLEVEL,
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          optionContext = ConverterHint.LOG_LEVEL,
-          help = CliStrings.EXPORT_LOGS__LOGLEVEL__HELP) String logLevel,
-      @CliOption(key = CliStrings.EXPORT_LOGS__UPTO_LOGLEVEL, unspecifiedDefaultValue = "false",
-          help = CliStrings.EXPORT_LOGS__UPTO_LOGLEVEL__HELP) boolean onlyLogLevel,
-      @CliOption(key = CliStrings.EXPORT_LOGS__MERGELOG, unspecifiedDefaultValue = "false",
-          help = CliStrings.EXPORT_LOGS__MERGELOG__HELP) boolean mergeLog,
-      @CliOption(key = CliStrings.EXPORT_LOGS__STARTTIME,
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.EXPORT_LOGS__STARTTIME__HELP) String start,
-      @CliOption(key = CliStrings.EXPORT_LOGS__ENDTIME,
-          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.EXPORT_LOGS__ENDTIME__HELP) String end) {
-    Result result = null;
-    try {
-      Set<DistributedMember> targetMembers =
-          CliUtil.findMembersIncludingLocators(groups, memberIds);
-
-
-      Map<String, Path> zipFilesFromMembers = new HashMap<>();
-      for (DistributedMember server : targetMembers) {
-        Region region = ExportLogsFunction.createOrGetExistingExportLogsRegion(true);
-
-        ExportLogsCacheWriter cacheWriter =
-            (ExportLogsCacheWriter) region.getAttributes().getCacheWriter();
-
-        cacheWriter.startFile(server.getName());
-
-        CliUtil
-            .executeFunction(new ExportLogsFunction(),
-                new ExportLogsFunction.Args(start, end, logLevel, onlyLogLevel), server)
-            .getResult();
-        Path zipFile = cacheWriter.endFile();
-        ExportLogsFunction.destroyExportLogsRegion();
-        logger.info("Recieved zip file from member " + server.getId() + ": " + zipFile.toString());
-        zipFilesFromMembers.put(server.getId(), zipFile);
-      }
-
-      Path tempDir = Files.createTempDirectory("exportedLogs");
-      Path exportedLogsDir = tempDir.resolve("exportedLogs");
-
-      for (Path zipFile : zipFilesFromMembers.values()) {
-        Path unzippedMemberDir =
-            exportedLogsDir.resolve(zipFile.getFileName().toString().replace(".zip", ""));
-        ZipUtils.unzip(zipFile.toAbsolutePath().toString(), unzippedMemberDir.toString());
-        FileUtils.deleteQuietly(zipFile.toFile());
-      }
-
-      Path workingDir = Paths.get(System.getProperty("user.dir"));
-      Path exportedLogsZipFile = workingDir
-          .resolve("exportedLogs_" + System.currentTimeMillis() + ".zip").toAbsolutePath();
-
-      logger.info("Zipping into: " + exportedLogsZipFile.toString());
-      ZipUtils.zipDirectory(exportedLogsDir, exportedLogsZipFile);
-      FileUtils.deleteDirectory(tempDir.toFile());
-      result = ResultBuilder.createInfoResult(exportedLogsZipFile.toString());
-    } catch (Exception ex) {
-      logger.error(ex, ex);
-      result = ResultBuilder.createUserErrorResult(ex.getMessage());
-    } finally {
-      ExportLogsFunction.destroyExportLogsRegion();
-    }
-    logger.debug("Exporting logs returning = {}", result);
-    return result;
-  }
-
-  /**
-   * after the export logs, will need to copy the tempFile to the desired location and delete the
-   * temp file.
-   */
-  public static class ExportLogsInterceptor extends AbstractCliAroundInterceptor {
-    @Override
-    public Result postExecution(GfshParseResult parseResult, Result commandResult, Path tempFile) {
-      // in the command over http case, the command result is in the downloaded temp file
-      if (tempFile != null) {
-        Path dirPath;
-        String dirName = parseResult.getParamValueStrings().get("dir");
-        if (StringUtils.isBlank(dirName)) {
-          dirPath = Paths.get(System.getProperty("user.dir"));
-        } else {
-          dirPath = Paths.get(dirName);
-        }
-        String fileName = "exportedLogs_" + System.currentTimeMillis() + ".zip";
-        File exportedLogFile = dirPath.resolve(fileName).toFile();
-        try {
-          FileUtils.copyFile(tempFile.toFile(), exportedLogFile);
-          FileUtils.deleteQuietly(tempFile.toFile());
-          commandResult = ResultBuilder
-              .createInfoResult("Logs exported to: " + exportedLogFile.getAbsolutePath());
-        } catch (IOException e) {
-          logger.error(e.getMessage(), e);
-          commandResult = ResultBuilder.createGemFireErrorResult(e.getMessage());
-        }
-      } else {
-        commandResult = ResultBuilder.createInfoResult(
-            "Logs exported to the connected member's file system: " + commandResult.nextLine());
-      }
-      return commandResult;
-    }
-  }
-
   /****
    * Current implementation supports writing it to a file and returning the location of the file
    *
@@ -899,11 +765,6 @@ public class MiscellaneousCommands implements CommandMarker {
         return ResultBuilder.createInfoResult("");
       }
     }
-
-    @Override
-    public Result postExecution(GfshParseResult parseResult, Result commandResult) {
-      return commandResult;
-    }
   }
 
   /***

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/ExportLogsFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/ExportLogsFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/ExportLogsFunction.java
index 560e8aa..62026fb 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/ExportLogsFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/ExportLogsFunction.java
@@ -16,41 +16,44 @@
 
 package org.apache.geode.management.internal.cli.functions;
 
-import static java.util.stream.Collectors.toSet;
-
+import org.apache.commons.lang.StringUtils;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.InternalEntity;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalRegionArguments;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.LogWriterImpl;
-import org.apache.geode.management.internal.cli.commands.MiscellaneousCommands;
+import org.apache.geode.management.internal.cli.commands.ExportLogCommand;
 import org.apache.geode.management.internal.cli.util.ExportLogsCacheWriter;
 import org.apache.geode.management.internal.cli.util.LogExporter;
 import org.apache.geode.management.internal.cli.util.LogFilter;
 import org.apache.geode.management.internal.configuration.domain.Configuration;
+import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
 
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.time.LocalDateTime;
 import java.time.ZoneId;
 import java.util.Arrays;
-import java.util.Set;
-import java.util.stream.Stream;
 
+/**
+ * this function extracts the logs using a LogExporter which creates a zip file, and then writes the
+ * zip file bytes into a replicated region, this in effect, "stream" the zip file bytes to the
+ * locator
+ *
+ * The function only extracts .log and .gfs files under server's working directory
+ */
 public class ExportLogsFunction implements Function, InternalEntity {
   public static final String EXPORT_LOGS_REGION = "__exportLogsRegion";
   private static final Logger LOGGER = LogService.getLogger();
@@ -61,20 +64,34 @@ public class ExportLogsFunction implements Function, InternalEntity {
   @Override
   public void execute(final FunctionContext context) {
     try {
-      // TODO: change this to get cache from FunctionContext when it becomes available
       GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+      DistributionConfig config = cache.getDistributedSystem().getConfig();
 
       String memberId = cache.getDistributedSystem().getMemberId();
       LOGGER.info("ExportLogsFunction started for member {}", memberId);
 
-      Region exportLogsRegion = createOrGetExistingExportLogsRegion(false);
+      Region exportLogsRegion = createOrGetExistingExportLogsRegion(false, cache);
 
       Args args = (Args) context.getArguments();
-      LogFilter logFilter =
-          new LogFilter(args.getPermittedLogLevels(), args.getStartTime(), args.getEndTime());
-      Path workingDir = Paths.get(System.getProperty("user.dir"));
+      File baseLogFile = null;
+      File baseStatsFile = null;
+      if (args.isIncludeLogs() && !config.getLogFile().toString().isEmpty()) {
+        baseLogFile = config.getLogFile().getAbsoluteFile();
+      }
+      if (args.isIncludeStats() && !config.getStatisticArchiveFile().toString().isEmpty()) {
+        baseStatsFile = config.getStatisticArchiveFile().getAbsoluteFile();
+      }
+
+      LogFilter logFilter = new LogFilter(args.getLogLevel(), args.isThisLogLevelOnly(),
+          args.getStartTime(), args.getEndTime());
 
-      Path exportedZipFile = new LogExporter(logFilter).export(workingDir);
+      Path exportedZipFile = new LogExporter(logFilter, baseLogFile, baseStatsFile).export();
+
+      // nothing to return back
+      if (exportedZipFile == null) {
+        context.getResultSender().lastResult(null);
+        return;
+      }
 
       LOGGER.info("Streaming zipped file: " + exportedZipFile.toString());
       try (FileInputStream inputStream = new FileInputStream(exportedZipFile.toFile())) {
@@ -93,14 +110,14 @@ public class ExportLogsFunction implements Function, InternalEntity {
       context.getResultSender().lastResult(null);
 
     } catch (Exception e) {
+      e.printStackTrace();
       LOGGER.error(e);
       context.getResultSender().sendException(e);
     }
   }
 
-  public static Region createOrGetExistingExportLogsRegion(boolean isInitiatingMember)
-      throws IOException, ClassNotFoundException {
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+  public static Region createOrGetExistingExportLogsRegion(boolean isInitiatingMember,
+      GemFireCacheImpl cache) throws IOException, ClassNotFoundException {
 
     Region exportLogsRegion = cache.getRegion(EXPORT_LOGS_REGION);
     if (exportLogsRegion == null) {
@@ -121,16 +138,13 @@ public class ExportLogsFunction implements Function, InternalEntity {
     return exportLogsRegion;
   }
 
-  public static void destroyExportLogsRegion() {
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+  public static void destroyExportLogsRegion(GemFireCacheImpl cache) {
 
     Region exportLogsRegion = cache.getRegion(EXPORT_LOGS_REGION);
     if (exportLogsRegion == null) {
       return;
     }
-
     exportLogsRegion.destroyRegion();
-
   }
 
   @Override
@@ -139,59 +153,68 @@ public class ExportLogsFunction implements Function, InternalEntity {
   }
 
   public static class Args implements Serializable {
-    private String startTime;
-    private String endTime;
-    private String logLevel;
-    private boolean logLevelOnly;
-
-    public Args(String startTime, String endTime, String logLevel, boolean logLevelOnly) {
-      this.startTime = startTime;
-      this.endTime = endTime;
-      this.logLevel = logLevel;
-      this.logLevelOnly = logLevelOnly;
+    private LocalDateTime startTime;
+    private LocalDateTime endTime;
+    private Level logLevel;
+    private boolean thisLogLevelOnly;
+    private boolean includeLogs;
+    private boolean includeStats;
+
+    public Args(String startTime, String endTime, String logLevel, boolean logLevelOnly,
+        boolean logsOnly, boolean statsOnly) {
+      this.startTime = parseTime(startTime);
+      this.endTime = parseTime(endTime);
+
+      if (StringUtils.isBlank(logLevel)) {
+        this.logLevel = Level.INFO;
+      } else {
+        this.logLevel = Level.getLevel(logLevel.toUpperCase());
+      }
+      this.thisLogLevelOnly = logLevelOnly;
+
+      this.includeLogs = !statsOnly;
+      this.includeStats = !logsOnly;
     }
 
     public LocalDateTime getStartTime() {
-      return parseTime(startTime);
+      return startTime;
     }
 
     public LocalDateTime getEndTime() {
-      return parseTime(endTime);
+      return endTime;
     }
 
-    public Set<String> getPermittedLogLevels() {
-      if (logLevel == null || StringUtils.isBlank(logLevel)) {
-        return LogFilter.allLogLevels();
-      }
+    public Level getLogLevel() {
+      return logLevel;
+    }
 
-      if (logLevelOnly) {
-        return Stream.of(logLevel).collect(toSet());
-      }
+    public boolean isThisLogLevelOnly() {
+      return thisLogLevelOnly;
+    }
 
-      // Return all log levels lower than or equal to the specified logLevel
-      return Arrays.stream(InternalLogWriter.levelNames).filter((String level) -> {
-        int logLevelCode = LogWriterImpl.levelNameToCode(level);
-        int logLevelCodeThreshold = LogWriterImpl.levelNameToCode(logLevel);
+    public boolean isIncludeLogs() {
+      return includeLogs;
+    }
 
-        return logLevelCode >= logLevelCodeThreshold;
-      }).collect(toSet());
+    public boolean isIncludeStats() {
+      return includeStats;
     }
+  }
 
-    private static LocalDateTime parseTime(String dateString) {
-      if (dateString == null) {
-        return null;
-      }
+  public static LocalDateTime parseTime(String dateString) {
+    if (dateString == null) {
+      return null;
+    }
 
+    try {
+      SimpleDateFormat df = new SimpleDateFormat(ExportLogCommand.FORMAT);
+      return df.parse(dateString).toInstant().atZone(ZoneId.systemDefault()).toLocalDateTime();
+    } catch (ParseException e) {
       try {
-        SimpleDateFormat df = new SimpleDateFormat(MiscellaneousCommands.FORMAT);
+        SimpleDateFormat df = new SimpleDateFormat(ExportLogCommand.ONLY_DATE_FORMAT);
         return df.parse(dateString).toInstant().atZone(ZoneId.systemDefault()).toLocalDateTime();
-      } catch (ParseException e) {
-        try {
-          SimpleDateFormat df = new SimpleDateFormat(MiscellaneousCommands.ONLY_DATE_FORMAT);
-          return df.parse(dateString).toInstant().atZone(ZoneId.systemDefault()).toLocalDateTime();
-        } catch (ParseException e1) {
-          return null;
-        }
+      } catch (ParseException e1) {
+        return null;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/LogFileFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/LogFileFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/LogFileFunction.java
deleted file mode 100644
index 41ffeb4..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/LogFileFunction.java
+++ /dev/null
@@ -1,315 +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.functions;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.GemFireIOException;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionContext;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.internal.InternalEntity;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.management.internal.MBeanJMXAdapter;
-import org.apache.geode.management.internal.cli.GfshParser;
-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.ReadWriteFile;
-
-// TODO:LOG:CONVERT: supports log-file only -- update to support Log4J 2 as well?
-/**
- * 
- * @since GemFire 7.0
- */
-public class LogFileFunction implements Function, InternalEntity {
-  private static final Logger logger = LogService.getLogger();
-
-  public static final String ID = LogFileFunction.class.getName();
-  private static final long serialVersionUID = 1L;
-
-  @Override
-  public void execute(final FunctionContext context) {
-
-    Thread waiting = new Thread(new Runnable() {
-      public void run() {
-        final boolean isDebugEnabled = logger.isDebugEnabled();
-        try {
-          Cache cache = CacheFactory.getAnyInstance();
-          if (isDebugEnabled) {
-            logger.debug("Exporting logs LogFileFunction");
-          }
-          Object[] args = (Object[]) context.getArguments();
-          String targetDirName = ((String) args[0]);
-          String logLevel = ((String) args[1]);
-          String onlyLogLevel = ((String) args[2]);
-          int numOfLogFilesForTesting = ((Number) args[5]).intValue();
-
-          InternalDistributedSystem ds = (InternalDistributedSystem) cache.getDistributedSystem();
-          if (ds != null && ds.isConnected()) {
-            // write all the file in the same directory with extension .log
-            String filterString = ds.getConfig().getLogFile().getName();
-            final String filterStr = filterString.substring(0, filterString.lastIndexOf(".") > 0
-                ? filterString.lastIndexOf(".") : filterString.length() - 1);
-
-            File dir = ds.getConfig().getLogFile(); // get log file object
-            if (dir == null) {
-              context.getResultSender()
-                  .lastResult(CliStrings.format(
-                      CliStrings.EXPORT_LOGS__MSG__FAILED_TO_EXPORT_LOG_FILES_FOR_MEMBER_0,
-                      ds.getMemberId()));
-              return;
-            }
-            try {
-              dir = dir.getAbsoluteFile(); // get absolute log file
-            } catch (SecurityException se) {
-              context.getResultSender().lastResult(se.getMessage());
-              return;
-            }
-            String logFileDir = dir.getParent(); // get log file directory
-            if (logFileDir == null)
-              logFileDir = "/"; // this works in Windows too
-            if (isDebugEnabled) {
-              logger.debug(
-                  "For member={}: Exporting logs LogFileFunction logFileDir={}, filterStr={}",
-                  ds.getMemberId(), logFileDir, filterStr);
-            }
-
-            dir = new File(logFileDir); // get log file directory object
-            FilenameFilter filter = new FilenameFilter() {
-              public boolean accept(File dir, String name) {
-                // get those files which start with the name of the log file
-                return name.startsWith(filterStr) && name.endsWith(".log");
-              }
-            };
-            // create target directory if does not exists
-            if (isDebugEnabled) {
-              logger.debug("For member={}: Exporting logs LogFileFunction targetDirName={}",
-                  ds.getMemberId(), targetDirName);
-            }
-            File targetDir = new File(targetDirName);
-            if (targetDir.exists() == false) {
-              if (targetDir.mkdirs() == true) {
-                String logsWritten = processLogs(dir, logFileDir, targetDirName, cache, logLevel,
-                    onlyLogLevel, filter, ((Number) args[3]).toString(),
-                    ((Number) args[4]).toString(), numOfLogFilesForTesting);
-                if (isDebugEnabled) {
-                  logger.debug(
-                      "For member={}: Done with Exporting logs LogFileFunction targetDirName={}",
-                      ds.getMemberId(), targetDirName);
-                }
-                context.getResultSender().lastResult(logsWritten);
-              } else {
-                if (isDebugEnabled) {
-                  logger.debug("For member={}{} {}", ds.getMemberId(),
-                      CliStrings.EXPORT_LOGS__MSG__TARGET_DIR_CANNOT_BE_CREATED, targetDirName);
-                }
-                context.getResultSender().lastResult(ResultBuilder.createInfoResult(CliStrings
-                    .format(CliStrings.EXPORT_LOGS__MSG__TARGET_DIR_CANNOT_BE_CREATED, targetDir)));
-              }
-            } else {
-              String logsWritten = processLogs(dir, logFileDir, targetDirName, cache, logLevel,
-                  onlyLogLevel, filter, ((Number) args[3]).toString(),
-                  ((Number) args[4]).toString(), numOfLogFilesForTesting);
-              if (isDebugEnabled) {
-                logger.debug(
-                    "For member={}: Done with Exporting logs LogFileFunction targetDirName={} logsWritten={}",
-                    ds.getMemberId(), targetDirName, logsWritten);
-              }
-              context.getResultSender().lastResult(logsWritten);
-            }
-          } else {
-            context.getResultSender().lastResult(
-                LocalizedStrings.InternalDistributedSystem_THIS_CONNECTION_TO_A_DISTRIBUTED_SYSTEM_HAS_BEEN_DISCONNECTED
-                    .toLocalizedString());
-          }
-
-        } catch (Exception e) {
-          context.getResultSender().lastResult(e.getMessage());
-        }
-      }
-    });
-    try {
-      final boolean isDebugEnabled = logger.isDebugEnabled();
-      if (isDebugEnabled) {
-        logger.debug("For member={}: started copying log files",
-            ((InternalDistributedSystem) CacheFactory.getAnyInstance().getDistributedSystem())
-                .getMemberId());
-      }
-      waiting.start();
-      waiting.join();
-      if (isDebugEnabled) {
-        logger.debug("For member={}: done with Exporting all log files",
-            ((InternalDistributedSystem) CacheFactory.getAnyInstance().getDistributedSystem())
-                .getMemberId());
-      }
-    } catch (Exception e) {
-      context.getResultSender().lastResult(e.getMessage());
-    } finally {
-      if (waiting.isAlive()) {
-        waiting.interrupt();
-      }
-    }
-  }
-
-  public String processLogs(File dir, String logFileDir, String targetDirName, Cache cache,
-      String logLevel, String onlyLogLevel, FilenameFilter filter, String startTime, String endTime,
-      int numOfLogFilesForTesting) {
-    try {
-      final boolean isDebugEnabled = logger.isDebugEnabled();
-      String[] logsInDir = dir.list(filter);
-      if (isDebugEnabled) {
-        logger.debug("LogFileFunction  processLogs logsInDir={} sample={}", logsInDir.length,
-            logsInDir[0]);
-      }
-      StringBuilder logsWritten = new StringBuilder();
-      if (logsInDir.length > 0) {
-        SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss-ms");
-        // numOfLogFilesForTesting is used to limit the files that get copied while running entire
-        // dunit suite
-        // from user perspective numOfLogFilesForTesting is of no use
-        if (isDebugEnabled) {
-          logger.debug("LogFileFunction  before copy  numOfLogFilesForTesting={}",
-              numOfLogFilesForTesting);
-        }
-
-        for (int i = 0, j = 0; i < logsInDir.length
-            && (j < numOfLogFilesForTesting || numOfLogFilesForTesting == 0); i++, j++) {
-          String fileName = new String(logFileDir + File.separator + logsInDir[i]);
-
-          String logToBeWritten = targetDirName + File.separator
-              + MBeanJMXAdapter
-                  .getMemberNameOrId(cache.getDistributedSystem().getDistributedMember())
-              + "_" + logsInDir[i].substring(0, logsInDir[i].length() - 4) + "_"
-              + sdf.format(new java.util.Date())
-              + logsInDir[i].substring(logsInDir[i].length() - 4, logsInDir[i].length());
-
-          // create a new process for log read and write
-          if (isDebugEnabled) {
-            logger.debug("LogFileFunction  processLogs fileName={} logToBeWritten={}", fileName,
-                logToBeWritten);
-          }
-          List<String> commandList = new ArrayList<String>();
-          commandList.add(System.getProperty("java.home") + File.separatorChar + "bin"
-              + File.separatorChar + "java");
-          commandList.add("-classpath");
-          commandList.add(System.getProperty("java.class.path", "."));
-          commandList.add(ReadWriteFile.class.getName());
-          commandList.add(fileName);
-          commandList.add(logToBeWritten);
-          commandList.add(logLevel);
-          commandList.add(onlyLogLevel);
-          commandList.add(startTime);
-          commandList.add(endTime);
-          ProcessBuilder procBuilder = new ProcessBuilder(commandList);
-
-          StringBuilder output = new StringBuilder();
-          String errorString = new String(), resultString = new String();
-          try {
-            Process copyLogProcess = procBuilder.redirectErrorStream(true).start();
-            if (isDebugEnabled) {
-              logger.debug("LogFileFunction  processLogs fileName before process waitfor");
-            }
-            int compacterProcessStatus = copyLogProcess.waitFor();
-            if (isDebugEnabled) {
-              logger.debug(
-                  "LogFileFunction  processLogs fileName after process waitfor destroy compacterProcessStatus={}",
-                  compacterProcessStatus);
-            }
-            InputStream inputStream = copyLogProcess.getInputStream();
-            BufferedReader br = new BufferedReader(new InputStreamReader(inputStream));
-            String line = null;
-
-            while ((line = br.readLine()) != null) {
-              output.append(line).append(GfshParser.LINE_SEPARATOR);
-            }
-            copyLogProcess.destroy();
-            if (isDebugEnabled) {
-              logger.debug(
-                  "LogFileFunction  processLogs fileName after process waitfor after destroy compacterProcessStatus={}",
-                  compacterProcessStatus);
-            }
-
-          } catch (IOException e) {
-            errorString =
-                (new GemFireIOException(" Exception in LogFileFunction is " + e, e)).toString();
-          } finally {
-            if (errorString != null) {
-              output.append(errorString).append(GfshParser.LINE_SEPARATOR);
-            }
-            resultString = output.toString();
-          }
-          // merge files which are written successfully
-          if (resultString.contains("Sucessfully written file")) {
-            if (isDebugEnabled) {
-              logger.debug("LogFileFunction wrote file logToBeWritten={}", logToBeWritten);
-            }
-            logsWritten.append(logToBeWritten + ";");
-          } else {
-            if (isDebugEnabled) {
-              logger.debug("LogFileFunction wrote file logToBeWritten={} resultString={}",
-                  logToBeWritten, resultString);
-            }
-          }
-        }
-
-      } else {
-        if (isDebugEnabled) {
-          logger
-              .debug(CliStrings.format("No file was found for exporting in export logs function"));
-        }
-      }
-      if (isDebugEnabled) {
-        logger.debug(CliStrings.format("logsWritten=" + logsWritten));
-      }
-      return logsWritten.toString();
-    } catch (Exception e) {
-      return ("Exception in LogFileFunction processLogs " + e.getMessage());
-    }
-  }
-
-  @Override
-  public String getId() {
-    return LogFileFunction.ID;
-  }
-
-  @Override
-  public boolean hasResult() {
-    return true;
-  }
-
-  @Override
-  public boolean optimizeForWrite() {
-    // no need of optimization since read-only.
-    return false;
-  }
-
-  @Override
-  public boolean isHA() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index 3dc42d6..7696aa8 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -1401,7 +1401,7 @@ public class CliStrings {
   public static final String EXPORT_LOGS__MSG__CANNOT_EXECUTE = "Cannot execute";
   public static final String EXPORT_LOGS__LOGLEVEL = LOG_LEVEL;
   public static final String EXPORT_LOGS__LOGLEVEL__HELP =
-      "Minimum level of log entries to export. Valid values are: none, error, info, config , fine, finer and finest.  The default is \"info\".";
+      "Minimum level of log entries to export. Valid values are: fatal, error, warn, info, debug, trace and all.  The default is \"INFO\".";
   public static final String EXPORT_LOGS__UPTO_LOGLEVEL = "only-log-level";
   public static final String EXPORT_LOGS__UPTO_LOGLEVEL__HELP =
       "Whether to only include those entries that exactly match the --log-level specified.";
@@ -1430,6 +1430,10 @@ public class CliStrings {
       "Groups specified have no members";
   public static final String EXPORT_LOGS__MSG__FAILED_TO_EXPORT_LOG_FILES_FOR_MEMBER_0 =
       "Could not export log files for member {0}";
+  public static final String EXPORT_LOGS__LOGSONLY = "logs-only";
+  public static final String EXPORT_LOGS__STATSONLY = "stats-only";
+  public static final String EXPORT_LOGS__LOGSONLY__HELP = "Whether to only export logs";
+  public static final String EXPORT_LOGS__STATSONLY__HELP = "Whether to only export statistics";
 
   /* export stack-trace command */
   public static final String EXPORT_STACKTRACE = "export stack-traces";

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/RemoteExecutionStrategy.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/RemoteExecutionStrategy.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/RemoteExecutionStrategy.java
index 4b827a9..fa0f3b2 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/RemoteExecutionStrategy.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/remote/RemoteExecutionStrategy.java
@@ -97,7 +97,7 @@ public class RemoteExecutionStrategy {
         }
 
         if (interceptor != null) {
-          Result postExecResult = interceptor.postExecution(gfshParseResult, result);
+          Result postExecResult = interceptor.postExecution(gfshParseResult, result, null);
           if (postExecResult != null) {
             if (Status.ERROR.equals(postExecResult.getStatus())) {
               logWrapper.warning(postExecResult.toString(), null);

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/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 6445454..ad78efd 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
@@ -115,11 +115,6 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
       } else {
         logWrapper.warning(e.getMessage());
       }
-    } catch (RuntimeException e) {
-      Gfsh.getCurrentInstance().logWarning("Exception occurred. " + e.getMessage(), e);
-      // Log other runtime exception in gfsh log
-      logWrapper.warning("Error occurred while executing command : "
-          + ((GfshParseResult) parseResult).getUserInput(), e);
     } catch (Exception e) {
       Gfsh.getCurrentInstance().logWarning("Unexpected exception occurred. " + e.getMessage(), e);
       // Log other exceptions in gfsh log

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/ExportLogsCacheWriter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/ExportLogsCacheWriter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/ExportLogsCacheWriter.java
index 178bb4a..f6e42ab 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/ExportLogsCacheWriter.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/ExportLogsCacheWriter.java
@@ -30,6 +30,7 @@ import java.nio.file.Path;
 
 public class ExportLogsCacheWriter extends CacheWriterAdapter implements Serializable {
   private Path currentFile;
+  private boolean isEmpty = true;
   private BufferedOutputStream currentOutputStream;
 
   @Override
@@ -46,6 +47,7 @@ public class ExportLogsCacheWriter extends CacheWriterAdapter implements Seriali
             "Value must be a byte[].  Recieved " + newValue.getClass().getCanonicalName());
       }
       currentOutputStream.write((byte[]) newValue);
+      isEmpty = false;
     } catch (IOException e) {
       throw new CacheWriterException(e);
     }
@@ -58,6 +60,7 @@ public class ExportLogsCacheWriter extends CacheWriterAdapter implements Seriali
 
     currentFile = Files.createTempDirectory(memberId).resolve(memberId + ".zip");
     currentOutputStream = new BufferedOutputStream(new FileOutputStream(currentFile.toFile()));
+    isEmpty = true;
   }
 
   public Path endFile() {
@@ -66,7 +69,8 @@ public class ExportLogsCacheWriter extends CacheWriterAdapter implements Seriali
     IOUtils.closeQuietly(currentOutputStream);
     currentOutputStream = null;
     currentFile = null;
-
+    if (isEmpty)
+      return null;
     return completedFile;
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogExporter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogExporter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogExporter.java
index f30d686..a4bef75 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogExporter.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogExporter.java
@@ -19,62 +19,76 @@ package org.apache.geode.management.internal.cli.util;
 import static java.util.stream.Collectors.toList;
 
 import org.apache.commons.io.FileUtils;
-import org.apache.geode.cache.AttributesFactory;
-import org.apache.geode.cache.DataPolicy;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.Scope;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.InternalRegionArguments;
 import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.management.internal.configuration.domain.Configuration;
 import org.apache.geode.management.internal.configuration.utils.ZipUtils;
 import org.apache.logging.log4j.Logger;
 
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
+import java.io.File;
 import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.text.ParseException;
 import java.util.List;
 import java.util.function.Predicate;
 import java.util.stream.Stream;
 
+/**
+ * this LogExporter only finds the .log and .gfs files under in the same directory of the base files
+ * it doesn't use the base file's filename patterns to find the related logs/stats yet.
+ */
 public class LogExporter {
   private static final Logger LOGGER = LogService.getLogger();
 
   private final LogFilter logFilter;
-
-  public LogExporter(LogFilter logFilter) throws ParseException {
+  private final File baseLogFile;
+  private final File baseStatsFile;
+
+  /**
+   * @param logFilter: the filter that's used to check if we need to accept the file or the logLine
+   * @param baseLogFile: if not null, we will export the logs in that directory
+   * @param baseStatsFile: if not null, we will export stats in that directory
+   * @throws ParseException
+   */
+  public LogExporter(LogFilter logFilter, File baseLogFile, File baseStatsFile)
+      throws ParseException {
+    assert logFilter != null;
     this.logFilter = logFilter;
+    this.baseLogFile = baseLogFile;
+    this.baseStatsFile = baseStatsFile;
   }
 
-  public Path export(Path workingDir) throws IOException {
-    LOGGER.debug("Working directory is {}", workingDir);
-
+  /**
+   *
+   * @return Path to the zip file that has all the filtered files, null if no files are selected to
+   *         export.
+   * @throws IOException
+   */
+  public Path export() throws IOException {
     Path tempDirectory = Files.createTempDirectory("exportLogs");
 
-    for (Path logFile : findLogFiles(workingDir)) {
-      Path filteredLogFile = tempDirectory.resolve(logFile.getFileName());
-
-      if (this.logFilter == null) {
-        Files.copy(logFile, filteredLogFile);
-      } else {
+    if (baseLogFile != null) {
+      for (Path logFile : findLogFiles(baseLogFile.toPath().getParent())) {
+        Path filteredLogFile = tempDirectory.resolve(logFile.getFileName());
         writeFilteredLogFile(logFile, filteredLogFile);
       }
     }
 
-    for (Path statFile : findStatFiles(workingDir)) {
-      Files.copy(statFile, tempDirectory);
+    if (baseStatsFile != null) {
+      for (Path statFile : findStatFiles(baseStatsFile.toPath().getParent())) {
+        Files.copy(statFile, tempDirectory.resolve(statFile.getFileName()));
+      }
     }
 
-    Path zipFile = Files.createTempFile("logExport", ".zip");
-    ZipUtils.zipDirectory(tempDirectory, zipFile);
-    LOGGER.info("Zipped files to: " + zipFile);
-
+    Path zipFile = null;
+    if (tempDirectory.toFile().listFiles().length > 0) {
+      zipFile = Files.createTempFile("logExport", ".zip");
+      ZipUtils.zipDirectory(tempDirectory, zipFile);
+      LOGGER.info("Zipped files to: " + zipFile);
+    }
 
     FileUtils.deleteDirectory(tempDirectory.toFile());
 

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogFilter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogFilter.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogFilter.java
index 2436530..0da6ddf 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogFilter.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogFilter.java
@@ -16,19 +16,16 @@
 
 package org.apache.geode.management.internal.cli.util;
 
-import static java.util.stream.Collectors.toSet;
-
-import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.logging.LogService;
+import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
 
-import java.io.IOException;
+import java.nio.file.Files;
 import java.nio.file.Path;
+import java.nio.file.attribute.BasicFileAttributes;
 import java.time.Instant;
 import java.time.LocalDateTime;
 import java.time.ZoneId;
-import java.util.Arrays;
-import java.util.Set;
 
 public class LogFilter {
   public enum LineFilterResult {
@@ -37,15 +34,22 @@ public class LogFilter {
 
   private static final Logger LOGGER = LogService.getLogger();
 
-  private final Set<String> permittedLogLevels;
+  private final Level thisLogLevel;
+  private final boolean thisLevelOnly;
   private final LocalDateTime startDate;
   private final LocalDateTime endDate;
 
   private LineFilterResult resultOfPreviousLine = LineFilterResult.LINE_ACCEPTED;
 
-  public LogFilter(Set<String> permittedLogLevels, LocalDateTime startDate, LocalDateTime endDate) {
-    this.permittedLogLevels = (permittedLogLevels == null || permittedLogLevels.isEmpty())
-        ? allLogLevels() : permittedLogLevels;
+  public LogFilter(Level logLevel, LocalDateTime startDate, LocalDateTime endDate) {
+    this(logLevel, false, startDate, endDate);
+  }
+
+  public LogFilter(Level logLevel, boolean thisLevelOnly, LocalDateTime startDate,
+      LocalDateTime endDate) {
+    assert logLevel != null;
+    this.thisLogLevel = logLevel;
+    this.thisLevelOnly = thisLevelOnly;
     this.startDate = startDate;
     this.endDate = endDate;
   }
@@ -68,7 +72,7 @@ public class LogFilter {
     return acceptsLogEntry(result.getLogLevel(), result.getLogTimestamp());
   }
 
-  protected LineFilterResult acceptsLogEntry(String logLevel, LocalDateTime logTimestamp) {
+  protected LineFilterResult acceptsLogEntry(Level logLevel, LocalDateTime logTimestamp) {
     if (logTimestamp == null || logLevel == null) {
       throw new IllegalArgumentException();
     }
@@ -80,8 +84,13 @@ public class LogFilter {
     } else if (startDate != null && logTimestamp.isBefore(startDate)) {
       result = LineFilterResult.LINE_REJECTED;
     } else {
-      result = permittedLogLevels.contains(logLevel) ? LineFilterResult.LINE_ACCEPTED
-          : LineFilterResult.LINE_REJECTED;
+      if (thisLevelOnly) {
+        result = logLevel.intLevel() == thisLogLevel.intLevel() ? LineFilterResult.LINE_ACCEPTED
+            : LineFilterResult.LINE_REJECTED;
+      } else {
+        result = logLevel.isMoreSpecificThan(thisLogLevel) ? LineFilterResult.LINE_ACCEPTED
+            : LineFilterResult.LINE_REJECTED;
+      }
     }
 
     resultOfPreviousLine = result;
@@ -90,24 +99,42 @@ public class LogFilter {
   }
 
   public boolean acceptsFile(Path file) {
-    if (startDate == null) {
+    if (startDate == null && endDate == null) {
       return true;
     }
-    try {
-      return (getEndTimeOf(file).isAfter(startDate));
-    } catch (IOException e) {
-      LOGGER.error("Unable to determine lastModified time", e);
-      return true;
+
+    if (endDate == null) {
+      return getEndTimeOf(file).isAfter(startDate);
+    }
+
+    if (startDate == null) {
+      return getStartTimeOf(file).isBefore(endDate);
     }
+
+    return (getEndTimeOf(file).isAfter(startDate) && getStartTimeOf(file).isBefore(endDate));
+
   }
 
-  private static LocalDateTime getEndTimeOf(Path file) throws IOException {
-    long lastModifiedMillis = file.toFile().lastModified();
-    return Instant.ofEpochMilli(lastModifiedMillis).atZone(ZoneId.systemDefault())
-        .toLocalDateTime();
+  private static LocalDateTime getEndTimeOf(Path file) {
+    try {
+      long lastModifiedMillis = file.toFile().lastModified();
+      return Instant.ofEpochMilli(lastModifiedMillis).atZone(ZoneId.systemDefault())
+          .toLocalDateTime();
+    } catch (Exception e) {
+      LOGGER.error("Unable to determine lastModified time", e);
+      return LocalDateTime.MAX;
+    }
   }
 
-  public static Set<String> allLogLevels() {
-    return Arrays.stream(InternalLogWriter.levelNames).collect(toSet());
+  private static LocalDateTime getStartTimeOf(Path file) {
+    try {
+      BasicFileAttributes attributes = Files.readAttributes(file, BasicFileAttributes.class);
+      long lastModifiedMillis = attributes.creationTime().toMillis();
+      return Instant.ofEpochMilli(lastModifiedMillis).atZone(ZoneId.systemDefault())
+          .toLocalDateTime();
+    } catch (Exception e) {
+      LOGGER.error("Unable to determine creation time", e);
+      return LocalDateTime.MIN;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogLevelExtractor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogLevelExtractor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogLevelExtractor.java
index 6c15d15..bbeb5c7 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogLevelExtractor.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/util/LogLevelExtractor.java
@@ -16,11 +16,19 @@
 
 package org.apache.geode.management.internal.cli.util;
 
+import org.apache.logging.log4j.Level;
+
 import java.time.LocalDateTime;
 import java.time.format.DateTimeFormatter;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+/**
+ * this will extract convert the deprecated InternalLogWriter's level into log4j level as well.
+ */
 public class LogLevelExtractor {
   private static Pattern LOG_PATTERN =
       Pattern.compile("^\\[(\\S*)\\s+([\\d\\/]+)\\s+([\\d:\\.]+)\\s+(\\S+)");
@@ -44,15 +52,15 @@ public class LogLevelExtractor {
   }
 
   public static class Result {
-    private String logLevel;
+    private Level logLevel;
     private LocalDateTime logTimestamp;
 
     public Result(String logLevel, LocalDateTime logTimestamp) {
-      this.logLevel = logLevel;
+      this.logLevel = LogLevelExtractor.getLevel(logLevel);
       this.logTimestamp = logTimestamp;
     }
 
-    public String getLogLevel() {
+    public Level getLogLevel() {
       return logLevel;
     }
 
@@ -61,5 +69,26 @@ public class LogLevelExtractor {
     }
 
   }
+
+  private static Map<String, Level> LEVELS = new HashMap<>();
+  static {
+    // put all the log4j levels in the map first
+    Arrays.stream(Level.values()).forEach(level -> {
+      LEVELS.put(level.name(), level);
+    });
+    // put all the other levels geode has been using and map them to log4j levels
+    LEVELS.put("SEVERE", Level.FATAL);
+    LEVELS.put("WARNING", Level.WARN);
+    LEVELS.put("CONFIG", Level.DEBUG);
+    LEVELS.put("FINE", Level.DEBUG);
+    LEVELS.put("FINER", Level.TRACE);
+    LEVELS.put("FINEST", Level.TRACE);
+  }
+
+  public static Level getLevel(String level) {
+    Level log4jLevel = LEVELS.get(level.toUpperCase());
+    // make sure any unrecognizable log level is assigned a most specific level
+    return log4jLevel == null ? Level.OFF : log4jLevel;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/bf788176/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
new file mode 100644
index 0000000..f3a5934
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ExportLogController.java
@@ -0,0 +1,101 @@
+/*
+ * 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 org.apache.geode.internal.lang.StringUtils;
+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.ResponseEntity;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.RequestParam;
+
+import java.io.File;
+import java.io.FileInputStream;
+
+@Controller("exportLogController")
+@RequestMapping(AbstractCommandsController.REST_API_VERSION)
+public class ExportLogController extends AbstractCommandsController {
+
+  @RequestMapping(method = RequestMethod.GET, value = "/logs")
+  public ResponseEntity<InputStreamResource> exportLogs(
+      @RequestParam(value = CliStrings.EXPORT_LOGS__DIR, required = false) final String directory,
+      @RequestParam(value = CliStrings.EXPORT_LOGS__GROUP, required = false) final String[] groups,
+      @RequestParam(value = CliStrings.EXPORT_LOGS__MEMBER,
+          required = false) final String memberNameId,
+      @RequestParam(value = CliStrings.EXPORT_LOGS__LOGLEVEL,
+          required = false) final String logLevel,
+      @RequestParam(value = CliStrings.EXPORT_LOGS__UPTO_LOGLEVEL,
+          defaultValue = "false") final Boolean onlyLogLevel,
+      @RequestParam(value = CliStrings.EXPORT_LOGS__MERGELOG,
+          defaultValue = "false") final Boolean mergeLog,
+      @RequestParam(value = CliStrings.EXPORT_LOGS__STARTTIME,
+          required = false) final String startTime,
+      @RequestParam(value = CliStrings.EXPORT_LOGS__ENDTIME, required = false) final String endTime,
+      @RequestParam(value = CliStrings.EXPORT_LOGS__LOGSONLY,
+          required = false) final boolean logsOnly,
+      @RequestParam(value = CliStrings.EXPORT_LOGS__STATSONLY,
+          required = false) final boolean statsOnly) {
+    final CommandStringBuilder command = new CommandStringBuilder(CliStrings.EXPORT_LOGS);
+
+    command.addOption(CliStrings.EXPORT_LOGS__DIR, decode(directory));
+
+    if (hasValue(groups)) {
+      command.addOption(CliStrings.EXPORT_LOGS__GROUP,
+          StringUtils.concat(groups, StringUtils.COMMA_DELIMITER));
+    }
+
+    if (hasValue(memberNameId)) {
+      command.addOption(CliStrings.EXPORT_LOGS__MEMBER, memberNameId);
+    }
+
+    if (hasValue(logLevel)) {
+      command.addOption(CliStrings.EXPORT_LOGS__LOGLEVEL, logLevel);
+    }
+
+    command.addOption(CliStrings.EXPORT_LOGS__UPTO_LOGLEVEL, String.valueOf(onlyLogLevel));
+    command.addOption(CliStrings.EXPORT_LOGS__MERGELOG, String.valueOf(mergeLog));
+    command.addOption(CliStrings.EXPORT_LOGS__LOGSONLY, String.valueOf(logsOnly));
+    command.addOption(CliStrings.EXPORT_LOGS__STATSONLY, String.valueOf(statsOnly));
+
+    if (hasValue(startTime)) {
+      command.addOption(CliStrings.EXPORT_LOGS__STARTTIME, startTime);
+    }
+
+    if (hasValue(endTime)) {
+      command.addOption(CliStrings.EXPORT_LOGS__ENDTIME, endTime);
+    }
+
+    // the result is json string from CommandResult
+    String result = processCommand(command.toString());
+
+    // 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();
+
+    HttpHeaders respHeaders = new HttpHeaders();
+    try {
+      InputStreamResource isr = new InputStreamResource(new FileInputStream(new File(filePath)));
+      return new ResponseEntity<InputStreamResource>(isr, respHeaders, HttpStatus.OK);
+    } catch (Exception ex) {
+      throw new RuntimeException("IOError writing file to output stream", ex);
+    }
+  }
+}