You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by je...@apache.org on 2016/03/21 22:33:06 UTC

[2/3] incubator-geode git commit: GEODE-980: Refactor destroyRegion command

GEODE-980: Refactor destroyRegion command

- Remove a bunch of logic separating local regions vs. other types. For
  distributed regions we just need to find one member ('cos the destroy
  region call gets distributed) and for local regions we choose each
  member hosting the named region Then do a function call against all
  these members.


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

Branch: refs/heads/develop
Commit: ae7d4da9c992db0810a9c37baa5d5a63db1ff75e
Parents: 7e8da34
Author: Jens Deppe <jd...@pivotal.io>
Authored: Thu Mar 17 07:24:48 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Mar 21 14:32:42 2016 -0700

----------------------------------------------------------------------
 .../CreateAlterDestroyRegionCommands.java       | 236 ++++++++-----------
 ...eateAlterDestroyRegionCommandsDUnitTest.java | 221 +++++++++--------
 2 files changed, 213 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ae7d4da9/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
index bf1fdf0..29de9ee 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
@@ -16,28 +16,6 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
-import java.lang.reflect.Array;
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.regex.Pattern;
-
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-
-import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -54,7 +32,6 @@ import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.ClassPathLoader;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.xmlcache.CacheXml;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.management.DistributedRegionMXBean;
@@ -83,6 +60,25 @@ import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.management.internal.cli.util.RegionPath;
 import com.gemstone.gemfire.management.internal.configuration.SharedConfigurationWriter;
 import com.gemstone.gemfire.management.internal.configuration.domain.XmlEntity;
+import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.regex.Pattern;
 
 /**
  * 
@@ -1001,118 +997,75 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           mandatory = true,
           help = CliStrings.DESTROY_REGION__REGION__HELP)
       String regionPath) {
+
+    if (regionPath == null) {
+      return ResultBuilder.createInfoResult(CliStrings.DESTROY_REGION__MSG__SPECIFY_REGIONPATH_TO_DESTROY);
+    }
+
+    if (regionPath.trim().isEmpty() || regionPath.equals(Region.SEPARATOR)) {
+      return ResultBuilder.createInfoResult(
+          CliStrings.format(CliStrings.DESTROY_REGION__MSG__REGIONPATH_0_NOT_VALID, new Object[]{regionPath}));
+    }
+
     Result result = null;
     XmlEntity xmlEntity = null;
     try {
-      if (regionPath != null) {
-        if (regionPath.trim().isEmpty()) { // unlikely through gfsh
-          result = ResultBuilder.createInfoResult(CliStrings.format(CliStrings.DESTROY_REGION__MSG__REGIONPATH_0_NOT_VALID,
-              new Object[] { regionPath }));
-        } else if (regionPath.equals(Region.SEPARATOR)) {
-          result = ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.DESTROY_REGION__MSG__REGIONPATH_0_NOT_VALID,
-              new Object[] { regionPath }));
-        } else {
-          boolean isRegionDestroyed = false;
-          String message = "";
-          Cache cache = CacheFactory.getAnyInstance();
-          ManagementService managementService = ManagementService.getExistingManagementService(cache);
-          String regionPathToUse = regionPath;
-          String regionName = regionPath.startsWith(Region.SEPARATOR) ? regionPath.substring(1) : regionPath;
-          
-          if (!regionPathToUse.startsWith(Region.SEPARATOR)) {
-            regionPathToUse = Region.SEPARATOR + regionPathToUse;
-          }
-          Set<DistributedMember>[] regionMembersList = findMembersForRegion(cache, managementService, regionPathToUse);
-          if (regionMembersList != null) {
-            Set<DistributedMember> regionMembers      = regionMembersList[0];
-            Set<DistributedMember> localRegionMembers = regionMembersList[1];
-            
-            if (regionMembers != null && !regionMembers.isEmpty()) { // Distributed Region with this region path exists
-              Region<?, ?> regionInThisCache = cache.getRegion(regionPath);
-              if (regionInThisCache != null) {// exists in local cache?
-                regionInThisCache.destroyRegion(); // this is for a Distributed Region & destroy event will be propagated
-                message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__REGION_0_1_DESTROYED, new Object[] { regionPath, /*subRegionMessage*/"" });
-                xmlEntity = new XmlEntity(CacheXml.REGION, "name", regionName);
-                isRegionDestroyed = true;
-              } else {
-                CliFunctionResult destroyRegionResult = null;
-                Iterator<DistributedMember> it = regionMembers.iterator();
-                DistributedMember firstMember = it.next(); // regionMembers is not empty
-
-                ResultCollector<?, ?> resultCollector = CliUtil.executeFunction(RegionDestroyFunction.INSTANCE, regionPath, firstMember);
-                List<CliFunctionResult> destroyRegionResults = (List<CliFunctionResult>) resultCollector.getResult();
-
-                destroyRegionResult = destroyRegionResults.get(0);
-
-                if (destroyRegionResult.isSuccessful()) {
-                  message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__REGION_0_1_DESTROYED, new Object[] { regionPath, /*subRegionMessage*/"" });
-                  xmlEntity = destroyRegionResult.getXmlEntity();
-                  isRegionDestroyed = true;
-                } else if (destroyRegionResult.getThrowable() != null) {
-                  Throwable t = destroyRegionResult.getThrowable();
-                  LogWrapper.getInstance().info(t.getMessage(), t);
-                  message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__ERROR_OCCURRED_WHILE_DESTROYING_0_REASON_1,
-                      new Object[] { regionPath, t.getMessage() });
-                  isRegionDestroyed = false;
-                } else {
-                  message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__UNKNOWN_RESULT_WHILE_DESTROYING_REGION_0_REASON_1,
-                      new Object[] { regionPath, destroyRegionResult.getMessage() });
-                  isRegionDestroyed = false;
-                }
-              }
-            }
+      String message = "";
+      Cache cache = CacheFactory.getAnyInstance();
+      ManagementService managementService = ManagementService.getExistingManagementService(cache);
+      String regionPathToUse = regionPath;
 
-            if (localRegionMembers != null && !localRegionMembers.isEmpty()) { // Local Regions with this region path also exist
-              Region<?, ?> regionInThisCache = cache.getRegion(regionPath);
-              if (regionInThisCache != null) {// exists in local cache?
-                regionInThisCache.destroyRegion();
-                message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__REGION_0_1_DESTROYED, new Object[] { regionPath, /*subRegionMessage*/"" });
-                xmlEntity = new XmlEntity(CacheXml.REGION, "name", regionName);
-                isRegionDestroyed = true;
-              }
-              DistributedMember distributedMember = cache.getDistributedSystem().getDistributedMember();
-              localRegionMembers.remove(distributedMember); //remove this member from the found members
-              
-              CliFunctionResult destroyRegionResult = null;
-
-              ResultCollector<?, ?> resultCollector = CliUtil.executeFunction(RegionDestroyFunction.INSTANCE, regionPath, localRegionMembers);
-              List<CliFunctionResult> resultsList = (List<CliFunctionResult>) resultCollector.getResult();
-              for (int i = 0; i < resultsList.size(); i++) {
-                destroyRegionResult = resultsList.get(i) ;
-                if (destroyRegionResult.isSuccessful()) {
-                  message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__REGION_0_1_DESTROYED, new Object[] { regionPath, /*subRegionMessage*/"" });
-                  xmlEntity = destroyRegionResult.getXmlEntity();
-                  isRegionDestroyed = true;
-                } else if (destroyRegionResult.getThrowable() != null) {
-                  Throwable t = destroyRegionResult.getThrowable();
-                  LogWrapper.getInstance().info(t.getMessage(), t);
-                  message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__ERROR_OCCURRED_WHILE_DESTROYING_0_REASON_1,
-                      new Object[] { regionPath, t.getMessage() });
-                  isRegionDestroyed = false;
-                } else {
-                  message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__UNKNOWN_RESULT_WHILE_DESTROYING_REGION_0_REASON_1,
-                      new Object[] { regionPath, destroyRegionResult.getMessage() });
-                  isRegionDestroyed = false;
-                }
-              }
-            }
-          } else {
-            message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__COULDNOT_FIND_REGIONPATH_0_IN_GEMFIRE, new Object[] { regionPath, "jmx-manager-update-rate milliseconds" });
-            isRegionDestroyed = false;
-          }
-          if (isRegionDestroyed) {
-            result = ResultBuilder.createInfoResult(message);
-          } else {
-            result = ResultBuilder.createUserErrorResult(message);
-          }
+      if (!regionPathToUse.startsWith(Region.SEPARATOR)) {
+        regionPathToUse = Region.SEPARATOR + regionPathToUse;
+      }
+
+      Set<DistributedMember> regionMembersList = findMembersForRegion(cache, managementService, regionPathToUse);
+
+      if (regionMembersList.size() == 0) {
+        return ResultBuilder.createUserErrorResult(
+            CliStrings.format(CliStrings.DESTROY_REGION__MSG__COULDNOT_FIND_REGIONPATH_0_IN_GEMFIRE,
+                new Object[]{regionPath, "jmx-manager-update-rate milliseconds"}));
+      }
+
+      CliFunctionResult destroyRegionResult = null;
+
+      ResultCollector<?, ?> resultCollector = CliUtil.executeFunction(RegionDestroyFunction.INSTANCE, regionPath,
+          regionMembersList);
+      List<CliFunctionResult> resultsList = (List<CliFunctionResult>) resultCollector.getResult();
+      message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__REGION_0_1_DESTROYED,
+          new Object[]{regionPath, /*subRegionMessage*/""});
+
+      // Only if there is an error is this set to false
+      boolean isRegionDestroyed = true;
+      for (int i = 0; i < resultsList.size(); i++) {
+        destroyRegionResult = resultsList.get(i);
+        if (destroyRegionResult.isSuccessful()) {
+          xmlEntity = destroyRegionResult.getXmlEntity();
+        } else if (destroyRegionResult.getThrowable() != null) {
+          Throwable t = destroyRegionResult.getThrowable();
+          LogWrapper.getInstance().info(t.getMessage(), t);
+          message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__ERROR_OCCURRED_WHILE_DESTROYING_0_REASON_1,
+              new Object[]{regionPath, t.getMessage()});
+          isRegionDestroyed = false;
+        } else {
+          message = CliStrings.format(CliStrings.DESTROY_REGION__MSG__UNKNOWN_RESULT_WHILE_DESTROYING_REGION_0_REASON_1,
+              new Object[]{regionPath, destroyRegionResult.getMessage()});
+          isRegionDestroyed = false;
         }
+      }
+      if (isRegionDestroyed) {
+        result = ResultBuilder.createInfoResult(message);
       } else {
-        result = ResultBuilder.createInfoResult(CliStrings.DESTROY_REGION__MSG__SPECIFY_REGIONPATH_TO_DESTROY);
+        result = ResultBuilder.createUserErrorResult(message);
       }
     } catch (IllegalStateException e) {
-      result = ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.DESTROY_REGION__MSG__ERROR_WHILE_DESTROYING_REGION_0_REASON_1, new Object[] {regionPath, e.getMessage()}));
+      result = ResultBuilder.createUserErrorResult(
+          CliStrings.format(CliStrings.DESTROY_REGION__MSG__ERROR_WHILE_DESTROYING_REGION_0_REASON_1,
+              new Object[]{regionPath, e.getMessage()}));
     } catch (Exception e) {
-      result = ResultBuilder.createGemFireErrorResult(CliStrings.format(CliStrings.DESTROY_REGION__MSG__ERROR_WHILE_DESTROYING_REGION_0_REASON_1, new Object[] {regionPath, e.getMessage()}));
+      result = ResultBuilder.createGemFireErrorResult(
+          CliStrings.format(CliStrings.DESTROY_REGION__MSG__ERROR_WHILE_DESTROYING_REGION_0_REASON_1,
+              new Object[]{regionPath, e.getMessage()}));
     }
 
     if (xmlEntity != null) {
@@ -1121,32 +1074,36 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     return result;
   }
 
-  private Set<DistributedMember>[] findMembersForRegion(Cache cache, ManagementService managementService, String regionPath) {
-    Set<DistributedMember>[] membersList = null;
+  private Set<DistributedMember> findMembersForRegion(Cache cache, ManagementService managementService, String regionPath) {
+    Set<DistributedMember> membersList = new HashSet<>();
     Set<String> regionMemberIds = Collections.emptySet();
-    Set<String> localRegionMemberIds = Collections.emptySet();
     MBeanServer mbeanServer = MBeanJMXAdapter.mbeanServer;
     String queryExp = MessageFormat.format(MBeanJMXAdapter.OBJECTNAME__REGION_MXBEAN, new Object[] {regionPath, "*"});
-//    String queryExp = MessageFormat.format(MBeanJMXAdapter.OBJECTNAME__REGION_MXBEAN, new Object[] {MBeanJMXAdapter.makeCompliantRegionNameAppender(regionPath), "*"});
+
     try {
       ObjectName queryExpON = new ObjectName(queryExp);
       Set<ObjectName> queryNames = mbeanServer.queryNames(null, queryExpON);
       if (queryNames != null && queryNames.size() != 0) {
-        regionMemberIds      = new HashSet<String>();
-        localRegionMemberIds = new HashSet<String>();
-
-        membersList = (Set<DistributedMember>[])Array.newInstance(HashSet.class, 2);
+        regionMemberIds      = new HashSet<>();
       }
+
+      boolean addedOneRemote = false;
       for (ObjectName regionMBeanObjectName : queryNames) {
         try {
           RegionMXBean regionMXBean = managementService.getMBeanInstance(regionMBeanObjectName, RegionMXBean.class);
           if (regionMXBean != null) {
             RegionAttributesData regionAttributes = regionMXBean.listRegionAttributes();
             String scope = regionAttributes.getScope();
+            // For Scope.LOCAL regions we need to identify each hosting member, but for
+            // other scopes we just need a single member as the region destroy will be
+            // propagated.
             if (Scope.LOCAL.equals(Scope.fromString(scope))) {
-              localRegionMemberIds.add(regionMXBean.getMember());
-            } else {
               regionMemberIds.add(regionMXBean.getMember());
+            } else {
+              if (!addedOneRemote) {
+                regionMemberIds.add(regionMXBean.getMember());
+                addedOneRemote = true;
+              }
             }
           }
         } catch (ClassCastException e) {
@@ -1156,13 +1113,10 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           }
         }
       }
+
       if (!regionMemberIds.isEmpty()) {
-        membersList[0] = getMembersByIds(cache, regionMemberIds);
+        membersList = getMembersByIds(cache, regionMemberIds);
       }
-      if (!localRegionMemberIds.isEmpty()) {
-        membersList[1] = getMembersByIds(cache, localRegionMemberIds);
-      }
-      
     } catch (MalformedObjectNameException e) {
       LogWrapper.getInstance().info(e.getMessage(), e);
     } catch (NullPointerException e) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ae7d4da9/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
index 7319934..10b37c1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
@@ -22,6 +22,7 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEvent;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventListener;
 import com.gemstone.gemfire.cache.wan.GatewaySenderFactory;
@@ -45,8 +46,7 @@ import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
-import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.jayway.awaitility.Awaitility;
 
 import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
@@ -59,7 +59,9 @@ import java.text.MessageFormat;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
+import java.util.concurrent.Callable;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
 
 public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBase {
   private static final long serialVersionUID = 1L;
@@ -181,7 +183,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     assertEquals(Result.Status.OK, cmdResult.getStatus());
   }
 
-  public void testDestroyRegion() {
+  public void testDestroyDistributedRegion() {
     createDefaultSetup(null);
 
     for (int i = 1; i <= 2; i++) {
@@ -198,32 +200,8 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
       });
     }
 
-    // Make sure that the region has been registered with the Manager MXBean
-    Host.getHost(0).getVM(0).invoke(() -> {
-      WaitCriterion wc = new WaitCriterion() {
-        @Override
-        public boolean done() {
-          try {
-            MBeanServer mbeanServer = MBeanJMXAdapter.mbeanServer;
-            String queryExp = MessageFormat.format(ManagementConstants.OBJECTNAME__REGION_MXBEAN,
-                new Object[]{"/Customer", "*"});
-            ObjectName queryExpON = new ObjectName(queryExp);
-            return !(mbeanServer.queryNames(null, queryExpON).isEmpty());
-          } catch (MalformedObjectNameException mone) {
-            LogWriterUtils.getLogWriter().error(mone);
-            fail(mone.getMessage());
-            return false;
-          }
-        }
-
-        @Override
-        public String description() {
-          return "Waiting for the region to be registed with the MXBean";
-        }
-      };
-
-      Wait.waitForCriterion(wc, 5000, 500, true);
-    });
+    waitForRegionMBeanCreation("/Customer", 2);
+    waitForRegionMBeanCreation("/Order", 2);
 
     // Test failure when region not found
     String command = "destroy region --name=DOESNOTEXIST";
@@ -260,6 +238,111 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     assertEquals(Result.Status.OK, cmdResult.getStatus());
   }
 
+  public void testDestroyLocalRegions() {
+    createDefaultSetup(null);
+
+    for (int i = 1; i <= 3; i++) {
+      Host.getHost(0).getVM(i).invoke(() -> {
+        final Cache cache = getCache();
+
+        RegionFactory<Object, Object> factory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+        factory.setScope(Scope.LOCAL);
+        factory.create("Customer");
+      });
+    }
+
+    waitForRegionMBeanCreation("/Customer", 3);
+
+    // Test failure when region not found
+    String command = "destroy region --name=DOESNOTEXIST";
+    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    String strr = commandResultToString(cmdResult);
+    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    assertTrue(stringContainsLine(strr, "Could not find.*\"DOESNOTEXIST\".*"));
+    assertEquals(Result.Status.ERROR, cmdResult.getStatus());
+
+    command = "destroy region --name=/Customer";
+    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    cmdResult = executeCommand(command);
+    strr = commandResultToString(cmdResult);
+    assertTrue(stringContainsLine(strr, ".*Customer.*destroyed successfully.*"));
+    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    for (int i = 1; i <= 3; i++) {
+      final int x = i;
+      Host.getHost(0).getVM(i).invoke(() -> {
+        assertNull("Region still exists in VM " + x, getCache().getRegion("Customer"));
+      });
+    }
+  }
+
+  public void testDestroyLocalAndDistributedRegions() {
+    createDefaultSetup(null);
+
+    for (int i = 1; i <= 2; i++) {
+      Host.getHost(0).getVM(i).invoke(() -> {
+        final Cache cache = getCache();
+        RegionFactory<Object, Object> factory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        factory.create("Customer");
+      });
+    }
+
+    Host.getHost(0).getVM(3).invoke(() -> {
+      final Cache cache = getCache();
+      RegionFactory<Object, Object> factory = cache.createRegionFactory(RegionShortcut.REPLICATE);
+      factory.setScope(Scope.LOCAL);
+      factory.create("Customer");
+    });
+
+    waitForRegionMBeanCreation("/Customer", 3);
+
+    // Test failure when region not found
+    String command = "destroy region --name=DOESNOTEXIST";
+    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    CommandResult cmdResult = executeCommand(command);
+    String strr = commandResultToString(cmdResult);
+    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    assertTrue(stringContainsLine(strr, "Could not find.*\"DOESNOTEXIST\".*"));
+    assertEquals(Result.Status.ERROR, cmdResult.getStatus());
+
+    command = "destroy region --name=/Customer";
+    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    cmdResult = executeCommand(command);
+    strr = commandResultToString(cmdResult);
+    assertTrue(stringContainsLine(strr, ".*Customer.*destroyed successfully.*"));
+    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    assertEquals(Result.Status.OK, cmdResult.getStatus());
+
+    for (int i = 1; i <= 3; i++) {
+      final int x = i;
+      Host.getHost(0).getVM(i).invoke(() -> {
+        assertNull("Region still exists in VM " + x, getCache().getRegion("Customer"));
+      });
+    }
+  }
+
+  private void waitForRegionMBeanCreation(final String regionPath, final int mbeanCount) {
+    Host.getHost(0).getVM(0).invoke(() -> {
+      Awaitility.waitAtMost(5, TimeUnit.SECONDS).until(newRegionMBeanIsCreated(regionPath, mbeanCount));
+    });
+  }
+
+  private Callable<Boolean> newRegionMBeanIsCreated(final String regionPath, final int mbeanCount) {
+    return () -> {
+      try {
+        MBeanServer mbeanServer = MBeanJMXAdapter.mbeanServer;
+        String queryExp = MessageFormat.format(ManagementConstants.OBJECTNAME__REGION_MXBEAN, new Object[]{regionPath, "*"});
+        ObjectName queryExpON = new ObjectName(queryExp);
+        return mbeanServer.queryNames(null, queryExpON).size() == mbeanCount;
+      } catch (MalformedObjectNameException mone) {
+        LogWriterUtils.getLogWriter().error(mone);
+        fail(mone.getMessage());
+        return false;
+      }
+    };
+  }
 
   public void testCreateRegion46391() throws IOException {
     createDefaultSetup(null);
@@ -669,6 +752,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     disconnectAllFromDS();
 
     final String regionName = "testRegionSharedConfigRegion";
+    final String regionPath = "/" + regionName;
     final String groupName = "testRegionSharedConfigGroup";
 
     // Start the Locator and wait for shared configuration to be available
@@ -684,18 +768,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
         final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
             locatorProps);
 
-        WaitCriterion wc = new WaitCriterion() {
-          @Override
-          public boolean done() {
-            return locator.isSharedConfigurationRunning();
-          }
-
-          @Override
-          public String description() {
-            return "Waiting for shared configuration to be started";
-          }
-        };
-        Wait.waitForCriterion(wc, 5000, 500, true);
+        Awaitility.waitAtMost(5, TimeUnit.SECONDS).until(() -> locator.isSharedConfigurationRunning());
       } catch (IOException ioex) {
         fail("Unable to create a locator with a shared configuration");
       }
@@ -728,31 +801,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     assertEquals(Result.Status.OK, cmdResult.getStatus());
 
     // Make sure that the region has been registered with the Manager MXBean
-    Host.getHost(0).getVM(0).invoke(() -> {
-      WaitCriterion wc = new WaitCriterion() {
-        @Override
-        public boolean done() {
-          try {
-            MBeanServer mbeanServer = MBeanJMXAdapter.mbeanServer;
-            String queryExp = MessageFormat.format(ManagementConstants.OBJECTNAME__REGION_MXBEAN,
-                new Object[]{"/" + regionName, "*"});
-            ObjectName queryExpON = new ObjectName(queryExp);
-            return !(mbeanServer.queryNames(null, queryExpON).isEmpty());
-          } catch (MalformedObjectNameException mone) {
-            LogWriterUtils.getLogWriter().error(mone);
-            fail(mone.getMessage());
-            return false;
-          }
-        }
-
-        @Override
-        public String description() {
-          return "Waiting for the region to be registed with the MXBean";
-        }
-      };
-
-      Wait.waitForCriterion(wc, 5000, 500, true);
-    });
+    waitForRegionMBeanCreation(regionPath, 1);
 
     // Make sure the region exists in the shared config
     Host.getHost(0).getVM(3).invoke(() -> {
@@ -836,6 +885,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     disconnectAllFromDS();
 
     final String regionName = "testRegionSharedConfigRegion";
+    final String regionPath = "/" + regionName;
     final String groupName = "testRegionSharedConfigGroup";
 
     // Start the Locator and wait for shared configuration to be available
@@ -851,18 +901,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
         final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
             locatorProps);
 
-        WaitCriterion wc = new WaitCriterion() {
-          @Override
-          public boolean done() {
-            return locator.isSharedConfigurationRunning();
-          }
-
-          @Override
-          public String description() {
-            return "Waiting for shared configuration to be started";
-          }
-        };
-        Wait.waitForCriterion(wc, 5000, 500, true);
+        Awaitility.waitAtMost(5, TimeUnit.SECONDS).until(() -> locator.isSharedConfigurationRunning());
       } catch (IOException ioex) {
         fail("Unable to create a locator with a shared configuration");
       }
@@ -895,31 +934,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     assertEquals(Result.Status.OK, cmdResult.getStatus());
 
     // Make sure that the region has been registered with the Manager MXBean
-    Host.getHost(0).getVM(0).invoke(() -> {
-      WaitCriterion wc = new WaitCriterion() {
-        @Override
-        public boolean done() {
-          try {
-            MBeanServer mbeanServer = MBeanJMXAdapter.mbeanServer;
-            String queryExp = MessageFormat.format(ManagementConstants.OBJECTNAME__REGION_MXBEAN,
-                new Object[]{"/" + regionName, "*"});
-            ObjectName queryExpON = new ObjectName(queryExp);
-            return !(mbeanServer.queryNames(null, queryExpON).isEmpty());
-          } catch (MalformedObjectNameException mone) {
-            LogWriterUtils.getLogWriter().error(mone);
-            fail(mone.getMessage());
-            return false;
-          }
-        }
-
-        @Override
-        public String description() {
-          return "Waiting for the region to be registed with the MXBean";
-        }
-      };
-
-      Wait.waitForCriterion(wc, 5000, 500, true);
-    });
+    waitForRegionMBeanCreation(regionPath, 1);
 
     // Make sure the region exists in the shared config
     Host.getHost(0).getVM(3).invoke(() -> {