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 2016/09/21 22:01:56 UTC

incubator-geode git commit: GEODE-1128: Add missing regions to the missing-disk-stores command

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 813fa909c -> 09cd4ad27


GEODE-1128: Add missing regions to the missing-disk-stores command

Refactored \u201cshow missing-disk-stores command\u201d and added a section to the
output to show missing colocated regions. Corrected messages in
LocalizedStrings, change \u201cgemfire list-missing-disk-stores\u201d to
\u201cgfsh show missing-disk-stores\u201d. Added new command function class to gather
both missing disk-store and missing colocated region information on a
distributed system member.

Parameterized show missing-disk-store error message.

Added new DataSerializable class for sending missing colocated region
results to gfsh command.

Refactored ColocationLogger. Class originally deferred updating the list
of missing colocated children (i.e. removing regions from the missing
list that are now present) for a region until the next logging
interval. This has been changed to update the list on-demand, whenever
getMissingChildRegions() is called.

Added DUnit test for new functionality in show missing-disk-store
command.

Added JUnit tests for new classes

This closes #244


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

Branch: refs/heads/develop
Commit: 09cd4ad27298ebf20441e3285636f8e4c7e772c7
Parents: 813fa90
Author: Ken Howe <kh...@pivotal.io>
Authored: Mon Sep 19 11:57:58 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Sep 21 15:01:13 2016 -0700

----------------------------------------------------------------------
 .../geode/internal/cache/ColocationLogger.java  |  20 +-
 .../geode/internal/cache/PartitionedRegion.java |   2 +-
 .../partitioned/ColocatedRegionDetails.java     | 204 +++++++++
 .../geode/internal/i18n/LocalizedStrings.java   |  10 +-
 .../cli/commands/DiskStoreCommands.java         | 114 ++++-
 .../ShowMissingDiskStoresFunction.java          | 107 +++++
 .../internal/cli/i18n/CliStrings.java           |   1 +
 .../ColocatedRegionDetailsJUnitTest.java        | 206 +++++++++
 .../commands/DiskStoreCommandsDUnitTest.java    | 245 ++++++++++-
 .../ShowMissingDiskStoresFunctionJUnitTest.java | 438 +++++++++++++++++++
 .../sanctionedDataSerializables.txt             |   4 +
 .../codeAnalysis/sanctionedSerializables.txt    |   1 +
 12 files changed, 1314 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/main/java/org/apache/geode/internal/cache/ColocationLogger.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ColocationLogger.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ColocationLogger.java
index 0a37547..412211c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ColocationLogger.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ColocationLogger.java
@@ -18,7 +18,7 @@ package org.apache.geode.internal.cache;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
+import java.util.Set;
 
 import org.apache.logging.log4j.Logger;
 
@@ -107,9 +107,6 @@ public class ColocationLogger implements Runnable {
           //Terminate the logging thread, recoverycomplete is only true when there are no missing colocated regions
           break;
         }
-        List<String>  existingRegions;
-        Map coloHierarchy = ColocationHelper.getAllColocationRegions(region);
-        missingChildren.removeAll(coloHierarchy.keySet());
         if(missingChildren.isEmpty()) {
           break;
         }
@@ -140,7 +137,20 @@ public class ColocationLogger implements Runnable {
     }
   }
 
-  public List<String> getMissingChildRegions() {
+  /**
+   * Updates the missing colocated child region list and returns a copy of the list.
+   * <p>
+   * The list of missing child regions is normally updated lazily, only when this logger thread periodically wakes up to
+   * log warnings about the colocated regions that are still missing. This method performs an on-demand update of the
+   * list so if called between logging intervals the returned list is current.
+   *
+   * @return missingChildren
+   */
+  public List<String> updateAndGetMissingChildRegions() {
+    synchronized (loggerLock) {
+      Set<String> childRegions = (Set<String>) ColocationHelper.getAllColocationRegions(this.region).keySet();
+      missingChildren.removeAll(childRegions);
+    }
     return new ArrayList<String>(missingChildren);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
index 6e4fcd5..baab79f 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
@@ -7532,7 +7532,7 @@ public class PartitionedRegion extends LocalRegion implements
   public List<String> getMissingColocatedChildren() {
     ColocationLogger regionLogger = missingColocatedRegionLogger;
     if (regionLogger != null) {
-      return regionLogger.getMissingChildRegions();
+      return regionLogger.updateAndGetMissingChildRegions();
     }
     return Collections.emptyList();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/ColocatedRegionDetails.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/ColocatedRegionDetails.java b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/ColocatedRegionDetails.java
new file mode 100644
index 0000000..53a69c9
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/partitioned/ColocatedRegionDetails.java
@@ -0,0 +1,204 @@
+/*
+ * 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.internal.cache.partitioned;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.geode.DataSerializable;
+import org.apache.geode.DataSerializer;
+import org.apache.geode.distributed.DistributedMember;
+
+/**
+ * Composite data type used to record colocation relationships
+ */
+public class ColocatedRegionDetails implements DataSerializable {
+  private String host;
+  private String member;
+  private String parent;
+  private String child;
+
+  public ColocatedRegionDetails(final String host, final String member, final String parent, final String child) {
+    this.host = host;
+    this.member = member;
+    this.parent = parent;
+    this.child = child;
+  }
+
+  //Used for deserialization only
+  public ColocatedRegionDetails() {
+  }
+
+  /**
+   * Returns the canonical name of the host machine
+   * 
+   * @return parent
+   */
+  public String getHost() {
+    return host;
+  }
+
+  /**
+   * Returns the name of the {@link DistributedMember}
+   * 
+   * @return parent
+   */
+  public String getMember() {
+    return member;
+  }
+
+  /**
+   * Returns the name of the parent region of a colocated pair
+   * 
+   * @return parent
+   */
+  public String getParent() {
+    return parent;
+  }
+
+  /**
+   * Returns the name of the child region of a colocated pair
+   * 
+   * @return child
+   */
+  public String getChild() {
+    return child;
+  }
+
+  @Override
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    boolean hasHost = in.readBoolean();
+    if(hasHost) {
+      host = DataSerializer.readString(in);
+    }
+    boolean hasMember = in.readBoolean();
+    if (hasMember) {
+      member = DataSerializer.readString(in);
+    }
+    boolean hasParent = in.readBoolean();
+    if (hasParent) {
+      parent = DataSerializer.readString(in);
+    }
+    boolean hasChild = in.readBoolean();
+    if (hasChild) {
+      child = DataSerializer.readString(in);
+    }
+  }
+
+  @Override
+  public void toData(DataOutput out) throws IOException {
+    out.writeBoolean(host != null);
+    if(host != null) {
+      DataSerializer.writeString(host, out);
+    }
+    out.writeBoolean(member != null);
+    if (member != null) {
+      DataSerializer.writeString(member, out);
+    }
+    out.writeBoolean(parent != null);
+    if (parent != null) {
+      DataSerializer.writeString(parent, out);
+    }
+    out.writeBoolean(child != null);
+    if (child != null) {
+      DataSerializer.writeString(child, out);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append("[");
+    if(host != null) {
+      result.append("host:" + host.toString());
+    } else {
+      result.append("");
+    }
+    if(member != null) {
+      result.append(", member:" + member.toString());
+    } else {
+      result.append(",");
+    }
+    if(parent != null) {
+      result.append(", parent:" + parent.toString());
+    } else {
+      result.append(",");
+    }
+    if(child != null) {
+      result.append(", child:" + child.toString());
+    } else {
+      result.append(",");
+    }
+    result.append("]");
+
+    return result.toString();
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((host == null) ? 0 : host.hashCode());
+    result = prime * result + ((member == null) ? 0 : member.hashCode());
+    result = prime * result + ((parent == null) ? 0 : parent.hashCode());
+    result = prime * result + ((child == null) ? 0 : child.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ColocatedRegionDetails other = (ColocatedRegionDetails) obj;
+    if (host == null) {
+      if (other.host != null) {
+        return false;
+      }
+    } else if (!host.equals(other.host)) {
+      return false;
+    }
+    if (member == null) {
+      if (other.member != null) {
+        return false;
+      }
+    } else if (!member.equals(other.member)) {
+      return false;
+    }
+    if (parent == null) {
+      if (other.parent != null) {
+        return false;
+      }
+    } else if (!parent.equals(other.parent)) {
+      return false;
+    }
+    if (child == null) {
+      if (other.child != null) {
+        return false;
+      }
+    } else if (!child.equals(other.child)) {
+      return false;
+    }
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
index b61201e..29dc500 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
@@ -3243,10 +3243,10 @@ public class LocalizedStrings {
   public static final StringId PartitionedRegion_REGION_WITH_PRID_0_FAILED_INITIALIZATION_ON_THIS_NODE = new StringId(5016, "Region with prId= {0}  failed initialization on this node");
   public static final StringId PersistenceAdvisorImpl_MEMBER_REVOKED = new StringId(5017, "The following persistent member has been revoked:\n{0}");
   public static final StringId CreatePersistentRegionProcessor_DONE_WAITING_FOR_BUCKET_MEMBERS = new StringId(5018, "Region {0} has successfully completed waiting for other members to recover the latest data.\nMy persistent member information:{1}");
-  public static final StringId CreatePersistentRegionProcessor_WAITING_FOR_OFFLINE_BUCKET_MEMBERS = new StringId(5019, "Region {0} (and any colocated sub-regions) has potentially stale data.  Buckets {1} are waiting for another offline member to recover the latest data.\nMy persistent id is:{2}\nOffline members with potentially new data:\n{3}\nUse the \"gemfire list-missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
+  public static final StringId CreatePersistentRegionProcessor_WAITING_FOR_OFFLINE_BUCKET_MEMBERS = new StringId(5019, "Region {0} (and any colocated sub-regions) has potentially stale data.  Buckets {1} are waiting for another offline member to recover the latest data.\nMy persistent id is:{2}\nOffline members with potentially new data:\n{3}\nUse the \"gfsh show missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
   public static final StringId CreatePersistentRegionProcessor_SPLIT_DISTRIBUTED_SYSTEM = new StringId(5020, "Region {0} remote member {1} with persistent data {2} was not part of the same distributed system as the local data from {3}");
   public static final StringId CreatePersistentRegionProcessor_INITIALIZING_FROM_OLD_DATA = new StringId(5021, "Region {0} refusing to initialize from member {1} with persistent data {2} which was offline when the local data from {3} was last online");
-  public static final StringId CreatePersistentRegionProcessor_WAITING_FOR_LATEST_MEMBER = new StringId(5022, "Region {0} has potentially stale data. It is waiting for another member to recover the latest data.\nMy persistent id:\n{1}\nMembers with potentially new data:\n{2}\nUse the \"gemfire list-missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
+  public static final StringId CreatePersistentRegionProcessor_WAITING_FOR_LATEST_MEMBER = new StringId(5022, "Region {0} has potentially stale data. It is waiting for another member to recover the latest data.\nMy persistent id:\n{1}\nMembers with potentially new data:\n{2}\nUse the \"gfsh show missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
   public static final StringId PersistenceAdvisorImpl_UNABLE_TO_PERSIST_MEMBERSHIP_CHANGE = new StringId(5023, "Unable to persist membership change");
   public static final StringId DistributedRegion_ERROR_CLEANING_UP_FAILED_INITIALIZATION = new StringId(5024, "Error cleaning up after failed region initialization of region {0}");
   public static final StringId PartitionedRegionDataStore_DATA_OFFLINE_MESSAGE = new StringId(5025, "Region {0} bucket {1} has persistent data that is no longer online stored at these locations: {2}");
@@ -3263,7 +3263,7 @@ public class LocalizedStrings {
   public static final StringId FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER = new StringId(5037, "Function attributes at client and server don''t match");
   public static final StringId Region_PutAll_Applied_PartialKeys_0_1 = new StringId(5038, "Region {0} putAll: {1}");
   public static final StringId Region_PutAll_Applied_PartialKeys_At_Server_0 = new StringId(5039, "Region {0} putAll at server applied partial keys due to exception.");
-  public static final StringId BucketPersistenceAdvisor_WAITING_FOR_LATEST_MEMBER = new StringId(5040, "Region {0}, bucket {1} has potentially stale data.  It is waiting for another member to recover the latest data.\nMy persistent id:\n{2}\nMembers with potentially new data:\n{3}\nUse the \"gemfire list-missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
+  public static final StringId BucketPersistenceAdvisor_WAITING_FOR_LATEST_MEMBER = new StringId(5040, "Region {0}, bucket {1} has potentially stale data.  It is waiting for another member to recover the latest data.\nMy persistent id:\n{2}\nMembers with potentially new data:\n{3}\nUse the \"gfsh show missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
 
   public static final StringId AgentConfigImpl_TCP_PORT = new StringId(5041, "TCP/IP port number to use in the agent''s distributed system");
 
@@ -3458,7 +3458,7 @@ public class LocalizedStrings {
   public static final StringId PersistenceAdvisorImpl_PERSISTENT_VIEW = new StringId(5233, "The following persistent member has gone offline for region {0}:\n{1}\nRemaining participating members for the region include:\n{2}");
 
   public static final StringId DefaultQuery_A_QUERY_ON_A_PARTITIONED_REGION_0_MAY_NOT_REFERENCE_ANY_OTHER_NON_COLOCATED_PARTITIONED_REGION_1 = new StringId(5234, "A query on a Partitioned Region ( {0} ) may not reference any other region except Co-located Partitioned Region. PR region (1) is not collocated with other PR region in the query.");
-  public static final StringId CreatePersistentRegionProcessor_WAITING_FOR_ONLINE_BUCKET_MEMBERS = new StringId(5235, "Region {0} (and any colocated sub-regions) has potentially stale data.  Buckets {1} are waiting for another online member to recover the latest data.\nMy persistent id is:{2}\nOnline members with potentially new data:\n{3}\nUse the \"gemfire list-missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
+  public static final StringId CreatePersistentRegionProcessor_WAITING_FOR_ONLINE_BUCKET_MEMBERS = new StringId(5235, "Region {0} (and any colocated sub-regions) has potentially stale data.  Buckets {1} are waiting for another online member to recover the latest data.\nMy persistent id is:{2}\nOnline members with potentially new data:\n{3}\nUse the \"gfsh show missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
 
   // Localized Strings for the AbstractLauncher, Locator and Server Launcher classes
   public static final StringId Launcher_ATTACH_API_NOT_FOUND_ERROR_MESSAGE = new StringId(5236, "The Attach API classes could not be found on the classpath.  Please include JDK tools.jar on the classpath or add the JDK tools.jar to the jre/lib/ext directory.");
@@ -3532,7 +3532,7 @@ public class LocalizedStrings {
   public static final StringId CqService_ERROR_SENDING_CQ_CONNECTION_STATUS = new StringId(5401, "Error while sending connection status to cq listeners");
   public static final StringId AbstractGatewaySender_SENDER_0_GOT_REMOTE_LOCATOR_INFORMATION_FOR_SITE_1 = new StringId(5402, "GatewaySender \"{0}\" got remote locator information for remote site \"{1}\" after {2} failures in connecting to remote site.");
 
-  public static final StringId CreatePersistentRegionProcessor_WAITING_FOR_ONLINE_LATEST_MEMBER = new StringId(5403, "Region {0} has potentially stale data. It is waiting for another online member to recover the latest data.\nMy persistent id:\n{1}\nMembers with potentially new data:\n{2}\nUse the \"gemfire list-missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
+  public static final StringId CreatePersistentRegionProcessor_WAITING_FOR_ONLINE_LATEST_MEMBER = new StringId(5403, "Region {0} has potentially stale data. It is waiting for another online member to recover the latest data.\nMy persistent id:\n{1}\nMembers with potentially new data:\n{2}\nUse the \"gfsh show missing-disk-stores\" command to see all disk stores that are being waited on by other members.");
   public static final StringId AttributesFactory_CONCURRENCY_CHECKS_MUST_BE_ENABLED = new StringId(5404, "Concurrency checks cannot be disabled for regions that use persistence");
 
   public static final StringId CqService_UNABLE_TO_RETRIEVE_DURABLE_CQS_FOR_CLIENT_PROXY_ID = new StringId(5406, "Unable to retrieve durable CQs for client proxy id {0}");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java
index 53f9108..d9ae2c2 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommands.java
@@ -55,6 +55,9 @@ import org.apache.geode.internal.cache.DiskStoreAttributes;
 import org.apache.geode.internal.cache.DiskStoreImpl;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
+import org.apache.geode.internal.cache.partitioned.ColocatedRegionDetails;
+import org.apache.geode.internal.cache.persistence.PersistentMemberID;
+import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
 import org.apache.geode.internal.lang.ClassUtils;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.DistributedSystemMXBean;
@@ -73,6 +76,7 @@ import org.apache.geode.management.internal.cli.functions.CreateDiskStoreFunctio
 import org.apache.geode.management.internal.cli.functions.DescribeDiskStoreFunction;
 import org.apache.geode.management.internal.cli.functions.DestroyDiskStoreFunction;
 import org.apache.geode.management.internal.cli.functions.ListDiskStoresFunction;
+import org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.CommandResultException;
 import org.apache.geode.management.internal.cli.result.CompositeResultData;
@@ -1026,39 +1030,101 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   public Result showMissingDiskStore() {
 
     try {
-      TabularResultData tabularData = ResultBuilder.createTabularResultData();
-      boolean accumulatedData = false;
+      Set<DistributedMember> dataMembers = getNormalMembers(getCache());
 
-      DistributedSystemMXBean dsMXBean = ManagementService.getManagementService(CacheFactory.getAnyInstance())
-          .getDistributedSystemMXBean();
-      PersistentMemberDetails[] allPersistentMemberDetails = dsMXBean.listMissingDiskStores();
-      if (allPersistentMemberDetails != null) {
-        for (PersistentMemberDetails peristentMemberDetails : allPersistentMemberDetails) {
-          tabularData.accumulate("Disk Store ID", peristentMemberDetails.getDiskStoreId());
-          tabularData.accumulate("Host", peristentMemberDetails.getHost());
-          tabularData.accumulate("Directory", peristentMemberDetails.getDirectory());
-          accumulatedData = true;
-        }
+      if (dataMembers.isEmpty()) {
+        return ResultBuilder.createInfoResult(CliStrings.NO_CACHING_MEMBERS_FOUND_MESSAGE);
       }
-
-      if (!accumulatedData) {
-        return ResultBuilder.createInfoResult("No missing disk store found");
-     }
-
-      return ResultBuilder.buildResult(tabularData);
+      List<Object> results = getMissingDiskStoresList(dataMembers);
+      return toMissingDiskStoresTabularResult(results);
+    } catch (FunctionInvocationTargetException ignore) {
+      return ResultBuilder.createGemFireErrorResult(CliStrings.format(CliStrings.COULD_NOT_EXECUTE_COMMAND_TRY_AGAIN,
+          CliStrings.SHOW_MISSING_DISK_STORE));
     } catch (VirtualMachineError e) {
       SystemFailure.initiateFailure(e);
       throw e;
-    } catch (Throwable th) {
+    } catch (Throwable t) {
       SystemFailure.checkFailure();
-      if (th.getMessage() == null) {
-        return ResultBuilder.createGemFireErrorResult("An error occurred while showing missing disk stores: " + th);
+    if (t.getMessage() == null) {
+      return ResultBuilder.createGemFireErrorResult(String.format(CliStrings.SHOW_MISSING_DISK_STORE__ERROR_MESSAGE, t));
+    }
+    return ResultBuilder.createGemFireErrorResult(String.format(CliStrings.SHOW_MISSING_DISK_STORE__ERROR_MESSAGE, t.getMessage()));
+    }
+  }
+
+  protected List<Object> getMissingDiskStoresList(Set<DistributedMember> members) {
+    final Execution membersFunctionExecutor = getMembersFunctionExecutor(members);
+    if (membersFunctionExecutor instanceof AbstractExecution) {
+      ((AbstractExecution) membersFunctionExecutor).setIgnoreDepartedMembers(true);
+    }
+
+    final ResultCollector<?, ?> resultCollector = membersFunctionExecutor.execute(new ShowMissingDiskStoresFunction());
+
+    final List<?> results = (List<?>) resultCollector.getResult();
+    final List<Object> distributedPersistentRecoveryDetails = new ArrayList<Object>(results.size());
+    for (final Object result: results) {
+      if (result instanceof Set) { // ignore FunctionInvocationTargetExceptions and other Exceptions...
+        distributedPersistentRecoveryDetails.addAll((Set<Object>) result);
       }
-      return ResultBuilder.createGemFireErrorResult("An error occurred while showing missing disk stores: " + th.getMessage());
     }
+    return distributedPersistentRecoveryDetails;
   }
-  
-  
+
+  protected Result toMissingDiskStoresTabularResult(final List<Object> resultDetails) throws ResultDataException {
+    CompositeResultData crd = ResultBuilder.createCompositeResultData();
+    List<PersistentMemberPattern> missingDiskStores = new ArrayList<PersistentMemberPattern>();
+    List<ColocatedRegionDetails> missingColocatedRegions = new ArrayList<ColocatedRegionDetails>();
+
+    for (Object detail : resultDetails) {
+      if (detail instanceof PersistentMemberPattern) {
+        missingDiskStores.add((PersistentMemberPattern) detail);
+      } else if (detail instanceof ColocatedRegionDetails) {
+        missingColocatedRegions.add((ColocatedRegionDetails) detail);
+      } else {
+        throw new ResultDataException("Unknown type of PersistentRecoveryFailures result");
+      }
+    }
+
+    boolean hasMissingDiskStores = !missingDiskStores.isEmpty();
+    boolean hasMissingColocatedRegions = !missingColocatedRegions.isEmpty();
+    if (hasMissingDiskStores) {
+      SectionResultData missingDiskStoresSection = crd.addSection();
+      missingDiskStoresSection.setHeader("Missing Disk Stores");
+      TabularResultData missingDiskStoreData = missingDiskStoresSection.addTable();
+
+      for (PersistentMemberPattern peristentMemberDetails : missingDiskStores) {
+        missingDiskStoreData.accumulate("Disk Store ID", peristentMemberDetails.getUUID());
+        missingDiskStoreData.accumulate("Host", peristentMemberDetails.getHost());
+        missingDiskStoreData.accumulate("Directory", peristentMemberDetails.getDirectory());
+      }
+    } else {
+      SectionResultData noMissingDiskStores = crd.addSection();
+      noMissingDiskStores.setHeader("No missing disk store found");
+    }
+    if (hasMissingDiskStores || hasMissingColocatedRegions) {
+      // For clarity, separate disk store and colocated region information
+      crd.addSection().setHeader("\n");
+    }
+
+    if (hasMissingColocatedRegions) {
+      SectionResultData missingRegionsSection = crd.addSection();
+      missingRegionsSection.setHeader("Missing Colocated Regions");
+      TabularResultData missingRegionData = missingRegionsSection.addTable();
+
+      for (ColocatedRegionDetails colocatedRegionDetails:missingColocatedRegions) {
+        missingRegionData.accumulate("Host", colocatedRegionDetails.getHost());
+        missingRegionData.accumulate("Distributed Member", colocatedRegionDetails.getMember());
+        missingRegionData.accumulate("Parent Region", colocatedRegionDetails.getParent());
+        missingRegionData.accumulate("Missing Colocated Region", colocatedRegionDetails.getChild());
+      }
+    } else {
+      SectionResultData noMissingColocatedRegions = crd.addSection();
+      noMissingColocatedRegions.setHeader("No missing colocated region found");
+    }
+
+    return ResultBuilder.buildResult(crd);
+  }
+
   @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(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunction.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunction.java
new file mode 100644
index 0000000..7fc6965
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunction.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.management.internal.cli.functions;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.execute.FunctionAdapter;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.internal.InternalEntity;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.partitioned.ColocatedRegionDetails;
+import org.apache.geode.internal.cache.persistence.PersistentMemberID;
+import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
+import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
+
+public class ShowMissingDiskStoresFunction extends FunctionAdapter implements InternalEntity {
+
+  protected Cache getCache() {
+    return CacheFactory.getAnyInstance();
+  }
+
+  @Override
+  public void execute(FunctionContext context) {
+    final Set<PersistentMemberPattern> memberMissingIDs = new HashSet<PersistentMemberPattern>();
+    Set<ColocatedRegionDetails> missingColocatedRegions = new HashSet<ColocatedRegionDetails>();
+
+    if (context == null) {
+      throw new RuntimeException();
+    }
+    try {
+      final Cache cache = getCache();
+
+      if (cache instanceof InternalCache) {
+        final InternalCache gemfireCache = (InternalCache) cache;
+
+        final DistributedMember member = gemfireCache.getMyId();
+
+        GemFireCacheImpl gfci = GemFireCacheImpl.getInstance();
+        if(gfci != null && !gfci.isClosed()) {
+          // Missing DiskStores
+          PersistentMemberManager mm = gfci.getPersistentMemberManager();
+          Map<String, Set<PersistentMemberID>> waitingRegions = mm.getWaitingRegions();
+          for (Map.Entry<String, Set<PersistentMemberID>> entry : waitingRegions.entrySet()) {
+            for(PersistentMemberID id : entry.getValue()) {
+              memberMissingIDs.add(new PersistentMemberPattern(id));
+            }
+          }
+          // Missing colocated regions
+          Set<PartitionedRegion> prs = gfci.getPartitionedRegions();
+          for (PartitionedRegion pr: prs) {
+            List<String> missingChildRegions = pr.getMissingColocatedChildren();
+            for (String child:missingChildRegions) {
+              missingColocatedRegions.add(new ColocatedRegionDetails(member.getHost(), member.getName(),pr.getFullPath(), child));
+            }
+          }
+        }
+
+      }
+
+      if (memberMissingIDs.isEmpty() && missingColocatedRegions.isEmpty()) {
+        context.getResultSender().lastResult(null);
+      } else {
+        if (!memberMissingIDs.isEmpty()) {
+          if (missingColocatedRegions.isEmpty()) {
+            context.getResultSender().lastResult(memberMissingIDs);
+          } else {
+            context.getResultSender().sendResult(memberMissingIDs);
+          }
+        }
+        if (!missingColocatedRegions.isEmpty()) {
+          context.getResultSender().lastResult(missingColocatedRegions);
+        }
+      }
+    }
+    catch (Exception e) {
+      context.getResultSender().sendException(e);
+    }
+  }
+
+  @Override
+  public String getId() {
+    return getClass().getName();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index 1f0420b..51887cf 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -1433,6 +1433,7 @@ public class CliStrings {
   /* 'show missing-disk-store' command */
   public static final String SHOW_MISSING_DISK_STORE = "show missing-disk-stores";
   public static final String SHOW_MISSING_DISK_STORE__HELP = "Display a summary of the disk stores that are currently missing from a distributed system.";
+  public static final String SHOW_MISSING_DISK_STORE__ERROR_MESSAGE = "An error occurred while showing missing disk stores and missing colocated regions: %1$s";
 
   /* 'shutdown' command */
   public static final String SHUTDOWN = "shutdown";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/ColocatedRegionDetailsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/ColocatedRegionDetailsJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/ColocatedRegionDetailsJUnitTest.java
new file mode 100644
index 0000000..a258bdd
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/ColocatedRegionDetailsJUnitTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.internal.cache.partitioned;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class ColocatedRegionDetailsJUnitTest {
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.internal.cache.partitioned.ColocatedRegionDetails#ColocatedRegionDetails(java.lang.String, java.lang.String, java.lang.String, java.lang.String)}.
+   */
+  @Test
+  public final void testColocatedRegionDetailsConstructor() {
+    ColocatedRegionDetails crd = new ColocatedRegionDetails("host", "member name", "parent region", "child region");
+    assertNotNull(crd);
+    assertEquals("host", crd.getHost());
+    assertEquals("member name", crd.getMember());
+    assertEquals("parent region", crd.getParent());
+    assertEquals("child region", crd.getChild());
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.internal.cache.partitioned.ColocatedRegionDetails#ColocatedRegionDetails()}.
+   */
+  @Test
+  public final void testColocatedRegion0ArgConstructor() {
+    ColocatedRegionDetails crd = new ColocatedRegionDetails();
+    assertNotNull(crd);
+    assertNull(crd.getHost());
+    assertNull(crd.getMember());
+    assertNull(crd.getParent());
+    assertNull(crd.getChild());
+    
+  }
+
+  @Test
+  public final void testContructingWithNulls() {
+    ColocatedRegionDetails crd1 = new ColocatedRegionDetails(null, "member name", "parent region", "child region");
+    ColocatedRegionDetails crd2 = new ColocatedRegionDetails("host", null, "parent region", "child region");
+    ColocatedRegionDetails crd3 = new ColocatedRegionDetails("host", "member name", null, "child region");
+    ColocatedRegionDetails crd4 = new ColocatedRegionDetails("host", "member name", "parent region", null);
+    
+    assertNotNull(crd1);
+    assertNotNull(crd2);
+    assertNotNull(crd3);
+    assertNotNull(crd4);
+  }
+  
+  /**
+   * Test method for {@link org.apache.geode.internal.cache.partitioned.ColocatedRegionDetails#toData(java.io.DataOutput)}.
+   * @throws IOException 
+   * @throws ClassNotFoundException 
+   */
+  @Test
+  public final void testSerialization() throws IOException, ClassNotFoundException {
+    ColocatedRegionDetails crd = new ColocatedRegionDetails("host", "member name", "parent region", "child region");
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutput out = new DataOutputStream(baos);
+    crd.toData(out);
+    ColocatedRegionDetails crdIn = new ColocatedRegionDetails();
+    crdIn.fromData(new DataInputStream(new ByteArrayInputStream(baos.toByteArray())));
+
+    assertEquals(crd,crdIn);
+  }
+
+  @Test
+  public final void testSerializationOfEmptyColocatedRegionDetails() throws IOException, ClassNotFoundException {
+    ColocatedRegionDetails crd = new ColocatedRegionDetails();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutput out = new DataOutputStream(baos);
+    crd.toData(out);
+    ColocatedRegionDetails crdIn = new ColocatedRegionDetails();
+    crdIn.fromData(new DataInputStream(new ByteArrayInputStream(baos.toByteArray())));
+
+    assertEquals(crd,crdIn);
+  }
+  
+  @Test
+  public final void testHostNotEquals() {
+    ColocatedRegionDetails crd1 = new ColocatedRegionDetails();
+    ColocatedRegionDetails crd2 = new ColocatedRegionDetails("host1", "member name", "parent region", "child region");
+    ColocatedRegionDetails crd3 = new ColocatedRegionDetails("host2", "member name", "parent region", "child region");
+    assertNotEquals(crd1, crd2);
+    assertNotEquals(crd2, crd3);
+    assertNotEquals(crd3, crd2);
+  }
+  
+  @Test
+  public final void testMemberNotEquals() {
+    ColocatedRegionDetails crd1 = new ColocatedRegionDetails("host", null, "parent region", "child region");
+    ColocatedRegionDetails crd2 = new ColocatedRegionDetails("host", "member1", "parent region", "child region");
+    ColocatedRegionDetails crd3 = new ColocatedRegionDetails("host", "member2", "parent region", "child region");
+    assertNotEquals(crd1, crd2);
+    assertNotEquals(crd2, crd3);
+    assertNotEquals(crd3, crd2);
+  }
+  
+  @Test
+  public final void testParentNotEquals() {
+    ColocatedRegionDetails crd1 = new ColocatedRegionDetails("host", "member1", null, "child region");
+    ColocatedRegionDetails crd2 = new ColocatedRegionDetails("host", "member1", "parent1", "child region");
+    ColocatedRegionDetails crd3 = new ColocatedRegionDetails("host", "member1", "parent2", "child region");
+    assertNotEquals(crd1, crd2);
+    assertNotEquals(crd2, crd3);
+    assertNotEquals(crd3, crd2);
+  }
+  
+  @Test
+  public final void testChildNotEquals() {
+    ColocatedRegionDetails crd1 = new ColocatedRegionDetails("host", "member1", "parent region", null);
+    ColocatedRegionDetails crd2 = new ColocatedRegionDetails("host", "member1", "parent region", "child1");
+    ColocatedRegionDetails crd3 = new ColocatedRegionDetails("host", "member1", "parent region", "child2");
+    assertNotEquals(crd1, crd2);
+    assertNotEquals(crd2, crd3);
+    assertNotEquals(crd3, crd2);
+  }
+  
+  @Test
+  public final void testClassInequality() {
+    ColocatedRegionDetails crd1 = new ColocatedRegionDetails("host", "member1", "parent region", null);
+    String crd2 = crd1.toString();
+    assertNotEquals(crd1, crd2);
+    assertNotEquals(crd2, crd1);
+  }
+  
+  @Test
+  public final void nullColocatedRegionDetailsEqualsTests() {
+    ColocatedRegionDetails crd1 = null;
+    ColocatedRegionDetails crd2 = new ColocatedRegionDetails("host", "member1", "parent region", "child1");
+    assertEquals(crd1, crd1);
+    assertEquals(crd2, crd2);
+    assertNotEquals(crd1, crd2);
+    assertNotEquals(crd2, crd1);
+  }
+  
+  @Test
+  public final void testToString() {
+    ColocatedRegionDetails crd = new ColocatedRegionDetails("host1", "member name", "parent region", "child region");
+    assertEquals("[host:host1, member:member name, parent:parent region, child:child region]", crd.toString()); 
+  }
+
+  @Test
+  public final void testToStringOfEmptyColocatedRegionDetails() {
+    ColocatedRegionDetails crd = new ColocatedRegionDetails();
+    assertEquals("[,,,]", crd.toString()); 
+  }
+
+  @Test
+  public final void testHashCode() {
+    ColocatedRegionDetails crd1 = new ColocatedRegionDetails();
+    ColocatedRegionDetails crd2 = new ColocatedRegionDetails("host1", "member name", "parent region", "child region");
+    ColocatedRegionDetails crd3 = new ColocatedRegionDetails("host2", "member name", "parent region", "child region");
+    
+    assertNotEquals(crd1.hashCode(), crd2.hashCode());
+    assertNotEquals(crd1.hashCode(), crd3.hashCode());
+    assertNotEquals(crd2.hashCode(), crd3.hashCode());
+
+    assertEquals(923521, crd1.hashCode());    
+    assertEquals(2077348855, crd2.hashCode());   
+    assertEquals(2077378646, crd3.hashCode());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
index 9926ae6..fdf8e73 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
@@ -30,6 +30,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.StringTokenizer;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -41,6 +42,8 @@ import org.apache.geode.cache.DiskStore;
 import org.apache.geode.cache.DiskStoreFactory;
 import org.apache.geode.cache.EvictionAction;
 import org.apache.geode.cache.EvictionAttributes;
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
@@ -56,6 +59,7 @@ import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.FileUtil;
 import org.apache.geode.internal.cache.DiskStoreImpl;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.SnapshotTestUtil;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.management.cli.Result;
@@ -63,6 +67,7 @@ import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.cli.result.CommandResult;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.dunit.SerializableRunnable;
@@ -208,12 +213,12 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
 
     String stringResult = commandResultToString(cmdResult);
     System.out.println("command result=" + stringResult);
-    assertEquals(3, countLinesInString(stringResult, false));
+    assertEquals(5, countLinesInString(stringResult, false));
     assertTrue(stringContainsLine(stringResult, "Disk Store ID.*Host.*Directory"));
     assertTrue(stringContainsLine(stringResult, ".*" + diskStoreName + vm1.getPid()));
 
     // Extract the id from the returned missing disk store
-    String line = getLineFromString(stringResult, 3);
+    String line = getLineFromString(stringResult, 4);
     assertFalse(line.contains("---------"));
     StringTokenizer resultTokenizer = new StringTokenizer(line);
     String id = resultTokenizer.nextToken();
@@ -241,6 +246,240 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
   }
 
   @Test
+  public void testMissingDiskStoreCommandWithColocation() {
+    final String regionName = "testShowPersistentRecoveryFailuresRegion";
+    final String childName = "childRegion";
+
+    setUpJmxManagerOnVm0ThenConnect(null);
+
+    final VM vm0 = Host.getHost(0).getVM(0);
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final String vm1Name = "VM" + vm1.getPid();
+    final String diskStoreName = "DiskStoreCommandsDUnitTest";
+
+    // Default setup creates a cache in the Manager, now create a cache in VM1
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Properties localProps = new Properties();
+        localProps.setProperty(NAME, vm1Name);
+        getSystem(localProps);
+        Cache cache = getCache();
+      }
+    });
+
+    // Create a disk store and region in the Manager (VM0) and VM1 VMs
+    for (final VM vm : (new VM[]{vm0, vm1})) {
+      final String vmName = "VM" + vm.getPid();
+      vm.invoke(new SerializableRunnable() {
+        public void run() {
+          Cache cache = getCache();
+
+          File diskStoreDirFile = new File(diskStoreName + vm.getPid());
+          diskStoreDirFile.mkdirs();
+
+          DiskStoreFactory diskStoreFactory = cache.createDiskStoreFactory();
+          diskStoreFactory.setDiskDirs(new File[]{diskStoreDirFile});
+          diskStoreFactory.setMaxOplogSize(1);
+          diskStoreFactory.setAllowForceCompaction(true);
+          diskStoreFactory.setAutoCompact(false);
+          diskStoreFactory.create(regionName);
+          diskStoreFactory.create(childName);
+
+          RegionFactory regionFactory = cache.createRegionFactory();
+          regionFactory.setDiskStoreName(regionName);
+          regionFactory.setDiskSynchronous(true);
+          regionFactory.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
+          regionFactory.create(regionName);
+
+          PartitionAttributes pa = new PartitionAttributesFactory().setColocatedWith(regionName).create();
+          RegionFactory childRegionFactory = cache.createRegionFactory();
+          childRegionFactory.setPartitionAttributes(pa);
+          childRegionFactory.setDiskStoreName(childName);
+          childRegionFactory.setDiskSynchronous(true);
+          childRegionFactory.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
+          childRegionFactory.create(childName);
+        }
+      });
+    }
+
+    // Add data to the region
+    vm0.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(regionName);
+        region.put("A", "a");
+        region.put("B", "b");
+      }
+    });
+
+    // Make sure that everything thus far is okay and there are no missing disk stores
+    CommandResult cmdResult = executeCommand(CliStrings.SHOW_MISSING_DISK_STORE);
+    System.out.println("command result=\n" + commandResultToString(cmdResult));
+
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+    assertTrue(cmdResult.toString(), commandResultToString(cmdResult).contains("No missing disk store found"));
+
+    // Close the regions in the Manager (VM0) VM
+    vm0.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(childName);
+        region.close();
+        region = cache.getRegion(regionName);
+        region.close();
+      }
+    });
+
+    // Add data to VM1 and then close the region
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+        Cache cache = getCache();
+        Region childRegion = cache.getRegion(childName);
+        PartitionedRegion parentRegion = (PartitionedRegion)(cache.getRegion(regionName));
+        try {
+          parentRegion.put("A", "C");
+        } catch (Exception e) {
+          //Ignore any exception on the put
+        }
+        childRegion.close();
+        parentRegion.close();
+      }
+    });
+
+    SerializableRunnable restartParentRegion = new SerializableRunnable("Restart parent region on") {
+      public void run() {
+        Cache cache = getCache();
+
+        RegionFactory regionFactory = cache.createRegionFactory();
+        regionFactory.setDiskStoreName(regionName);
+        regionFactory.setDiskSynchronous(true);
+        regionFactory.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
+        try {
+          regionFactory.create(regionName);
+        } catch (Exception e) {
+          // okay to ignore
+        }
+      }
+    };
+
+    SerializableRunnable restartChildRegion = new SerializableRunnable("Restart child region") {
+      public void run() {
+        Cache cache = getCache();
+
+        PartitionAttributes pa = new PartitionAttributesFactory().setColocatedWith(regionName).create();
+        RegionFactory regionFactory = cache.createRegionFactory();
+        regionFactory.setPartitionAttributes(pa);
+        regionFactory.setDiskStoreName(childName);
+        regionFactory.setDiskSynchronous(true);
+        regionFactory.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
+        try {
+          regionFactory.create(childName);
+        } catch (Exception e) {
+          // okay to ignore
+          e.printStackTrace();
+        }
+      }
+    };
+
+    // Add the region back to the Manager (VM0) VM
+    AsyncInvocation async0 = vm0.invokeAsync(restartParentRegion);
+    AsyncInvocation async1 = vm1.invokeAsync(restartParentRegion);
+
+    // Wait for the region in the Manager (VM0) to come online
+    vm0.invoke(new SerializableRunnable("WaitForRegionInVm0") {
+      public void run() {
+        WaitCriterion waitCriterion = new WaitCriterion() {
+          public boolean done() {
+            Cache cache = getCache();
+            PersistentMemberManager memberManager = ((GemFireCacheImpl) cache).getPersistentMemberManager();
+            return !memberManager.getWaitingRegions().isEmpty();
+          }
+
+          public String description() {
+            return "Waiting for another persistent member to come online";
+          }
+        };
+        try {
+          waitForCriterion(waitCriterion, 5000, 100, true);
+        } catch (AssertionError ae) {
+          // Ignore. waitForCriterion is expected to timeout in this test
+        }
+      }
+    });
+
+    // Validate that there is a missing disk store on VM1
+    try {
+      cmdResult = executeCommand(CliStrings.SHOW_MISSING_DISK_STORE);
+      assertNotNull("Expect command result != null", cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      String stringResult = commandResultToString(cmdResult);
+      System.out.println("command result=\n" + stringResult);
+      // Expect 2 result sections with header lines and 4 information lines in the first section
+      assertEquals(6, countLinesInString(stringResult, false));
+      assertTrue(stringContainsLine(stringResult, "Host.*Distributed Member.*Parent Region.*Missing Colocated Region"));
+      assertTrue(stringContainsLine(stringResult, ".*" + regionName + ".*" + childName));
+
+      AsyncInvocation async0b = vm0.invokeAsync(restartChildRegion);
+      try {
+        async0b.get(5000, TimeUnit.MILLISECONDS);
+      } catch (Exception e) {
+        // Expected timeout - Region recovery is still waiting on vm1 child region and disk-store to come online
+      }
+
+      cmdResult = executeCommand(CliStrings.SHOW_MISSING_DISK_STORE);
+      assertNotNull("Expect command result != null", cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      stringResult = commandResultToString(cmdResult);
+      System.out.println("command result=\n" + stringResult);
+
+      // Extract the id from the returned missing disk store
+      String line = getLineFromString(stringResult, 4);
+      assertFalse(line.contains("---------"));
+      StringTokenizer resultTokenizer = new StringTokenizer(line);
+      String id = resultTokenizer.nextToken();
+
+      AsyncInvocation async1b = vm1.invokeAsync(restartChildRegion);
+      try {
+        async1b.get(5000, TimeUnit.MILLISECONDS);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+      cmdResult = executeCommand(CliStrings.SHOW_MISSING_DISK_STORE);
+      assertNotNull("Expect command result != null", cmdResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+      stringResult = commandResultToString(cmdResult);
+      System.out.println("command result=\n" + stringResult);
+
+    } finally {
+      // Verify that the invokeAsync thread terminated
+      try {
+        async0.get(10000, TimeUnit.MILLISECONDS);
+        async1.get(10000, TimeUnit.MILLISECONDS);
+      } catch (Exception e) {
+        fail("Unexpected timeout waitiong for invokeAsync threads to terminate: " + e.getMessage());
+      }
+    }
+
+    // Do our own cleanup so that the disk store directories can be removed
+    super.destroyDefaultSetup();
+    for (final VM vm : (new VM[]{vm0, vm1})) {
+      final String vmName = "VM" + vm.getPid();
+      vm.invoke(new SerializableRunnable() {
+        public void run() {
+          try {
+            FileUtil.delete((new File(diskStoreName + vm.getPid())));
+          } catch (IOException iex) {
+            // There's nothing else we can do
+          }
+        }
+      });
+    }
+  }
+
+  @Test
   public void testDescribeOfflineDiskStore() {
     setUpJmxManagerOnVm0ThenConnect(null);
 
@@ -806,7 +1045,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
     File incrementalBackUpDir = new File(incrementalBackUpName);
     incrementalBackUpDir.mkdir();
 
-    //Perform an incremental backup 
+    //Perform an incremental backup
     final String incrementalBackUpDirPath = incrementalBackUpDir.getCanonicalPath();
     filesToBeDeleted.add(incrementalBackUpDirPath);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunctionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunctionJUnitTest.java
new file mode 100644
index 0000000..ce0761f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunctionJUnitTest.java
@@ -0,0 +1,438 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.geode.management.internal.cli.functions;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.logging.log4j.core.Appender;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+import org.mockito.ArgumentCaptor;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.execute.ResultSender;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.cache.DistributedRegion;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionRegionConfig;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.execute.FunctionContextImpl;
+import org.apache.geode.internal.cache.partitioned.ColocatedRegionDetails;
+import org.apache.geode.internal.cache.persistence.DiskStoreID;
+import org.apache.geode.internal.cache.persistence.PersistentMemberID;
+import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
+import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
+import org.apache.geode.test.fake.Fakes;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class ShowMissingDiskStoresFunctionJUnitTest {
+  private GemFireCacheImpl cache;
+  private GemFireCacheImpl oldCacheInstance;
+  private InternalDistributedSystem system;
+  private PartitionedRegion pr1;
+  private PartitionedRegion pr2;
+  private DistributedRegion prRoot;
+  private PartitionAttributes pa;
+  private PartitionRegionConfig prc;
+  private Logger logger;
+  private Appender mockAppender;
+  private ArgumentCaptor<LogEvent> loggingEventCaptor;
+  private FunctionContext context;
+  private TestResultSender resultSender;
+  private PersistentMemberManager memberManager;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Before
+  public void setUp() throws Exception {
+    cache = Fakes.cache();
+    system = (InternalDistributedSystem) cache.getDistributedSystem();
+    pr1 = mock(PartitionedRegion.class);
+    pr2 = mock(PartitionedRegion.class);
+    prRoot = mock(DistributedRegion.class);
+    pa = mock(PartitionAttributes.class);
+    prc = mock(PartitionRegionConfig.class);
+    cache = Fakes.cache();
+    oldCacheInstance = GemFireCacheImpl.setInstanceForTests(cache);
+    resultSender = new TestResultSender();
+    context = new FunctionContextImpl("testFunction", null, resultSender);
+    memberManager = mock(PersistentMemberManager.class);
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @After
+  public void tearDown() throws Exception {
+    GemFireCacheImpl.setInstanceForTests(oldCacheInstance);
+  }
+
+  private class TestSPRFFunc1 extends ShowMissingDiskStoresFunction {
+    @Override
+    protected Cache getCache() {
+      return null;
+    }
+  }
+
+  private class TestSPRFFunc2 extends ShowMissingDiskStoresFunction {
+    @Override
+    protected Cache getCache() {
+      return cache;
+    }
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#getCache()}.
+   */
+  @Test
+  public final void testGetCache() {
+    ShowMissingDiskStoresFunction rff = new ShowMissingDiskStoresFunction();
+    assertTrue(rff.getCache() instanceof Cache);
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#execute(org.apache.geode.cache.execute.FunctionContext)}.
+   */
+  @Test
+  public final void testExecute() {
+    ShowMissingDiskStoresFunction rff = new ShowMissingDiskStoresFunction();
+    List<?> results = null;
+
+    when(cache.getPersistentMemberManager()).thenReturn(memberManager);
+
+    rff.execute(context);
+    try {
+      results = resultSender.getResults();
+    } catch (Throwable e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    }
+    assertNotNull(results);
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#execute(org.apache.geode.cache.execute.FunctionContext)}.
+   */
+  @Test
+  public final void testExecuteWithNullContextThrowsRuntimeException() {
+    expectedException.expect(RuntimeException.class);
+
+    ShowMissingDiskStoresFunction rff = new ShowMissingDiskStoresFunction();
+    rff.execute(null);
+    fail("Missing expected RuntimeException");
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#execute(org.apache.geode.cache.execute.FunctionContext)}.
+   */
+  @Test
+  public final void testExecuteWithNullCacheInstanceHasEmptyResults() {
+    TestSPRFFunc1 rff = new TestSPRFFunc1();
+    List<?> results = null;
+
+    rff.execute(context);
+    try {
+      results = resultSender.getResults();
+    } catch (Throwable e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    }
+    assertNotNull(results);
+    assertEquals(1,results.size());
+    assertNull(results.get(0));
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#execute(org.apache.geode.cache.execute.FunctionContext)}.
+   */
+  @Test
+  public final void testExecuteWithNullGFCIResultValueIsNull() {
+    TestSPRFFunc2 rff = new TestSPRFFunc2();
+    List<?> results = null;
+
+    when(cache.getPersistentMemberManager()).thenReturn(memberManager);
+    GemFireCacheImpl.setInstanceForTests(null);
+
+    rff.execute(context);
+    try {
+      results = resultSender.getResults();
+    } catch (Throwable e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    }
+    assertNotNull(results);
+    assertEquals(1,results.size());
+    assertNull(results.get(0));
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#execute(org.apache.geode.cache.execute.FunctionContext)}.
+   */
+  @Test
+  public final void testExecuteWhenGFCIClosedResultValueIsNull() {
+    TestSPRFFunc2 rff = new TestSPRFFunc2();
+    List<?> results = null;
+
+    when(cache.getPersistentMemberManager()).thenReturn(memberManager);
+    when(((GemFireCacheImpl)cache).isClosed()).thenReturn(true);
+    rff.execute(context);
+    try {
+      results = resultSender.getResults();
+    } catch (Throwable e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    }
+    assertNotNull(results);
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#execute(org.apache.geode.cache.execute.FunctionContext)}.
+   * @throws UnknownHostException 
+   */
+  @Test
+  public final void testExecuteReturnsMissingDiskStores() throws UnknownHostException {
+    ShowMissingDiskStoresFunction rff = new ShowMissingDiskStoresFunction();
+    List<?> results = null;
+
+    when(cache.getPersistentMemberManager()).thenReturn(memberManager);
+
+    // Fake missing disk-stores
+    Set<PersistentMemberID> regions1 = new HashSet<PersistentMemberID>();
+    regions1.add(new PersistentMemberID(new DiskStoreID(), InetAddress.getLocalHost(), "/diskStore1", 1L, (short)1));
+    regions1.add(new PersistentMemberID(new DiskStoreID(), InetAddress.getLocalHost(), "/diskStore2", 2L, (short)2));
+    Map<String,Set<PersistentMemberID>> mapMember1 = new HashMap<String,Set<PersistentMemberID>>();;
+    mapMember1.put("member1", regions1);
+    when(memberManager.getWaitingRegions()).thenReturn(mapMember1);
+
+    rff.execute(context);
+    try {
+      results = resultSender.getResults();
+    } catch (Throwable e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    }
+    assertNotNull(results);
+    assertEquals(1,results.size());
+    Set<?> detailSet = (Set<?>)results.get(0);
+    assertEquals(2, detailSet.toArray().length);
+    assertTrue(detailSet.toArray()[0] instanceof PersistentMemberPattern);
+    assertTrue(detailSet.toArray()[1] instanceof PersistentMemberPattern);
+    // Results are not sorted so verify results in either order
+    if (((PersistentMemberPattern)detailSet.toArray()[0]).getDirectory().equals("/diskStore1")) {
+      assertEquals("/diskStore2", ((PersistentMemberPattern)detailSet.toArray()[1]).getDirectory());
+    } else if (((PersistentMemberPattern)detailSet.toArray()[0]).getDirectory().equals("/diskStore2")) {
+      assertEquals("/diskStore1", ((PersistentMemberPattern)detailSet.toArray()[1]).getDirectory());
+    } else {
+      fail("Incorrect missing colocated region results");
+    }
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#execute(org.apache.geode.cache.execute.FunctionContext)}.
+   */
+  @Test
+  public final void testExecuteReturnsMissingColocatedRegions() {
+    ShowMissingDiskStoresFunction rff = new ShowMissingDiskStoresFunction();
+    List<?> results = null;
+
+    when(cache.getPersistentMemberManager()).thenReturn(memberManager);
+
+    // Fake missing colocated regions
+    Set<PartitionedRegion> prs = new HashSet<PartitionedRegion>();
+    prs.add(pr1);
+    prs.add(pr2);
+    List<String> missing1 = new ArrayList<String>(Arrays.asList("child1", "child2"));
+    when(cache.getPartitionedRegions()).thenReturn(prs);
+    when(pr1.getMissingColocatedChildren()).thenReturn(missing1);
+    when(pr1.getFullPath()).thenReturn("/pr1");
+
+    rff.execute(context);
+    try {
+      results = resultSender.getResults();
+    } catch (Throwable e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    }
+    assertEquals(1,results.size());
+    Set<?> detailSet = (Set<?>)results.get(0);
+    assertEquals(2, detailSet.toArray().length);
+    assertTrue(detailSet.toArray()[0] instanceof ColocatedRegionDetails);
+    assertTrue(detailSet.toArray()[1] instanceof ColocatedRegionDetails);
+    assertEquals("/pr1", ((ColocatedRegionDetails)detailSet.toArray()[0]).getParent());
+    assertEquals("/pr1", ((ColocatedRegionDetails)detailSet.toArray()[1]).getParent());
+    // Results are not sorted so verify results in either order
+    if (((ColocatedRegionDetails)detailSet.toArray()[0]).getChild().equals("child1")) {
+      assertEquals("child2", ((ColocatedRegionDetails)detailSet.toArray()[1]).getChild());
+    } else if (((ColocatedRegionDetails)detailSet.toArray()[0]).getChild().equals("child2")) {
+      assertEquals("child1", ((ColocatedRegionDetails)detailSet.toArray()[1]).getChild());
+    } else {
+      fail("Incorrect missing colocated region results");
+    }
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#execute(org.apache.geode.cache.execute.FunctionContext)}.
+   * @throws UnknownHostException 
+   */
+  @Test
+  public final void testExecuteReturnsMissingStoresAndRegions() throws UnknownHostException {
+    ShowMissingDiskStoresFunction rff = new ShowMissingDiskStoresFunction();
+    List<?> results = null;
+
+    when(cache.getPersistentMemberManager()).thenReturn(memberManager);
+
+    // Fake missing disk-stores
+    Set<PersistentMemberID> regions1 = new HashSet<PersistentMemberID>();
+    regions1.add(new PersistentMemberID(new DiskStoreID(), InetAddress.getLocalHost(), "/diskStore1", 1L, (short)1));
+    regions1.add(new PersistentMemberID(new DiskStoreID(), InetAddress.getLocalHost(), "/diskStore2", 2L, (short)2));
+    Map<String,Set<PersistentMemberID>> mapMember1 = new HashMap<String,Set<PersistentMemberID>>();;
+    mapMember1.put("member1", regions1);
+    when(memberManager.getWaitingRegions()).thenReturn(mapMember1);
+
+    // Fake missing colocated regions
+    Set<PartitionedRegion> prs = new HashSet<PartitionedRegion>();
+    prs.add(pr1);
+    prs.add(pr2);
+    List<String> missing1 = new ArrayList<String>(Arrays.asList("child1", "child2"));
+    when(cache.getPartitionedRegions()).thenReturn(prs);
+    when(pr1.getMissingColocatedChildren()).thenReturn(missing1);
+    when(pr1.getFullPath()).thenReturn("/pr1");
+
+    rff.execute(context);
+    try {
+      results = resultSender.getResults();
+    } catch (Throwable e) {
+      e.printStackTrace();
+      fail("Unexpected exception");
+    }
+    assertEquals(2,results.size());
+    for (Object result:results) {
+      Set<?> detailSet = (Set<?>)result;
+      if (detailSet.toArray()[0] instanceof PersistentMemberPattern) {
+        assertEquals(2, detailSet.toArray().length);
+        assertTrue(detailSet.toArray()[1] instanceof PersistentMemberPattern);
+        // Results are not sorted so verify results in either order
+        if (((PersistentMemberPattern)detailSet.toArray()[0]).getDirectory().equals("/diskStore1")) {
+          assertEquals("/diskStore2", ((PersistentMemberPattern)detailSet.toArray()[1]).getDirectory());
+        } else if (((PersistentMemberPattern)detailSet.toArray()[0]).getDirectory().equals("/diskStore2")) {
+          assertEquals("/diskStore1", ((PersistentMemberPattern)detailSet.toArray()[1]).getDirectory());
+        } else {
+          fail("Incorrect missing colocated region results");
+        }
+      } else if (detailSet.toArray()[0] instanceof ColocatedRegionDetails) {
+        assertEquals(2, detailSet.toArray().length);
+        assertTrue(detailSet.toArray()[1] instanceof ColocatedRegionDetails);
+        assertEquals("/pr1", ((ColocatedRegionDetails)detailSet.toArray()[0]).getParent());
+        assertEquals("/pr1", ((ColocatedRegionDetails)detailSet.toArray()[1]).getParent());
+        // Results are not sorted so verify results in either order
+        if (((ColocatedRegionDetails)detailSet.toArray()[0]).getChild().equals("child1")) {
+          assertEquals("child2", ((ColocatedRegionDetails)detailSet.toArray()[1]).getChild());
+        } else if (((ColocatedRegionDetails)detailSet.toArray()[0]).getChild().equals("child2")) {
+          assertEquals("child1", ((ColocatedRegionDetails)detailSet.toArray()[1]).getChild());
+        } else {
+          fail("Incorrect missing colocated region results");
+        }
+      } else {
+        fail("Unexpected result type: " + detailSet.toArray()[0].getClass());
+      }
+    }
+  }
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#execute(org.apache.geode.cache.execute.FunctionContext)}.
+   * @throws Throwable 
+   */
+  @Test
+  public final void testExecuteCatchesExceptions() throws Throwable {
+    expectedException.expect(RuntimeException.class);
+
+    ShowMissingDiskStoresFunction rff = new ShowMissingDiskStoresFunction();
+
+    when(cache.getPersistentMemberManager()).thenThrow(new RuntimeException());
+
+    rff.execute(context);
+    List<?> results = resultSender.getResults();
+    fail("Failed to catch expected RuntimeException");
+  }
+
+
+  /**
+   * Test method for {@link org.apache.geode.management.internal.cli.functions.ShowMissingDiskStoresFunction#getId()}.
+   */
+  @Test
+  public final void testGetId() {
+    ShowMissingDiskStoresFunction rff = new ShowMissingDiskStoresFunction();
+    assertEquals(ShowMissingDiskStoresFunction.class.getName(), rff.getId());
+  }
+
+  private static class TestResultSender implements ResultSender {
+
+    private final List<Object> results = new LinkedList<Object>();
+
+    private Throwable t;
+
+    protected List<Object> getResults() throws Throwable {
+      if (t != null) {
+        throw t;
+      }
+      return Collections.unmodifiableList(results);
+    }
+
+    @Override
+    public void lastResult(final Object lastResult) {
+      results.add(lastResult);
+    }
+
+    @Override
+    public void sendResult(final Object oneResult) {
+      results.add(oneResult);
+    }
+
+    @Override
+    public void sendException(final Throwable t) {
+      this.t = t;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
index 2240daf..ecb6529 100644
--- a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
+++ b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedDataSerializables.txt
@@ -1527,6 +1527,10 @@ org/apache/geode/internal/cache/partitioned/BucketSizeMessage$BucketSizeReplyMes
 fromData,26,2a2bb7001b2a2bb9001c0100b500022a2bb9001d0100b50003b1
 toData,26,2a2bb7001e2b2ab40002b9001f02002b2ab40003b900200300b1
 
+org/apache/geode/internal/cache/partitioned/ColocatedRegionDetails,2
+fromData,81,2bb9000601003d1c99000b2a2bb80007b500022bb9000601003e1d99000b2a2bb80007b500032bb9000601003604150499000b2a2bb80007b500042bb9000601003605150599000b2a2bb80007b50005b1
+toData,133,2b2ab40002c6000704a7000403b9000802002ab40002c6000b2ab400022bb800092b2ab40003c6000704a7000403b9000802002ab40003c6000b2ab400032bb800092b2ab40004c6000704a7000403b9000802002ab40004c6000b2ab400042bb800092b2ab40005c6000704a7000403b9000802002ab40005c6000b2ab400052bb80009b1
+
 org/apache/geode/internal/cache/partitioned/ContainsKeyValueMessage,2
 fromData,37,2a2bb700352a2bb80036b500052a2bb900370100b500042a2bb900380100b80039b50006b1
 toData,37,2a2bb7003a2ab400052bb8003b2b2ab40004b9003c02002b2ab40006b6003db9003e0200b1

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09cd4ad2/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
index 58b4395..8ca0b5b 100644
--- a/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
+++ b/geode-core/src/test/resources/org/apache/geode/codeAnalysis/sanctionedSerializables.txt
@@ -605,6 +605,7 @@ org/apache/geode/management/internal/cli/functions/RegionFunctionArgs,true,-5158
 org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$ExpirationAttrs,true,1474255033398008062,action:org/apache/geode/cache/ExpirationAction,time:java/lang/Integer,type:org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$ExpirationAttrs$ExpirationFor
 org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$ExpirationAttrs$ExpirationFor,false
 org/apache/geode/management/internal/cli/functions/RegionFunctionArgs$PartitionArgs,true,5907052187323280919,hasPartitionAttributes:boolean,isSetPRLocalMaxMemory:boolean,isSetPRRecoveryDelay:boolean,isSetPRRedundantCopies:boolean,isSetPRStartupRecoveryDelay:boolean,isSetPRTotalMaxMemory:boolean,isSetPRTotalNumBuckets:boolean,prColocatedWith:java/lang/String,prLocalMaxMemory:int,prRecoveryDelay:long,prRedundantCopies:int,prStartupRecoveryDelay:long,prTotalMaxMemory:long,prTotalNumBuckets:int,userSpecifiedPartitionAttributes:java/util/Set
+org/apache/geode/management/internal/cli/functions/ShowMissingDiskStoresFunction,false
 org/apache/geode/management/internal/cli/functions/ShutDownFunction,true,1
 org/apache/geode/management/internal/cli/functions/UndeployFunction,true,1
 org/apache/geode/management/internal/cli/functions/UnregisterFunction,true,1