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

[26/47] geode git commit: GEODE-3436: Restore refactoring of DiskStoreCommands

GEODE-3436: Restore refactoring of DiskStoreCommands

* See initial commit GEODE-3258 (5d6cad7755ec3c4fe931e3d0f8e89fb181038543)


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

Branch: refs/heads/feature/GEODE-3543
Commit: ca808714ef93ba5232ce5be5f7e2533106bcfc34
Parents: 0dc67f0
Author: YehEmily <em...@gmail.com>
Authored: Thu Aug 3 09:00:08 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Tue Aug 29 09:27:28 2017 -0700

----------------------------------------------------------------------
 .../commands/AlterOfflineDiskStoreCommand.java  |  141 ++
 .../cli/commands/BackupDiskStoreCommand.java    |  142 ++
 .../cli/commands/CompactDiskStoreCommand.java   |  185 +++
 .../CompactOfflineDiskStoreCommand.java         |  176 +++
 .../cli/commands/CreateDiskStoreCommand.java    |  166 ++
 .../cli/commands/DescribeDiskStoreCommand.java  |  177 +++
 .../DescribeOfflineDiskStoreCommand.java        |   75 +
 .../cli/commands/DestroyDiskStoreCommand.java   |  106 ++
 .../cli/commands/DiskStoreCommands.java         | 1433 ------------------
 .../cli/commands/DiskStoreCommandsUtils.java    |   60 +
 .../commands/ExportOfflineDiskStoreCommand.java |   68 +
 .../cli/commands/ListDiskStoresCommand.java     |  112 ++
 .../commands/RevokeMissingDiskStoreCommand.java |   61 +
 .../commands/ShowMissingDiskStoreCommand.java   |  149 ++
 .../UpgradeOfflineDiskStoreCommand.java         |  177 +++
 .../cli/commands/ValidateDiskStoreCommand.java  |  104 ++
 .../DiskStoreCommandsController.java            |   18 +-
 .../commands/DiskStoreCommandsDUnitTest.java    |   46 +-
 .../commands/DiskStoreCommandsJUnitTest.java    |  112 +-
 ...stAndDescribeDiskStoreCommandsDUnitTest.java |   36 +-
 20 files changed, 2041 insertions(+), 1503 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/ca808714/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterOfflineDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterOfflineDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterOfflineDiskStoreCommand.java
new file mode 100644
index 0000000..ce7594e
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/AlterOfflineDiskStoreCommand.java
@@ -0,0 +1,141 @@
+/*
+ * 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 org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.CacheExistsException;
+import org.apache.geode.cache.Region;
+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.ErrorResultData;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+
+public class AlterOfflineDiskStoreCommand implements GfshCommand {
+  @CliCommand(value = CliStrings.ALTER_DISK_STORE, help = CliStrings.ALTER_DISK_STORE__HELP)
+  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
+  public Result alterOfflineDiskStore(
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__DISKSTORENAME, mandatory = true,
+          help = CliStrings.ALTER_DISK_STORE__DISKSTORENAME__HELP) String diskStoreName,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__REGIONNAME, mandatory = true,
+          help = CliStrings.ALTER_DISK_STORE__REGIONNAME__HELP) String regionName,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__DISKDIRS,
+          help = CliStrings.ALTER_DISK_STORE__DISKDIRS__HELP, mandatory = true) String[] diskDirs,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__COMPRESSOR, specifiedDefaultValue = "none",
+          help = CliStrings.ALTER_DISK_STORE__COMPRESSOR__HELP) String compressorClassName,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__CONCURRENCY__LEVEL,
+          help = CliStrings.ALTER_DISK_STORE__CONCURRENCY__LEVEL__HELP) Integer concurrencyLevel,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__STATISTICS__ENABLED,
+          help = CliStrings.ALTER_DISK_STORE__STATISTICS__ENABLED__HELP) Boolean statisticsEnabled,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__INITIAL__CAPACITY,
+          help = CliStrings.ALTER_DISK_STORE__INITIAL__CAPACITY__HELP) Integer initialCapacity,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__LOAD__FACTOR,
+          help = CliStrings.ALTER_DISK_STORE__LOAD__FACTOR__HELP) Float loadFactor,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__LRU__EVICTION__ACTION,
+          help = CliStrings.ALTER_DISK_STORE__LRU__EVICTION__ACTION__HELP) String lruEvictionAction,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__LRU__EVICTION__ALGORITHM,
+          help = CliStrings.ALTER_DISK_STORE__LRU__EVICTION__ALGORITHM__HELP) String lruEvictionAlgo,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__LRU__EVICTION__LIMIT,
+          help = CliStrings.ALTER_DISK_STORE__LRU__EVICTION__LIMIT__HELP) Integer lruEvictionLimit,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__OFF_HEAP,
+          help = CliStrings.ALTER_DISK_STORE__OFF_HEAP__HELP) Boolean offHeap,
+      @CliOption(key = CliStrings.ALTER_DISK_STORE__REMOVE,
+          help = CliStrings.ALTER_DISK_STORE__REMOVE__HELP, specifiedDefaultValue = "true",
+          unspecifiedDefaultValue = "false") boolean remove) {
+
+    Result result;
+
+    try {
+      File[] dirs = null;
+
+      if (diskDirs != null) {
+        dirs = new File[diskDirs.length];
+        for (int i = 0; i < diskDirs.length; i++) {
+          dirs[i] = new File((diskDirs[i]));
+        }
+      }
+
+      if (regionName.equals(Region.SEPARATOR)) {
+        return ResultBuilder.createUserErrorResult(CliStrings.INVALID_REGION_NAME);
+      }
+
+      if ((lruEvictionAlgo != null) || (lruEvictionAction != null) || (lruEvictionLimit != null)
+          || (concurrencyLevel != null) || (initialCapacity != null) || (loadFactor != null)
+          || (compressorClassName != null) || (offHeap != null) || (statisticsEnabled != null)) {
+        if (!remove) {
+          String lruEvictionLimitString =
+              lruEvictionLimit == null ? null : lruEvictionLimit.toString();
+          String concurrencyLevelString =
+              concurrencyLevel == null ? null : concurrencyLevel.toString();
+          String initialCapacityString =
+              initialCapacity == null ? null : initialCapacity.toString();
+          String loadFactorString = loadFactor == null ? null : loadFactor.toString();
+          String statisticsEnabledString =
+              statisticsEnabled == null ? null : statisticsEnabled.toString();
+          String offHeapString = offHeap == null ? null : offHeap.toString();
+
+          if ("none".equals(compressorClassName)) {
+            compressorClassName = "";
+          }
+
+          String resultMessage = DiskStoreImpl.modifyRegion(diskStoreName, dirs, "/" + regionName,
+              lruEvictionAlgo, lruEvictionAction, lruEvictionLimitString, concurrencyLevelString,
+              initialCapacityString, loadFactorString, compressorClassName, statisticsEnabledString,
+              offHeapString, false);
+
+          result = ResultBuilder.createInfoResult(resultMessage);
+        } else {
+          result = ResultBuilder.createParsingErrorResult(
+              "Cannot use the --remove=true parameter with any other parameters");
+        }
+      } else {
+        if (remove) {
+          DiskStoreImpl.destroyRegion(diskStoreName, dirs, "/" + regionName);
+          result = ResultBuilder.createInfoResult("The region " + regionName
+              + " was successfully removed from the disk store " + diskStoreName);
+        } else {
+          // Please provide an option
+          result = ResultBuilder.createParsingErrorResult("Please provide a relevant parameter");
+        }
+      }
+      // Catch the IllegalArgumentException thrown by the modifyDiskStore function and sent the
+    } catch (IllegalArgumentException e) {
+      String message = "Please check the parameters";
+      message += "\n" + e.getMessage();
+      result = ResultBuilder.createGemFireErrorResult(message);
+    } catch (IllegalStateException e) {
+      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
+    } catch (CacheExistsException e) {
+      // Indicates that the command is being used when a cache is open
+      result = ResultBuilder.createGemFireErrorResult("Cannot execute "
+          + CliStrings.ALTER_DISK_STORE + " when a cache exists (Offline command)");
+    } catch (Exception e) {
+      result = createErrorResult(e.getMessage());
+    }
+    return result;
+  }
+
+  private Result createErrorResult(String message) {
+    ErrorResultData erd = ResultBuilder.createErrorResultData();
+    erd.addLine(message);
+    return ResultBuilder.buildResult(erd);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ca808714/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/BackupDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/BackupDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/BackupDiskStoreCommand.java
new file mode 100644
index 0000000..6fc5df1
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/BackupDiskStoreCommand.java
@@ -0,0 +1,142 @@
+/*
+ * 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.util.Map;
+import java.util.Set;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.admin.BackupStatus;
+import org.apache.geode.admin.internal.AdminDistributedSystemImpl;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.DM;
+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.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.cli.result.TabularResultData;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class BackupDiskStoreCommand implements GfshCommand {
+  /**
+   * Internally, we also verify the resource operation permissions CLUSTER:WRITE:DISK if the region
+   * is persistent
+   */
+  @CliCommand(value = CliStrings.BACKUP_DISK_STORE, help = CliStrings.BACKUP_DISK_STORE__HELP)
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
+  @ResourceOperation(resource = ResourcePermission.Resource.DATA,
+      operation = ResourcePermission.Operation.READ)
+  public Result backupDiskStore(
+      @CliOption(key = CliStrings.BACKUP_DISK_STORE__DISKDIRS,
+          help = CliStrings.BACKUP_DISK_STORE__DISKDIRS__HELP, mandatory = true) String targetDir,
+      @CliOption(key = CliStrings.BACKUP_DISK_STORE__BASELINEDIR,
+          help = CliStrings.BACKUP_DISK_STORE__BASELINEDIR__HELP) String baselineDir) {
+
+    getSecurityService().authorize(ResourcePermission.Resource.CLUSTER,
+        ResourcePermission.Operation.WRITE, ResourcePermission.Target.DISK);
+    Result result;
+    try {
+      InternalCache cache = getCache();
+      DM dm = cache.getDistributionManager();
+      BackupStatus backupStatus;
+
+      if (baselineDir != null && !baselineDir.isEmpty()) {
+        backupStatus = AdminDistributedSystemImpl.backupAllMembers(dm, new File(targetDir),
+            new File(baselineDir));
+      } else {
+        backupStatus = AdminDistributedSystemImpl.backupAllMembers(dm, new File(targetDir), null);
+      }
+
+      Map<DistributedMember, Set<PersistentID>> backedupMemberDiskstoreMap =
+          backupStatus.getBackedUpDiskStores();
+
+      Set<DistributedMember> backedupMembers = backedupMemberDiskstoreMap.keySet();
+      CompositeResultData crd = ResultBuilder.createCompositeResultData();
+
+      if (!backedupMembers.isEmpty()) {
+        CompositeResultData.SectionResultData backedupDiskStoresSection = crd.addSection();
+        backedupDiskStoresSection.setHeader(CliStrings.BACKUP_DISK_STORE_MSG_BACKED_UP_DISK_STORES);
+        TabularResultData backedupDiskStoresTable = backedupDiskStoresSection.addTable();
+
+        for (DistributedMember member : backedupMembers) {
+          Set<PersistentID> backedupDiskStores = backedupMemberDiskstoreMap.get(member);
+          boolean printMember = true;
+          String memberName = member.getName();
+
+          if (memberName == null || memberName.isEmpty()) {
+            memberName = member.getId();
+          }
+          for (PersistentID persistentId : backedupDiskStores) {
+            if (persistentId != null) {
+
+              String UUID = persistentId.getUUID().toString();
+              String hostName = persistentId.getHost().getHostName();
+              String directory = persistentId.getDirectory();
+
+              if (printMember) {
+                writeToBackupDiskStoreTable(backedupDiskStoresTable, memberName, UUID, hostName,
+                    directory);
+                printMember = false;
+              } else {
+                writeToBackupDiskStoreTable(backedupDiskStoresTable, "", UUID, hostName, directory);
+              }
+            }
+          }
+        }
+      } else {
+        CompositeResultData.SectionResultData noMembersBackedUp = crd.addSection();
+        noMembersBackedUp.setHeader(CliStrings.BACKUP_DISK_STORE_MSG_NO_DISKSTORES_BACKED_UP);
+      }
+
+      Set<PersistentID> offlineDiskStores = backupStatus.getOfflineDiskStores();
+
+      if (!offlineDiskStores.isEmpty()) {
+        CompositeResultData.SectionResultData offlineDiskStoresSection = crd.addSection();
+        TabularResultData offlineDiskStoresTable = offlineDiskStoresSection.addTable();
+
+        offlineDiskStoresSection.setHeader(CliStrings.BACKUP_DISK_STORE_MSG_OFFLINE_DISK_STORES);
+        for (PersistentID offlineDiskStore : offlineDiskStores) {
+          offlineDiskStoresTable.accumulate(CliStrings.BACKUP_DISK_STORE_MSG_UUID,
+              offlineDiskStore.getUUID().toString());
+          offlineDiskStoresTable.accumulate(CliStrings.BACKUP_DISK_STORE_MSG_HOST,
+              offlineDiskStore.getHost().getHostName());
+          offlineDiskStoresTable.accumulate(CliStrings.BACKUP_DISK_STORE_MSG_DIRECTORY,
+              offlineDiskStore.getDirectory());
+        }
+      }
+      result = ResultBuilder.buildResult(crd);
+
+    } catch (Exception e) {
+      result = ResultBuilder.createGemFireErrorResult(e.getMessage());
+    }
+    return result;
+  }
+
+  private void writeToBackupDiskStoreTable(TabularResultData backedupDiskStoreTable,
+      String memberId, String UUID, String host, String directory) {
+    backedupDiskStoreTable.accumulate(CliStrings.BACKUP_DISK_STORE_MSG_MEMBER, memberId);
+    backedupDiskStoreTable.accumulate(CliStrings.BACKUP_DISK_STORE_MSG_UUID, UUID);
+    backedupDiskStoreTable.accumulate(CliStrings.BACKUP_DISK_STORE_MSG_DIRECTORY, directory);
+    backedupDiskStoreTable.accumulate(CliStrings.BACKUP_DISK_STORE_MSG_HOST, host);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ca808714/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CompactDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CompactDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CompactDiskStoreCommand.java
new file mode 100644
index 0000000..34cc6c3
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CompactDiskStoreCommand.java
@@ -0,0 +1,185 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+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.cache.persistence.PersistentID;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+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.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.CompositeResultData;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.messages.CompactRequest;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class CompactDiskStoreCommand implements GfshCommand {
+  @CliCommand(value = CliStrings.COMPACT_DISK_STORE, help = CliStrings.COMPACT_DISK_STORE__HELP)
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.MANAGE, target = ResourcePermission.Target.DISK)
+  public Result compactDiskStore(
+      @CliOption(key = CliStrings.COMPACT_DISK_STORE__NAME, mandatory = true,
+          optionContext = ConverterHint.DISKSTORE,
+          help = CliStrings.COMPACT_DISK_STORE__NAME__HELP) String diskStoreName,
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          help = CliStrings.COMPACT_DISK_STORE__GROUP__HELP) String[] groups) {
+    Result result;
+
+    try {
+      // disk store exists validation
+      if (!diskStoreExists(diskStoreName)) {
+        result = ResultBuilder.createUserErrorResult(
+            CliStrings.format(CliStrings.COMPACT_DISK_STORE__DISKSTORE_0_DOES_NOT_EXIST,
+                new Object[] {diskStoreName}));
+      } else {
+        InternalDistributedSystem ds = getCache().getInternalDistributedSystem();
+
+        Map<DistributedMember, PersistentID> overallCompactInfo = new HashMap<>();
+
+        Set<?> otherMembers = ds.getDistributionManager().getOtherNormalDistributionManagerIds();
+        Set<InternalDistributedMember> allMembers = new HashSet<>();
+
+        for (Object member : otherMembers) {
+          allMembers.add((InternalDistributedMember) member);
+        }
+        allMembers.add(ds.getDistributedMember());
+
+        String groupInfo = "";
+        // if groups are specified, find members in the specified group
+        if (groups != null && groups.length > 0) {
+          groupInfo = CliStrings.format(CliStrings.COMPACT_DISK_STORE__MSG__FOR_GROUP,
+              new Object[] {Arrays.toString(groups) + "."});
+          final Set<InternalDistributedMember> selectedMembers = new HashSet<>();
+          List<String> targetedGroups = Arrays.asList(groups);
+          for (InternalDistributedMember member : allMembers) {
+            List<String> memberGroups = member.getGroups();
+            if (!Collections.disjoint(targetedGroups, memberGroups)) {
+              selectedMembers.add(member);
+            }
+          }
+
+          allMembers = selectedMembers;
+        }
+
+        // allMembers should not be empty when groups are not specified - it'll
+        // have at least one member
+        if (allMembers.isEmpty()) {
+          result = ResultBuilder.createUserErrorResult(
+              CliStrings.format(CliStrings.COMPACT_DISK_STORE__NO_MEMBERS_FOUND_IN_SPECIFED_GROUP,
+                  new Object[] {Arrays.toString(groups)}));
+        } else {
+          // first invoke on local member if it exists in the targeted set
+          if (allMembers.remove(ds.getDistributedMember())) {
+            PersistentID compactedDiskStoreId = CompactRequest.compactDiskStore(diskStoreName);
+            if (compactedDiskStoreId != null) {
+              overallCompactInfo.put(ds.getDistributedMember(), compactedDiskStoreId);
+            }
+          }
+
+          // was this local member the only one? Then don't try to send
+          // CompactRequest. Otherwise, send the request to others
+          if (!allMembers.isEmpty()) {
+            // Invoke compact on all 'other' members
+            Map<DistributedMember, PersistentID> memberCompactInfo =
+                CompactRequest.send(ds.getDistributionManager(), diskStoreName, allMembers);
+            if (memberCompactInfo != null && !memberCompactInfo.isEmpty()) {
+              overallCompactInfo.putAll(memberCompactInfo);
+              memberCompactInfo.clear();
+            }
+            String notExecutedMembers = CompactRequest.getNotExecutedMembers();
+            if (notExecutedMembers != null && !notExecutedMembers.isEmpty()) {
+              LogWrapper.getInstance()
+                  .info("compact disk-store \"" + diskStoreName
+                      + "\" message was scheduled to be sent to but was not send to "
+                      + notExecutedMembers);
+            }
+          }
+
+          // If compaction happened at all, then prepare the summary
+          if (overallCompactInfo != null && !overallCompactInfo.isEmpty()) {
+            CompositeResultData compositeResultData = ResultBuilder.createCompositeResultData();
+            CompositeResultData.SectionResultData section;
+
+            Set<Map.Entry<DistributedMember, PersistentID>> entries = overallCompactInfo.entrySet();
+
+            for (Map.Entry<DistributedMember, PersistentID> entry : entries) {
+              String memberId = entry.getKey().getId();
+              section = compositeResultData.addSection(memberId);
+              section.addData("On Member", memberId);
+
+              PersistentID persistentID = entry.getValue();
+              if (persistentID != null) {
+                CompositeResultData.SectionResultData subSection =
+                    section.addSection("DiskStore" + memberId);
+                subSection.addData("UUID", persistentID.getUUID());
+                subSection.addData("Host", persistentID.getHost().getHostName());
+                subSection.addData("Directory", persistentID.getDirectory());
+              }
+            }
+            compositeResultData.setHeader("Compacted " + diskStoreName + groupInfo);
+            result = ResultBuilder.buildResult(compositeResultData);
+          } else {
+            result = ResultBuilder.createInfoResult(
+                CliStrings.COMPACT_DISK_STORE__COMPACTION_ATTEMPTED_BUT_NOTHING_TO_COMPACT);
+          }
+        } // all members' if
+      } // disk store exists' if
+    } catch (RuntimeException e) {
+      LogWrapper.getInstance().info(e.getMessage(), e);
+      result = ResultBuilder.createGemFireErrorResult(
+          CliStrings.format(CliStrings.COMPACT_DISK_STORE__ERROR_WHILE_COMPACTING_REASON_0,
+              new Object[] {e.getMessage()}));
+    }
+    return result;
+  }
+
+  private boolean diskStoreExists(String diskStoreName) {
+    InternalCache cache = getCache();
+    ManagementService managementService = ManagementService.getExistingManagementService(cache);
+    DistributedSystemMXBean dsMXBean = managementService.getDistributedSystemMXBean();
+    Map<String, String[]> diskstore = dsMXBean.listMemberDiskstore();
+
+    Set<Map.Entry<String, String[]>> entrySet = diskstore.entrySet();
+
+    for (Map.Entry<String, String[]> entry : entrySet) {
+      String[] value = entry.getValue();
+      if (CliUtil.contains(value, diskStoreName)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ca808714/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CompactOfflineDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CompactOfflineDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CompactOfflineDiskStoreCommand.java
new file mode 100644
index 0000000..ae73440
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CompactOfflineDiskStoreCommand.java
@@ -0,0 +1,176 @@
+/*
+ * 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.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.GemFireIOException;
+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.GfshParser;
+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.shell.Gfsh;
+import org.apache.geode.management.internal.cli.util.DiskStoreCompacter;
+
+public class CompactOfflineDiskStoreCommand implements GfshCommand {
+  @CliCommand(value = CliStrings.COMPACT_OFFLINE_DISK_STORE,
+      help = CliStrings.COMPACT_OFFLINE_DISK_STORE__HELP)
+  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
+  public Result compactOfflineDiskStore(
+      @CliOption(key = CliStrings.COMPACT_OFFLINE_DISK_STORE__NAME, mandatory = true,
+          help = CliStrings.COMPACT_OFFLINE_DISK_STORE__NAME__HELP) String diskStoreName,
+      @CliOption(key = CliStrings.COMPACT_OFFLINE_DISK_STORE__DISKDIRS, mandatory = true,
+          help = CliStrings.COMPACT_OFFLINE_DISK_STORE__DISKDIRS__HELP) String[] diskDirs,
+      @CliOption(key = CliStrings.COMPACT_OFFLINE_DISK_STORE__MAXOPLOGSIZE,
+          unspecifiedDefaultValue = "-1",
+          help = CliStrings.COMPACT_OFFLINE_DISK_STORE__MAXOPLOGSIZE__HELP) long maxOplogSize,
+      @CliOption(key = CliStrings.COMPACT_OFFLINE_DISK_STORE__J,
+          help = CliStrings.COMPACT_OFFLINE_DISK_STORE__J__HELP) String[] jvmProps) {
+    Result result;
+    LogWrapper logWrapper = LogWrapper.getInstance();
+
+    StringBuilder output = new StringBuilder();
+    StringBuilder error = new StringBuilder();
+    StringBuilder errorMessage = new StringBuilder();
+    Process compacterProcess = null;
+
+    try {
+      String validatedDirectories = DiskStoreCommandsUtils.validatedDirectories(diskDirs);
+      if (validatedDirectories != null) {
+        throw new IllegalArgumentException(
+            "Could not find " + CliStrings.COMPACT_OFFLINE_DISK_STORE__DISKDIRS + ": \""
+                + validatedDirectories + "\"");
+      }
+
+      List<String> commandList = new ArrayList<>();
+      commandList.add(System.getProperty("java.home") + File.separatorChar + "bin"
+          + File.separatorChar + "java");
+
+      DiskStoreCommandsUtils.configureLogging(commandList);
+
+      if (jvmProps != null && jvmProps.length != 0) {
+        commandList.addAll(Arrays.asList(jvmProps));
+      }
+      commandList.add("-classpath");
+      commandList.add(System.getProperty("java.class.path", "."));
+      commandList.add(DiskStoreCompacter.class.getName());
+
+      commandList.add(CliStrings.COMPACT_OFFLINE_DISK_STORE__NAME + "=" + diskStoreName);
+
+      if (diskDirs != null && diskDirs.length != 0) {
+        StringBuilder builder = new StringBuilder();
+        int arrayLength = diskDirs.length;
+        for (int i = 0; i < arrayLength; i++) {
+          if (File.separatorChar == '\\') {
+            builder.append(diskDirs[i].replace("\\", "/")); // see 46120
+          } else {
+            builder.append(diskDirs[i]);
+          }
+          if (i + 1 != arrayLength) {
+            builder.append(',');
+          }
+        }
+        commandList.add(CliStrings.COMPACT_OFFLINE_DISK_STORE__DISKDIRS + "=" + builder.toString());
+      }
+      // -1 is ignore as maxOplogSize
+      commandList.add(CliStrings.COMPACT_OFFLINE_DISK_STORE__MAXOPLOGSIZE + "=" + maxOplogSize);
+
+      ProcessBuilder procBuilder = new ProcessBuilder(commandList);
+      compacterProcess = procBuilder.start();
+      InputStream inputStream = compacterProcess.getInputStream();
+      InputStream errorStream = compacterProcess.getErrorStream();
+      BufferedReader inputReader = new BufferedReader(new InputStreamReader(inputStream));
+      BufferedReader errorReader = new BufferedReader(new InputStreamReader(errorStream));
+
+      String line;
+      while ((line = inputReader.readLine()) != null) {
+        output.append(line).append(GfshParser.LINE_SEPARATOR);
+      }
+
+      boolean switchToStackTrace = false;
+      while ((line = errorReader.readLine()) != null) {
+        if (!switchToStackTrace && DiskStoreCompacter.STACKTRACE_START.equals(line)) {
+          switchToStackTrace = true;
+        } else if (switchToStackTrace) {
+          error.append(line).append(GfshParser.LINE_SEPARATOR);
+        } else {
+          errorMessage.append(line);
+        }
+      }
+
+      if (errorMessage.length() > 0) {
+        throw new GemFireIOException(errorMessage.toString());
+      }
+
+      // do we have to waitFor??
+      compacterProcess.destroy();
+      result = ResultBuilder.createInfoResult(output.toString());
+    } catch (IOException e) {
+      if (output.length() != 0) {
+        Gfsh.println(output.toString());
+      }
+      String fieldsMessage = (maxOplogSize != -1
+          ? CliStrings.COMPACT_OFFLINE_DISK_STORE__MAXOPLOGSIZE + "=" + maxOplogSize + "," : "");
+      fieldsMessage += CliUtil.arrayToString(diskDirs);
+      String errorString = CliStrings.format(
+          CliStrings.COMPACT_OFFLINE_DISK_STORE__MSG__ERROR_WHILE_COMPACTING_DISKSTORE_0_WITH_1_REASON_2,
+          diskStoreName, fieldsMessage);
+      result = ResultBuilder.createUserErrorResult(errorString);
+      if (logWrapper.fineEnabled()) {
+        logWrapper.fine(e.getMessage(), e);
+      }
+    } catch (GemFireIOException e) {
+      if (output.length() != 0) {
+        Gfsh.println(output.toString());
+      }
+      result = ResultBuilder.createUserErrorResult(errorMessage.toString());
+      if (logWrapper.fineEnabled()) {
+        logWrapper.fine(error.toString());
+      }
+    } catch (IllegalArgumentException e) {
+      if (output.length() != 0) {
+        Gfsh.println(output.toString());
+      }
+      result = ResultBuilder.createUserErrorResult(e.getMessage());
+    } finally {
+      if (compacterProcess != null) {
+        try {
+          // just to check whether the process has exited
+          // Process.exitValue() throws IllegalThreadStateException if Process
+          // is alive
+          compacterProcess.exitValue();
+        } catch (IllegalThreadStateException ise) {
+          // not yet terminated, destroy the process
+          compacterProcess.destroy();
+        }
+      }
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ca808714/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java
new file mode 100644
index 0000000..19784d6
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateDiskStoreCommand.java
@@ -0,0 +1,166 @@
+/*
+ * 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.util.List;
+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.internal.cache.DiskStoreAttributes;
+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.functions.CliFunctionResult;
+import org.apache.geode.management.internal.cli.functions.CreateDiskStoreFunction;
+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;
+
+public class CreateDiskStoreCommand implements GfshCommand {
+  @CliCommand(value = CliStrings.CREATE_DISK_STORE, help = CliStrings.CREATE_DISK_STORE__HELP)
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.MANAGE, target = ResourcePermission.Target.DISK)
+  public Result createDiskStore(
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__NAME, mandatory = true,
+          optionContext = ConverterHint.DISKSTORE,
+          help = CliStrings.CREATE_DISK_STORE__NAME__HELP) String name,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__ALLOW_FORCE_COMPACTION,
+          specifiedDefaultValue = "true", unspecifiedDefaultValue = "false",
+          help = CliStrings.CREATE_DISK_STORE__ALLOW_FORCE_COMPACTION__HELP) boolean allowForceCompaction,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__AUTO_COMPACT, specifiedDefaultValue = "true",
+          unspecifiedDefaultValue = "true",
+          help = CliStrings.CREATE_DISK_STORE__AUTO_COMPACT__HELP) boolean autoCompact,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__COMPACTION_THRESHOLD,
+          unspecifiedDefaultValue = "50",
+          help = CliStrings.CREATE_DISK_STORE__COMPACTION_THRESHOLD__HELP) int compactionThreshold,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__MAX_OPLOG_SIZE,
+          unspecifiedDefaultValue = "1024",
+          help = CliStrings.CREATE_DISK_STORE__MAX_OPLOG_SIZE__HELP) int maxOplogSize,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__QUEUE_SIZE, unspecifiedDefaultValue = "0",
+          help = CliStrings.CREATE_DISK_STORE__QUEUE_SIZE__HELP) int queueSize,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__TIME_INTERVAL,
+          unspecifiedDefaultValue = "1000",
+          help = CliStrings.CREATE_DISK_STORE__TIME_INTERVAL__HELP) long timeInterval,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__WRITE_BUFFER_SIZE,
+          unspecifiedDefaultValue = "32768",
+          help = CliStrings.CREATE_DISK_STORE__WRITE_BUFFER_SIZE__HELP) int writeBufferSize,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE, mandatory = true,
+          help = CliStrings.CREATE_DISK_STORE__DIRECTORY_AND_SIZE__HELP) String[] directoriesAndSizes,
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          help = CliStrings.CREATE_DISK_STORE__GROUP__HELP,
+          optionContext = ConverterHint.MEMBERGROUP) String[] groups,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__DISK_USAGE_WARNING_PCT,
+          unspecifiedDefaultValue = "90",
+          help = CliStrings.CREATE_DISK_STORE__DISK_USAGE_WARNING_PCT__HELP) float diskUsageWarningPercentage,
+      @CliOption(key = CliStrings.CREATE_DISK_STORE__DISK_USAGE_CRITICAL_PCT,
+          unspecifiedDefaultValue = "99",
+          help = CliStrings.CREATE_DISK_STORE__DISK_USAGE_CRITICAL_PCT__HELP) float diskUsageCriticalPercentage) {
+
+    try {
+      DiskStoreAttributes diskStoreAttributes = new DiskStoreAttributes();
+      diskStoreAttributes.allowForceCompaction = allowForceCompaction;
+      diskStoreAttributes.autoCompact = autoCompact;
+      diskStoreAttributes.compactionThreshold = compactionThreshold;
+      diskStoreAttributes.maxOplogSizeInBytes = maxOplogSize * (1024 * 1024);
+      diskStoreAttributes.queueSize = queueSize;
+      diskStoreAttributes.timeInterval = timeInterval;
+      diskStoreAttributes.writeBufferSize = writeBufferSize;
+
+      File[] directories = new File[directoriesAndSizes.length];
+      int[] sizes = new int[directoriesAndSizes.length];
+      for (int i = 0; i < directoriesAndSizes.length; i++) {
+        final int hashPosition = directoriesAndSizes[i].indexOf('#');
+        if (hashPosition == -1) {
+          directories[i] = new File(directoriesAndSizes[i]);
+          sizes[i] = Integer.MAX_VALUE;
+        } else {
+          directories[i] = new File(directoriesAndSizes[i].substring(0, hashPosition));
+          sizes[i] = Integer.parseInt(directoriesAndSizes[i].substring(hashPosition + 1));
+        }
+      }
+      diskStoreAttributes.diskDirs = directories;
+      diskStoreAttributes.diskDirSizes = sizes;
+
+      diskStoreAttributes.setDiskUsageWarningPercentage(diskUsageWarningPercentage);
+      diskStoreAttributes.setDiskUsageCriticalPercentage(diskUsageCriticalPercentage);
+
+      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);
+      }
+
+      ResultCollector<?, ?> rc = CliUtil.executeFunction(new CreateDiskStoreFunction(),
+          new Object[] {name, diskStoreAttributes}, 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(Result.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 disk store(s).");
+      }
+
+      Result result = ResultBuilder.buildResult(tabularData);
+
+      if (xmlEntity.get() != null) {
+        persistClusterConfiguration(result,
+            () -> getSharedConfiguration().addXmlEntity(xmlEntity.get(), groups));
+      }
+
+      return ResultBuilder.buildResult(tabularData);
+    } catch (VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+      throw e;
+    } catch (Throwable th) {
+      SystemFailure.checkFailure();
+      return ResultBuilder.createGemFireErrorResult(
+          CliStrings.format(CliStrings.CREATE_DISK_STORE__ERROR_WHILE_CREATING_REASON_0,
+              new Object[] {th.getMessage()}));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ca808714/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeDiskStoreCommand.java
new file mode 100644
index 0000000..4efd973
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeDiskStoreCommand.java
@@ -0,0 +1,177 @@
+/*
+ * 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.Collections;
+import java.util.List;
+
+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.lang.ClassUtils;
+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.domain.DiskStoreDetails;
+import org.apache.geode.management.internal.cli.functions.DescribeDiskStoreFunction;
+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.cli.result.TabularResultData;
+import org.apache.geode.management.internal.cli.util.DiskStoreNotFoundException;
+import org.apache.geode.management.internal.cli.util.MemberNotFoundException;
+import org.apache.geode.management.internal.security.ResourceOperation;
+import org.apache.geode.security.ResourcePermission;
+
+public class DescribeDiskStoreCommand implements GfshCommand {
+  @CliCommand(value = CliStrings.DESCRIBE_DISK_STORE, help = CliStrings.DESCRIBE_DISK_STORE__HELP)
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.READ)
+  public Result describeDiskStore(
+      @CliOption(key = CliStrings.MEMBER, mandatory = true,
+          optionContext = ConverterHint.MEMBERIDNAME,
+          help = CliStrings.DESCRIBE_DISK_STORE__MEMBER__HELP) final String memberName,
+      @CliOption(key = CliStrings.DESCRIBE_DISK_STORE__NAME, mandatory = true,
+          optionContext = ConverterHint.DISKSTORE,
+          help = CliStrings.DESCRIBE_DISK_STORE__NAME__HELP) final String diskStoreName) {
+    try {
+      return toCompositeResult(getDiskStoreDescription(memberName, diskStoreName));
+    } catch (DiskStoreNotFoundException | MemberNotFoundException e) {
+      return ResultBuilder.createShellClientErrorResult(e.getMessage());
+    } catch (FunctionInvocationTargetException ignore) {
+      return ResultBuilder.createGemFireErrorResult(CliStrings
+          .format(CliStrings.COULD_NOT_EXECUTE_COMMAND_TRY_AGAIN, CliStrings.DESCRIBE_DISK_STORE));
+    } catch (VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+      throw e;
+    } catch (Throwable t) {
+      SystemFailure.checkFailure();
+      return ResultBuilder
+          .createGemFireErrorResult(String.format(CliStrings.DESCRIBE_DISK_STORE__ERROR_MESSAGE,
+              memberName, diskStoreName, toString(t, isDebugging())));
+    }
+  }
+
+  DiskStoreDetails getDiskStoreDescription(final String memberName, final String diskStoreName) {
+    final DistributedMember member = getMember(getCache(), memberName);
+
+    final ResultCollector<?, ?> resultCollector =
+        getMembersFunctionExecutor(Collections.singleton(member)).setArguments(diskStoreName)
+            .execute(new DescribeDiskStoreFunction());
+
+    final Object result = ((List<?>) resultCollector.getResult()).get(0);
+
+    if (result instanceof DiskStoreDetails) { // disk store details in hand...
+      return (DiskStoreDetails) result;
+    } else if (result instanceof DiskStoreNotFoundException) { // bad disk store name...
+      throw (DiskStoreNotFoundException) result;
+    } else { // unknown and unexpected return type...
+      final Throwable cause = (result instanceof Throwable ? (Throwable) result : null);
+
+      if (isLogging()) {
+        if (cause != null) {
+          getGfsh().logSevere(String.format(
+              "Exception (%1$s) occurred while executing '%2$s' on member (%3$s) with disk store (%4$s).",
+              ClassUtils.getClassName(cause), CliStrings.DESCRIBE_DISK_STORE, memberName,
+              diskStoreName), cause);
+        } else {
+          getGfsh().logSevere(String.format(
+              "Received an unexpected result of type (%1$s) while executing '%2$s' on member (%3$s) with disk store (%4$s).",
+              ClassUtils.getClassName(result), CliStrings.DESCRIBE_DISK_STORE, memberName,
+              diskStoreName), null);
+        }
+      }
+
+      throw new RuntimeException(
+          CliStrings.format(CliStrings.UNEXPECTED_RETURN_TYPE_EXECUTING_COMMAND_ERROR_MESSAGE,
+              ClassUtils.getClassName(result), CliStrings.DESCRIBE_DISK_STORE),
+          cause);
+    }
+  }
+
+  private Result toCompositeResult(final DiskStoreDetails diskStoreDetails) {
+    final CompositeResultData diskStoreData = ResultBuilder.createCompositeResultData();
+
+    final CompositeResultData.SectionResultData diskStoreSection = diskStoreData.addSection();
+
+    diskStoreSection.addData("Disk Store ID", diskStoreDetails.getId());
+    diskStoreSection.addData("Disk Store Name", diskStoreDetails.getName());
+    diskStoreSection.addData("Member ID", diskStoreDetails.getMemberId());
+    diskStoreSection.addData("Member Name", diskStoreDetails.getMemberName());
+    diskStoreSection.addData("Allow Force Compaction",
+        toString(diskStoreDetails.isAllowForceCompaction(), "Yes", "No"));
+    diskStoreSection.addData("Auto Compaction",
+        toString(diskStoreDetails.isAutoCompact(), "Yes", "No"));
+    diskStoreSection.addData("Compaction Threshold", diskStoreDetails.getCompactionThreshold());
+    diskStoreSection.addData("Max Oplog Size", diskStoreDetails.getMaxOplogSize());
+    diskStoreSection.addData("Queue Size", diskStoreDetails.getQueueSize());
+    diskStoreSection.addData("Time Interval", diskStoreDetails.getTimeInterval());
+    diskStoreSection.addData("Write Buffer Size", diskStoreDetails.getWriteBufferSize());
+    diskStoreSection.addData("Disk Usage Warning Percentage",
+        diskStoreDetails.getDiskUsageWarningPercentage());
+    diskStoreSection.addData("Disk Usage Critical Percentage",
+        diskStoreDetails.getDiskUsageCriticalPercentage());
+    diskStoreSection.addData("PDX Serialization Meta-Data Stored",
+        toString(diskStoreDetails.isPdxSerializationMetaDataStored(), "Yes", "No"));
+
+    final TabularResultData diskDirTable = diskStoreData.addSection().addTable();
+
+    for (DiskStoreDetails.DiskDirDetails diskDirDetails : diskStoreDetails) {
+      diskDirTable.accumulate("Disk Directory", diskDirDetails.getAbsolutePath());
+      diskDirTable.accumulate("Size", diskDirDetails.getSize());
+    }
+
+    final TabularResultData regionTable = diskStoreData.addSection().addTable();
+
+    for (DiskStoreDetails.RegionDetails regionDetails : diskStoreDetails.iterateRegions()) {
+      regionTable.accumulate("Region Path", regionDetails.getFullPath());
+      regionTable.accumulate("Region Name", regionDetails.getName());
+      regionTable.accumulate("Persistent", toString(regionDetails.isPersistent(), "Yes", "No"));
+      regionTable.accumulate("Overflow To Disk",
+          toString(regionDetails.isOverflowToDisk(), "Yes", "No"));
+    }
+
+    final TabularResultData cacheServerTable = diskStoreData.addSection().addTable();
+
+    for (DiskStoreDetails.CacheServerDetails cacheServerDetails : diskStoreDetails
+        .iterateCacheServers()) {
+      cacheServerTable.accumulate("Bind Address", cacheServerDetails.getBindAddress());
+      cacheServerTable.accumulate("Hostname for Clients", cacheServerDetails.getHostName());
+      cacheServerTable.accumulate("Port", cacheServerDetails.getPort());
+    }
+
+    final TabularResultData gatewayTable = diskStoreData.addSection().addTable();
+
+    for (DiskStoreDetails.GatewayDetails gatewayDetails : diskStoreDetails.iterateGateways()) {
+      gatewayTable.accumulate("Gateway ID", gatewayDetails.getId());
+      gatewayTable.accumulate("Persistent", toString(gatewayDetails.isPersistent(), "Yes", "No"));
+    }
+
+    final TabularResultData asyncEventQueueTable = diskStoreData.addSection().addTable();
+
+    for (DiskStoreDetails.AsyncEventQueueDetails asyncEventQueueDetails : diskStoreDetails
+        .iterateAsyncEventQueues()) {
+      asyncEventQueueTable.accumulate("Async Event Queue ID", asyncEventQueueDetails.getId());
+    }
+
+    return ResultBuilder.buildResult(diskStoreData);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ca808714/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeOfflineDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeOfflineDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeOfflineDiskStoreCommand.java
new file mode 100644
index 0000000..904a677
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DescribeOfflineDiskStoreCommand.java
@@ -0,0 +1,75 @@
+/*
+ * 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 org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.SystemFailure;
+import org.apache.geode.cache.Region;
+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;
+
+public class DescribeOfflineDiskStoreCommand implements GfshCommand {
+  @CliCommand(value = CliStrings.DESCRIBE_OFFLINE_DISK_STORE,
+      help = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__HELP)
+  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
+  public Result describeOfflineDiskStore(
+      @CliOption(key = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKSTORENAME, mandatory = true,
+          help = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKSTORENAME__HELP) String diskStoreName,
+      @CliOption(key = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKDIRS, mandatory = true,
+          help = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKDIRS__HELP) String[] diskDirs,
+      @CliOption(key = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__PDX_TYPES,
+          help = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__PDX_TYPES__HELP) Boolean listPdxTypes,
+      @CliOption(key = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__REGIONNAME,
+          help = CliStrings.DESCRIBE_OFFLINE_DISK_STORE__REGIONNAME__HELP) String regionName) {
+
+    try {
+      final File[] dirs = new File[diskDirs.length];
+      for (int i = 0; i < diskDirs.length; i++) {
+        dirs[i] = new File((diskDirs[i]));
+      }
+
+      if (Region.SEPARATOR.equals(regionName)) {
+        return ResultBuilder.createUserErrorResult(CliStrings.INVALID_REGION_NAME);
+      }
+
+      ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+      PrintStream printStream = new PrintStream(outputStream);
+
+      DiskStoreImpl.dumpInfo(printStream, diskStoreName, dirs, regionName, listPdxTypes);
+      return ResultBuilder.createInfoResult(outputStream.toString());
+    } catch (VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+      throw e;
+    } catch (Throwable th) {
+      SystemFailure.checkFailure();
+      if (th.getMessage() == null) {
+        return ResultBuilder.createGemFireErrorResult(
+            "An error occurred while describing offline disk stores: " + th);
+      }
+      return ResultBuilder.createGemFireErrorResult(
+          "An error occurred while describing offline disk stores: " + th.getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/ca808714/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyDiskStoreCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyDiskStoreCommand.java
new file mode 100644
index 0000000..2f24736
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DestroyDiskStoreCommand.java
@@ -0,0 +1,106 @@
+/*
+ * 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 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.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.functions.CliFunctionResult;
+import org.apache.geode.management.internal.cli.functions.DestroyDiskStoreFunction;
+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;
+
+public class DestroyDiskStoreCommand implements GfshCommand {
+  @CliCommand(value = CliStrings.DESTROY_DISK_STORE, help = CliStrings.DESTROY_DISK_STORE__HELP)
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE})
+  @ResourceOperation(resource = ResourcePermission.Resource.CLUSTER,
+      operation = ResourcePermission.Operation.MANAGE, target = ResourcePermission.Target.DISK)
+  public Result destroyDiskStore(
+      @CliOption(key = CliStrings.DESTROY_DISK_STORE__NAME, mandatory = true,
+          help = CliStrings.DESTROY_DISK_STORE__NAME__HELP) String name,
+      @CliOption(key = {CliStrings.GROUP, CliStrings.GROUPS},
+          help = CliStrings.DESTROY_DISK_STORE__GROUP__HELP,
+          optionContext = ConverterHint.MEMBERGROUP) String[] groups) {
+    try {
+      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);
+      }
+
+      ResultCollector<?, ?> rc = CliUtil.executeFunction(new DestroyDiskStoreFunction(),
+          new Object[] {name}, 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(Result.Status.ERROR);
+        } else if (result.getMessage() != null) {
+          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("No matching disk stores found.");
+      }
+
+      Result result = ResultBuilder.buildResult(tabularData);
+      if (xmlEntity.get() != null) {
+        persistClusterConfiguration(result,
+            () -> getSharedConfiguration().deleteXmlEntity(xmlEntity.get(), groups));
+      }
+
+      return result;
+    } catch (VirtualMachineError e) {
+      SystemFailure.initiateFailure(e);
+      throw e;
+    } catch (Throwable th) {
+      SystemFailure.checkFailure();
+      return ResultBuilder.createGemFireErrorResult(
+          CliStrings.format(CliStrings.DESTROY_DISK_STORE__ERROR_WHILE_DESTROYING_REASON_0,
+              new Object[] {th.getMessage()}));
+    }
+  }
+}