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/08/11 23:05:59 UTC

[1/3] geode git commit: GEODE-3254: Refactoring ConfigCommands

Repository: geode
Updated Branches:
  refs/heads/develop bdc8de00f -> d27f8b956


GEODE-3254: Refactoring ConfigCommands

This closes #665


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

Branch: refs/heads/develop
Commit: 97c4e9a59f17c7bc914e39dd048b0a4cd96293c4
Parents: bdc8de0
Author: YehEmily <em...@gmail.com>
Authored: Wed Jul 26 11:07:09 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Aug 11 15:59:58 2017 -0700

----------------------------------------------------------------------
 .../org/apache/geode/BundledJarsJUnitTest.java  |  44 +-
 .../cli/commands/AlterRuntimeConfigCommand.java | 246 ++++++++++
 .../cli/commands/AlterRuntimeInterceptor.java   |  39 ++
 .../internal/cli/commands/ConfigCommands.java   | 489 -------------------
 .../cli/commands/DescribeConfigCommand.java     | 153 ++++++
 .../cli/commands/ExportConfigCommand.java       | 159 ++++++
 .../controllers/ConfigCommandsController.java   |  17 +-
 .../controllers/DeployCommandsController.java   |  13 +-
 .../cli/commands/LogLevelInterceptorTest.java   |   2 +-
 .../internal/security/TestCommand.java          |   2 +-
 10 files changed, 636 insertions(+), 528 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/geode-assembly/src/test/java/org/apache/geode/BundledJarsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/org/apache/geode/BundledJarsJUnitTest.java b/geode-assembly/src/test/java/org/apache/geode/BundledJarsJUnitTest.java
index 3f0e2c0..3a5538c 100644
--- a/geode-assembly/src/test/java/org/apache/geode/BundledJarsJUnitTest.java
+++ b/geode-assembly/src/test/java/org/apache/geode/BundledJarsJUnitTest.java
@@ -16,14 +16,6 @@ package org.apache.geode;
 
 import static org.junit.Assert.assertTrue;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.apache.geode.test.junit.categories.RestAPITest;
-import org.apache.geode.util.test.TestUtil;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
@@ -36,11 +28,20 @@ import java.util.jar.JarFile;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import org.apache.commons.io.FileUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.apache.geode.test.junit.categories.RestAPITest;
+import org.apache.geode.util.test.TestUtil;
+
 @Category({IntegrationTest.class, RestAPITest.class})
 public class BundledJarsJUnitTest {
 
   private static final String VERSION_PATTERN = "[0-9-_.v]{3,}.*\\.jar$";
-  protected static final String GEODE_HOME = System.getenv("GEODE_HOME");
+  private static final String GEODE_HOME = System.getenv("GEODE_HOME");
   private Set<String> expectedJars;
 
   @Before
@@ -65,27 +66,22 @@ public class BundledJarsJUnitTest {
     TreeSet<String> missingJars = new TreeSet<String>(expectedJars);
     missingJars.removeAll(bundledJarNames);
 
-    StringBuilder message = new StringBuilder();
-    message.append(
-        "The bundled jars have changed. Please make sure you update the licence and notice");
-    message.append(
-        "\nas described in https://cwiki.apache.org/confluence/display/GEODE/License+Guide+for+Contributors");
-    message.append("\nWhen fixed, copy geode-assembly/build/test/bundled_jars.txt");
-    message.append("\nto src/test/resources/expected_jars.txt");
-    message.append("\nRemoved Jars\n--------------\n");
-    message.append(String.join("\n", missingJars));
-    message.append("\n\nAdded Jars\n--------------\n");
-    message.append(String.join("\n", newJars));
-    message.append("\n\n");
-
-    assertTrue(message.toString(), expectedJars.equals(bundledJarNames));
+    String message =
+        "The bundled jars have changed. Please make sure you update the licence and notice"
+            + "\nas described in https://cwiki.apache.org/confluence/display/GEODE/License+Guide+for+Contributors"
+            + "\nWhen fixed, copy geode-assembly/build/test/bundled_jars.txt"
+            + "\nto src/test/resources/expected_jars.txt" + "\nRemoved Jars\n--------------\n"
+            + String.join("\n", missingJars) + "\n\nAdded Jars\n--------------\n"
+            + String.join("\n", newJars) + "\n\n";
+
+    assertTrue(message, expectedJars.equals(bundledJarNames));
 
   }
 
   /**
    * Find all of the jars bundled with the project. Key is the name of the jar, value is the path.
    */
-  protected TreeMap<String, String> getBundledJars() {
+  private TreeMap<String, String> getBundledJars() {
     File geodeHomeDirectory = new File(GEODE_HOME);
 
     assertTrue(

http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeConfigCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeConfigCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeConfigCommand.java
new file mode 100644
index 0000000..4415968
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeConfigCommand.java
@@ -0,0 +1,246 @@
+/*
+ * 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.STATISTIC_SAMPLING_ENABLED;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.Logger;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.cache.xmlcache.CacheXml;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.logging.log4j.LogLevel;
+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.AlterRuntimeConfigFunction;
+import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.configuration.domain.XmlEntity;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class AlterRuntimeConfigCommand implements GfshCommand {
+  private final AlterRuntimeConfigFunction alterRunTimeConfigFunction =
+      new AlterRuntimeConfigFunction();
+  private static Logger logger = LogService.getLogger();
+
+  @CliCommand(value = {CliStrings.ALTER_RUNTIME_CONFIG},
+      help = CliStrings.ALTER_RUNTIME_CONFIG__HELP)
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG},
+      interceptor = "org.apache.geode.management.internal.cli.commands.AlterRuntimeConfigCommand$AlterRuntimeInterceptor")
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.MANAGE)
+  public Result alterRuntimeConfig(
+      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
+          optionContext = ConverterHint.ALL_MEMBER_IDNAME,
+          help = CliStrings.ALTER_RUNTIME_CONFIG__MEMBER__HELP) String[] memberNameOrId,
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          optionContext = ConverterHint.MEMBERGROUP,
+          help = CliStrings.ALTER_RUNTIME_CONFIG__GROUP__HELP) String[] group,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT__HELP) Integer archiveDiskSpaceLimit,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT__HELP) Integer archiveFileSizeLimit,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT__HELP) Integer logDiskSpaceLimit,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT__HELP) Integer logFileSizeLimit,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL},
+          optionContext = ConverterHint.LOG_LEVEL,
+          help = CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL__HELP) String logLevel,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE__HELP) String statisticArchiveFile,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE__HELP) Integer statisticSampleRate,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLING__ENABLED},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLING__ENABLED__HELP) Boolean statisticSamplingEnabled,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ},
+          specifiedDefaultValue = "false",
+          help = CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ__HELP) Boolean setCopyOnRead,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE__HELP) Integer lockLease,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT__HELP) Integer lockTimeout,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL__HELP) Integer messageSyncInterval,
+      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT},
+          help = CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT__HELP) Integer searchTimeout) {
+
+    Map<String, String> runTimeDistributionConfigAttributes = new HashMap<>();
+    Map<String, String> rumTimeCacheAttributes = new HashMap<>();
+    Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
+
+    if (targetMembers.isEmpty()) {
+      return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+    }
+
+    if (archiveDiskSpaceLimit != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT,
+          archiveDiskSpaceLimit.toString());
+    }
+
+    if (archiveFileSizeLimit != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT,
+          archiveFileSizeLimit.toString());
+    }
+
+    if (logDiskSpaceLimit != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, logDiskSpaceLimit.toString());
+    }
+
+    if (logFileSizeLimit != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT, logFileSizeLimit.toString());
+    }
+
+    if (logLevel != null && !logLevel.isEmpty()) {
+      runTimeDistributionConfigAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL,
+          logLevel);
+    }
+
+    if (statisticArchiveFile != null && !statisticArchiveFile.isEmpty()) {
+      runTimeDistributionConfigAttributes
+          .put(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE, statisticArchiveFile);
+    }
+
+    if (statisticSampleRate != null) {
+      runTimeDistributionConfigAttributes.put(
+          CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE, statisticSampleRate.toString());
+    }
+
+    if (statisticSamplingEnabled != null) {
+      runTimeDistributionConfigAttributes.put(STATISTIC_SAMPLING_ENABLED,
+          statisticSamplingEnabled.toString());
+    }
+
+
+    // Attributes that are set on the cache.
+    if (setCopyOnRead != null) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ,
+          setCopyOnRead.toString());
+    }
+
+    if (lockLease != null && lockLease > 0 && lockLease < Integer.MAX_VALUE) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE,
+          lockLease.toString());
+    }
+
+    if (lockTimeout != null && lockTimeout > 0 && lockTimeout < Integer.MAX_VALUE) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT,
+          lockTimeout.toString());
+    }
+
+    if (messageSyncInterval != null && messageSyncInterval > 0
+        && messageSyncInterval < Integer.MAX_VALUE) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL,
+          messageSyncInterval.toString());
+    }
+
+    if (searchTimeout != null && searchTimeout > 0 && searchTimeout < Integer.MAX_VALUE) {
+      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT,
+          searchTimeout.toString());
+    }
+
+    if (runTimeDistributionConfigAttributes.isEmpty() && rumTimeCacheAttributes.isEmpty()) {
+      return ResultBuilder
+          .createUserErrorResult(CliStrings.ALTER_RUNTIME_CONFIG__RELEVANT__OPTION__MESSAGE);
+    }
+
+    Map<String, String> allRunTimeAttributes = new HashMap<>();
+    allRunTimeAttributes.putAll(runTimeDistributionConfigAttributes);
+    allRunTimeAttributes.putAll(rumTimeCacheAttributes);
+
+    ResultCollector<?, ?> rc =
+        CliUtil.executeFunction(alterRunTimeConfigFunction, allRunTimeAttributes, targetMembers);
+    List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
+    Set<String> successfulMembers = new TreeSet<>();
+    Set<String> errorMessages = new TreeSet<>();
+
+    for (CliFunctionResult result : results) {
+      if (result.getThrowable() != null) {
+        logger.info("Function failed: " + result.getThrowable());
+        errorMessages.add(result.getThrowable().getMessage());
+      } else {
+        successfulMembers.add(result.getMemberIdOrName());
+      }
+    }
+    final String lineSeparator = System.getProperty("line.separator");
+    if (!successfulMembers.isEmpty()) {
+      StringBuilder successMessageBuilder = new StringBuilder();
+
+      successMessageBuilder.append(CliStrings.ALTER_RUNTIME_CONFIG__SUCCESS__MESSAGE);
+      successMessageBuilder.append(lineSeparator);
+
+      for (String member : successfulMembers) {
+        successMessageBuilder.append(member);
+        successMessageBuilder.append(lineSeparator);
+      }
+
+      Properties properties = new Properties();
+      properties.putAll(runTimeDistributionConfigAttributes);
+
+      Result result = ResultBuilder.createInfoResult(successMessageBuilder.toString());
+
+      // Set the Cache attributes to be modified
+      final XmlEntity xmlEntity = XmlEntity.builder().withType(CacheXml.CACHE)
+          .withAttributes(rumTimeCacheAttributes).build();
+      persistClusterConfiguration(result,
+          () -> getSharedConfiguration().modifyXmlAndProperties(properties, xmlEntity, group));
+      return result;
+    } else {
+      StringBuilder errorMessageBuilder = new StringBuilder();
+      errorMessageBuilder.append("Following errors occurred while altering runtime config");
+      errorMessageBuilder.append(lineSeparator);
+
+      for (String errorMessage : errorMessages) {
+        errorMessageBuilder.append(errorMessage);
+        errorMessageBuilder.append(lineSeparator);
+      }
+      return ResultBuilder.createUserErrorResult(errorMessageBuilder.toString());
+    }
+  }
+
+  public static class AlterRuntimeInterceptor extends AbstractCliAroundInterceptor {
+    @Override
+    public Result preExecution(GfshParseResult parseResult) {
+      Map<String, String> arguments = parseResult.getParamValueStrings();
+      // validate log level
+      String logLevel = arguments.get("log-level");
+      if (StringUtils.isNotBlank(logLevel) && (LogLevel.getLevel(logLevel) == null)) {
+        return ResultBuilder.createUserErrorResult("Invalid log level: " + logLevel);
+      }
+      return ResultBuilder.createInfoResult("");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeInterceptor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeInterceptor.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeInterceptor.java
new file mode 100644
index 0000000..37d4dbb
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterRuntimeInterceptor.java
@@ -0,0 +1,39 @@
+/*
+ * 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 java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+
+import org.apache.geode.internal.logging.log4j.LogLevel;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.AbstractCliAroundInterceptor;
+import org.apache.geode.management.internal.cli.GfshParseResult;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+
+public class AlterRuntimeInterceptor extends AbstractCliAroundInterceptor {
+  @Override
+  public Result preExecution(GfshParseResult parseResult) {
+    Map<String, String> arguments = parseResult.getParamValueStrings();
+    // validate log level
+    String logLevel = arguments.get("log-level");
+    if (StringUtils.isNotBlank(logLevel) && (LogLevel.getLevel(logLevel) == null)) {
+      return ResultBuilder.createUserErrorResult("Invalid log level: " + logLevel);
+    }
+    return ResultBuilder.createInfoResult("");
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/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
deleted file mode 100644
index 40450c0..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ConfigCommands.java
+++ /dev/null
@@ -1,489 +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.STATISTIC_SAMPLING_ENABLED;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.logging.log4j.Logger;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import org.apache.geode.SystemFailure;
-import org.apache.geode.cache.execute.FunctionInvocationTargetException;
-import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.internal.cache.xmlcache.CacheXml;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.log4j.LogLevel;
-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.domain.MemberConfigurationInfo;
-import org.apache.geode.management.internal.cli.functions.AlterRuntimeConfigFunction;
-import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
-import org.apache.geode.management.internal.cli.functions.ExportConfigFunction;
-import org.apache.geode.management.internal.cli.functions.GetMemberConfigInformationFunction;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.CompositeResultData;
-import org.apache.geode.management.internal.cli.result.CompositeResultData.SectionResultData;
-import org.apache.geode.management.internal.cli.result.ErrorResultData;
-import org.apache.geode.management.internal.cli.result.InfoResultData;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.result.TabularResultData;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.configuration.domain.XmlEntity;
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-/****
- * @since GemFire 7.0
- *
- */
-public class ConfigCommands implements GfshCommand {
-  private final ExportConfigFunction exportConfigFunction = new ExportConfigFunction();
-  private final GetMemberConfigInformationFunction getMemberConfigFunction =
-      new GetMemberConfigInformationFunction();
-  private final AlterRuntimeConfigFunction alterRunTimeConfigFunction =
-      new AlterRuntimeConfigFunction();
-  private static Logger logger = LogService.getLogger();
-
-  @CliCommand(value = {CliStrings.DESCRIBE_CONFIG}, help = CliStrings.DESCRIBE_CONFIG__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-  public Result describeConfig(
-      @CliOption(key = CliStrings.MEMBER, optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-          help = CliStrings.DESCRIBE_CONFIG__MEMBER__HELP, mandatory = true) String memberNameOrId,
-      @CliOption(key = CliStrings.DESCRIBE_CONFIG__HIDE__DEFAULTS,
-          help = CliStrings.DESCRIBE_CONFIG__HIDE__DEFAULTS__HELP, unspecifiedDefaultValue = "true",
-          specifiedDefaultValue = "true") boolean hideDefaults) {
-
-    Result result = null;
-    try {
-      DistributedMember targetMember = null;
-
-      if (memberNameOrId != null && !memberNameOrId.isEmpty()) {
-        targetMember = CliUtil.getDistributedMemberByNameOrId(memberNameOrId);
-      }
-      if (targetMember != null) {
-        ResultCollector<?, ?> rc = CliUtil.executeFunction(getMemberConfigFunction,
-            new Boolean(hideDefaults), targetMember);
-        ArrayList<?> output = (ArrayList<?>) rc.getResult();
-        Object obj = output.get(0);
-
-        if (obj != null && obj instanceof MemberConfigurationInfo) {
-          MemberConfigurationInfo memberConfigInfo = (MemberConfigurationInfo) obj;
-
-          CompositeResultData crd = ResultBuilder.createCompositeResultData();
-          crd.setHeader(
-              CliStrings.format(CliStrings.DESCRIBE_CONFIG__HEADER__TEXT, memberNameOrId));
-
-          List<String> jvmArgsList = memberConfigInfo.getJvmInputArguments();
-          TabularResultData jvmInputArgs = crd.addSection().addSection().addTable();
-
-          for (String jvmArg : jvmArgsList) {
-            jvmInputArgs.accumulate("JVM command line arguments", jvmArg);
-          }
-
-          addSection(crd, memberConfigInfo.getGfePropsSetUsingApi(),
-              "GemFire properties defined using the API");
-          addSection(crd, memberConfigInfo.getGfePropsRuntime(),
-              "GemFire properties defined at the runtime");
-          addSection(crd, memberConfigInfo.getGfePropsSetFromFile(),
-              "GemFire properties defined with the property file");
-          addSection(crd, memberConfigInfo.getGfePropsSetWithDefaults(),
-              "GemFire properties using default values");
-          addSection(crd, memberConfigInfo.getCacheAttributes(), "Cache attributes");
-
-          List<Map<String, String>> cacheServerAttributesList =
-              memberConfigInfo.getCacheServerAttributes();
-
-          if (cacheServerAttributesList != null && !cacheServerAttributesList.isEmpty()) {
-            SectionResultData cacheServerSection = crd.addSection();
-            cacheServerSection.setHeader("Cache-server attributes");
-
-            for (Map<String, String> cacheServerAttributes : cacheServerAttributesList) {
-              addSubSection(cacheServerSection, cacheServerAttributes, "");
-            }
-          }
-          result = ResultBuilder.buildResult(crd);
-        }
-
-      } else {
-        ErrorResultData erd = ResultBuilder.createErrorResultData();
-        erd.addLine(CliStrings.format(CliStrings.DESCRIBE_CONFIG__MEMBER__NOT__FOUND,
-            new Object[] {memberNameOrId}));
-        result = ResultBuilder.buildResult(erd);
-      }
-    } catch (FunctionInvocationTargetException e) {
-      result = ResultBuilder.createGemFireErrorResult(CliStrings
-          .format(CliStrings.COULD_NOT_EXECUTE_COMMAND_TRY_AGAIN, CliStrings.DESCRIBE_CONFIG));
-    } catch (Exception e) {
-      ErrorResultData erd = ResultBuilder.createErrorResultData();
-      erd.addLine(e.getMessage());
-      result = ResultBuilder.buildResult(erd);
-    }
-    return result;
-  }
-
-
-  private void addSection(CompositeResultData crd, Map<String, String> attrMap, String headerText) {
-    if (attrMap != null && !attrMap.isEmpty()) {
-      SectionResultData section = crd.addSection();
-      section.setHeader(headerText);
-      section.addSeparator('.');
-      Set<String> attributes = new TreeSet<>(attrMap.keySet());
-
-      for (String attribute : attributes) {
-        String attributeValue = attrMap.get(attribute);
-        section.addData(attribute, attributeValue);
-      }
-    }
-  }
-
-  private void addSubSection(SectionResultData section, Map<String, String> attrMap,
-      String headerText) {
-    if (!attrMap.isEmpty()) {
-      SectionResultData subSection = section.addSection();
-      Set<String> attributes = new TreeSet<>(attrMap.keySet());
-      subSection.setHeader(headerText);
-
-      for (String attribute : attributes) {
-        String attributeValue = attrMap.get(attribute);
-        subSection.addData(attribute, attributeValue);
-      }
-    }
-  }
-
-  /**
-   * Export the cache configuration in XML format.
-   *
-   * @param member Member for which to write the configuration
-   * @param group Group or groups for which to write the configuration
-   * @return Results of the attempt to write the configuration
-   */
-  @CliCommand(value = {CliStrings.EXPORT_CONFIG}, help = CliStrings.EXPORT_CONFIG__HELP)
-  @CliMetaData(
-      interceptor = "org.apache.geode.management.internal.cli.commands.ConfigCommands$Interceptor",
-      relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-  public Result exportConfig(
-      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
-          optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-          help = CliStrings.EXPORT_CONFIG__MEMBER__HELP) String[] member,
-      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
-          optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.EXPORT_CONFIG__GROUP__HELP) String[] group,
-      @CliOption(key = {CliStrings.EXPORT_CONFIG__DIR},
-          help = CliStrings.EXPORT_CONFIG__DIR__HELP) String dir) {
-    InfoResultData infoData = ResultBuilder.createInfoResultData();
-
-    Set<DistributedMember> targetMembers = CliUtil.findMembers(group, member);
-    if (targetMembers.isEmpty()) {
-      return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
-    }
-
-    try {
-      ResultCollector<?, ?> rc =
-          CliUtil.executeFunction(this.exportConfigFunction, null, targetMembers);
-      List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
-
-      for (CliFunctionResult result : results) {
-        if (result.getThrowable() != null) {
-          infoData.addLine(CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__EXCEPTION,
-              result.getMemberIdOrName(), result.getThrowable()));
-        } else if (result.isSuccessful()) {
-          String cacheFileName = result.getMemberIdOrName() + "-cache.xml";
-          String propsFileName = result.getMemberIdOrName() + "-gf.properties";
-          String[] fileContent = (String[]) result.getSerializables();
-          infoData.addAsFile(cacheFileName, fileContent[0], "Downloading Cache XML file: {0}",
-              false);
-          infoData.addAsFile(propsFileName, fileContent[1], "Downloading properties file: {0}",
-              false);
-        }
-      }
-      return ResultBuilder.buildResult(infoData);
-    } catch (VirtualMachineError e) {
-      SystemFailure.initiateFailure(e);
-      throw e;
-    } catch (Throwable th) {
-      SystemFailure.checkFailure();
-      th.printStackTrace(System.err);
-      return ResultBuilder
-          .createGemFireErrorResult(CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__EXCEPTION,
-              th.getClass().getName() + ": " + th.getMessage()));
-    }
-  }
-
-
-  @CliCommand(value = {CliStrings.ALTER_RUNTIME_CONFIG},
-      help = CliStrings.ALTER_RUNTIME_CONFIG__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG},
-      interceptor = "org.apache.geode.management.internal.cli.commands.ConfigCommands$AlterRuntimeInterceptor")
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE)
-  public Result alterRuntimeConfig(
-      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
-          optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-          help = CliStrings.ALTER_RUNTIME_CONFIG__MEMBER__HELP) String[] memberNameOrId,
-      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
-          optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.ALTER_RUNTIME_CONFIG__GROUP__HELP) String[] group,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT__HELP) Integer archiveDiskSpaceLimit,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT__HELP) Integer archiveFileSizeLimit,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT__HELP) Integer logDiskSpaceLimit,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT__HELP) Integer logFileSizeLimit,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL},
-          optionContext = ConverterHint.LOG_LEVEL,
-          help = CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL__HELP) String logLevel,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE__HELP) String statisticArchiveFile,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE__HELP) Integer statisticSampleRate,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLING__ENABLED},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLING__ENABLED__HELP) Boolean statisticSamplingEnabled,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ},
-          specifiedDefaultValue = "false",
-          help = CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ__HELP) Boolean setCopyOnRead,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE__HELP) Integer lockLease,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT__HELP) Integer lockTimeout,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL__HELP) Integer messageSyncInterval,
-      @CliOption(key = {CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT},
-          help = CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT__HELP) Integer searchTimeout) {
-
-    Map<String, String> runTimeDistributionConfigAttributes = new HashMap<>();
-    Map<String, String> rumTimeCacheAttributes = new HashMap<>();
-    Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
-
-    if (targetMembers.isEmpty()) {
-      return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
-    }
-
-    if (archiveDiskSpaceLimit != null) {
-      runTimeDistributionConfigAttributes.put(
-          CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__DISK__SPACE__LIMIT,
-          archiveDiskSpaceLimit.toString());
-    }
-
-    if (archiveFileSizeLimit != null) {
-      runTimeDistributionConfigAttributes.put(
-          CliStrings.ALTER_RUNTIME_CONFIG__ARCHIVE__FILE__SIZE__LIMIT,
-          archiveFileSizeLimit.toString());
-    }
-
-    if (logDiskSpaceLimit != null) {
-      runTimeDistributionConfigAttributes.put(
-          CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, logDiskSpaceLimit.toString());
-    }
-
-    if (logFileSizeLimit != null) {
-      runTimeDistributionConfigAttributes.put(
-          CliStrings.ALTER_RUNTIME_CONFIG__LOG__FILE__SIZE__LIMIT, logFileSizeLimit.toString());
-    }
-
-    if (logLevel != null && !logLevel.isEmpty()) {
-      runTimeDistributionConfigAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOG__LEVEL,
-          logLevel);
-    }
-
-    if (statisticArchiveFile != null && !statisticArchiveFile.isEmpty()) {
-      runTimeDistributionConfigAttributes
-          .put(CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__ARCHIVE__FILE, statisticArchiveFile);
-    }
-
-    if (statisticSampleRate != null) {
-      runTimeDistributionConfigAttributes.put(
-          CliStrings.ALTER_RUNTIME_CONFIG__STATISTIC__SAMPLE__RATE, statisticSampleRate.toString());
-    }
-
-    if (statisticSamplingEnabled != null) {
-      runTimeDistributionConfigAttributes.put(STATISTIC_SAMPLING_ENABLED,
-          statisticSamplingEnabled.toString());
-    }
-
-
-    // Attributes that are set on the cache.
-    if (setCopyOnRead != null) {
-      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__COPY__ON__READ,
-          setCopyOnRead.toString());
-    }
-
-    if (lockLease != null && lockLease > 0 && lockLease < Integer.MAX_VALUE) {
-      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__LEASE,
-          lockLease.toString());
-    }
-
-    if (lockTimeout != null && lockTimeout > 0 && lockTimeout < Integer.MAX_VALUE) {
-      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__LOCK__TIMEOUT,
-          lockTimeout.toString());
-    }
-
-    if (messageSyncInterval != null && messageSyncInterval > 0
-        && messageSyncInterval < Integer.MAX_VALUE) {
-      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__MESSAGE__SYNC__INTERVAL,
-          messageSyncInterval.toString());
-    }
-
-    if (searchTimeout != null && searchTimeout > 0 && searchTimeout < Integer.MAX_VALUE) {
-      rumTimeCacheAttributes.put(CliStrings.ALTER_RUNTIME_CONFIG__SEARCH__TIMEOUT,
-          searchTimeout.toString());
-    }
-
-    if (runTimeDistributionConfigAttributes.isEmpty() && rumTimeCacheAttributes.isEmpty()) {
-      return ResultBuilder
-          .createUserErrorResult(CliStrings.ALTER_RUNTIME_CONFIG__RELEVANT__OPTION__MESSAGE);
-    }
-
-    Map<String, String> allRunTimeAttributes = new HashMap<>();
-    allRunTimeAttributes.putAll(runTimeDistributionConfigAttributes);
-    allRunTimeAttributes.putAll(rumTimeCacheAttributes);
-
-    ResultCollector<?, ?> rc =
-        CliUtil.executeFunction(alterRunTimeConfigFunction, allRunTimeAttributes, targetMembers);
-    List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
-    Set<String> successfulMembers = new TreeSet<>();
-    Set<String> errorMessages = new TreeSet<>();
-
-    for (CliFunctionResult result : results) {
-      if (result.getThrowable() != null) {
-        logger.info("Function failed: " + result.getThrowable());
-        errorMessages.add(result.getThrowable().getMessage());
-      } else {
-        successfulMembers.add(result.getMemberIdOrName());
-      }
-    }
-    final String lineSeparator = System.getProperty("line.separator");
-    if (!successfulMembers.isEmpty()) {
-      StringBuilder successMessageBuilder = new StringBuilder();
-
-      successMessageBuilder.append(CliStrings.ALTER_RUNTIME_CONFIG__SUCCESS__MESSAGE);
-      successMessageBuilder.append(lineSeparator);
-
-      for (String member : successfulMembers) {
-        successMessageBuilder.append(member);
-        successMessageBuilder.append(lineSeparator);
-      }
-
-      Properties properties = new Properties();
-      properties.putAll(runTimeDistributionConfigAttributes);
-
-      Result result = ResultBuilder.createInfoResult(successMessageBuilder.toString());
-
-      // Set the Cache attributes to be modified
-      final XmlEntity xmlEntity = XmlEntity.builder().withType(CacheXml.CACHE)
-          .withAttributes(rumTimeCacheAttributes).build();
-      persistClusterConfiguration(result,
-          () -> getSharedConfiguration().modifyXmlAndProperties(properties, xmlEntity, group));
-      return result;
-    } else {
-      StringBuilder errorMessageBuilder = new StringBuilder();
-      errorMessageBuilder.append("Following errors occurred while altering runtime config");
-      errorMessageBuilder.append(lineSeparator);
-
-      for (String errorMessage : errorMessages) {
-        errorMessageBuilder.append(errorMessage);
-        errorMessageBuilder.append(lineSeparator);
-      }
-      return ResultBuilder.createUserErrorResult(errorMessageBuilder.toString());
-    }
-  }
-
-  public static class AlterRuntimeInterceptor extends AbstractCliAroundInterceptor {
-    @Override
-    public Result preExecution(GfshParseResult parseResult) {
-      Map<String, String> arguments = parseResult.getParamValueStrings();
-      // validate log level
-      String logLevel = arguments.get("log-level");
-      if (StringUtils.isNotBlank(logLevel) && (LogLevel.getLevel(logLevel) == null)) {
-        return ResultBuilder.createUserErrorResult("Invalid log level: " + logLevel);
-      }
-
-      return ResultBuilder.createInfoResult("");
-    }
-  }
-
-  /**
-   * Interceptor used by gfsh to intercept execution of export config command at "shell".
-   */
-  public static class Interceptor extends AbstractCliAroundInterceptor {
-    private String saveDirString;
-
-    @Override
-    public Result preExecution(GfshParseResult parseResult) {
-      Map<String, String> paramValueMap = parseResult.getParamValueStrings();
-      String dir = paramValueMap.get("dir");
-      dir = (dir == null) ? null : dir.trim();
-
-      File saveDirFile = new File(".");
-      if (dir != null && !dir.isEmpty()) {
-        saveDirFile = new File(dir);
-        if (saveDirFile.exists()) {
-          if (!saveDirFile.isDirectory())
-            return ResultBuilder.createGemFireErrorResult(
-                CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__NOT_A_DIRECTORY, dir));
-        } else if (!saveDirFile.mkdirs()) {
-          return ResultBuilder.createGemFireErrorResult(
-              CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__CANNOT_CREATE_DIR, dir));
-        }
-      }
-      try {
-        if (!saveDirFile.canWrite()) {
-          return ResultBuilder.createGemFireErrorResult(CliStrings.format(
-              CliStrings.EXPORT_CONFIG__MSG__NOT_WRITEABLE, saveDirFile.getCanonicalPath()));
-        }
-      } catch (IOException ioex) {
-        return ResultBuilder.createGemFireErrorResult(
-            CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__NOT_WRITEABLE, saveDirFile.getName()));
-      }
-
-      saveDirString = saveDirFile.getAbsolutePath();
-      return ResultBuilder.createInfoResult("OK");
-    }
-
-    @Override
-    public Result postExecution(GfshParseResult parseResult, Result commandResult, Path tempFile) {
-      if (commandResult.hasIncomingFiles()) {
-        try {
-          commandResult.saveIncomingFiles(saveDirString);
-        } catch (IOException ioex) {
-          Gfsh.getCurrentInstance().logSevere("Unable to export config", ioex);
-        }
-      }
-
-      return commandResult;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeConfigCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeConfigCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeConfigCommand.java
new file mode 100644
index 0000000..824063a
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeConfigCommand.java
@@ -0,0 +1,153 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.execute.FunctionInvocationTargetException;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+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.CliUtil;
+import org.apache.geode.management.internal.cli.domain.MemberConfigurationInfo;
+import org.apache.geode.management.internal.cli.functions.GetMemberConfigInformationFunction;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.CompositeResultData;
+import org.apache.geode.management.internal.cli.result.ErrorResultData;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.result.TabularResultData;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class DescribeConfigCommand implements GfshCommand {
+  private final GetMemberConfigInformationFunction getMemberConfigFunction =
+      new GetMemberConfigInformationFunction();
+
+  @CliCommand(value = {CliStrings.DESCRIBE_CONFIG}, help = CliStrings.DESCRIBE_CONFIG__HELP)
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.READ)
+  public Result describeConfig(
+      @CliOption(key = CliStrings.MEMBER, optionContext = ConverterHint.ALL_MEMBER_IDNAME,
+          help = CliStrings.DESCRIBE_CONFIG__MEMBER__HELP, mandatory = true) String memberNameOrId,
+      @CliOption(key = CliStrings.DESCRIBE_CONFIG__HIDE__DEFAULTS,
+          help = CliStrings.DESCRIBE_CONFIG__HIDE__DEFAULTS__HELP, unspecifiedDefaultValue = "true",
+          specifiedDefaultValue = "true") boolean hideDefaults) {
+
+    Result result = null;
+    try {
+      DistributedMember targetMember = null;
+
+      if (memberNameOrId != null && !memberNameOrId.isEmpty()) {
+        targetMember = CliUtil.getDistributedMemberByNameOrId(memberNameOrId);
+      }
+      if (targetMember != null) {
+        ResultCollector<?, ?> rc =
+            CliUtil.executeFunction(getMemberConfigFunction, hideDefaults, targetMember);
+        ArrayList<?> output = (ArrayList<?>) rc.getResult();
+        Object obj = output.get(0);
+
+        if (obj != null && obj instanceof MemberConfigurationInfo) {
+          MemberConfigurationInfo memberConfigInfo = (MemberConfigurationInfo) obj;
+
+          CompositeResultData crd = ResultBuilder.createCompositeResultData();
+          crd.setHeader(
+              CliStrings.format(CliStrings.DESCRIBE_CONFIG__HEADER__TEXT, memberNameOrId));
+
+          List<String> jvmArgsList = memberConfigInfo.getJvmInputArguments();
+          TabularResultData jvmInputArgs = crd.addSection().addSection().addTable();
+
+          for (String jvmArg : jvmArgsList) {
+            jvmInputArgs.accumulate("JVM command line arguments", jvmArg);
+          }
+
+          addSection(crd, memberConfigInfo.getGfePropsSetUsingApi(),
+              "GemFire properties defined using the API");
+          addSection(crd, memberConfigInfo.getGfePropsRuntime(),
+              "GemFire properties defined at the runtime");
+          addSection(crd, memberConfigInfo.getGfePropsSetFromFile(),
+              "GemFire properties defined with the property file");
+          addSection(crd, memberConfigInfo.getGfePropsSetWithDefaults(),
+              "GemFire properties using default values");
+          addSection(crd, memberConfigInfo.getCacheAttributes(), "Cache attributes");
+
+          List<Map<String, String>> cacheServerAttributesList =
+              memberConfigInfo.getCacheServerAttributes();
+
+          if (cacheServerAttributesList != null && !cacheServerAttributesList.isEmpty()) {
+            CompositeResultData.SectionResultData cacheServerSection = crd.addSection();
+            cacheServerSection.setHeader("Cache-server attributes");
+
+            for (Map<String, String> cacheServerAttributes : cacheServerAttributesList) {
+              addSubSection(cacheServerSection, cacheServerAttributes);
+            }
+          }
+          result = ResultBuilder.buildResult(crd);
+        }
+
+      } else {
+        ErrorResultData erd = ResultBuilder.createErrorResultData();
+        erd.addLine(CliStrings.format(CliStrings.DESCRIBE_CONFIG__MEMBER__NOT__FOUND,
+            new Object[] {memberNameOrId}));
+        result = ResultBuilder.buildResult(erd);
+      }
+    } catch (FunctionInvocationTargetException e) {
+      result = ResultBuilder.createGemFireErrorResult(CliStrings
+          .format(CliStrings.COULD_NOT_EXECUTE_COMMAND_TRY_AGAIN, CliStrings.DESCRIBE_CONFIG));
+    } catch (Exception e) {
+      ErrorResultData erd = ResultBuilder.createErrorResultData();
+      erd.addLine(e.getMessage());
+      result = ResultBuilder.buildResult(erd);
+    }
+    return result;
+  }
+
+  private void addSection(CompositeResultData crd, Map<String, String> attrMap, String headerText) {
+    if (attrMap != null && !attrMap.isEmpty()) {
+      CompositeResultData.SectionResultData section = crd.addSection();
+      section.setHeader(headerText);
+      section.addSeparator('.');
+      Set<String> attributes = new TreeSet<>(attrMap.keySet());
+
+      for (String attribute : attributes) {
+        String attributeValue = attrMap.get(attribute);
+        section.addData(attribute, attributeValue);
+      }
+    }
+  }
+
+  private void addSubSection(CompositeResultData.SectionResultData section,
+      Map<String, String> attrMap) {
+    if (!attrMap.isEmpty()) {
+      CompositeResultData.SectionResultData subSection = section.addSection();
+      Set<String> attributes = new TreeSet<>(attrMap.keySet());
+      subSection.setHeader("");
+
+      for (String attribute : attributes) {
+        String attributeValue = attrMap.get(attribute);
+        subSection.addData(attribute, attributeValue);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportConfigCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportConfigCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportConfigCommand.java
new file mode 100644
index 0000000..672ec88
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ExportConfigCommand.java
@@ -0,0 +1,159 @@
+/*
+ * 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 java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.SystemFailure;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+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.CliFunctionResult;
+import org.apache.geode.management.internal.cli.functions.ExportConfigFunction;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.InfoResultData;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class ExportConfigCommand implements GfshCommand {
+  private final ExportConfigFunction exportConfigFunction = new ExportConfigFunction();
+
+  /**
+   * Export the cache configuration in XML format.
+   *
+   * @param member Member for which to write the configuration
+   * @param group Group or groups for which to write the configuration
+   * @return Results of the attempt to write the configuration
+   */
+  @CliCommand(value = {CliStrings.EXPORT_CONFIG}, help = CliStrings.EXPORT_CONFIG__HELP)
+  @CliMetaData(
+      interceptor = "org.apache.geode.management.internal.cli.commands.ExportConfigCommand$Interceptor",
+      relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG})
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.READ)
+  public Result exportConfig(
+      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
+          optionContext = ConverterHint.ALL_MEMBER_IDNAME,
+          help = CliStrings.EXPORT_CONFIG__MEMBER__HELP) String[] member,
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          optionContext = ConverterHint.MEMBERGROUP,
+          help = CliStrings.EXPORT_CONFIG__GROUP__HELP) String[] group,
+      @CliOption(key = {CliStrings.EXPORT_CONFIG__DIR},
+          help = CliStrings.EXPORT_CONFIG__DIR__HELP) String dir) {
+    InfoResultData infoData = ResultBuilder.createInfoResultData();
+
+    Set<DistributedMember> targetMembers = CliUtil.findMembers(group, member);
+    if (targetMembers.isEmpty()) {
+      return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+    }
+
+    try {
+      ResultCollector<?, ?> rc =
+          CliUtil.executeFunction(this.exportConfigFunction, null, targetMembers);
+      List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
+
+      for (CliFunctionResult result : results) {
+        if (result.getThrowable() != null) {
+          infoData.addLine(CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__EXCEPTION,
+              result.getMemberIdOrName(), result.getThrowable()));
+        } else if (result.isSuccessful()) {
+          String cacheFileName = result.getMemberIdOrName() + "-cache.xml";
+          String propsFileName = result.getMemberIdOrName() + "-gf.properties";
+          String[] fileContent = (String[]) result.getSerializables();
+          infoData.addAsFile(cacheFileName, fileContent[0], "Downloading Cache XML file: {0}",
+              false);
+          infoData.addAsFile(propsFileName, fileContent[1], "Downloading properties file: {0}",
+              false);
+        }
+      }
+      return ResultBuilder.buildResult(infoData);
+    } catch (VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+      throw e;
+    } catch (Throwable th) {
+      SystemFailure.checkFailure();
+      th.printStackTrace(System.err);
+      return ResultBuilder
+          .createGemFireErrorResult(CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__EXCEPTION,
+              th.getClass().getName() + ": " + th.getMessage()));
+    }
+  }
+
+  /**
+   * Interceptor used by gfsh to intercept execution of export config command at "shell".
+   */
+  public static class Interceptor extends AbstractCliAroundInterceptor {
+    private String saveDirString;
+
+    @Override
+    public Result preExecution(GfshParseResult parseResult) {
+      Map<String, String> paramValueMap = parseResult.getParamValueStrings();
+      String dir = paramValueMap.get("dir");
+      dir = (dir == null) ? null : dir.trim();
+
+      File saveDirFile = new File(".");
+      if (dir != null && !dir.isEmpty()) {
+        saveDirFile = new File(dir);
+        if (saveDirFile.exists()) {
+          if (!saveDirFile.isDirectory())
+            return ResultBuilder.createGemFireErrorResult(
+                CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__NOT_A_DIRECTORY, dir));
+        } else if (!saveDirFile.mkdirs()) {
+          return ResultBuilder.createGemFireErrorResult(
+              CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__CANNOT_CREATE_DIR, dir));
+        }
+      }
+      try {
+        if (!saveDirFile.canWrite()) {
+          return ResultBuilder.createGemFireErrorResult(CliStrings.format(
+              CliStrings.EXPORT_CONFIG__MSG__NOT_WRITEABLE, saveDirFile.getCanonicalPath()));
+        }
+      } catch (IOException ioex) {
+        return ResultBuilder.createGemFireErrorResult(
+            CliStrings.format(CliStrings.EXPORT_CONFIG__MSG__NOT_WRITEABLE, saveDirFile.getName()));
+      }
+
+      saveDirString = saveDirFile.getAbsolutePath();
+      return ResultBuilder.createInfoResult("OK");
+    }
+
+    @Override
+    public Result postExecution(GfshParseResult parseResult, Result commandResult, Path tempFile) {
+      if (commandResult.hasIncomingFiles()) {
+        try {
+          commandResult.saveIncomingFiles(saveDirString);
+        } catch (IOException ioex) {
+          Gfsh.getCurrentInstance().logSevere("Unable to export config", ioex);
+        }
+      }
+      return commandResult;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ConfigCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ConfigCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ConfigCommandsController.java
index 9f06ea3..d223a9f 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ConfigCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/ConfigCommandsController.java
@@ -14,10 +14,9 @@
  */
 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.util.CommandStringBuilder;
-import org.apache.geode.management.internal.web.util.ConvertUtils;
+import java.io.IOException;
+import java.util.concurrent.Callable;
+
 import org.springframework.http.ResponseEntity;
 import org.springframework.stereotype.Controller;
 import org.springframework.web.bind.annotation.PathVariable;
@@ -27,15 +26,19 @@ import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 import org.springframework.web.multipart.MultipartFile;
 
-import java.io.IOException;
-import java.util.concurrent.Callable;
+import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.management.internal.web.util.ConvertUtils;
 
 /**
  * The ConfigCommandsController class implements GemFire Management REST API web service endpoints
  * for the Gfsh Config Commands.
  * <p/>
  * 
- * @see org.apache.geode.management.internal.cli.commands.ConfigCommands
+ * @see org.apache.geode.management.internal.cli.commands.AlterRuntimeConfigCommand
+ * @see org.apache.geode.management.internal.cli.commands.DescribeConfigCommand
+ * @see org.apache.geode.management.internal.cli.commands.ExportConfigCommand
  * @see org.apache.geode.management.internal.web.controllers.AbstractMultiPartCommandsController
  * @see org.springframework.stereotype.Controller
  * @see org.springframework.web.bind.annotation.PathVariable

http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DeployCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DeployCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DeployCommandsController.java
index 65c22a9..9d3e086 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DeployCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DeployCommandsController.java
@@ -14,10 +14,8 @@
  */
 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.util.CommandStringBuilder;
-import org.apache.geode.management.internal.web.util.ConvertUtils;
+import java.io.IOException;
+
 import org.springframework.stereotype.Controller;
 import org.springframework.web.bind.annotation.RequestMapping;
 import org.springframework.web.bind.annotation.RequestMethod;
@@ -25,13 +23,16 @@ import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 import org.springframework.web.multipart.MultipartFile;
 
-import java.io.IOException;
+import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.management.internal.web.util.ConvertUtils;
 
 /**
  * The DeployCommandsController class implements the GemFire Management REST API web service
  * endpoints for the Gfsh Deploy Commands.
  * <p/>
- * 
+ *
  * @see org.apache.geode.management.internal.cli.commands.DeployCommand
  * @see org.apache.geode.management.internal.cli.commands.UndeployCommand
  * @see org.apache.geode.management.internal.cli.commands.ListDeployedCommand

http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/LogLevelInterceptorTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/LogLevelInterceptorTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/LogLevelInterceptorTest.java
index 9feac1a..9188582 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/LogLevelInterceptorTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/LogLevelInterceptorTest.java
@@ -43,7 +43,7 @@ public class LogLevelInterceptorTest {
   @Before
   public void before() {
     interceptors.add(new ExportLogsInterceptor());
-    interceptors.add(new ConfigCommands.AlterRuntimeInterceptor());
+    interceptors.add(new AlterRuntimeInterceptor());
     interceptors.add(new ChangeLogLevelCommand.ChangeLogLevelCommandInterceptor());
 
     parseResult = Mockito.mock(GfshParseResult.class);

http://git-wip-us.apache.org/repos/asf/geode/blob/97c4e9a5/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
index 7426645..43e3c04 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
@@ -123,7 +123,7 @@ public class TestCommand {
     createTestCommand("list clients", clusterRead);
     createTestCommand("describe client --clientID=172.16.196.144", clusterRead);
 
-    // ConfigCommands
+    // AlterRuntimeConfigCommand, DescribeConfigCommand, ExportConfigCommand (config commands)
     createTestCommand("alter runtime", clusterManage);
     createTestCommand("describe config --member=Member1", clusterRead);
     createTestCommand("export config --member=member1", clusterRead);


[2/3] geode git commit: GEODE-3259: Refactoring DurableClientCommands

Posted by kl...@apache.org.
GEODE-3259: Refactoring DurableClientCommands

This closes #689


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

Branch: refs/heads/develop
Commit: 440c87f81fab96f9ce38a2d53ded75e5fe8390d7
Parents: 97c4e9a
Author: YehEmily <em...@gmail.com>
Authored: Mon Aug 7 11:52:14 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Aug 11 16:00:31 2017 -0700

----------------------------------------------------------------------
 .../cli/commands/CloseDurableCQsCommand.java    |  83 ++++
 .../cli/commands/CloseDurableClientCommand.java |  77 ++++
 .../commands/CountDurableCQEventsCommand.java   |  88 ++++
 .../cli/commands/DurableClientCommands.java     | 420 -------------------
 .../DurableClientCommandsResultBuilder.java     | 164 ++++++++
 .../commands/ListDurableClientCQsCommand.java   | 121 ++++++
 .../DurableClientCommandsController.java        |  15 +-
 .../internal/security/TestCommand.java          |   5 +-
 8 files changed, 545 insertions(+), 428 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/440c87f8/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CloseDurableCQsCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CloseDurableCQsCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CloseDurableCQsCommand.java
new file mode 100644
index 0000000..61dd914
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CloseDurableCQsCommand.java
@@ -0,0 +1,83 @@
+/*
+ * 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 java.util.List;
+import java.util.Set;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+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.CliUtil;
+import org.apache.geode.management.internal.cli.domain.MemberResult;
+import org.apache.geode.management.internal.cli.functions.CloseDurableCqFunction;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class CloseDurableCQsCommand implements GfshCommand {
+  DurableClientCommandsResultBuilder builder = new DurableClientCommandsResultBuilder();
+
+  @CliCommand(value = CliStrings.CLOSE_DURABLE_CQS, help = CliStrings.CLOSE_DURABLE_CQS__HELP)
+  @CliMetaData()
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.MANAGE, target = ResourcePermission.Target.QUERY)
+  public Result closeDurableCqs(@CliOption(key = CliStrings.CLOSE_DURABLE_CQS__DURABLE__CLIENT__ID,
+      mandatory = true,
+      help = CliStrings.CLOSE_DURABLE_CQS__DURABLE__CLIENT__ID__HELP) final String durableClientId,
+
+      @CliOption(key = CliStrings.CLOSE_DURABLE_CQS__NAME, mandatory = true,
+          help = CliStrings.CLOSE_DURABLE_CQS__NAME__HELP) final String cqName,
+
+      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
+          help = CliStrings.CLOSE_DURABLE_CQS__MEMBER__HELP,
+          optionContext = ConverterHint.MEMBERIDNAME) final String[] memberNameOrId,
+
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          help = CliStrings.CLOSE_DURABLE_CQS__GROUP__HELP,
+          optionContext = ConverterHint.MEMBERGROUP) final String[] group) {
+    Result result;
+    try {
+      Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
+
+      if (targetMembers.isEmpty()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+      }
+
+      String[] params = new String[2];
+      params[0] = durableClientId;
+      params[1] = cqName;
+
+      final ResultCollector<?, ?> rc =
+          CliUtil.executeFunction(new CloseDurableCqFunction(), params, targetMembers);
+      final List<MemberResult> results = (List<MemberResult>) rc.getResult();
+      String failureHeader =
+          CliStrings.format(CliStrings.CLOSE_DURABLE_CQS__FAILURE__HEADER, cqName, durableClientId);
+      String successHeader =
+          CliStrings.format(CliStrings.CLOSE_DURABLE_CQS__SUCCESS, cqName, durableClientId);
+      result = builder.buildResult(results, successHeader, failureHeader);
+    } catch (Exception e) {
+      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/440c87f8/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CloseDurableClientCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CloseDurableClientCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CloseDurableClientCommand.java
new file mode 100644
index 0000000..14c9731
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CloseDurableClientCommand.java
@@ -0,0 +1,77 @@
+/*
+ * 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 java.util.List;
+import java.util.Set;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+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.CliUtil;
+import org.apache.geode.management.internal.cli.domain.MemberResult;
+import org.apache.geode.management.internal.cli.functions.CloseDurableClientFunction;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class CloseDurableClientCommand implements GfshCommand {
+  DurableClientCommandsResultBuilder builder = new DurableClientCommandsResultBuilder();
+
+  @CliCommand(value = CliStrings.CLOSE_DURABLE_CLIENTS,
+      help = CliStrings.CLOSE_DURABLE_CLIENTS__HELP)
+  @CliMetaData()
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.MANAGE, target = ResourcePermission.Target.QUERY)
+  public Result closeDurableClient(
+      @CliOption(key = CliStrings.CLOSE_DURABLE_CLIENTS__CLIENT__ID, mandatory = true,
+          help = CliStrings.CLOSE_DURABLE_CLIENTS__CLIENT__ID__HELP) final String durableClientId,
+      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
+          help = CliStrings.CLOSE_DURABLE_CLIENTS__MEMBER__HELP,
+          optionContext = ConverterHint.MEMBERIDNAME) final String[] memberNameOrId,
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          help = CliStrings.COUNT_DURABLE_CQ_EVENTS__GROUP__HELP,
+          optionContext = ConverterHint.MEMBERGROUP) final String[] group) {
+
+    Result result;
+    try {
+
+      Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
+
+      if (targetMembers.isEmpty()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+      }
+
+      final ResultCollector<?, ?> rc =
+          CliUtil.executeFunction(new CloseDurableClientFunction(), durableClientId, targetMembers);
+      final List<MemberResult> results = (List<MemberResult>) rc.getResult();
+      String failureHeader =
+          CliStrings.format(CliStrings.CLOSE_DURABLE_CLIENTS__FAILURE__HEADER, durableClientId);
+      String successHeader =
+          CliStrings.format(CliStrings.CLOSE_DURABLE_CLIENTS__SUCCESS, durableClientId);
+      result = builder.buildResult(results, successHeader, failureHeader);
+    } catch (Exception e) {
+      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/440c87f8/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CountDurableCQEventsCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CountDurableCQEventsCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CountDurableCQEventsCommand.java
new file mode 100644
index 0000000..8ebc4da
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CountDurableCQEventsCommand.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.management.internal.cli.commands;
+
+import java.util.List;
+import java.util.Set;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+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.CliUtil;
+import org.apache.geode.management.internal.cli.domain.SubscriptionQueueSizeResult;
+import org.apache.geode.management.internal.cli.functions.GetSubscriptionQueueSizeFunction;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class CountDurableCQEventsCommand implements GfshCommand {
+  DurableClientCommandsResultBuilder builder = new DurableClientCommandsResultBuilder();
+
+  @CliCommand(value = CliStrings.COUNT_DURABLE_CQ_EVENTS,
+      help = CliStrings.COUNT_DURABLE_CQ_EVENTS__HELP)
+  @CliMetaData()
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.READ)
+  public Result countDurableCqEvents(
+      @CliOption(key = CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CLIENT__ID, mandatory = true,
+          help = CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CLIENT__ID__HELP) final String durableClientId,
+      @CliOption(key = CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CQ__NAME,
+          help = CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CQ__NAME__HELP) final String cqName,
+      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
+          help = CliStrings.COUNT_DURABLE_CQ_EVENTS__MEMBER__HELP,
+          optionContext = ConverterHint.MEMBERIDNAME) final String[] memberNameOrId,
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          help = CliStrings.COUNT_DURABLE_CQ_EVENTS__GROUP__HELP,
+          optionContext = ConverterHint.MEMBERGROUP) final String[] group) {
+
+    Result result;
+    try {
+      Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
+
+      if (targetMembers.isEmpty()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+      }
+
+      String[] params = new String[2];
+      params[0] = durableClientId;
+      params[1] = cqName;
+      final ResultCollector<?, ?> rc =
+          CliUtil.executeFunction(new GetSubscriptionQueueSizeFunction(), params, targetMembers);
+      final List<SubscriptionQueueSizeResult> funcResults =
+          (List<SubscriptionQueueSizeResult>) rc.getResult();
+
+      String queueSizeColumnName;
+
+      if (cqName != null && !cqName.isEmpty()) {
+        queueSizeColumnName = CliStrings
+            .format(CliStrings.COUNT_DURABLE_CQ_EVENTS__SUBSCRIPTION__QUEUE__SIZE__CLIENT, cqName);
+      } else {
+        queueSizeColumnName = CliStrings.format(
+            CliStrings.COUNT_DURABLE_CQ_EVENTS__SUBSCRIPTION__QUEUE__SIZE__CLIENT, durableClientId);
+      }
+      result = builder.buildTableResultForQueueSize(funcResults, queueSizeColumnName);
+    } catch (Exception e) {
+      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/440c87f8/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DurableClientCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DurableClientCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DurableClientCommands.java
deleted file mode 100644
index d66212b..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DurableClientCommands.java
+++ /dev/null
@@ -1,420 +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 java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.distributed.DistributedMember;
-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.CliUtil;
-import org.apache.geode.management.internal.cli.domain.DurableCqNamesResult;
-import org.apache.geode.management.internal.cli.domain.MemberResult;
-import org.apache.geode.management.internal.cli.domain.SubscriptionQueueSizeResult;
-import org.apache.geode.management.internal.cli.functions.CloseDurableClientFunction;
-import org.apache.geode.management.internal.cli.functions.CloseDurableCqFunction;
-import org.apache.geode.management.internal.cli.functions.GetSubscriptionQueueSizeFunction;
-import org.apache.geode.management.internal.cli.functions.ListDurableCqNamesFunction;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ErrorResultData;
-import org.apache.geode.management.internal.cli.result.InfoResultData;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.result.TabularResultData;
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.geode.security.ResourcePermission.Target;
-
-/**
- * The DurableClientCommands class encapsulates all GemFire shell (Gfsh) commands related to durable
- * clients and cqs defined in GemFire.
- * </p>
- */
-@SuppressWarnings("unused")
-public class DurableClientCommands implements GfshCommand {
-
-  private static final ListDurableCqNamesFunction listDurableCqNamesFunction =
-      new ListDurableCqNamesFunction();
-  private static final CloseDurableClientFunction closeDurableClientFunction =
-      new CloseDurableClientFunction();
-  private static final CloseDurableCqFunction closeDurableCqFunction = new CloseDurableCqFunction();
-  private static final GetSubscriptionQueueSizeFunction countDurableCqEvents =
-      new GetSubscriptionQueueSizeFunction();
-
-  @CliCommand(value = CliStrings.LIST_DURABLE_CQS, help = CliStrings.LIST_DURABLE_CQS__HELP)
-  @CliMetaData()
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-  public Result listDurableClientCqs(
-      @CliOption(key = CliStrings.LIST_DURABLE_CQS__DURABLECLIENTID, mandatory = true,
-          help = CliStrings.LIST_DURABLE_CQS__DURABLECLIENTID__HELP) final String durableClientId,
-
-      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
-          help = CliStrings.LIST_DURABLE_CQS__MEMBER__HELP,
-          optionContext = ConverterHint.MEMBERIDNAME) final String[] memberNameOrId,
-
-      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
-          help = CliStrings.LIST_DURABLE_CQS__GROUP__HELP,
-          optionContext = ConverterHint.MEMBERGROUP) final String[] group) {
-    Result result;
-    try {
-
-      boolean noResults = true;
-      Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
-
-      if (targetMembers.isEmpty()) {
-        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
-      }
-
-      final ResultCollector<?, ?> rc =
-          CliUtil.executeFunction(new ListDurableCqNamesFunction(), durableClientId, targetMembers);
-      final List<DurableCqNamesResult> results = (List<DurableCqNamesResult>) rc.getResult();
-      Map<String, List<String>> memberCqNamesMap = new TreeMap<>();
-      Map<String, List<String>> errorMessageNodes = new HashMap<>();
-      Map<String, List<String>> exceptionMessageNodes = new HashMap<>();
-
-      for (DurableCqNamesResult memberResult : results) {
-        if (memberResult != null) {
-          if (memberResult.isSuccessful()) {
-            memberCqNamesMap.put(memberResult.getMemberNameOrId(), memberResult.getCqNamesList());
-          } else {
-            if (memberResult.isOpPossible()) {
-              groupByMessage(memberResult.getExceptionMessage(), memberResult.getMemberNameOrId(),
-                  exceptionMessageNodes);
-            } else {
-              groupByMessage(memberResult.getErrorMessage(), memberResult.getMemberNameOrId(),
-                  errorMessageNodes);
-            }
-          }
-        }
-      }
-
-      if (!memberCqNamesMap.isEmpty()) {
-        TabularResultData table = ResultBuilder.createTabularResultData();
-        Set<String> members = memberCqNamesMap.keySet();
-
-        for (String member : members) {
-          boolean isFirst = true;
-          List<String> cqNames = memberCqNamesMap.get(member);
-          for (String cqName : cqNames) {
-            if (isFirst) {
-              isFirst = false;
-              table.accumulate(CliStrings.MEMBER, member);
-            } else {
-              table.accumulate(CliStrings.MEMBER, "");
-            }
-            table.accumulate(CliStrings.LIST_DURABLE_CQS__NAME, cqName);
-          }
-        }
-        result = ResultBuilder.buildResult(table);
-      } else {
-        String errorHeader =
-            CliStrings.format(CliStrings.LIST_DURABLE_CQS__FAILURE__HEADER, durableClientId);
-        result = ResultBuilder.buildResult(
-            buildFailureData(null, exceptionMessageNodes, errorMessageNodes, errorHeader));
-      }
-    } catch (Exception e) {
-      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
-    }
-
-    return result;
-  }
-
-  @CliCommand(value = CliStrings.COUNT_DURABLE_CQ_EVENTS,
-      help = CliStrings.COUNT_DURABLE_CQ_EVENTS__HELP)
-  @CliMetaData()
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-  public Result countDurableCqEvents(
-      @CliOption(key = CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CLIENT__ID, mandatory = true,
-          help = CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CLIENT__ID__HELP) final String durableClientId,
-      @CliOption(key = CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CQ__NAME,
-          help = CliStrings.COUNT_DURABLE_CQ_EVENTS__DURABLE__CQ__NAME__HELP) final String cqName,
-      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
-          help = CliStrings.COUNT_DURABLE_CQ_EVENTS__MEMBER__HELP,
-          optionContext = ConverterHint.MEMBERIDNAME) final String[] memberNameOrId,
-      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
-          help = CliStrings.COUNT_DURABLE_CQ_EVENTS__GROUP__HELP,
-          optionContext = ConverterHint.MEMBERGROUP) final String[] group) {
-
-    Result result;
-    try {
-      Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
-
-      if (targetMembers.isEmpty()) {
-        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
-      }
-
-      String[] params = new String[2];
-      params[0] = durableClientId;
-      params[1] = cqName;
-      final ResultCollector<?, ?> rc =
-          CliUtil.executeFunction(new GetSubscriptionQueueSizeFunction(), params, targetMembers);
-      final List<SubscriptionQueueSizeResult> funcResults =
-          (List<SubscriptionQueueSizeResult>) rc.getResult();
-
-      String queueSizeColumnName;
-
-      if (cqName != null && !cqName.isEmpty()) {
-        queueSizeColumnName = CliStrings
-            .format(CliStrings.COUNT_DURABLE_CQ_EVENTS__SUBSCRIPTION__QUEUE__SIZE__CLIENT, cqName);
-      } else {
-        queueSizeColumnName = CliStrings.format(
-            CliStrings.COUNT_DURABLE_CQ_EVENTS__SUBSCRIPTION__QUEUE__SIZE__CLIENT, durableClientId);
-      }
-      result = buildTableResultForQueueSize(funcResults, queueSizeColumnName);
-    } catch (Exception e) {
-      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
-    }
-
-    return result;
-  }
-
-  @CliCommand(value = CliStrings.CLOSE_DURABLE_CLIENTS,
-      help = CliStrings.CLOSE_DURABLE_CLIENTS__HELP)
-  @CliMetaData()
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
-      target = Target.QUERY)
-  public Result closeDurableClient(
-      @CliOption(key = CliStrings.CLOSE_DURABLE_CLIENTS__CLIENT__ID, mandatory = true,
-          help = CliStrings.CLOSE_DURABLE_CLIENTS__CLIENT__ID__HELP) final String durableClientId,
-      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
-          help = CliStrings.CLOSE_DURABLE_CLIENTS__MEMBER__HELP,
-          optionContext = ConverterHint.MEMBERIDNAME) final String[] memberNameOrId,
-      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
-          help = CliStrings.CLOSE_DURABLE_CLIENTS__GROUP__HELP,
-          optionContext = ConverterHint.MEMBERGROUP) final String[] group) {
-
-    Result result;
-    try {
-
-      Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
-
-      if (targetMembers.isEmpty()) {
-        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
-      }
-
-      final ResultCollector<?, ?> rc =
-          CliUtil.executeFunction(new CloseDurableClientFunction(), durableClientId, targetMembers);
-      final List<MemberResult> results = (List<MemberResult>) rc.getResult();
-      String failureHeader =
-          CliStrings.format(CliStrings.CLOSE_DURABLE_CLIENTS__FAILURE__HEADER, durableClientId);
-      String successHeader =
-          CliStrings.format(CliStrings.CLOSE_DURABLE_CLIENTS__SUCCESS, durableClientId);
-      result = buildResult(results, successHeader, failureHeader);
-    } catch (Exception e) {
-      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
-    }
-    return result;
-  }
-
-
-  @CliCommand(value = CliStrings.CLOSE_DURABLE_CQS, help = CliStrings.CLOSE_DURABLE_CQS__HELP)
-  @CliMetaData()
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE,
-      target = Target.QUERY)
-  public Result closeDurableCqs(@CliOption(key = CliStrings.CLOSE_DURABLE_CQS__DURABLE__CLIENT__ID,
-      mandatory = true,
-      help = CliStrings.CLOSE_DURABLE_CQS__DURABLE__CLIENT__ID__HELP) final String durableClientId,
-
-      @CliOption(key = CliStrings.CLOSE_DURABLE_CQS__NAME, mandatory = true,
-          help = CliStrings.CLOSE_DURABLE_CQS__NAME__HELP) final String cqName,
-
-      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
-          help = CliStrings.CLOSE_DURABLE_CQS__MEMBER__HELP,
-          optionContext = ConverterHint.MEMBERIDNAME) final String[] memberNameOrId,
-
-      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
-          help = CliStrings.CLOSE_DURABLE_CQS__GROUP__HELP,
-          optionContext = ConverterHint.MEMBERGROUP) final String[] group) {
-    Result result;
-    try {
-      Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
-
-      if (targetMembers.isEmpty()) {
-        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
-      }
-
-      String[] params = new String[2];
-      params[0] = durableClientId;
-      params[1] = cqName;
-
-      final ResultCollector<?, ?> rc =
-          CliUtil.executeFunction(new CloseDurableCqFunction(), params, targetMembers);
-      final List<MemberResult> results = (List<MemberResult>) rc.getResult();
-      String failureHeader =
-          CliStrings.format(CliStrings.CLOSE_DURABLE_CQS__FAILURE__HEADER, cqName, durableClientId);
-      String successHeader =
-          CliStrings.format(CliStrings.CLOSE_DURABLE_CQS__SUCCESS, cqName, durableClientId);
-      result = buildResult(results, successHeader, failureHeader);
-    } catch (Exception e) {
-      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
-    }
-    return result;
-  }
-
-
-  private Result buildResult(List<MemberResult> results, String successHeader,
-      String failureHeader) {
-    Result result;
-    boolean failure = true;
-    boolean partialFailure = false;
-    Map<String, List<String>> errorMap = new HashMap<>();
-    Map<String, List<String>> successMap = new HashMap<>();
-    Map<String, List<String>> exceptionMap = new HashMap<>();
-
-    /*
-     * Aggregate the results from the members
-     */
-    for (MemberResult memberResult : results) {
-
-      if (memberResult.isSuccessful()) {
-        failure = false;
-        groupByMessage(memberResult.getSuccessMessage(), memberResult.getMemberNameOrId(),
-            successMap);
-      } else {
-
-        if (memberResult.isOpPossible()) {
-          partialFailure = true;
-          groupByMessage(memberResult.getExceptionMessage(), memberResult.getMemberNameOrId(),
-              exceptionMap);
-
-        } else {
-          groupByMessage(memberResult.getErrorMessage(), memberResult.getMemberNameOrId(),
-              errorMap);
-        }
-      }
-    }
-
-    if (!failure && !partialFailure) {
-      result = ResultBuilder.buildResult(buildSuccessData(successMap));
-    } else {
-      result = ResultBuilder
-          .buildResult(buildFailureData(successMap, exceptionMap, errorMap, failureHeader));
-    }
-    return result;
-  }
-
-  private Result buildTableResultForQueueSize(List<SubscriptionQueueSizeResult> results,
-      String queueSizeColumnName) {
-    Result result;
-    boolean failure = true;
-
-    Map<String, List<String>> failureMap = new HashMap<>();
-    Map<String, Long> memberQueueSizeTable = new TreeMap<>();
-
-    /*
-     * Aggregate the results from the members
-     */
-    for (SubscriptionQueueSizeResult memberResult : results) {
-
-      if (memberResult.isSuccessful()) {
-        failure = false;
-        memberQueueSizeTable.put(memberResult.getMemberNameOrId(),
-            memberResult.getSubscriptionQueueSize());
-      } else {
-        groupByMessage(memberResult.getErrorMessage(), memberResult.getMemberNameOrId(),
-            failureMap);
-      }
-    }
-
-    if (!failure) {
-      TabularResultData table = ResultBuilder.createTabularResultData();
-
-      Set<String> members = memberQueueSizeTable.keySet();
-
-      for (String member : members) {
-        long queueSize = memberQueueSizeTable.get(member);
-        table.accumulate(CliStrings.MEMBER, member);
-        table.accumulate(queueSizeColumnName, queueSize);
-      }
-      result = ResultBuilder.buildResult(table);
-
-    } else {
-      ErrorResultData erd = ResultBuilder.createErrorResultData();
-      buildErrorResult(erd, failureMap);
-      result = ResultBuilder.buildResult(erd);
-    }
-    return result;
-  }
-
-  private void groupByMessage(String message, String memberNameOrId,
-      Map<String, List<String>> map) {
-    List<String> members = map.get(message);
-
-    if (members == null) {
-      members = new LinkedList<>();
-    }
-    members.add(memberNameOrId);
-    map.put(message, members);
-  }
-
-
-  private InfoResultData buildSuccessData(Map<String, List<String>> successMap) {
-    InfoResultData ird = ResultBuilder.createInfoResultData();
-    Set<String> successMessages = successMap.keySet();
-
-    for (String successMessage : successMessages) {
-      ird.addLine(CliStrings.format(CliStrings.ACTION_SUCCEEDED_ON_MEMBER, successMessage));
-
-      List<String> successfulMembers = successMap.get(successMessage);
-      int num = 0;
-      for (String member : successfulMembers) {
-        ird.addLine("" + ++num + "." + member);
-      }
-      ird.addLine("\n");
-    }
-    return ird;
-  }
-
-  private ErrorResultData buildFailureData(Map<String, List<String>> successMap,
-      Map<String, List<String>> exceptionMap, Map<String, List<String>> errorMap,
-      String errorHeader) {
-    ErrorResultData erd = ResultBuilder.createErrorResultData();
-    buildErrorResult(erd, successMap);
-    erd.addLine("\n");
-    erd.addLine(errorHeader);
-    buildErrorResult(erd, exceptionMap);
-    buildErrorResult(erd, errorMap);
-    return erd;
-  }
-
-  private void buildErrorResult(ErrorResultData erd, Map<String, List<String>> resultMap) {
-    if (resultMap != null && !resultMap.isEmpty()) {
-      Set<String> messages = resultMap.keySet();
-
-      for (String message : messages) {
-        erd.addLine("\n");
-        erd.addLine(message);
-        erd.addLine(CliStrings.OCCURRED_ON_MEMBERS);
-        List<String> members = resultMap.get(message);
-        int num = 0;
-        for (String member : members) {
-          ++num;
-          erd.addLine("" + num + "." + member);
-        }
-      }
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/geode/blob/440c87f8/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DurableClientCommandsResultBuilder.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DurableClientCommandsResultBuilder.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DurableClientCommandsResultBuilder.java
new file mode 100644
index 0000000..743602a
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DurableClientCommandsResultBuilder.java
@@ -0,0 +1,164 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.domain.MemberResult;
+import org.apache.geode.management.internal.cli.domain.SubscriptionQueueSizeResult;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.ErrorResultData;
+import org.apache.geode.management.internal.cli.result.InfoResultData;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.result.TabularResultData;
+
+public class DurableClientCommandsResultBuilder {
+  public Result buildResult(List<MemberResult> results, String successHeader,
+      String failureHeader) {
+    Result result;
+    boolean failure = true;
+    boolean partialFailure = false;
+    Map<String, List<String>> errorMap = new HashMap<>();
+    Map<String, List<String>> successMap = new HashMap<>();
+    Map<String, List<String>> exceptionMap = new HashMap<>();
+
+    // Aggregate the results from the members
+    for (MemberResult memberResult : results) {
+      if (memberResult.isSuccessful()) {
+        failure = false;
+        groupByMessage(memberResult.getSuccessMessage(), memberResult.getMemberNameOrId(),
+            successMap);
+      } else {
+        if (memberResult.isOpPossible()) {
+          partialFailure = true;
+          groupByMessage(memberResult.getExceptionMessage(), memberResult.getMemberNameOrId(),
+              exceptionMap);
+        } else {
+          groupByMessage(memberResult.getErrorMessage(), memberResult.getMemberNameOrId(),
+              errorMap);
+        }
+      }
+    }
+
+    if (!failure && !partialFailure) {
+      result = ResultBuilder.buildResult(buildSuccessData(successMap));
+    } else {
+      result = ResultBuilder
+          .buildResult(buildFailureData(successMap, exceptionMap, errorMap, failureHeader));
+    }
+    return result;
+  }
+
+  Result buildTableResultForQueueSize(List<SubscriptionQueueSizeResult> results,
+      String queueSizeColumnName) {
+    Result result;
+    boolean failure = true;
+
+    Map<String, List<String>> failureMap = new HashMap<>();
+    Map<String, Long> memberQueueSizeTable = new TreeMap<>();
+
+    // Aggregate the results from the members
+    for (SubscriptionQueueSizeResult memberResult : results) {
+      if (memberResult.isSuccessful()) {
+        failure = false;
+        memberQueueSizeTable.put(memberResult.getMemberNameOrId(),
+            memberResult.getSubscriptionQueueSize());
+      } else {
+        groupByMessage(memberResult.getErrorMessage(), memberResult.getMemberNameOrId(),
+            failureMap);
+      }
+    }
+
+    if (!failure) {
+      TabularResultData table = ResultBuilder.createTabularResultData();
+      Set<String> members = memberQueueSizeTable.keySet();
+
+      for (String member : members) {
+        long queueSize = memberQueueSizeTable.get(member);
+        table.accumulate(CliStrings.MEMBER, member);
+        table.accumulate(queueSizeColumnName, queueSize);
+      }
+      result = ResultBuilder.buildResult(table);
+    } else {
+      ErrorResultData erd = ResultBuilder.createErrorResultData();
+      buildErrorResult(erd, failureMap);
+      result = ResultBuilder.buildResult(erd);
+    }
+    return result;
+  }
+
+  void groupByMessage(String message, String memberNameOrId, Map<String, List<String>> map) {
+    List<String> members = map.get(message);
+    if (members == null) {
+      members = new LinkedList<>();
+    }
+    members.add(memberNameOrId);
+    map.put(message, members);
+  }
+
+
+  private InfoResultData buildSuccessData(Map<String, List<String>> successMap) {
+    InfoResultData ird = ResultBuilder.createInfoResultData();
+    Set<String> successMessages = successMap.keySet();
+
+    for (String successMessage : successMessages) {
+      ird.addLine(CliStrings.format(CliStrings.ACTION_SUCCEEDED_ON_MEMBER, successMessage));
+      List<String> successfulMembers = successMap.get(successMessage);
+      int num = 0;
+      for (String member : successfulMembers) {
+        ird.addLine("" + ++num + "." + member);
+      }
+      ird.addLine("\n");
+    }
+    return ird;
+  }
+
+  ErrorResultData buildFailureData(Map<String, List<String>> successMap,
+      Map<String, List<String>> exceptionMap, Map<String, List<String>> errorMap,
+      String errorHeader) {
+    ErrorResultData erd = ResultBuilder.createErrorResultData();
+    buildErrorResult(erd, successMap);
+    erd.addLine("\n");
+    erd.addLine(errorHeader);
+    buildErrorResult(erd, exceptionMap);
+    buildErrorResult(erd, errorMap);
+    return erd;
+  }
+
+  private void buildErrorResult(ErrorResultData erd, Map<String, List<String>> resultMap) {
+    if (resultMap != null && !resultMap.isEmpty()) {
+      Set<String> messages = resultMap.keySet();
+
+      for (String message : messages) {
+        erd.addLine("\n");
+        erd.addLine(message);
+        erd.addLine(CliStrings.OCCURRED_ON_MEMBERS);
+        List<String> members = resultMap.get(message);
+        int num = 0;
+        for (String member : members) {
+          ++num;
+          erd.addLine("" + num + "." + member);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/440c87f8/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListDurableClientCQsCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListDurableClientCQsCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListDurableClientCQsCommand.java
new file mode 100644
index 0000000..011bfde
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListDurableClientCQsCommand.java
@@ -0,0 +1,121 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+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.CliUtil;
+import org.apache.geode.management.internal.cli.domain.DurableCqNamesResult;
+import org.apache.geode.management.internal.cli.functions.ListDurableCqNamesFunction;
+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.result.TabularResultData;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class ListDurableClientCQsCommand implements GfshCommand {
+  DurableClientCommandsResultBuilder builder = new DurableClientCommandsResultBuilder();
+
+  @CliCommand(value = CliStrings.LIST_DURABLE_CQS, help = CliStrings.LIST_DURABLE_CQS__HELP)
+  @CliMetaData()
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.READ)
+  public Result listDurableClientCQs(
+      @CliOption(key = CliStrings.LIST_DURABLE_CQS__DURABLECLIENTID, mandatory = true,
+          help = CliStrings.LIST_DURABLE_CQS__DURABLECLIENTID__HELP) final String durableClientId,
+
+      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
+          help = CliStrings.LIST_DURABLE_CQS__MEMBER__HELP,
+          optionContext = ConverterHint.MEMBERIDNAME) final String[] memberNameOrId,
+
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          help = CliStrings.LIST_DURABLE_CQS__GROUP__HELP,
+          optionContext = ConverterHint.MEMBERGROUP) final String[] group) {
+    Result result;
+    try {
+
+      boolean noResults = true;
+      Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
+
+      if (targetMembers.isEmpty()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+      }
+
+      final ResultCollector<?, ?> rc =
+          CliUtil.executeFunction(new ListDurableCqNamesFunction(), durableClientId, targetMembers);
+      final List<DurableCqNamesResult> results = (List<DurableCqNamesResult>) rc.getResult();
+      Map<String, List<String>> memberCqNamesMap = new TreeMap<>();
+      Map<String, List<String>> errorMessageNodes = new HashMap<>();
+      Map<String, List<String>> exceptionMessageNodes = new HashMap<>();
+
+      for (DurableCqNamesResult memberResult : results) {
+        if (memberResult != null) {
+          if (memberResult.isSuccessful()) {
+            memberCqNamesMap.put(memberResult.getMemberNameOrId(), memberResult.getCqNamesList());
+          } else {
+            if (memberResult.isOpPossible()) {
+              builder.groupByMessage(memberResult.getExceptionMessage(),
+                  memberResult.getMemberNameOrId(), exceptionMessageNodes);
+            } else {
+              builder.groupByMessage(memberResult.getErrorMessage(),
+                  memberResult.getMemberNameOrId(), errorMessageNodes);
+            }
+          }
+        }
+      }
+
+      if (!memberCqNamesMap.isEmpty()) {
+        TabularResultData table = ResultBuilder.createTabularResultData();
+        Set<String> members = memberCqNamesMap.keySet();
+
+        for (String member : members) {
+          boolean isFirst = true;
+          List<String> cqNames = memberCqNamesMap.get(member);
+          for (String cqName : cqNames) {
+            if (isFirst) {
+              isFirst = false;
+              table.accumulate(CliStrings.MEMBER, member);
+            } else {
+              table.accumulate(CliStrings.MEMBER, "");
+            }
+            table.accumulate(CliStrings.LIST_DURABLE_CQS__NAME, cqName);
+          }
+        }
+        result = ResultBuilder.buildResult(table);
+      } else {
+        String errorHeader =
+            CliStrings.format(CliStrings.LIST_DURABLE_CQS__FAILURE__HEADER, durableClientId);
+        result = ResultBuilder.buildResult(
+            builder.buildFailureData(null, exceptionMessageNodes, errorMessageNodes, errorHeader));
+      }
+    } catch (Exception e) {
+      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/440c87f8/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DurableClientCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DurableClientCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DurableClientCommandsController.java
index 4562cd5..0e62e71 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DurableClientCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DurableClientCommandsController.java
@@ -14,10 +14,6 @@
  */
 package org.apache.geode.management.internal.web.controllers;
 
-import org.apache.geode.distributed.ConfigurationProperties;
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 import org.springframework.stereotype.Controller;
 import org.springframework.web.bind.annotation.PathVariable;
 import org.springframework.web.bind.annotation.RequestMapping;
@@ -25,13 +21,20 @@ import org.springframework.web.bind.annotation.RequestMethod;
 import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 
+import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+
 /**
  * The DurableClientCommandsController class implements GemFire Management REST API web service
  * endpoints for the durable client/CQs Gfsh commands.
  * <p/>
  * 
- * @see org.apache.geode.management.internal.cli.commands.DurableClientCommands
- * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
+ * @see org.apache.geode.management.internal.cli.commands.CloseDurableClientCommand
+ * @see org.apache.geode.management.internal.cli.commands.CloseDurableCQsCommand
+ * @see org.apache.geode.management.internal.cli.commands.CountDurableCQEventsCommand
+ * @see org.apache.geode.management.internal.cli.commands.ListDurableClientCQsCommand
  * @see org.springframework.stereotype.Controller
  * @see org.springframework.web.bind.annotation.PathVariable
  * @see org.springframework.web.bind.annotation.RequestMapping

http://git-wip-us.apache.org/repos/asf/geode/blob/440c87f8/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
index 43e3c04..2b23930 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
@@ -171,14 +171,15 @@ public class TestCommand {
     createTestCommand("alter disk-store --name=foo --region=xyz --disk-dirs=bar");
     createTestCommand("destroy disk-store --name=foo", clusterManageDisk);
 
-    // DurableClientCommands
+    // CloseDurableClientCommand, CloseDurableCQsCommand, CountDurableCQEventsCommand,
+    // ListDurableClientCQsCommand
     createTestCommand("close durable-client --durable-client-id=client1", clusterManageQuery);
     createTestCommand("close durable-cq --durable-client-id=client1 --durable-cq-name=cq1",
         clusterManageQuery);
     createTestCommand("show subscription-queue-size --durable-client-id=client1", clusterRead);
     createTestCommand("list durable-cqs --durable-client-id=client1", clusterRead);
 
-    // ExportIMportSharedConfigurationCommands
+    // ExportImportSharedConfigurationCommands
     createTestCommand("export cluster-configuration --zip-file-name=mySharedConfig.zip",
         clusterRead);
     createTestCommand("import cluster-configuration --zip-file-name=value.zip", clusterManage);


[3/3] geode git commit: GEODE-3264: Refactoring MemberCommands

Posted by kl...@apache.org.
GEODE-3264: Refactoring MemberCommands

This closes #692


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

Branch: refs/heads/develop
Commit: d27f8b956de7d9c5d95ebdc68dfc67ee8b2d7b51
Parents: 440c87f
Author: YehEmily <em...@gmail.com>
Authored: Mon Aug 7 13:09:42 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Aug 11 16:01:23 2017 -0700

----------------------------------------------------------------------
 .../cli/commands/DescribeMemberCommand.java     | 133 +++++++++++++
 .../cli/commands/ListMemberCommand.java         |  77 ++++++++
 .../internal/cli/commands/MemberCommands.java   | 196 -------------------
 .../controllers/MemberCommandsController.java   |  23 +--
 .../internal/security/TestCommand.java          |   2 +-
 5 files changed, 216 insertions(+), 215 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/d27f8b95/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeMemberCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeMemberCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeMemberCommand.java
new file mode 100644
index 0000000..236dd23e
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeMemberCommand.java
@@ -0,0 +1,133 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+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.CliUtil;
+import org.apache.geode.management.internal.cli.domain.CacheServerInfo;
+import org.apache.geode.management.internal.cli.domain.MemberInformation;
+import org.apache.geode.management.internal.cli.functions.GetMemberInformationFunction;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.CompositeResultData;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class DescribeMemberCommand implements GfshCommand {
+  private static final GetMemberInformationFunction getMemberInformation =
+      new GetMemberInformationFunction();
+
+  @CliCommand(value = {CliStrings.DESCRIBE_MEMBER}, help = CliStrings.DESCRIBE_MEMBER__HELP)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_SERVER)
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.READ)
+  public Result describeMember(@CliOption(key = CliStrings.DESCRIBE_MEMBER__IDENTIFIER,
+      optionContext = ConverterHint.ALL_MEMBER_IDNAME,
+      help = CliStrings.DESCRIBE_MEMBER__IDENTIFIER__HELP,
+      mandatory = true) String memberNameOrId) {
+    Result result = null;
+
+    try {
+      DistributedMember memberToBeDescribed =
+          CliUtil.getDistributedMemberByNameOrId(memberNameOrId);
+
+      if (memberToBeDescribed != null) {
+        ResultCollector<?, ?> rc =
+            CliUtil.executeFunction(getMemberInformation, null, memberToBeDescribed);
+
+        ArrayList<?> output = (ArrayList<?>) rc.getResult();
+        Object obj = output.get(0);
+
+        if (obj != null && (obj instanceof MemberInformation)) {
+
+          CompositeResultData crd = ResultBuilder.createCompositeResultData();
+
+          MemberInformation memberInformation = (MemberInformation) obj;
+          memberInformation.setName(memberToBeDescribed.getName());
+          memberInformation.setId(memberToBeDescribed.getId());
+          memberInformation.setHost(memberToBeDescribed.getHost());
+          memberInformation.setProcessId("" + memberToBeDescribed.getProcessId());
+
+          CompositeResultData.SectionResultData section = crd.addSection();
+          section.addData("Name", memberInformation.getName());
+          section.addData("Id", memberInformation.getId());
+          section.addData("Host", memberInformation.getHost());
+          section.addData("Regions",
+              CliUtil.convertStringSetToString(memberInformation.getHostedRegions(), '\n'));
+          section.addData("PID", memberInformation.getProcessId());
+          section.addData("Groups", memberInformation.getGroups());
+          section.addData("Used Heap", memberInformation.getHeapUsage() + "M");
+          section.addData("Max Heap", memberInformation.getMaxHeapSize() + "M");
+
+          String offHeapMemorySize = memberInformation.getOffHeapMemorySize();
+          if (offHeapMemorySize != null && !offHeapMemorySize.isEmpty()) {
+            section.addData("Off Heap Size", offHeapMemorySize);
+          }
+
+          section.addData("Working Dir", memberInformation.getWorkingDirPath());
+          section.addData("Log file", memberInformation.getLogFilePath());
+
+          section.addData("Locators", memberInformation.getLocators());
+
+          if (memberInformation.isServer()) {
+            CompositeResultData.SectionResultData clientServiceSection = crd.addSection();
+            List<CacheServerInfo> csList = memberInformation.getCacheServeInfo();
+
+            if (csList != null) {
+              Iterator<CacheServerInfo> iters = csList.iterator();
+              clientServiceSection.setHeader("Cache Server Information");
+
+              while (iters.hasNext()) {
+                CacheServerInfo cacheServerInfo = iters.next();
+                clientServiceSection.addData("Server Bind", cacheServerInfo.getBindAddress());
+                clientServiceSection.addData("Server Port", cacheServerInfo.getPort());
+                clientServiceSection.addData("Running", cacheServerInfo.isRunning());
+              }
+
+              clientServiceSection.addData("Client Connections",
+                  memberInformation.getClientCount());
+            }
+          }
+          result = ResultBuilder.buildResult(crd);
+
+        } else {
+          result = ResultBuilder.createInfoResult(CliStrings.format(
+              CliStrings.DESCRIBE_MEMBER__MSG__INFO_FOR__0__COULD_NOT_BE_RETRIEVED,
+              new Object[] {memberNameOrId}));
+        }
+      } else {
+        result = ResultBuilder.createInfoResult(CliStrings
+            .format(CliStrings.DESCRIBE_MEMBER__MSG__NOT_FOUND, new Object[] {memberNameOrId}));
+      }
+    } catch (CacheClosedException ignored) {
+    } catch (Exception e) {
+      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/d27f8b95/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMemberCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMemberCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMemberCommand.java
new file mode 100644
index 0000000..ea88c69
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ListMemberCommand.java
@@ -0,0 +1,77 @@
+/*
+ * 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 java.util.Set;
+import java.util.TreeSet;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+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.CliUtil;
+import org.apache.geode.management.internal.cli.LogWrapper;
+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.result.TabularResultData;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class ListMemberCommand implements GfshCommand {
+  @CliCommand(value = {CliStrings.LIST_MEMBER}, help = CliStrings.LIST_MEMBER__HELP)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_SERVER)
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.READ)
+  public Result listMember(@CliOption(key = {CliStrings.GROUP}, unspecifiedDefaultValue = "",
+      optionContext = ConverterHint.MEMBERGROUP,
+      help = CliStrings.LIST_MEMBER__GROUP__HELP) String group) {
+    Result result;
+
+    // TODO: Add the code for identifying the system services
+    try {
+      Set<DistributedMember> memberSet = new TreeSet<>();
+      InternalCache cache = getCache();
+
+      // default get all the members in the DS
+      if (group.isEmpty()) {
+        memberSet.addAll(CliUtil.getAllMembers(cache));
+      } else {
+        memberSet.addAll(cache.getDistributedSystem().getGroupMembers(group));
+      }
+
+      if (memberSet.isEmpty()) {
+        result = ResultBuilder.createInfoResult(CliStrings.LIST_MEMBER__MSG__NO_MEMBER_FOUND);
+      } else {
+        TabularResultData resultData = ResultBuilder.createTabularResultData();
+        for (DistributedMember member : memberSet) {
+          resultData.accumulate("Name", member.getName());
+          resultData.accumulate("Id", member.getId());
+        }
+
+        result = ResultBuilder.buildResult(resultData);
+      }
+    } catch (Exception e) {
+      result = ResultBuilder
+          .createGemFireErrorResult("Could not fetch the list of members. " + e.getMessage());
+      LogWrapper.getInstance().warning(e.getMessage(), e);
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/d27f8b95/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MemberCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MemberCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MemberCommands.java
deleted file mode 100644
index 70a5c79..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/MemberCommands.java
+++ /dev/null
@@ -1,196 +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 java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import org.apache.geode.cache.CacheClosedException;
-import org.apache.geode.cache.execute.FunctionInvocationTargetException;
-import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.internal.cache.InternalCache;
-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.CliUtil;
-import org.apache.geode.management.internal.cli.LogWrapper;
-import org.apache.geode.management.internal.cli.domain.CacheServerInfo;
-import org.apache.geode.management.internal.cli.domain.MemberInformation;
-import org.apache.geode.management.internal.cli.functions.GetMemberInformationFunction;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.CompositeResultData;
-import org.apache.geode.management.internal.cli.result.CompositeResultData.SectionResultData;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.result.TabularResultData;
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission.Operation;
-import org.apache.geode.security.ResourcePermission.Resource;
-
-/**
- * @since GemFire 7.0
- */
-public class MemberCommands implements GfshCommand {
-
-  private static final GetMemberInformationFunction getMemberInformation =
-      new GetMemberInformationFunction();
-
-  @CliCommand(value = {CliStrings.LIST_MEMBER}, help = CliStrings.LIST_MEMBER__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEODE_SERVER)
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-  public Result listMember(@CliOption(key = {CliStrings.GROUP}, unspecifiedDefaultValue = "",
-      optionContext = ConverterHint.MEMBERGROUP,
-      help = CliStrings.LIST_MEMBER__GROUP__HELP) String group) {
-    Result result = null;
-
-    // TODO: Add the code for identifying the system services
-    try {
-      Set<DistributedMember> memberSet = new TreeSet<DistributedMember>();
-      InternalCache cache = getCache();
-
-      // default get all the members in the DS
-      if (group.isEmpty()) {
-        memberSet.addAll(CliUtil.getAllMembers(cache));
-      } else {
-        memberSet.addAll(cache.getDistributedSystem().getGroupMembers(group));
-      }
-
-      if (memberSet.isEmpty()) {
-        result = ResultBuilder.createInfoResult(CliStrings.LIST_MEMBER__MSG__NO_MEMBER_FOUND);
-      } else {
-        TabularResultData resultData = ResultBuilder.createTabularResultData();
-        Iterator<DistributedMember> memberIters = memberSet.iterator();
-        while (memberIters.hasNext()) {
-          DistributedMember member = memberIters.next();
-          resultData.accumulate("Name", member.getName());
-          resultData.accumulate("Id", member.getId());
-        }
-
-        result = ResultBuilder.buildResult(resultData);
-      }
-    } catch (Exception e) {
-
-      result = ResultBuilder
-          .createGemFireErrorResult("Could not fetch the list of members. " + e.getMessage());
-      LogWrapper.getInstance().warning(e.getMessage(), e);
-    }
-
-    return result;
-  }
-
-  @CliCommand(value = {CliStrings.DESCRIBE_MEMBER}, help = CliStrings.DESCRIBE_MEMBER__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEODE_SERVER)
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
-  public Result describeMember(@CliOption(key = CliStrings.DESCRIBE_MEMBER__IDENTIFIER,
-      optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-      help = CliStrings.DESCRIBE_MEMBER__IDENTIFIER__HELP,
-      mandatory = true) String memberNameOrId) {
-    Result result = null;
-
-    try {
-      DistributedMember memberToBeDescribed =
-          CliUtil.getDistributedMemberByNameOrId(memberNameOrId);
-
-      if (memberToBeDescribed != null) {
-        // This information should be available through the MBeans too. We might not need
-        // the function.
-
-        // Yes, but then the command is subject to Mbean availability, which would be
-        // affected once MBean filters are used.
-
-        ResultCollector<?, ?> rc =
-            CliUtil.executeFunction(getMemberInformation, null, memberToBeDescribed);
-
-        ArrayList<?> output = (ArrayList<?>) rc.getResult();
-        Object obj = output.get(0);
-
-        if (obj != null && (obj instanceof MemberInformation)) {
-
-          CompositeResultData crd = ResultBuilder.createCompositeResultData();
-
-          MemberInformation memberInformation = (MemberInformation) obj;
-          memberInformation.setName(memberToBeDescribed.getName());
-          memberInformation.setId(memberToBeDescribed.getId());
-          memberInformation.setHost(memberToBeDescribed.getHost());
-          memberInformation.setProcessId("" + memberToBeDescribed.getProcessId());
-
-          SectionResultData section = crd.addSection();
-          section.addData("Name", memberInformation.getName());
-          section.addData("Id", memberInformation.getId());
-          section.addData("Host", memberInformation.getHost());
-          section.addData("Regions",
-              CliUtil.convertStringSetToString(memberInformation.getHostedRegions(), '\n'));
-          section.addData("PID", memberInformation.getProcessId());
-          section.addData("Groups", memberInformation.getGroups());
-          section.addData("Used Heap", memberInformation.getHeapUsage() + "M");
-          section.addData("Max Heap", memberInformation.getMaxHeapSize() + "M");
-
-          String offHeapMemorySize = memberInformation.getOffHeapMemorySize();
-          if (offHeapMemorySize != null && !offHeapMemorySize.isEmpty()) {
-            section.addData("Off Heap Size", offHeapMemorySize);
-          }
-
-          section.addData("Working Dir", memberInformation.getWorkingDirPath());
-          section.addData("Log file", memberInformation.getLogFilePath());
-
-          section.addData("Locators", memberInformation.getLocators());
-
-          if (memberInformation.isServer()) {
-            SectionResultData clientServiceSection = crd.addSection();
-            List<CacheServerInfo> csList = memberInformation.getCacheServeInfo();
-
-            if (csList != null) {
-              Iterator<CacheServerInfo> iters = csList.iterator();
-              clientServiceSection.setHeader("Cache Server Information");
-
-              while (iters.hasNext()) {
-                CacheServerInfo cacheServerInfo = iters.next();
-                clientServiceSection.addData("Server Bind", cacheServerInfo.getBindAddress());
-                clientServiceSection.addData("Server Port", cacheServerInfo.getPort());
-                clientServiceSection.addData("Running", cacheServerInfo.isRunning());
-              }
-
-              clientServiceSection.addData("Client Connections",
-                  memberInformation.getClientCount());
-            }
-          }
-          result = ResultBuilder.buildResult(crd);
-
-        } else {
-          result = ResultBuilder.createInfoResult(CliStrings.format(
-              CliStrings.DESCRIBE_MEMBER__MSG__INFO_FOR__0__COULD_NOT_BE_RETRIEVED,
-              new Object[] {memberNameOrId}));
-        }
-      } else {
-        result = ResultBuilder.createInfoResult(CliStrings
-            .format(CliStrings.DESCRIBE_MEMBER__MSG__NOT_FOUND, new Object[] {memberNameOrId}));
-      }
-    } catch (CacheClosedException e) {
-
-    } catch (FunctionInvocationTargetException e) {
-      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
-    } catch (Exception e) {
-      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
-    }
-    return result;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/d27f8b95/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
index 66e82ee..ba5c788 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/MemberCommandsController.java
@@ -14,9 +14,6 @@
  */
 package org.apache.geode.management.internal.web.controllers;
 
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-
 import org.springframework.stereotype.Controller;
 import org.springframework.web.bind.annotation.PathVariable;
 import org.springframework.web.bind.annotation.RequestMapping;
@@ -24,12 +21,16 @@ import org.springframework.web.bind.annotation.RequestMethod;
 import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+
 /**
  * The MemberCommandsController class implements GemFire Management REST API web service endpoints
  * for the Gfsh Member Commands.
  * <p/>
  * 
- * @see org.apache.geode.management.internal.cli.commands.MemberCommands
+ * @see org.apache.geode.management.internal.cli.commands.ListMemberCommand
+ * @see org.apache.geode.management.internal.cli.commands.DescribeMemberCommand
  * @see org.apache.geode.management.internal.web.controllers.AbstractCommandsController
  * @see org.springframework.stereotype.Controller
  * @see org.springframework.web.bind.annotation.PathVariable
@@ -46,25 +47,12 @@ public class MemberCommandsController extends AbstractCommandsController {
 
   @RequestMapping(method = RequestMethod.GET, value = "/members")
   @ResponseBody
-  // public String listMembers(@RequestBody MultiValueMap<String, String> requestParameters) {
-  // public String listMembers(@RequestParam(value = "group", required = false) final String
-  // groupName,
-  // @RequestParam(value = "group", required = false) final String[] groupNames) {
   public String listMembers(
       @RequestParam(value = CliStrings.GROUP, required = false) final String groupName) {
     CommandStringBuilder command = new CommandStringBuilder(CliStrings.LIST_MEMBER);
-
-    // logger.info(String.format("Request Body: %1$s", requestParameters));
-    // logger.info(String.format("Request Parameter (group): %1$s", groupName));
-    // logger.info(String.format("Request Parameter (group) as array: %1$s",
-    // ArrayUtils.toString(groupNames)));
-
-    // final String groupName = requestParameters.getFirst("group");
-
     if (hasValue(groupName)) {
       command.addOption(CliStrings.GROUP, groupName);
     }
-
     return processCommand(command.toString());
   }
 
@@ -75,5 +63,4 @@ public class MemberCommandsController extends AbstractCommandsController {
     command.addOption(CliStrings.DESCRIBE_MEMBER__IDENTIFIER, decode(memberNameId));
     return processCommand(command.toString());
   }
-
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/d27f8b95/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
index 2b23930..3aca001 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
@@ -218,7 +218,7 @@ public class TestCommand {
     // createTestCommand("stop locator --name=locator1", clusterManage);
     // createTestCommand("stop server --name=server1", clusterManage);
 
-    // MemberCommands
+    // DescribeMemberCommand, ListMemberCommand
     createTestCommand("describe member --name=server1", clusterRead);
     createTestCommand("list members", clusterRead);