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

[12/50] [abbrv] geode git commit: GEODE-3436: revert recent refactoring of GFSH commands

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/NetstatCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/NetstatCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/NetstatCommand.java
deleted file mode 100644
index a5d62fa..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/NetstatCommand.java
+++ /dev/null
@@ -1,212 +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.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.zip.DataFormatException;
-
-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.distributed.internal.InternalDistributedSystem;
-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.GfshParser;
-import org.apache.geode.management.internal.cli.LogWrapper;
-import org.apache.geode.management.internal.cli.functions.NetstatFunction;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.remote.CommandExecutionContext;
-import org.apache.geode.management.internal.cli.result.InfoResultData;
-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 NetstatCommand implements GfshCommand {
-  private static final String NETSTAT_FILE_REQUIRED_EXTENSION = ".txt";
-
-  @CliCommand(value = CliStrings.NETSTAT, help = CliStrings.NETSTAT__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DEBUG_UTIL})
-  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
-      operation = ResourcePermission.Operation.READ)
-  // TODO : Verify the auto-completion for multiple values.
-  public Result netstat(
-      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
-          optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-          help = CliStrings.NETSTAT__MEMBER__HELP) String[] members,
-      @CliOption(key = CliStrings.GROUP, optionContext = ConverterHint.MEMBERGROUP,
-          help = CliStrings.NETSTAT__GROUP__HELP) String group,
-      @CliOption(key = CliStrings.NETSTAT__FILE,
-          help = CliStrings.NETSTAT__FILE__HELP) String saveAs,
-      @CliOption(key = CliStrings.NETSTAT__WITHLSOF, specifiedDefaultValue = "true",
-          unspecifiedDefaultValue = "false",
-          help = CliStrings.NETSTAT__WITHLSOF__HELP) boolean withlsof) {
-    Result result;
-
-    Map<String, DistributedMember> hostMemberMap = new HashMap<>();
-    Map<String, List<String>> hostMemberListMap = new HashMap<>();
-
-    try {
-      if (members != null && members.length > 0 && group != null) {
-        throw new IllegalArgumentException(
-            CliStrings.NETSTAT__MSG__ONLY_ONE_OF_MEMBER_OR_GROUP_SHOULD_BE_SPECIFIED);
-      }
-      StringBuilder resultInfo = new StringBuilder();
-
-      // Execute for remote members whose id or name matches
-      InternalDistributedSystem system = InternalDistributedSystem.getConnectedInstance();
-
-      if (members != null) {
-        Set<String> notFoundMembers = new HashSet<>();
-        for (String memberIdOrName : members) {
-          Set<DistributedMember> membersToExecuteOn = CliUtil.getAllMembers(system);
-          boolean memberFound = false;
-          for (DistributedMember distributedMember : membersToExecuteOn) {
-            String memberName = distributedMember.getName();
-            String memberId = distributedMember.getId();
-            if (memberName.equals(memberIdOrName) || memberId.equals(memberIdOrName)) {
-              buildMaps(hostMemberMap, hostMemberListMap, memberIdOrName, distributedMember);
-
-              memberFound = true;
-              break;
-            }
-          }
-          if (!memberFound) {
-            notFoundMembers.add(memberIdOrName);
-          }
-        }
-        // if there are not found members, it's probably unknown member or member has departed
-        if (!notFoundMembers.isEmpty()) {
-          throw new IllegalArgumentException(
-              CliStrings.format(CliStrings.NETSTAT__MSG__COULD_NOT_FIND_MEMBERS_0,
-                  new Object[] {CliUtil.collectionToString(notFoundMembers, -1)}));
-        }
-      } else {
-        Set<DistributedMember> membersToExecuteOn;
-        if (group != null) {
-          membersToExecuteOn = system.getGroupMembers(group);
-        } else {
-          // consider all members
-          membersToExecuteOn = CliUtil.getAllMembers(system);
-        }
-
-        for (DistributedMember distributedMember : membersToExecuteOn) {
-          String memberName = distributedMember.getName();
-          String memberId = distributedMember.getId();
-          String memberIdOrName =
-              memberName != null && !memberName.isEmpty() ? memberName : memberId;
-
-          buildMaps(hostMemberMap, hostMemberListMap, memberIdOrName, distributedMember);
-        }
-      }
-
-      String lineSeparatorToUse;
-      lineSeparatorToUse = CommandExecutionContext.getShellLineSeparator();
-      if (lineSeparatorToUse == null) {
-        lineSeparatorToUse = GfshParser.LINE_SEPARATOR;
-      }
-      NetstatFunction.NetstatFunctionArgument nfa =
-          new NetstatFunction.NetstatFunctionArgument(lineSeparatorToUse, withlsof);
-
-      if (!hostMemberMap.isEmpty()) {
-        Set<DistributedMember> membersToExecuteOn = new HashSet<>(hostMemberMap.values());
-        ResultCollector<?, ?> netstatResult =
-            CliUtil.executeFunction(NetstatFunction.INSTANCE, nfa, membersToExecuteOn);
-        List<?> resultList = (List<?>) netstatResult.getResult();
-        for (Object aResultList : resultList) {
-          NetstatFunction.NetstatFunctionResult netstatFunctionResult =
-              (NetstatFunction.NetstatFunctionResult) aResultList;
-          CliUtil.DeflaterInflaterData deflaterInflaterData =
-              netstatFunctionResult.getCompressedBytes();
-          try {
-            String remoteHost = netstatFunctionResult.getHost();
-            List<String> membersList = hostMemberListMap.get(remoteHost);
-            resultInfo.append(MessageFormat.format(netstatFunctionResult.getHeaderInfo(),
-                CliUtil.collectionToString(membersList, 120)));
-            CliUtil.DeflaterInflaterData uncompressedBytes = CliUtil.uncompressBytes(
-                deflaterInflaterData.getData(), deflaterInflaterData.getDataLength());
-            resultInfo.append(new String(uncompressedBytes.getData()));
-          } catch (DataFormatException e) {
-            resultInfo.append("Error in some data. Reason : ").append(e.getMessage());
-          }
-        }
-      }
-
-      InfoResultData resultData = ResultBuilder.createInfoResultData();
-      if (saveAs != null && !saveAs.isEmpty()) {
-        String saveToFile = saveAs;
-        if (!saveAs.endsWith(NETSTAT_FILE_REQUIRED_EXTENSION)) {
-          saveToFile = saveAs + NETSTAT_FILE_REQUIRED_EXTENSION;
-        }
-        resultData.addAsFile(saveToFile, resultInfo.toString(),
-            CliStrings.NETSTAT__MSG__SAVED_OUTPUT_IN_0, false); // Note: substitution for {0} will
-        // happen on client side.
-      } else {
-        resultData.addLine(resultInfo.toString());
-      }
-      result = ResultBuilder.buildResult(resultData);
-    } catch (IllegalArgumentException e) {
-      LogWrapper.getInstance()
-          .info(CliStrings.format(
-              CliStrings.NETSTAT__MSG__ERROR_OCCURRED_WHILE_EXECUTING_NETSTAT_ON_0,
-              new Object[] {Arrays.toString(members)}));
-      result = ResultBuilder.createUserErrorResult(e.getMessage());
-    } catch (RuntimeException e) {
-      LogWrapper.getInstance()
-          .info(CliStrings.format(
-              CliStrings.NETSTAT__MSG__ERROR_OCCURRED_WHILE_EXECUTING_NETSTAT_ON_0,
-              new Object[] {Arrays.toString(members)}), e);
-      result = ResultBuilder.createGemFireErrorResult(
-          CliStrings.format(CliStrings.NETSTAT__MSG__ERROR_OCCURRED_WHILE_EXECUTING_NETSTAT_ON_0,
-              new Object[] {Arrays.toString(members)}));
-    } finally {
-      hostMemberMap.clear();
-      hostMemberListMap.clear();
-    }
-    return result;
-  }
-
-  private void buildMaps(Map<String, DistributedMember> hostMemberMap,
-      Map<String, List<String>> hostMemberListMap, String memberIdOrName,
-      DistributedMember distributedMember) {
-    String host = distributedMember.getHost();
-
-    // Maintain one member for a host - function execution purpose - once only for a host
-    if (!hostMemberMap.containsKey(host)) {
-      hostMemberMap.put(host, distributedMember);
-    }
-
-    // Maintain all members for a host - display purpose
-    List<String> list;
-    if (!hostMemberListMap.containsKey(host)) {
-      list = new ArrayList<>();
-      hostMemberListMap.put(host, list);
-    } else {
-      list = hostMemberListMap.get(host);
-    }
-    list.add(memberIdOrName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXCommands.java
new file mode 100644
index 0000000..e0bad15
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXCommands.java
@@ -0,0 +1,195 @@
+/*
+ * 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.ByteArrayOutputStream;
+import java.io.File;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.internal.cache.CacheConfig;
+import org.apache.geode.internal.cache.DiskStoreImpl;
+import org.apache.geode.internal.cache.xmlcache.CacheCreation;
+import org.apache.geode.internal.cache.xmlcache.CacheXml;
+import org.apache.geode.internal.cache.xmlcache.CacheXmlGenerator;
+import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.CliUtil;
+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.configuration.domain.XmlEntity;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.pdx.ReflectionBasedAutoSerializer;
+import org.apache.geode.pdx.internal.EnumInfo;
+import org.apache.geode.pdx.internal.PdxType;
+import org.apache.geode.security.ResourcePermission.Operation;
+import org.apache.geode.security.ResourcePermission.Resource;
+
+public class PDXCommands implements GfshCommand {
+
+
+  @CliCommand(value = CliStrings.CONFIGURE_PDX, help = CliStrings.CONFIGURE_PDX__HELP)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION)
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE)
+  public Result configurePDX(
+      @CliOption(key = CliStrings.CONFIGURE_PDX__READ__SERIALIZED,
+          help = CliStrings.CONFIGURE_PDX__READ__SERIALIZED__HELP) Boolean readSerialized,
+      @CliOption(key = CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS,
+          help = CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS__HELP) Boolean ignoreUnreadFields,
+      @CliOption(key = CliStrings.CONFIGURE_PDX__DISKSTORE, specifiedDefaultValue = "",
+          help = CliStrings.CONFIGURE_PDX__DISKSTORE__HELP) String diskStore,
+      @CliOption(key = CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES,
+          help = CliStrings.CONFIGURE_PDX__AUTO__SERIALIZER__CLASSES__HELP) String[] patterns,
+      @CliOption(key = CliStrings.CONFIGURE_PDX__PORTABLE__AUTO__SERIALIZER__CLASSES,
+          help = CliStrings.CONFIGURE_PDX__PORTABLE__AUTO__SERIALIZER__CLASSES__HELP) String[] portablePatterns) {
+    Result result;
+
+    try {
+      InfoResultData ird = ResultBuilder.createInfoResultData();
+      CacheCreation cache = new CacheCreation(true);
+
+      if ((portablePatterns != null && portablePatterns.length > 0)
+          && (patterns != null && patterns.length > 0)) {
+        return ResultBuilder.createUserErrorResult(CliStrings.CONFIGURE_PDX__ERROR__MESSAGE);
+      }
+      if (!CliUtil.getAllNormalMembers(CliUtil.getCacheIfExists()).isEmpty()) {
+        ird.addLine(CliStrings.CONFIGURE_PDX__NORMAL__MEMBERS__WARNING);
+      }
+      // Set persistent and the disk-store
+      if (diskStore != null) {
+        cache.setPdxPersistent(true);
+        ird.addLine(CliStrings.CONFIGURE_PDX__PERSISTENT + " = " + cache.getPdxPersistent());
+        if (!diskStore.equals("")) {
+          cache.setPdxDiskStore(diskStore);
+          ird.addLine(CliStrings.CONFIGURE_PDX__DISKSTORE + " = " + cache.getPdxDiskStore());
+        } else {
+          ird.addLine(CliStrings.CONFIGURE_PDX__DISKSTORE + " = " + "DEFAULT");
+        }
+      } else {
+        cache.setPdxPersistent(CacheConfig.DEFAULT_PDX_PERSISTENT);
+        ird.addLine(CliStrings.CONFIGURE_PDX__PERSISTENT + " = " + cache.getPdxPersistent());
+      }
+
+      // Set read-serialized
+      if (readSerialized != null) {
+        cache.setPdxReadSerialized(readSerialized);
+      } else {
+        cache.setPdxReadSerialized(CacheConfig.DEFAULT_PDX_READ_SERIALIZED);
+      }
+      ird.addLine(
+          CliStrings.CONFIGURE_PDX__READ__SERIALIZED + " = " + cache.getPdxReadSerialized());
+
+
+      // Set ignoreUnreadFields
+      if (ignoreUnreadFields != null) {
+        cache.setPdxIgnoreUnreadFields(ignoreUnreadFields);
+      } else {
+        cache.setPdxIgnoreUnreadFields(CacheConfig.DEFAULT_PDX_IGNORE_UNREAD_FIELDS);
+      }
+      ird.addLine(CliStrings.CONFIGURE_PDX__IGNORE__UNREAD_FIELDS + " = "
+          + cache.getPdxIgnoreUnreadFields());
+
+
+      if (portablePatterns != null) {
+        ReflectionBasedAutoSerializer autoSerializer =
+            new ReflectionBasedAutoSerializer(portablePatterns);
+        cache.setPdxSerializer(autoSerializer);
+        ird.addLine("PDX Serializer " + cache.getPdxSerializer().getClass().getName());
+        ird.addLine("Portable classes " + Arrays.toString(portablePatterns));
+      }
+
+      if (patterns != null) {
+        ReflectionBasedAutoSerializer nonPortableAutoSerializer =
+            new ReflectionBasedAutoSerializer(true, patterns);
+        cache.setPdxSerializer(nonPortableAutoSerializer);
+        ird.addLine("PDX Serializer : " + cache.getPdxSerializer().getClass().getName());
+        ird.addLine("Non portable classes :" + Arrays.toString(patterns));
+      }
+
+      final StringWriter stringWriter = new StringWriter();
+      final PrintWriter printWriter = new PrintWriter(stringWriter);
+      CacheXmlGenerator.generate(cache, printWriter, true, false, false);
+      printWriter.close();
+      String xmlDefinition = stringWriter.toString();
+      // TODO jbarrett - shouldn't this use the same loadXmlDefinition that other constructors use?
+      XmlEntity xmlEntity =
+          XmlEntity.builder().withType(CacheXml.PDX).withConfig(xmlDefinition).build();
+
+      result = ResultBuilder.buildResult(ird);
+      persistClusterConfiguration(result,
+          () -> getSharedConfiguration().addXmlEntity(xmlEntity, null));
+
+    } catch (Exception e) {
+      return ResultBuilder.createGemFireErrorResult(e.getMessage());
+    }
+
+    return result;
+  }
+
+  @CliCommand(value = CliStrings.PDX_RENAME, help = CliStrings.PDX_RENAME__HELP)
+  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
+  public Result pdxRename(@CliOption(key = CliStrings.PDX_RENAME_OLD, mandatory = true,
+      help = CliStrings.PDX_RENAME_OLD__HELP) String oldClassName,
+
+      @CliOption(key = CliStrings.PDX_RENAME_NEW, mandatory = true,
+          help = CliStrings.PDX_RENAME_NEW__HELP) String newClassName,
+
+      @CliOption(key = CliStrings.PDX_DISKSTORE, mandatory = true,
+          help = CliStrings.PDX_DISKSTORE__HELP) String diskStore,
+
+      @CliOption(key = CliStrings.PDX_DISKDIR, mandatory = true,
+          help = CliStrings.PDX_DISKDIR__HELP) String[] diskDirs) {
+
+    try {
+      final File[] dirs = new File[diskDirs.length];
+      for (int i = 0; i < diskDirs.length; i++) {
+        dirs[i] = new File((diskDirs[i]));
+      }
+
+      Collection<Object> results =
+          DiskStoreImpl.pdxRename(diskStore, dirs, oldClassName, newClassName);
+
+      if (results.isEmpty()) {
+        return ResultBuilder
+            .createGemFireErrorResult(CliStrings.format(CliStrings.PDX_RENAME__EMPTY));
+      }
+
+      ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+      PrintStream printStream = new PrintStream(outputStream);
+      for (Object p : results) {
+        if (p instanceof PdxType) {
+          ((PdxType) p).toStream(printStream, false);
+        } else {
+          ((EnumInfo) p).toStream(printStream);
+        }
+      }
+      String resultString =
+          CliStrings.format(CliStrings.PDX_RENAME__SUCCESS, outputStream.toString());
+      return ResultBuilder.createInfoResult(resultString);
+
+    } catch (Exception e) {
+      return ResultBuilder.createGemFireErrorResult(
+          CliStrings.format(CliStrings.PDX_RENAME__ERROR, e.getMessage()));
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXRenameCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXRenameCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXRenameCommand.java
deleted file mode 100644
index 8831597..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/PDXRenameCommand.java
+++ /dev/null
@@ -1,81 +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.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.PrintStream;
-import java.util.Collection;
-
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import org.apache.geode.internal.cache.DiskStoreImpl;
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.pdx.internal.EnumInfo;
-import org.apache.geode.pdx.internal.PdxType;
-
-public class PDXRenameCommand implements GfshCommand {
-  @CliCommand(value = CliStrings.PDX_RENAME, help = CliStrings.PDX_RENAME__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
-  public Result pdxRename(@CliOption(key = CliStrings.PDX_RENAME_OLD, mandatory = true,
-      help = CliStrings.PDX_RENAME_OLD__HELP) String oldClassName,
-
-      @CliOption(key = CliStrings.PDX_RENAME_NEW, mandatory = true,
-          help = CliStrings.PDX_RENAME_NEW__HELP) String newClassName,
-
-      @CliOption(key = CliStrings.PDX_DISKSTORE, mandatory = true,
-          help = CliStrings.PDX_DISKSTORE__HELP) String diskStore,
-
-      @CliOption(key = CliStrings.PDX_DISKDIR, mandatory = true,
-          help = CliStrings.PDX_DISKDIR__HELP) String[] diskDirs) {
-
-    try {
-      final File[] dirs = new File[diskDirs.length];
-      for (int i = 0; i < diskDirs.length; i++) {
-        dirs[i] = new File((diskDirs[i]));
-      }
-
-      Collection<Object> results =
-          DiskStoreImpl.pdxRename(diskStore, dirs, oldClassName, newClassName);
-
-      if (results.isEmpty()) {
-        return ResultBuilder
-            .createGemFireErrorResult(CliStrings.format(CliStrings.PDX_RENAME__EMPTY));
-      }
-
-      ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-      PrintStream printStream = new PrintStream(outputStream);
-      for (Object p : results) {
-        if (p instanceof PdxType) {
-          ((PdxType) p).toStream(printStream, false);
-        } else {
-          ((EnumInfo) p).toStream(printStream);
-        }
-      }
-      String resultString =
-          CliStrings.format(CliStrings.PDX_RENAME__SUCCESS, outputStream.toString());
-      return ResultBuilder.createInfoResult(resultString);
-
-    } catch (Exception e) {
-      return ResultBuilder.createGemFireErrorResult(
-          CliStrings.format(CliStrings.PDX_RENAME__ERROR, e.getMessage()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueueCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueueCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueueCommands.java
new file mode 100644
index 0000000..87c5bbc
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/QueueCommands.java
@@ -0,0 +1,259 @@
+/*
+ * 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.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+
+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.ConverterHint;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.cli.Result.Status;
+import org.apache.geode.management.internal.cli.CliUtil;
+import org.apache.geode.management.internal.cli.domain.AsyncEventQueueDetails;
+import org.apache.geode.management.internal.cli.functions.AsyncEventQueueFunctionArgs;
+import org.apache.geode.management.internal.cli.functions.CliFunctionResult;
+import org.apache.geode.management.internal.cli.functions.CreateAsyncEventQueueFunction;
+import org.apache.geode.management.internal.cli.functions.ListAsyncEventQueuesFunction;
+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.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;
+import org.apache.geode.security.ResourcePermission.Target;
+
+/**
+ * The QueueCommands class encapsulates all GemFire Queue commands in Gfsh.
+ * </p>
+ * 
+ * @since GemFire 8.0
+ */
+public class QueueCommands implements GfshCommand {
+
+  @CliCommand(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE,
+      help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__HELP)
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.MANAGE, target = Target.JAR)
+  public Result createAsyncEventQueue(
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ID, mandatory = true,
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ID__HELP) String id,
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          optionContext = ConverterHint.MEMBERGROUP,
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__GROUP__HELP) String[] groups,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL,
+          unspecifiedDefaultValue = "false", specifiedDefaultValue = "true",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL__HELP) Boolean parallel,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ENABLEBATCHCONFLATION,
+          unspecifiedDefaultValue = "false", specifiedDefaultValue = "true",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ENABLEBATCHCONFLATION__HELP) Boolean enableBatchConflation,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCH_SIZE,
+          unspecifiedDefaultValue = "100",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCH_SIZE__HELP) int batchSize,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCHTIMEINTERVAL,
+          unspecifiedDefaultValue = "1000",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCHTIMEINTERVAL__HELP) int batchTimeInterval,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT,
+          unspecifiedDefaultValue = "false", specifiedDefaultValue = "true",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT__HELP) boolean persistent,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISK_STORE,
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISK_STORE__HELP) String diskStore,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS,
+          unspecifiedDefaultValue = "true", specifiedDefaultValue = "true",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS__HELP) Boolean diskSynchronous,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY,
+          unspecifiedDefaultValue = "false", specifiedDefaultValue = "false",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY__HELP) Boolean ignoreEvictionAndExpiration,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY,
+          unspecifiedDefaultValue = "100",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY__HELP) int maxQueueMemory,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS,
+          unspecifiedDefaultValue = "1",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS__HELP) Integer dispatcherThreads,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY,
+          unspecifiedDefaultValue = "KEY",
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY__HELP) String orderPolicy,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER,
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER__HELP) String[] gatewayEventFilters,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER,
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER__HELP) String gatewaySubstitutionListener,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER, mandatory = true,
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER__HELP) String listener,
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE,
+          help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE__HELP) String[] listenerParamsAndValues) {
+
+    if (persistent) {
+      getSecurityService().authorize(Resource.CLUSTER, Operation.WRITE, Target.DISK);
+    }
+    Properties listenerProperties = new Properties();
+
+    try {
+      if (listenerParamsAndValues != null) {
+        for (String listenerParamsAndValue : listenerParamsAndValues) {
+          final int hashPosition = listenerParamsAndValue.indexOf('#');
+          if (hashPosition == -1) {
+            listenerProperties.put(listenerParamsAndValue, "");
+          } else {
+            listenerProperties.put(listenerParamsAndValue.substring(0, hashPosition),
+                listenerParamsAndValue.substring(hashPosition + 1));
+          }
+        }
+      }
+
+      TabularResultData tabularData = ResultBuilder.createTabularResultData();
+      boolean accumulatedData = false;
+
+      Set<DistributedMember> targetMembers = CliUtil.findMembers(groups, null);
+
+      if (targetMembers.isEmpty()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+      }
+
+      AsyncEventQueueFunctionArgs aeqArgs = new AsyncEventQueueFunctionArgs(id, parallel,
+          enableBatchConflation, batchSize, batchTimeInterval, persistent, diskStore,
+          diskSynchronous, maxQueueMemory, dispatcherThreads, orderPolicy, gatewayEventFilters,
+          gatewaySubstitutionListener, listener, listenerProperties, ignoreEvictionAndExpiration);
+
+      ResultCollector<?, ?> rc =
+          CliUtil.executeFunction(new CreateAsyncEventQueueFunction(), aeqArgs, targetMembers);
+
+      List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
+
+      AtomicReference<XmlEntity> xmlEntity = new AtomicReference<>();
+      for (CliFunctionResult result : results) {
+        if (result.getThrowable() != null) {
+          tabularData.accumulate("Member", result.getMemberIdOrName());
+          tabularData.accumulate("Result", "ERROR: " + result.getThrowable().getClass().getName()
+              + ": " + result.getThrowable().getMessage());
+          accumulatedData = true;
+          tabularData.setStatus(Status.ERROR);
+        } else if (result.isSuccessful()) {
+          tabularData.accumulate("Member", result.getMemberIdOrName());
+          tabularData.accumulate("Result", result.getMessage());
+          accumulatedData = true;
+
+          if (xmlEntity.get() == null) {
+            xmlEntity.set(result.getXmlEntity());
+          }
+        }
+      }
+
+      if (!accumulatedData) {
+        return ResultBuilder.createInfoResult("Unable to create async event queue(s).");
+      }
+
+      Result result = ResultBuilder.buildResult(tabularData);
+      if (xmlEntity.get() != null) {
+        persistClusterConfiguration(result,
+            () -> getSharedConfiguration().addXmlEntity(xmlEntity.get(), groups));
+      }
+      return result;
+    } catch (VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+      throw e;
+    } catch (Throwable th) {
+      SystemFailure.checkFailure();
+      return ResultBuilder.createGemFireErrorResult(
+          CliStrings.format(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ERROR_WHILE_CREATING_REASON_0,
+              new Object[] {th.getMessage()}));
+    }
+  }
+
+  @CliCommand(value = CliStrings.LIST_ASYNC_EVENT_QUEUES,
+      help = CliStrings.LIST_ASYNC_EVENT_QUEUES__HELP)
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
+  public Result listAsyncEventQueues() {
+    try {
+      TabularResultData tabularData = ResultBuilder.createTabularResultData();
+      boolean accumulatedData = false;
+
+      Set<DistributedMember> targetMembers = CliUtil.findMembers(null, null);
+
+      if (targetMembers.isEmpty()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+      }
+
+      ResultCollector<?, ?> rc = CliUtil.executeFunction(new ListAsyncEventQueuesFunction(),
+          new Object[] {}, targetMembers);
+      List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
+
+      for (CliFunctionResult result : results) {
+        if (result.getThrowable() != null) {
+          tabularData.accumulate("Member", result.getMemberIdOrName());
+          tabularData.accumulate("Result", "ERROR: " + result.getThrowable().getClass().getName()
+              + ": " + result.getThrowable().getMessage());
+          accumulatedData = true;
+          tabularData.setStatus(Status.ERROR);
+        } else {
+          AsyncEventQueueDetails[] details = (AsyncEventQueueDetails[]) result.getSerializables();
+          for (AsyncEventQueueDetails detail : details) {
+            tabularData.accumulate("Member", result.getMemberIdOrName());
+            tabularData.accumulate("ID", detail.getId());
+            tabularData.accumulate("Batch Size", detail.getBatchSize());
+            tabularData.accumulate("Persistent", detail.isPersistent());
+            tabularData.accumulate("Disk Store", detail.getDiskStoreName());
+            tabularData.accumulate("Max Memory", detail.getMaxQueueMemory());
+
+            Properties listenerProperties = detail.getListenerProperties();
+            if (listenerProperties == null || listenerProperties.size() == 0) {
+              tabularData.accumulate("Listener", detail.getListener());
+            } else {
+              StringBuilder propsStringBuilder = new StringBuilder();
+              propsStringBuilder.append('(');
+              boolean firstProperty = true;
+              for (Map.Entry<Object, Object> property : listenerProperties.entrySet()) {
+                if (!firstProperty) {
+                  propsStringBuilder.append(',');
+                } else {
+                  firstProperty = false;
+                }
+                propsStringBuilder.append(property.getKey()).append('=')
+                    .append(property.getValue());
+              }
+              propsStringBuilder.append(')');
+
+              tabularData.accumulate("Listener",
+                  detail.getListener() + propsStringBuilder.toString());
+            }
+            accumulatedData = true;
+          }
+        }
+      }
+
+      if (!accumulatedData) {
+        return ResultBuilder
+            .createInfoResult(CliStrings.LIST_ASYNC_EVENT_QUEUES__NO_QUEUES_FOUND_MESSAGE);
+      }
+
+      return ResultBuilder.buildResult(tabularData);
+    } catch (VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+      throw e;
+    } catch (Throwable th) {
+      SystemFailure.checkFailure();
+      return ResultBuilder.createGemFireErrorResult(
+          CliStrings.format(CliStrings.LIST_ASYNC_EVENT_QUEUES__ERROR_WHILE_LISTING_REASON_0,
+              new Object[] {th.getMessage()}));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RegionCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RegionCommands.java
new file mode 100644
index 0000000..3222666
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RegionCommands.java
@@ -0,0 +1,474 @@
+/*
+ * 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.LinkedHashSet;
+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.Region;
+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.FixedPartitionAttributesInfo;
+import org.apache.geode.management.internal.cli.domain.RegionDescription;
+import org.apache.geode.management.internal.cli.domain.RegionDescriptionPerMember;
+import org.apache.geode.management.internal.cli.domain.RegionInformation;
+import org.apache.geode.management.internal.cli.functions.GetRegionDescriptionFunction;
+import org.apache.geode.management.internal.cli.functions.GetRegionsFunction;
+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.cli.util.RegionAttributesNames;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission.Operation;
+import org.apache.geode.security.ResourcePermission.Resource;
+
+/**
+ * Class containing implementation of commands based on region:
+ * <ul>
+ * <li>list region
+ * <li>describe region
+ * </ul>
+ * 
+ * @since GemFire 7.0
+ */
+public class RegionCommands implements GfshCommand {
+  private static final GetRegionsFunction getRegionsFunction = new GetRegionsFunction();
+  private static final GetRegionDescriptionFunction getRegionDescription =
+      new GetRegionDescriptionFunction();
+
+  @CliCommand(value = {CliStrings.LIST_REGION}, help = CliStrings.LIST_REGION__HELP)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_REGION)
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
+  public Result listRegion(
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          optionContext = ConverterHint.MEMBERGROUP,
+          help = CliStrings.LIST_REGION__GROUP__HELP) String[] group,
+      @CliOption(key = {CliStrings.MEMBER, CliStrings.MEMBERS},
+          optionContext = ConverterHint.MEMBERIDNAME,
+          help = CliStrings.LIST_REGION__MEMBER__HELP) String[] memberNameOrId) {
+    Result result = null;
+    try {
+      Set<RegionInformation> regionInfoSet = new LinkedHashSet<>();
+      ResultCollector<?, ?> rc;
+
+      Set<DistributedMember> targetMembers = CliUtil.findMembers(group, memberNameOrId);
+
+      if (targetMembers.isEmpty()) {
+        return ResultBuilder.createUserErrorResult(CliStrings.NO_MEMBERS_FOUND_MESSAGE);
+      }
+
+      TabularResultData resultData = ResultBuilder.createTabularResultData();
+      rc = CliUtil.executeFunction(getRegionsFunction, null, targetMembers);
+
+      ArrayList<?> resultList = (ArrayList<?>) rc.getResult();
+
+      if (resultList != null) {
+
+        for (Object resultObj : resultList) {
+          if (resultObj != null) {
+            if (resultObj instanceof Object[]) {
+              Object[] resultObjectArray = (Object[]) resultObj;
+              for (Object regionInfo : resultObjectArray) {
+                if (regionInfo instanceof RegionInformation) {
+                  regionInfoSet.add((RegionInformation) regionInfo);
+                }
+              }
+            }
+          }
+        }
+
+        Set<String> regionNames = new TreeSet<>();
+
+        for (RegionInformation regionInfo : regionInfoSet) {
+          regionNames.add(regionInfo.getName());
+          Set<String> subRegionNames = regionInfo.getSubRegionNames();
+
+          regionNames.addAll(subRegionNames);
+        }
+
+        for (String regionName : regionNames) {
+          resultData.accumulate("List of regions", regionName);
+        }
+
+        if (!regionNames.isEmpty()) {
+          result = ResultBuilder.buildResult(resultData);
+
+        } else {
+          result = ResultBuilder.createInfoResult(CliStrings.LIST_REGION__MSG__NOT_FOUND);
+        }
+      }
+    } catch (FunctionInvocationTargetException e) {
+      result = ResultBuilder.createGemFireErrorResult(CliStrings
+          .format(CliStrings.COULD_NOT_EXECUTE_COMMAND_TRY_AGAIN, CliStrings.LIST_REGION));
+    } catch (Exception e) {
+      result = ResultBuilder
+          .createGemFireErrorResult(CliStrings.LIST_REGION__MSG__ERROR + " : " + e.getMessage());
+    }
+    return result;
+  }
+
+  @CliCommand(value = {CliStrings.DESCRIBE_REGION}, help = CliStrings.DESCRIBE_REGION__HELP)
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_CONFIG})
+  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
+  public Result describeRegion(
+      @CliOption(key = CliStrings.DESCRIBE_REGION__NAME, optionContext = ConverterHint.REGION_PATH,
+          help = CliStrings.DESCRIBE_REGION__NAME__HELP, mandatory = true) String regionName) {
+
+    Result result;
+    try {
+
+      if (regionName == null || regionName.isEmpty()) {
+        return ResultBuilder.createUserErrorResult("Please provide a region name");
+      }
+
+      if (regionName.equals(Region.SEPARATOR)) {
+        return ResultBuilder.createUserErrorResult(CliStrings.INVALID_REGION_NAME);
+      }
+
+      InternalCache cache = getCache();
+      ResultCollector<?, ?> rc =
+          CliUtil.executeFunction(getRegionDescription, regionName, CliUtil.getAllMembers(cache));
+
+      List<?> resultList = (List<?>) rc.getResult();
+
+      // The returned result could be a region description with per member and /or single local
+      // region
+      Object[] results = resultList.toArray();
+      List<RegionDescription> regionDescriptionList = new ArrayList<>();
+
+      for (int i = 0; i < results.length; i++) {
+
+        if (results[i] instanceof RegionDescriptionPerMember) {
+          RegionDescriptionPerMember regionDescPerMember = (RegionDescriptionPerMember) results[i];
+
+          if (regionDescPerMember != null) {
+            RegionDescription regionDescription = new RegionDescription();
+            regionDescription.add(regionDescPerMember);
+
+            for (int j = i + 1; j < results.length; j++) {
+              if (results[j] != null && results[j] instanceof RegionDescriptionPerMember) {
+                RegionDescriptionPerMember preyRegionDescPerMember =
+                    (RegionDescriptionPerMember) results[j];
+                if (regionDescription.add(preyRegionDescPerMember)) {
+                  results[j] = null;
+                }
+              }
+            }
+            regionDescriptionList.add(regionDescription);
+          }
+        } else if (results[i] instanceof Throwable) {
+          Throwable t = (Throwable) results[i];
+          LogWrapper.getInstance().info(t.getMessage(), t);
+        }
+      }
+
+      if (regionDescriptionList.isEmpty()) {
+        return ResultBuilder
+            .createUserErrorResult(CliStrings.format(CliStrings.REGION_NOT_FOUND, regionName));
+      }
+
+      CompositeResultData crd = ResultBuilder.createCompositeResultData();
+
+      for (RegionDescription regionDescription : regionDescriptionList) {
+        // No point in displaying the scope for PR's
+        if (regionDescription.isPartition()) {
+          regionDescription.getCndRegionAttributes().remove(RegionAttributesNames.SCOPE);
+        } else {
+          String scope =
+              regionDescription.getCndRegionAttributes().get(RegionAttributesNames.SCOPE);
+          if (scope != null) {
+            scope = scope.toLowerCase().replace('_', '-');
+            regionDescription.getCndRegionAttributes().put(RegionAttributesNames.SCOPE, scope);
+          }
+        }
+        SectionResultData regionSection = crd.addSection();
+        regionSection.addSeparator('-');
+        regionSection.addData("Name", regionDescription.getName());
+
+        String dataPolicy =
+            regionDescription.getDataPolicy().toString().toLowerCase().replace('_', ' ');
+        regionSection.addData("Data Policy", dataPolicy);
+
+        String memberType;
+
+        if (regionDescription.isAccessor()) {
+          memberType = CliStrings.DESCRIBE_REGION__ACCESSOR__MEMBER;
+        } else {
+          memberType = CliStrings.DESCRIBE_REGION__HOSTING__MEMBER;
+        }
+        regionSection.addData(memberType,
+            CliUtil.convertStringSetToString(regionDescription.getHostingMembers(), '\n'));
+        regionSection.addSeparator('.');
+
+        TabularResultData commonNonDefaultAttrTable = regionSection.addSection().addTable();
+
+        commonNonDefaultAttrTable.setHeader(CliStrings
+            .format(CliStrings.DESCRIBE_REGION__NONDEFAULT__COMMONATTRIBUTES__HEADER, memberType));
+        // Common Non Default Region Attributes
+        Map<String, String> cndRegionAttrsMap = regionDescription.getCndRegionAttributes();
+
+        // Common Non Default Eviction Attributes
+        Map<String, String> cndEvictionAttrsMap = regionDescription.getCndEvictionAttributes();
+
+        // Common Non Default Partition Attributes
+        Map<String, String> cndPartitionAttrsMap = regionDescription.getCndPartitionAttributes();
+
+        writeCommonAttributesToTable(commonNonDefaultAttrTable,
+            CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE__REGION, cndRegionAttrsMap);
+        writeCommonAttributesToTable(commonNonDefaultAttrTable,
+            CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE__EVICTION, cndEvictionAttrsMap);
+        writeCommonAttributesToTable(commonNonDefaultAttrTable,
+            CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE__PARTITION, cndPartitionAttrsMap);
+
+        // Member-wise non default Attributes
+        Map<String, RegionDescriptionPerMember> regDescPerMemberMap =
+            regionDescription.getRegionDescriptionPerMemberMap();
+        Set<String> members = regDescPerMemberMap.keySet();
+
+        TabularResultData table = regionSection.addSection().addTable();
+
+        boolean setHeader = false;
+        for (String member : members) {
+          RegionDescriptionPerMember regDescPerMem = regDescPerMemberMap.get(member);
+          Map<String, String> ndRa = regDescPerMem.getNonDefaultRegionAttributes();
+          Map<String, String> ndEa = regDescPerMem.getNonDefaultEvictionAttributes();
+          Map<String, String> ndPa = regDescPerMem.getNonDefaultPartitionAttributes();
+
+          // Get all the member-specific non-default attributes by removing the common keys
+          ndRa.keySet().removeAll(cndRegionAttrsMap.keySet());
+          ndEa.keySet().removeAll(cndEvictionAttrsMap.keySet());
+          ndPa.keySet().removeAll(cndPartitionAttrsMap.keySet());
+
+          // Scope is not valid for PR's
+          if (regionDescription.isPartition()) {
+            if (ndRa.get(RegionAttributesNames.SCOPE) != null) {
+              ndRa.remove(RegionAttributesNames.SCOPE);
+            }
+          }
+
+          List<FixedPartitionAttributesInfo> fpaList = regDescPerMem.getFixedPartitionAttributes();
+
+          if (!(ndRa.isEmpty() && ndEa.isEmpty() && ndPa.isEmpty()) || fpaList != null) {
+            setHeader = true;
+            boolean memberNameAdded;
+            memberNameAdded = writeAttributesToTable(table,
+                CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE__REGION, ndRa, member, false);
+            memberNameAdded =
+                writeAttributesToTable(table, CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE__EVICTION,
+                    ndEa, member, memberNameAdded);
+            memberNameAdded = writeAttributesToTable(table,
+                CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE__PARTITION, ndPa, member,
+                memberNameAdded);
+
+            writeFixedPartitionAttributesToTable(table, "", fpaList, member, memberNameAdded);
+          }
+        }
+
+        if (setHeader) {
+          table.setHeader(CliStrings.format(
+              CliStrings.DESCRIBE_REGION__NONDEFAULT__PERMEMBERATTRIBUTES__HEADER, memberType));
+        }
+      }
+
+      result = ResultBuilder.buildResult(crd);
+    } catch (FunctionInvocationTargetException e) {
+      result = ResultBuilder.createGemFireErrorResult(CliStrings
+          .format(CliStrings.COULD_NOT_EXECUTE_COMMAND_TRY_AGAIN, CliStrings.DESCRIBE_REGION));
+    } catch (Exception e) {
+      String errorMessage = CliStrings.format(CliStrings.EXCEPTION_CLASS_AND_MESSAGE,
+          e.getClass().getName(), e.getMessage());
+      result = ResultBuilder.createGemFireErrorResult(errorMessage);
+    }
+    return result;
+  }
+
+  private void writeCommonAttributesToTable(TabularResultData table, String attributeType,
+      Map<String, String> attributesMap) {
+    if (!attributesMap.isEmpty()) {
+      Set<String> attributes = attributesMap.keySet();
+      boolean isTypeAdded = false;
+      final String blank = "";
+
+      for (String attributeName : attributes) {
+        String attributeValue = attributesMap.get(attributeName);
+        String type, memName;
+
+        if (!isTypeAdded) {
+          type = attributeType;
+          isTypeAdded = true;
+        } else {
+          type = blank;
+        }
+        writeCommonAttributeToTable(table, type, attributeName, attributeValue);
+      }
+    }
+  }
+
+  private boolean writeFixedPartitionAttributesToTable(TabularResultData table,
+      String attributeType, List<FixedPartitionAttributesInfo> fpaList, String member,
+      boolean isMemberNameAdded) {
+
+    if (fpaList != null) {
+      boolean isTypeAdded = false;
+      final String blank = "";
+
+      Iterator<FixedPartitionAttributesInfo> fpaIter = fpaList.iterator();
+      String type, memName;
+
+      while (fpaIter.hasNext()) {
+        FixedPartitionAttributesInfo fpa = fpaIter.next();
+        StringBuilder fpaBuilder = new StringBuilder();
+        fpaBuilder.append(fpa.getPartitionName());
+        fpaBuilder.append(',');
+
+        if (fpa.isPrimary()) {
+          fpaBuilder.append("Primary");
+        } else {
+          fpaBuilder.append("Secondary");
+        }
+        fpaBuilder.append(',');
+        fpaBuilder.append(fpa.getNumBuckets());
+
+        if (!isTypeAdded) {
+          type = attributeType;
+          isTypeAdded = true;
+        } else {
+          type = blank;
+        }
+
+        if (!isMemberNameAdded) {
+          memName = member;
+          isMemberNameAdded = true;
+        } else {
+          memName = blank;
+        }
+
+        writeAttributeToTable(table, memName, type, "Fixed Partition", fpaBuilder.toString());
+      }
+    }
+
+    return isMemberNameAdded;
+  }
+
+  private boolean writeAttributesToTable(TabularResultData table, String attributeType,
+      Map<String, String> attributesMap, String member, boolean isMemberNameAdded) {
+    if (!attributesMap.isEmpty()) {
+      Set<String> attributes = attributesMap.keySet();
+      boolean isTypeAdded = false;
+      final String blank = "";
+
+      for (String attributeName : attributes) {
+        String attributeValue = attributesMap.get(attributeName);
+        String type, memName;
+
+        if (!isTypeAdded) {
+          type = attributeType;
+          isTypeAdded = true;
+        } else {
+          type = blank;
+        }
+
+        if (!isMemberNameAdded) {
+          memName = member;
+          isMemberNameAdded = true;
+        } else {
+          memName = blank;
+        }
+
+        writeAttributeToTable(table, memName, type, attributeName, attributeValue);
+      }
+    }
+
+    return isMemberNameAdded;
+  }
+
+  public void writeAttributeToTable(TabularResultData table, String member, String attributeType,
+      String attributeName, String attributeValue) {
+
+    final String blank = "";
+    if (attributeValue != null) {
+      // Tokenize the attributeValue
+      String[] attributeValues = attributeValue.split(",");
+      boolean isFirstValue = true;
+
+      for (String value : attributeValues) {
+        if (isFirstValue) {
+          table.accumulate(CliStrings.DESCRIBE_REGION__MEMBER, member);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE, attributeType);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__NAME, attributeName);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__VALUE, value);
+          isFirstValue = false;
+        } else {
+          table.accumulate(CliStrings.DESCRIBE_REGION__MEMBER, blank);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE, blank);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__NAME, blank);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__VALUE, value);
+        }
+      }
+    }
+  }
+
+
+  private void writeCommonAttributeToTable(TabularResultData table, String attributeType,
+      String attributeName, String attributeValue) {
+    final String blank = "";
+
+    if (attributeValue != null) {
+      String[] attributeValues = attributeValue.split(",");
+      boolean isFirstValue = true;
+      for (String value : attributeValues) {
+        if (isFirstValue) {
+          isFirstValue = false;
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE, attributeType);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__NAME, attributeName);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__VALUE, value);
+        } else {
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__TYPE, blank);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__NAME, blank);
+          table.accumulate(CliStrings.DESCRIBE_REGION__ATTRIBUTE__VALUE, value);
+        }
+      }
+    }
+  }
+
+  public void addChildSection(SectionResultData parentSection, Map<String, String> map,
+      String header) {
+    if (!map.isEmpty()) {
+      Set<String> attributes = map.keySet();
+      SectionResultData section = parentSection.addSection();
+      section.setHeader(header);
+      for (String attribute : attributes) {
+        section.addData(attribute, map.get(attribute));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RegionCommandsUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RegionCommandsUtils.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RegionCommandsUtils.java
deleted file mode 100644
index 49eb520..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RegionCommandsUtils.java
+++ /dev/null
@@ -1,59 +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.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.management.internal.cli.CliUtil;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-
-public class RegionCommandsUtils {
-
-  static void validateGroups(InternalCache cache, String[] groups) {
-    if (groups != null && groups.length != 0) {
-      Set<String> existingGroups = new HashSet<>();
-      Set<DistributedMember> members = CliUtil.getAllNormalMembers(cache);
-      for (DistributedMember distributedMember : members) {
-        List<String> memberGroups = distributedMember.getGroups();
-        existingGroups.addAll(memberGroups);
-      }
-      List<String> groupsList = new ArrayList<>(Arrays.asList(groups));
-      groupsList.removeAll(existingGroups);
-
-      if (!groupsList.isEmpty()) {
-        throw new IllegalArgumentException(
-            CliStrings.format(CliStrings.CREATE_REGION__MSG__GROUPS_0_ARE_INVALID,
-                new Object[] {String.valueOf(groupsList)}));
-      }
-    }
-  }
-
-  static boolean isClassNameValid(String fqcn) {
-    if (fqcn.isEmpty()) {
-      return true;
-    }
-    String regex = "([\\p{L}_$][\\p{L}\\p{N}_$]*\\.)*[\\p{L}_$][\\p{L}\\p{N}_$]*";
-    return Pattern.matches(regex, fqcn);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RevokeMissingDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RevokeMissingDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RevokeMissingDiskStoreCommand.java
deleted file mode 100644
index 98acf55..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/RevokeMissingDiskStoreCommand.java
+++ /dev/null
@@ -1,61 +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 org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import org.apache.geode.SystemFailure;
-import org.apache.geode.management.DistributedSystemMXBean;
-import org.apache.geode.management.ManagementService;
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission;
-
-public class RevokeMissingDiskStoreCommand implements GfshCommand {
-  @CliCommand(value = CliStrings.REVOKE_MISSING_DISK_STORE,
-      help = CliStrings.REVOKE_MISSING_DISK_STORE__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
-  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
-      operation = ResourcePermission.Operation.MANAGE, target = ResourcePermission.Target.DISK)
-  public Result revokeMissingDiskStore(@CliOption(key = CliStrings.REVOKE_MISSING_DISK_STORE__ID,
-      mandatory = true, help = CliStrings.REVOKE_MISSING_DISK_STORE__ID__HELP) String id) {
-
-    try {
-      DistributedSystemMXBean dsMXBean =
-          ManagementService.getManagementService(getCache()).getDistributedSystemMXBean();
-      if (dsMXBean.revokeMissingDiskStores(id)) {
-        return ResultBuilder.createInfoResult("Missing disk store successfully revoked");
-      }
-
-      return ResultBuilder.createUserErrorResult("Unable to find missing disk store to revoke");
-    } catch (VirtualMachineError e) {
-      SystemFailure.initiateFailure(e);
-      throw e;
-    } catch (Throwable th) {
-      SystemFailure.checkFailure();
-      if (th.getMessage() == null) {
-        return ResultBuilder.createGemFireErrorResult(
-            "An error occurred while revoking missing disk stores: " + th);
-      }
-      return ResultBuilder.createGemFireErrorResult(
-          "An error occurred while revoking missing disk stores: " + th.getMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
index 41e227b..84ee5db 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
@@ -319,6 +319,7 @@ public class ShellCommands implements GfshCommand {
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH})
   public Result history(
       @CliOption(key = {CliStrings.HISTORY__FILE},
+          unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.HISTORY__FILE__HELP) String saveHistoryTo,
       @CliOption(key = {CliStrings.HISTORY__CLEAR}, specifiedDefaultValue = "true",
           unspecifiedDefaultValue = "false",
@@ -330,7 +331,7 @@ public class ShellCommands implements GfshCommand {
     } else {
       // Process file option
       Gfsh gfsh = Gfsh.getCurrentInstance();
-      ErrorResultData errorResultData;
+      ErrorResultData errorResultData = null;
       StringBuilder contents = new StringBuilder();
       Writer output = null;
 
@@ -345,7 +346,7 @@ public class ShellCommands implements GfshCommand {
 
       while (it.hasNext()) {
         String line = it.next().toString();
-        if (!line.isEmpty()) {
+        if (line.isEmpty() == false) {
           if (flagForLineNumbers) {
             lineNumber++;
             contents.append(String.format("%" + historySizeWordLength + "s  ", lineNumber));

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowDeadlockCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowDeadlockCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowDeadlockCommand.java
deleted file mode 100644
index 3b5fd78..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowDeadlockCommand.java
+++ /dev/null
@@ -1,92 +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.text.MessageFormat;
-import java.util.Collection;
-import java.util.Set;
-
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.deadlock.DeadlockDetector;
-import org.apache.geode.distributed.internal.deadlock.Dependency;
-import org.apache.geode.distributed.internal.deadlock.DependencyGraph;
-import org.apache.geode.distributed.internal.deadlock.GemFireDeadlockDetector;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.management.cli.CliMetaData;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.CliUtil;
-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.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission;
-
-public class ShowDeadlockCommand implements GfshCommand {
-  @CliCommand(value = CliStrings.SHOW_DEADLOCK, help = CliStrings.SHOW_DEADLOCK__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DEBUG_UTIL})
-  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
-      operation = ResourcePermission.Operation.READ)
-  public Result showDeadlock(@CliOption(key = CliStrings.SHOW_DEADLOCK__DEPENDENCIES__FILE,
-      help = CliStrings.SHOW_DEADLOCK__DEPENDENCIES__FILE__HELP,
-      mandatory = true) String filename) {
-
-    Result result;
-    try {
-      if (!filename.endsWith(".txt")) {
-        return ResultBuilder
-            .createUserErrorResult(CliStrings.format(CliStrings.INVALID_FILE_EXTENSION, ".txt"));
-      }
-      InternalCache cache = getCache();
-
-      Set<DistributedMember> allMembers = CliUtil.getAllMembers(cache);
-      GemFireDeadlockDetector gfeDeadLockDetector = new GemFireDeadlockDetector(allMembers);
-      DependencyGraph dependencyGraph = gfeDeadLockDetector.find();
-      Collection<Dependency> deadlock = dependencyGraph.findCycle();
-      DependencyGraph deepest = null;
-      if (deadlock == null) {
-        deepest = dependencyGraph.findLongestCallChain();
-        if (deepest != null) {
-          deadlock = deepest.getEdges();
-        }
-      }
-      Set<Dependency> dependencies = (Set<Dependency>) dependencyGraph.getEdges();
-
-      InfoResultData resultData = ResultBuilder.createInfoResultData();
-
-      if (deadlock != null) {
-        if (deepest != null) {
-          resultData.addLine(CliStrings.SHOW_DEADLOCK__DEEPEST_FOUND);
-        } else {
-          resultData.addLine(CliStrings.SHOW_DEADLOCK__DEADLOCK__DETECTED);
-        }
-        resultData.addLine(DeadlockDetector.prettyFormat(deadlock));
-      } else {
-        resultData.addLine(CliStrings.SHOW_DEADLOCK__NO__DEADLOCK);
-      }
-      resultData.addAsFile(filename, DeadlockDetector.prettyFormat(dependencies),
-          MessageFormat.format(CliStrings.SHOW_DEADLOCK__DEPENDENCIES__REVIEW, filename), false);
-      result = ResultBuilder.buildResult(resultData);
-
-    } catch (Exception e) {
-      result = ResultBuilder
-          .createGemFireErrorResult(CliStrings.SHOW_DEADLOCK__ERROR + " : " + e.getMessage());
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/645a32d0/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowLogCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowLogCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowLogCommand.java
deleted file mode 100644
index 7780479..0000000
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShowLogCommand.java
+++ /dev/null
@@ -1,102 +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 javax.management.ObjectName;
-
-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.ManagementService;
-import org.apache.geode.management.MemberMXBean;
-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.ManagementConstants;
-import org.apache.geode.management.internal.SystemManagementService;
-import org.apache.geode.management.internal.cli.CliUtil;
-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.security.ResourceOperation;
-import org.apache.geode.security.ResourcePermission;
-
-public class ShowLogCommand implements GfshCommand {
-  @CliCommand(value = CliStrings.SHOW_LOG, help = CliStrings.SHOW_LOG_HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DEBUG_UTIL})
-  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
-      operation = ResourcePermission.Operation.READ)
-  public Result showLog(
-      @CliOption(key = CliStrings.MEMBER, optionContext = ConverterHint.ALL_MEMBER_IDNAME,
-          help = CliStrings.SHOW_LOG_MEMBER_HELP, mandatory = true) String memberNameOrId,
-      @CliOption(key = CliStrings.SHOW_LOG_LINE_NUM, unspecifiedDefaultValue = "0",
-          help = CliStrings.SHOW_LOG_LINE_NUM_HELP) int numberOfLines) {
-    Result result;
-    try {
-      InternalCache cache = getCache();
-      SystemManagementService service =
-          (SystemManagementService) ManagementService.getExistingManagementService(cache);
-      MemberMXBean bean;
-      DistributedMember memberToBeInvoked = CliUtil.getDistributedMemberByNameOrId(memberNameOrId);
-      if (memberToBeInvoked != null) {
-        String memberId = memberToBeInvoked.getId();
-
-        if (cache.getDistributedSystem().getDistributedMember().getId().equals(memberId)) {
-          bean = service.getMemberMXBean();
-        } else {
-          ObjectName objectName = service.getMemberMBeanName(memberToBeInvoked);
-          bean = service.getMBeanProxy(objectName, MemberMXBean.class);
-        }
-
-        if (numberOfLines > ManagementConstants.MAX_SHOW_LOG_LINES) {
-          numberOfLines = ManagementConstants.MAX_SHOW_LOG_LINES;
-        }
-        if (numberOfLines == 0 || numberOfLines < 0) {
-          numberOfLines = ManagementConstants.DEFAULT_SHOW_LOG_LINES;
-        }
-        InfoResultData resultData = ResultBuilder.createInfoResultData();
-        if (bean != null) {
-          String log = bean.showLog(numberOfLines);
-          if (log != null) {
-            resultData.addLine(log);
-          } else {
-            resultData.addLine(CliStrings.SHOW_LOG_NO_LOG);
-          }
-        } else {
-          ErrorResultData errorResultData =
-              ResultBuilder.createErrorResultData().setErrorCode(ResultBuilder.ERRORCODE_DEFAULT)
-                  .addLine(memberNameOrId + CliStrings.SHOW_LOG_MSG_MEMBER_NOT_FOUND);
-          return (ResultBuilder.buildResult(errorResultData));
-        }
-
-        result = ResultBuilder.buildResult(resultData);
-      } else {
-        ErrorResultData errorResultData =
-            ResultBuilder.createErrorResultData().setErrorCode(ResultBuilder.ERRORCODE_DEFAULT)
-                .addLine(memberNameOrId + CliStrings.SHOW_LOG_MSG_MEMBER_NOT_FOUND);
-        return (ResultBuilder.buildResult(errorResultData));
-      }
-
-    } catch (Exception e) {
-      result = ResultBuilder
-          .createGemFireErrorResult(CliStrings.SHOW_LOG_ERROR + CliUtil.stackTraceAsString(e));
-    }
-    return result;
-  }
-}